You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by rm...@apache.org on 2014/06/26 11:46:26 UTC

[01/53] [abbrv] Rework the Taskmanager to a slot based model and remove legacy cloud code

Repository: incubator-flink
Updated Branches:
  refs/heads/travis_test f8ec28c73 -> f94225397


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/GlobalSortingMixedOrderITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/GlobalSortingMixedOrderITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/GlobalSortingMixedOrderITCase.java
index 51d7a66..a9bda2b 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/GlobalSortingMixedOrderITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/GlobalSortingMixedOrderITCase.java
@@ -36,42 +36,44 @@ import eu.stratosphere.types.Key;
 public class GlobalSortingMixedOrderITCase extends RecordAPITestBase {
 	
 	private static final int NUM_RECORDS = 100000;
-	
+
 	private static final int RANGE_I1 = 100;
 	private static final int RANGE_I2 = 20;
 	private static final int RANGE_I3 = 20;
-	
+
 	private String recordsPath;
 	private String resultPath;
 
 	private String sortedRecords;
 
-
+	public GlobalSortingMixedOrderITCase(){
+		setTaskManagerNumSlots(DOP);
+	}
 
 	@Override
 	protected void preSubmit() throws Exception {
-		
+
 		ArrayList<TripleInt> records = new ArrayList<TripleInt>();
-		
+
 		//Generate records
 		final Random rnd = new Random(1988);
 		final StringBuilder sb = new StringBuilder(NUM_RECORDS * 7);
-		
-		
+
+
 		for (int j = 0; j < NUM_RECORDS; j++) {
 			TripleInt val = new TripleInt(rnd.nextInt(RANGE_I1), rnd.nextInt(RANGE_I2), rnd.nextInt(RANGE_I3));
 			records.add(val);
 			sb.append(val);
 			sb.append('\n');
 		}
-		
-		
+
+
 		this.recordsPath = createTempFile("records", sb.toString());
 		this.resultPath = getTempDirPath("result");
 
 		// create the sorted result;
 		Collections.sort(records);
-		
+
 		sb.setLength(0);
 		for (TripleInt val : records) {
 			sb.append(val);
@@ -83,7 +85,7 @@ public class GlobalSortingMixedOrderITCase extends RecordAPITestBase {
 	@Override
 	protected Plan getTestJob() {
 		GlobalSort globalSort = new GlobalSort();
-		return globalSort.getPlan("4", recordsPath, resultPath);
+		return globalSort.getPlan(new Integer(DOP).toString(), recordsPath, resultPath);
 	}
 
 	@Override
@@ -91,22 +93,22 @@ public class GlobalSortingMixedOrderITCase extends RecordAPITestBase {
 		// Test results
 		compareResultsByLinesInMemoryWithStrictOrder(this.sortedRecords, this.resultPath);
 	}
-	
-	
+
+
 	public static class TripleIntDistribution implements DataDistribution {
-		
+
 		private static final long serialVersionUID = 1L;
-		
+
 		private boolean ascendingI1, ascendingI2, ascendingI3;
-		
+
 		public TripleIntDistribution(Order orderI1, Order orderI2, Order orderI3) {
 			this.ascendingI1 = orderI1 != Order.DESCENDING;
 			this.ascendingI2 = orderI2 != Order.DESCENDING;
 			this.ascendingI3 = orderI3 != Order.DESCENDING;
 		}
-		
+
 		public TripleIntDistribution() {}
-		
+
 		@Override
 		public void write(DataOutput out) throws IOException {
 			out.writeBoolean(this.ascendingI1);
@@ -129,7 +131,7 @@ public class GlobalSortingMixedOrderITCase extends RecordAPITestBase {
 			if (!this.ascendingI1) {
 				boundVal = RANGE_I1 - boundVal;
 			}
-			
+
 			return new Key[] { new IntValue(boundVal), new IntValue(RANGE_I2), new IntValue(RANGE_I3) };
 		}
 
@@ -137,11 +139,11 @@ public class GlobalSortingMixedOrderITCase extends RecordAPITestBase {
 		public int getNumberOfFields() {
 			return 3;
 		}
-		
+
 	}
-	
+
 	private static class GlobalSort implements Program {
-		
+
 		private static final long serialVersionUID = 1L;
 
 		@Override
@@ -150,10 +152,10 @@ public class GlobalSortingMixedOrderITCase extends RecordAPITestBase {
 			final int numSubtasks     = (args.length > 0 ? Integer.parseInt(args[0]) : 1);
 			final String recordsPath = (args.length > 1 ? args[1] : "");
 			final String output      = (args.length > 2 ? args[2] : "");
-			
+
 			@SuppressWarnings("unchecked")
 			FileDataSource source = new FileDataSource(new CsvInputFormat(',', IntValue.class, IntValue.class, IntValue.class), recordsPath);
-			
+
 			FileDataSink sink = new FileDataSink(CsvOutputFormat.class, output);
 			CsvOutputFormat.configureRecordFormat(sink)
 				.recordDelimiter('\n')
@@ -162,34 +164,34 @@ public class GlobalSortingMixedOrderITCase extends RecordAPITestBase {
 				.field(IntValue.class, 0)
 				.field(IntValue.class, 1)
 				.field(IntValue.class, 2);
-			
+
 			sink.setGlobalOrder(
 				new Ordering(0, IntValue.class, Order.DESCENDING)
 					.appendOrdering(1, IntValue.class, Order.ASCENDING)
 					.appendOrdering(2, IntValue.class, Order.DESCENDING),
 				new TripleIntDistribution(Order.DESCENDING, Order.ASCENDING, Order.DESCENDING));
 			sink.setInput(source);
-			
+
 			Plan p = new Plan(sink);
 			p.setDefaultParallelism(numSubtasks);
 			return p;
 		}
 	}
-	
+
 	/**
 	 * Three integers sorting descending, ascending, descending.
 	 */
 	private static final class TripleInt implements Comparable<TripleInt> {
-		
+
 		private final int i1, i2, i3;
 
-		
+
 		private TripleInt(int i1, int i2, int i3) {
 			this.i1 = i1;
 			this.i2 = i2;
 			this.i3 = i3;
 		}
-		
+
 		@Override
 		public String toString() {
 			StringBuilder bld = new StringBuilder(32);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/GroupOrderReduceITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/GroupOrderReduceITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/GroupOrderReduceITCase.java
index d9b244c..48479ed 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/GroupOrderReduceITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/GroupOrderReduceITCase.java
@@ -60,6 +60,7 @@ public class GroupOrderReduceITCase extends RecordAPITestBase {
 	
 	public GroupOrderReduceITCase(Configuration config) {
 		super(config);
+		setTaskManagerNumSlots(DOP);
 	}
 
 	
@@ -104,7 +105,7 @@ public class GroupOrderReduceITCase extends RecordAPITestBase {
 	@Parameters
 	public static Collection<Object[]> getConfigurations() {
 		Configuration config = new Configuration();
-		config.setInteger("GroupOrderTest#NumSubtasks", 4);
+		config.setInteger("GroupOrderTest#NumSubtasks", DOP);
 		return toParameterList(config);
 	}
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/MergeOnlyJoinITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/MergeOnlyJoinITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/MergeOnlyJoinITCase.java
index 47a0f59..f19ceee 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/MergeOnlyJoinITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/MergeOnlyJoinITCase.java
@@ -62,6 +62,7 @@ public class MergeOnlyJoinITCase extends RecordAPITestBase {
 
 	public MergeOnlyJoinITCase(Configuration config) {
 		super(config);
+		setTaskManagerNumSlots(4);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/PairwiseSPITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/PairwiseSPITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/PairwiseSPITCase.java
index 38168a2..52de6ab 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/PairwiseSPITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/PairwiseSPITCase.java
@@ -66,7 +66,7 @@ public class PairwiseSPITCase extends RecordAPITestBase {
 	@Override
 	protected Plan getTestJob() {
 		PairwiseSP a2aSP = new PairwiseSP();
-		return a2aSP.getPlan(config.getString("All2AllSPTest#NoSubtasks", "4"),
+		return a2aSP.getPlan(config.getString("All2AllSPTest#NoSubtasks", new Integer(DOP).toString()),
 				rdfDataPath,
 				resultPath,
 				"true");
@@ -80,7 +80,7 @@ public class PairwiseSPITCase extends RecordAPITestBase {
 	@Parameters
 	public static Collection<Object[]> getConfigurations() {
 		Configuration config = new Configuration();
-		config.setInteger("All2AllSPTest#NoSubtasks", 4);
+		config.setInteger("All2AllSPTest#NoSubtasks", DOP);
 		return toParameterList(config);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery10ITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery10ITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery10ITCase.java
index a19cad1..5450498 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery10ITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery10ITCase.java
@@ -200,7 +200,7 @@ public class TPCHQuery10ITCase extends RecordAPITestBase {
 	@Parameters
 	public static Collection<Object[]> getConfigurations() {
 		Configuration config = new Configuration();
-		config.setInteger("TPCHQuery10Test#NoSubtasks", 4);
+		config.setInteger("TPCHQuery10Test#NoSubtasks", DOP);
 		return toParameterList(config);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery3ITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery3ITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery3ITCase.java
index cc1d16a..54a5fe1 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery3ITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery3ITCase.java
@@ -122,6 +122,7 @@ public class TPCHQuery3ITCase extends RecordAPITestBase {
 
 	public TPCHQuery3ITCase(Configuration config) {
 		super(config);
+		setTaskManagerNumSlots(DOP);
 	}
 
 	@Override
@@ -150,7 +151,7 @@ public class TPCHQuery3ITCase extends RecordAPITestBase {
 	@Parameters
 	public static Collection<Object[]> getConfigurations() {
 		Configuration config = new Configuration();
-		config.setInteger("dop", 4);
+		config.setInteger("dop", DOP);
 		return toParameterList(config);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery3WithUnionITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery3WithUnionITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery3WithUnionITCase.java
index b0c5200..df95484 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery3WithUnionITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery3WithUnionITCase.java
@@ -121,6 +121,10 @@ public class TPCHQuery3WithUnionITCase extends RecordAPITestBase {
 	
 	private static final String EXPECTED_RESULT = "5|0|147828.97\n" + "66|0|99188.09\n";
 
+	public TPCHQuery3WithUnionITCase(){
+		setTaskManagerNumSlots(DOP);
+	}
+
 
 	@Override
 	protected void preSubmit() throws Exception {
@@ -136,7 +140,7 @@ public class TPCHQuery3WithUnionITCase extends RecordAPITestBase {
 	protected Plan getTestJob() {
 		TPCHQuery3Unioned tpch3 = new TPCHQuery3Unioned();
 		return tpch3.getPlan(
-				"4",
+				new Integer(DOP).toString(),
 				orders1Path,
 				orders2Path,
 				partJoin1Path,

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery4ITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery4ITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery4ITCase.java
index d8b9275..50097be 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery4ITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery4ITCase.java
@@ -112,6 +112,10 @@ public class TPCHQuery4ITCase extends RecordAPITestBase {
 
 	private static final String EXPECTED_RESULT = "1-URGENT|2|\n" + "3-MEDIUM|2|\n" + "4-NOT SPECIFIED|4|";
 
+	public TPCHQuery4ITCase(){
+		setTaskManagerNumSlots(DOP);
+	}
+
 	@Override
 	protected void preSubmit() throws Exception {
 		ordersPath = createTempFile("orders", ORDERS);
@@ -122,7 +126,7 @@ public class TPCHQuery4ITCase extends RecordAPITestBase {
 	@Override
 	protected Plan getTestJob() {
 		TPCHQuery4 tpch4 = new TPCHQuery4();
-		return tpch4.getPlan("4", ordersPath, lineitemsPath, resultPath);
+		return tpch4.getPlan(new Integer(DOP).toString(), ordersPath, lineitemsPath, resultPath);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery9ITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery9ITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery9ITCase.java
index b1e9af6..a863236 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery9ITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery9ITCase.java
@@ -335,6 +335,10 @@ public class TPCHQuery9ITCase extends RecordAPITestBase {
 		+ "IRAN|1992|37970.953\n"
 		+ "IRAN|1993|83140.0\n"
 		+ "IRAN|1996|9672.556\n";
+
+	public TPCHQuery9ITCase(){
+		setTaskManagerNumSlots(DOP);
+	}
 		
 
 	@Override
@@ -352,7 +356,7 @@ public class TPCHQuery9ITCase extends RecordAPITestBase {
 	protected Plan getTestJob() {
 		TPCHQuery9 tpch9 = new TPCHQuery9();
 		return tpch9.getPlan(
-				"4",
+				new Integer(DOP).toString(),
 				partInputPath,
 				partSuppInputPath,
 				ordersInputPath,

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQueryAsterixITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQueryAsterixITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQueryAsterixITCase.java
index 9c2ba26..0b6220c 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQueryAsterixITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQueryAsterixITCase.java
@@ -62,6 +62,10 @@ public class TPCHQueryAsterixITCase extends RecordAPITestBase {
 		"2|MACHINERY\n" +
 		"2|FURNITURE\n";
 
+	public TPCHQueryAsterixITCase(){
+		setTaskManagerNumSlots(DOP);
+	}
+
 
 	@Override
 	protected void preSubmit() throws Exception {
@@ -73,7 +77,7 @@ public class TPCHQueryAsterixITCase extends RecordAPITestBase {
 	@Override
 	protected Plan getTestJob() {
 		TPCHQueryAsterix tpchBench = new TPCHQueryAsterix();
-		return tpchBench.getPlan("4", ordersPath, custPath, resultPath);
+		return tpchBench.getPlan(new Integer(DOP).toString(), ordersPath, custPath, resultPath);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TeraSortITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TeraSortITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TeraSortITCase.java
index 97db904..ab2ee7b 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TeraSortITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TeraSortITCase.java
@@ -29,7 +29,10 @@ public class TeraSortITCase extends RecordAPITestBase {
 	private static final String INPUT_DATA_FILE = "/testdata/terainput.txt";
 	
 	private String resultPath;
-	
+
+	public TeraSortITCase(){
+		setTaskManagerNumSlots(DOP);
+	}
 
 	@Override
 	protected void preSubmit() throws Exception {
@@ -41,7 +44,7 @@ public class TeraSortITCase extends RecordAPITestBase {
 		String testDataPath = getClass().getResource(INPUT_DATA_FILE).toString();
 		
 		TeraSort ts = new TeraSort();
-		return ts.getPlan("4", testDataPath, resultPath);
+		return ts.getPlan(new Integer(DOP).toString(), testDataPath, resultPath);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/WebLogAnalysisITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/WebLogAnalysisITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/WebLogAnalysisITCase.java
index 8b60d52..a0458c7 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/WebLogAnalysisITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/WebLogAnalysisITCase.java
@@ -148,6 +148,10 @@ public class WebLogAnalysisITCase extends RecordAPITestBase {
 
 	private static final String expected = "87|url_24|39\n" + "59|url_28|41\n";
 
+	public WebLogAnalysisITCase(){
+		setTaskManagerNumSlots(DOP);
+	}
+
 	@Override
 	protected void preSubmit() throws Exception {
 		docsPath   = createTempFile("docs", docs);
@@ -159,7 +163,7 @@ public class WebLogAnalysisITCase extends RecordAPITestBase {
 	@Override
 	protected Plan getTestJob() {
 		WebLogAnalysis relOLAP = new WebLogAnalysis();
-		return relOLAP.getPlan("4", docsPath, ranksPath, visitsPath, resultPath);
+		return relOLAP.getPlan(new Integer(DOP).toString(), docsPath, ranksPath, visitsPath, resultPath);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/WordCountITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/WordCountITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/WordCountITCase.java
index 1a2d183..1adf4f8 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/WordCountITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/WordCountITCase.java
@@ -23,6 +23,10 @@ public class WordCountITCase extends RecordAPITestBase {
 	protected String textPath;
 	protected String resultPath;
 
+	public WordCountITCase(){
+		setTaskManagerNumSlots(DOP);
+	}
+
 	
 	@Override
 	protected void preSubmit() throws Exception {
@@ -33,7 +37,7 @@ public class WordCountITCase extends RecordAPITestBase {
 	@Override
 	protected Plan getTestJob() {
 		WordCount wc = new WordCount();
-		return wc.getPlan("4", textPath, resultPath);
+		return wc.getPlan(new Integer(DOP).toString(), textPath, resultPath);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/WordCountUnionReduceITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/WordCountUnionReduceITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/WordCountUnionReduceITCase.java
index 30ce102..35a14c5 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/WordCountUnionReduceITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/WordCountUnionReduceITCase.java
@@ -46,6 +46,10 @@ public class WordCountUnionReduceITCase extends RecordAPITestBase {
 
 	private String outputPath;
 
+	public WordCountUnionReduceITCase(){
+		setTaskManagerNumSlots(DOP);
+	}
+
 
 	@Override
 	protected void preSubmit() throws Exception {
@@ -61,7 +65,7 @@ public class WordCountUnionReduceITCase extends RecordAPITestBase {
 	@Override
 	protected Plan getTestJob() {
 		WordCountUnionReduce wc = new WordCountUnionReduce();
-		return wc.getPlan(this.inputPath, this.outputPath, 4);
+		return wc.getPlan(this.inputPath, this.outputPath, DOP);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackThroughput.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackThroughput.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackThroughput.java
index fae3f99..a8ab311 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackThroughput.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackThroughput.java
@@ -30,6 +30,7 @@ import eu.stratosphere.runtime.io.api.RecordWriter;
 import eu.stratosphere.runtime.io.channels.ChannelType;
 import eu.stratosphere.test.util.RecordAPITestBase;
 import eu.stratosphere.util.LogUtils;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.junit.After;
@@ -53,9 +54,9 @@ public class NetworkStackThroughput extends RecordAPITestBase {
 
 	private static final String USE_FORWARDER_CONFIG_KEY = "use.forwarder";
 
-	private static final String NUM_SUBTASKS_CONFIG_KEY = "num.subtasks";
+	private static final String PARALLELISM_CONFIG_KEY = "num.subtasks";
 
-	private static final String NUM_SUBTASKS_PER_INSTANCE_CONFIG_KEY = "num.subtasks.instance";
+	private static final String NUM_SLOTS_PER_TM_CONFIG_KEY = "num.slots.per.tm";
 
 	private static final String IS_SLOW_SENDER_CONFIG_KEY = "is.slow.sender";
 
@@ -64,13 +65,35 @@ public class NetworkStackThroughput extends RecordAPITestBase {
 	private static final int IS_SLOW_SLEEP_MS = 10;
 
 	private static final int IS_SLOW_EVERY_NUM_RECORDS = (2 * 32 * 1024) / SpeedTestRecord.RECORD_SIZE;
+	
+	// ------------------------------------------------------------------------
+	
+	private int dataVolumeGb;
+	private boolean useForwarder;
+	private boolean isSlowSender;
+	private boolean isSlowReceiver;
+	private int parallelism;
 
 	// ------------------------------------------------------------------------
 
 	public NetworkStackThroughput(Configuration config) {
 		super(config);
-
-		setNumTaskManager(2);
+		
+		dataVolumeGb = this.config.getInteger(DATA_VOLUME_GB_CONFIG_KEY, 1);
+		useForwarder = this.config.getBoolean(USE_FORWARDER_CONFIG_KEY, true);
+		isSlowSender = this.config.getBoolean(IS_SLOW_SENDER_CONFIG_KEY, false);
+		isSlowReceiver = this.config.getBoolean(IS_SLOW_RECEIVER_CONFIG_KEY, false);
+		parallelism = config.getInteger(PARALLELISM_CONFIG_KEY, 1);
+		
+		int numSlots = config.getInteger(NUM_SLOTS_PER_TM_CONFIG_KEY, 1);
+		
+		if (parallelism % numSlots != 0) {
+			throw new RuntimeException("The test case defines a parallelism that is not a multiple of the slots per task manager.");
+		}
+		
+		setNumTaskTracker(parallelism / numSlots);
+		setTaskManagerNumSlots(numSlots);
+		
 		LogUtils.initializeDefaultConsoleLogger();
 	}
 
@@ -94,8 +117,8 @@ public class NetworkStackThroughput extends RecordAPITestBase {
 			config.setBoolean(USE_FORWARDER_CONFIG_KEY, (Boolean) p[1]);
 			config.setBoolean(IS_SLOW_SENDER_CONFIG_KEY, (Boolean) p[2]);
 			config.setBoolean(IS_SLOW_RECEIVER_CONFIG_KEY, (Boolean) p[3]);
-			config.setInteger(NUM_SUBTASKS_CONFIG_KEY, (Integer) p[4]);
-			config.setInteger(NUM_SUBTASKS_PER_INSTANCE_CONFIG_KEY, (Integer) p[5]);
+			config.setInteger(PARALLELISM_CONFIG_KEY, (Integer) p[4]);
+			config.setInteger(NUM_SLOTS_PER_TM_CONFIG_KEY, (Integer) p[5]);
 
 			configs.add(config);
 		}
@@ -107,14 +130,7 @@ public class NetworkStackThroughput extends RecordAPITestBase {
 
 	@Override
 	protected JobGraph getJobGraph() throws Exception {
-		int dataVolumeGb = this.config.getInteger(DATA_VOLUME_GB_CONFIG_KEY, 1);
-		boolean useForwarder = this.config.getBoolean(USE_FORWARDER_CONFIG_KEY, true);
-		boolean isSlowSender = this.config.getBoolean(IS_SLOW_SENDER_CONFIG_KEY, false);
-		boolean isSlowReceiver = this.config.getBoolean(IS_SLOW_RECEIVER_CONFIG_KEY, false);
-		int numSubtasks = this.config.getInteger(NUM_SUBTASKS_CONFIG_KEY, 1);
-		int numSubtasksPerInstance = this.config.getInteger(NUM_SUBTASKS_PER_INSTANCE_CONFIG_KEY, 1);
-
-		return createJobGraph(dataVolumeGb, useForwarder, isSlowSender, isSlowReceiver, numSubtasks, numSubtasksPerInstance);
+		return createJobGraph(dataVolumeGb, useForwarder, isSlowSender, isSlowReceiver, parallelism);
 	}
 
 	@After
@@ -133,14 +149,13 @@ public class NetworkStackThroughput extends RecordAPITestBase {
 	}
 
 	private JobGraph createJobGraph(int dataVolumeGb, boolean useForwarder, boolean isSlowSender, boolean isSlowReceiver,
-									int numSubtasks, int numSubtasksPerInstance) throws JobGraphDefinitionException {
+									int numSubtasks) throws JobGraphDefinitionException {
 
 		JobGraph jobGraph = new JobGraph("Speed Test");
 
 		JobInputVertex producer = new JobGenericInputVertex("Speed Test Producer", jobGraph);
 		producer.setInputClass(SpeedTestProducer.class);
 		producer.setNumberOfSubtasks(numSubtasks);
-		producer.setNumberOfSubtasksPerInstance(numSubtasksPerInstance);
 		producer.getConfiguration().setInteger(DATA_VOLUME_GB_CONFIG_KEY, dataVolumeGb);
 		producer.getConfiguration().setBoolean(IS_SLOW_SENDER_CONFIG_KEY, isSlowSender);
 
@@ -149,13 +164,11 @@ public class NetworkStackThroughput extends RecordAPITestBase {
 			forwarder = new JobTaskVertex("Speed Test Forwarder", jobGraph);
 			forwarder.setTaskClass(SpeedTestForwarder.class);
 			forwarder.setNumberOfSubtasks(numSubtasks);
-			forwarder.setNumberOfSubtasksPerInstance(numSubtasksPerInstance);
 		}
 
 		JobOutputVertex consumer = new JobOutputVertex("Speed Test Consumer", jobGraph);
 		consumer.setOutputClass(SpeedTestConsumer.class);
 		consumer.setNumberOfSubtasks(numSubtasks);
-		consumer.setNumberOfSubtasksPerInstance(numSubtasksPerInstance);
 		consumer.getConfiguration().setBoolean(IS_SLOW_RECEIVER_CONFIG_KEY, isSlowReceiver);
 
 		if (useForwarder) {


[04/53] [abbrv] Rework the Taskmanager to a slot based model and remove legacy cloud code

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/cluster/DefaultInstanceManagerTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/cluster/DefaultInstanceManagerTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/cluster/DefaultInstanceManagerTest.java
new file mode 100644
index 0000000..7460200
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/cluster/DefaultInstanceManagerTest.java
@@ -0,0 +1,232 @@
+/***********************************************************************************************************************
+ * 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.instance.cluster;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.net.InetAddress;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import eu.stratosphere.nephele.instance.*;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import eu.stratosphere.configuration.ConfigConstants;
+import eu.stratosphere.configuration.Configuration;
+import eu.stratosphere.configuration.GlobalConfiguration;
+import eu.stratosphere.nephele.jobgraph.JobID;
+import eu.stratosphere.util.LogUtils;
+
+/**
+ * Tests for {@link eu.stratosphere.nephele.instance.DefaultInstanceManager}.
+ */
+public class DefaultInstanceManagerTest {
+
+	@BeforeClass
+	public static void initLogging() {
+		LogUtils.initializeDefaultTestConsoleLogger();
+	}
+	
+	
+	@Test
+	public void testInstanceRegistering() {
+		try {
+			DefaultInstanceManager cm = new DefaultInstanceManager();
+			TestInstanceListener testInstanceListener = new TestInstanceListener();
+			cm.setInstanceListener(testInstanceListener);
+			
+			
+			int ipcPort = ConfigConstants.DEFAULT_TASK_MANAGER_IPC_PORT;
+			int dataPort = ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT;
+
+			HardwareDescription hardwareDescription = HardwareDescriptionFactory.construct(2, 2L * 1024L * 1024L * 1024L,
+																				2L * 1024L * 1024L * 1024L);
+
+			String hostname = "192.168.198.1";
+			InetAddress address = InetAddress.getByName("192.168.198.1");
+			
+			InstanceConnectionInfo ici1 = new InstanceConnectionInfo(address, hostname, null, ipcPort + 0, dataPort + 0);
+			InstanceConnectionInfo ici2 = new InstanceConnectionInfo(address, hostname, null, ipcPort + 15, dataPort + 15);
+			InstanceConnectionInfo ici3 = new InstanceConnectionInfo(address, hostname, null, ipcPort + 30, dataPort + 30);
+			
+			// register three instances
+			cm.registerTaskManager(ici1, hardwareDescription, 1);
+			cm.registerTaskManager(ici2, hardwareDescription, 1);
+			cm.registerTaskManager(ici3, hardwareDescription, 1);
+			
+
+			assertEquals(3, cm.getNumberOfSlots());
+
+			cm.shutdown();
+		}
+		catch (Exception e) {
+			System.err.println(e.getMessage());
+			e.printStackTrace();
+			Assert.fail("Test erroneous: " + e.getMessage());
+		}
+	}
+
+	@Test
+	public void testAllocationDeallocation() {
+		try {
+			DefaultInstanceManager cm = new DefaultInstanceManager();
+			TestInstanceListener testInstanceListener = new TestInstanceListener();
+			cm.setInstanceListener(testInstanceListener);
+			
+			
+			int ipcPort = ConfigConstants.DEFAULT_TASK_MANAGER_IPC_PORT;
+			int dataPort = ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT;
+
+			HardwareDescription hardwareDescription = HardwareDescriptionFactory.construct(2, 2L * 1024L * 1024L * 1024L,
+																				2L * 1024L * 1024L * 1024L);
+
+			String hostname = "192.168.198.1";
+			InetAddress address = InetAddress.getByName("192.168.198.1");
+			
+			InstanceConnectionInfo ici1 = new InstanceConnectionInfo(address, hostname, null, ipcPort + 0, dataPort + 0);
+			InstanceConnectionInfo ici2 = new InstanceConnectionInfo(address, hostname, null, ipcPort + 15, dataPort + 15);
+			
+			// register three instances
+			cm.registerTaskManager(ici1, hardwareDescription, 1);
+			cm.registerTaskManager(ici2, hardwareDescription, 1);
+
+			assertEquals(2, cm.getNumberOfSlots());
+			
+			
+			// allocate something
+			JobID jobID = new JobID();
+			Configuration conf = new Configuration();
+			cm.requestInstance(jobID, conf, 2);
+			
+			DefaultInstanceManagerTestUtils.waitForInstances(jobID, testInstanceListener, 3, 1000);
+			
+			List<AllocatedResource> allocatedResources = testInstanceListener.getAllocatedResourcesForJob(jobID);
+			assertEquals(2, allocatedResources.size());
+			
+			Iterator<AllocatedResource> it = allocatedResources.iterator();
+			Set<AllocationID> allocationIDs = new HashSet<AllocationID>();
+			while (it.hasNext()) {
+				AllocatedResource allocatedResource = it.next();
+
+				if (allocationIDs.contains(allocatedResource.getAllocationID())) {
+					fail("Discovered allocation ID " + allocatedResource.getAllocationID() + " at least twice");
+				} else {
+					allocationIDs.add(allocatedResource.getAllocationID());
+				}
+			}
+
+			// Try to allocate more resources which must result in an error
+			try {
+				cm.requestInstance(jobID, conf, 3);
+
+				fail("ClusterManager allowed to request more instances than actually available");
+
+			} catch (InstanceException ie) {
+				// Exception is expected and correct behavior here
+			}
+
+			// Release all allocated resources
+			it = allocatedResources.iterator();
+			while (it.hasNext()) {
+				final AllocatedResource allocatedResource = it.next();
+				cm.releaseAllocatedResource(allocatedResource);
+			}
+			
+			// Now further allocations should be possible
+			
+			cm.requestInstance(jobID, conf, 1);
+			
+			
+			cm.shutdown();
+		}
+		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 {
+			
+			final int CLEANUP_INTERVAL = 2;
+
+			// configure a short cleanup interval
+			Configuration config = new Configuration();
+			config.setInteger("instancemanager.cluster.cleanupinterval", CLEANUP_INTERVAL);
+			GlobalConfiguration.includeConfiguration(config);
+
+			DefaultInstanceManager cm = new DefaultInstanceManager();
+			TestInstanceListener testInstanceListener = new TestInstanceListener();
+			cm.setInstanceListener(testInstanceListener);
+
+
+			int ipcPort = ConfigConstants.DEFAULT_TASK_MANAGER_IPC_PORT;
+			int dataPort = ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT;
+
+			HardwareDescription hardwareDescription = HardwareDescriptionFactory.construct(2, 2L * 1024L * 1024L * 1024L,
+																				2L * 1024L * 1024L * 1024L);
+
+			String hostname = "192.168.198.1";
+			InetAddress address = InetAddress.getByName("192.168.198.1");
+
+			InstanceConnectionInfo ici1 = new InstanceConnectionInfo(address, hostname, null, ipcPort + 0, dataPort + 0);
+			InstanceConnectionInfo ici2 = new InstanceConnectionInfo(address, hostname, null, ipcPort + 15, dataPort + 15);
+			InstanceConnectionInfo ici3 = new InstanceConnectionInfo(address, hostname, null, ipcPort + 30, dataPort + 30);
+
+			// register three instances
+			cm.registerTaskManager(ici1, hardwareDescription, 1);
+			cm.registerTaskManager(ici2, hardwareDescription, 1);
+			cm.registerTaskManager(ici3, hardwareDescription, 1);
+
+			assertEquals(3, cm.getNumberOfSlots());
+
+			// request some instances
+			JobID jobID = new JobID();
+			Configuration conf = new Configuration();
+
+			cm.requestInstance(jobID, conf, 1);
+
+			DefaultInstanceManagerTestUtils.waitForInstances(jobID, testInstanceListener, 1, 1000);
+			assertEquals(1, testInstanceListener.getNumberOfAllocatedResourcesForJob(jobID));
+
+			// wait for the cleanup to kick in
+			Thread.sleep(2000 * CLEANUP_INTERVAL);
+
+			// check that the instances are gone
+			DefaultInstanceManagerTestUtils.waitForInstances(jobID, testInstanceListener, 0, 1000);
+			assertEquals(0, testInstanceListener.getNumberOfAllocatedResourcesForJob(jobID));
+
+
+			assertEquals(0, cm.getNumberOfSlots());
+
+			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/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/cluster/DefaultInstanceManagerTestUtils.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/cluster/DefaultInstanceManagerTestUtils.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/cluster/DefaultInstanceManagerTestUtils.java
new file mode 100644
index 0000000..ca3d971
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/cluster/DefaultInstanceManagerTestUtils.java
@@ -0,0 +1,66 @@
+/***********************************************************************************************************************
+ * 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.instance.cluster;
+
+import eu.stratosphere.nephele.instance.InstanceListener;
+import eu.stratosphere.nephele.jobgraph.JobID;
+
+/**
+ * This class contains utility methods used during the tests of the {@link eu.stratosphere.nephele.instance.DefaultInstanceManager} implementation.
+ * 
+ */
+public class DefaultInstanceManagerTestUtils {
+
+	/**
+	 * Granularity of the sleep time.
+	 */
+	private static final long SLEEP_TIME = 10; // 10 milliseconds
+
+	/**
+	 * Private constructor so the class cannot be instantiated.
+	 */
+	private DefaultInstanceManagerTestUtils() {
+	}
+
+	/**
+	 * Waits until a specific number of instances have registered or deregistrations with the given
+	 * {@link InstanceListener} object for a given job or the maximum wait time has elapsed.
+	 * 
+	 * @param jobID
+	 *        the ID of the job to check the instance registration for
+	 * @param instanceListener
+	 *        the listener which shall be notified when a requested instance is available for the job
+	 * @param numberOfInstances
+	 *        the number of registered instances to wait for
+	 * @param maxWaitTime
+	 *        the maximum wait time before this method returns
+	 */
+	public static void waitForInstances(JobID jobID, TestInstanceListener instanceListener,
+			int numberOfInstances, long maxWaitTime) {
+
+		final long startTime = System.currentTimeMillis();
+
+		while (instanceListener.getNumberOfAllocatedResourcesForJob(jobID) != numberOfInstances) {
+			try {
+				Thread.sleep(SLEEP_TIME);
+			} catch (InterruptedException e) {
+				break;
+			}
+
+			if ((System.currentTimeMillis() - startTime) >= maxWaitTime) {
+				break;
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/cluster/HostInClusterTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/cluster/HostInClusterTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/cluster/HostInClusterTest.java
index 952e588..1bac907 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/cluster/HostInClusterTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/cluster/HostInClusterTest.java
@@ -16,30 +16,22 @@ package eu.stratosphere.nephele.instance.cluster;
 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 java.net.UnknownHostException;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
+import java.util.*;
 
+import eu.stratosphere.nephele.instance.*;
 import org.junit.Test;
 
 import eu.stratosphere.configuration.ConfigConstants;
-import eu.stratosphere.nephele.instance.HardwareDescription;
-import eu.stratosphere.nephele.instance.HardwareDescriptionFactory;
-import eu.stratosphere.nephele.instance.InstanceConnectionInfo;
-import eu.stratosphere.nephele.instance.InstanceType;
-import eu.stratosphere.nephele.instance.InstanceTypeFactory;
 import eu.stratosphere.nephele.jobgraph.JobID;
 import eu.stratosphere.nephele.topology.NetworkTopology;
 
 /**
- * Tests for {@link ClusterInstance}.
+ * Tests for {@link eu.stratosphere.nephele.instance.Instance}.
  * 
  */
 public class HostInClusterTest {
@@ -49,7 +41,7 @@ public class HostInClusterTest {
 	 * 
 	 * @return a cluster instance of a special test type
 	 */
-	private ClusterInstance createTestClusterInstance() {
+	private Instance createTestClusterInstance() {
 
 		final int ipcPort = ConfigConstants.DEFAULT_TASK_MANAGER_IPC_PORT;
 		final int dataPort = ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT;
@@ -60,16 +52,8 @@ public class HostInClusterTest {
 			fail(e.getMessage());
 		}
 
-		final String identifier = "testtype";
-		final int numComputeUnits = 8;
 		final int numCores = 8;
 		final int memorySize = 32 * 1024;
-		final int diskCapacity = 200;
-		final int pricePerHour = 10;
-
-		final InstanceType capacity = InstanceTypeFactory.construct(identifier, numComputeUnits, numCores, memorySize,
-			diskCapacity,
-			pricePerHour);
 
 		final InstanceConnectionInfo instanceConnectionInfo = new InstanceConnectionInfo(inetAddress, ipcPort, dataPort);
 
@@ -77,8 +61,8 @@ public class HostInClusterTest {
 			memorySize * 1024L * 1024L, memorySize * 1024L * 1024L);
 
 		final NetworkTopology topology = NetworkTopology.createEmptyTopology();
-		ClusterInstance host = new ClusterInstance(instanceConnectionInfo, capacity, topology.getRootNode(), topology,
-			hardwareDescription);
+		Instance host = new Instance(instanceConnectionInfo, topology.getRootNode(), topology,
+			hardwareDescription, 8);
 
 		return host;
 	}
@@ -90,7 +74,7 @@ public class HostInClusterTest {
 	public void testHeartBeat() {
 		// check that heart beat is triggered correctly.
 
-		ClusterInstance host = createTestClusterInstance();
+		Instance host = createTestClusterInstance();
 
 		host.reportHeartBeat();
 
@@ -111,33 +95,33 @@ public class HostInClusterTest {
 	@Test
 	public void testAccounting() {
 		// check whether the accounting of capacity works correctly
-		final ClusterInstance host = createTestClusterInstance();
+		final Instance host = createTestClusterInstance();
 		final JobID jobID = new JobID();
-		final int numComputeUnits = 8 / 8;
-		final int numCores = 8 / 8;
-		final int memorySize = 32 * 1024 / 8;
-		final int diskCapacity = 200 / 8;
-		final InstanceType type = InstanceTypeFactory.construct("dummy", numComputeUnits, numCores, memorySize,
-			diskCapacity, -1);
 		for (int run = 0; run < 2; ++run) {
 			// do this twice to check that everything is correctly freed
-			AllocatedSlice[] slices = new AllocatedSlice[8];
+			AllocatedResource[] allocatedSlots = new AllocatedResource[8];
 			for (int i = 0; i < 8; ++i) {
-				slices[i] = host.createSlice(type, jobID);
-				assertNotNull(slices[i]);
-				assertEquals(numComputeUnits, slices[i].getType().getNumberOfComputeUnits());
-				assertEquals(numCores, slices[i].getType().getNumberOfCores());
-				assertEquals(memorySize, slices[i].getType().getMemorySize());
-				assertEquals(diskCapacity, slices[i].getType().getDiskCapacity());
+				try {
+					allocatedSlots[i] = host.allocateSlot(jobID);
+				}catch(InstanceException ex){
+					fail(ex.getMessage());
+				}
+
+				assertNotNull(allocatedSlots[i]);
 			}
 			// now no resources should be left
-			assertNull(host.createSlice(InstanceTypeFactory.construct("dummy", 1, 0, 0, 0, 0), jobID));
-			assertNull(host.createSlice(InstanceTypeFactory.construct("dummy", 0, 1, 0, 0, 0), jobID));
-			assertNull(host.createSlice(InstanceTypeFactory.construct("dummy", 0, 0, 1, 0, 0), jobID));
-			assertNull(host.createSlice(InstanceTypeFactory.construct("dummy", 0, 0, 0, 1, 0), jobID));
+			boolean instanceException = false;
+
+			try{
+				host.allocateSlot(jobID);
+			}catch(InstanceException ex){
+				instanceException = true;
+			}
+
+			assertTrue(instanceException);
 
 			for (int i = 0; i < 8; ++i) {
-				host.removeAllocatedSlice(slices[i].getAllocationID());
+				host.releaseSlot(allocatedSlots[i].getAllocationID());
 			}
 		}
 	}
@@ -149,47 +133,51 @@ public class HostInClusterTest {
 	public void testTermination() {
 		
 		// check whether the accounting of capacity works correctly if terminateAllInstances is called
-		final ClusterInstance host = createTestClusterInstance();
+		final Instance host = createTestClusterInstance();
 		final JobID jobID = new JobID();
-		final int numComputeUnits = 8 / 8;
-		final int numCores = 8 / 8;
-		final int memorySize = 32 * 1024 / 8;
-		final int diskCapacity = 200 / 8;
-		final InstanceType type = InstanceTypeFactory.construct("dummy", numComputeUnits, numCores, memorySize,
-			diskCapacity, -1);
 		for (int run = 0; run < 2; ++run) {
 			// do this twice to check that everything is correctly freed
-			AllocatedSlice[] slices = new AllocatedSlice[8];
+			AllocatedResource[] allocatedResources = new AllocatedResource[8];
 			for (int i = 0; i < 8; ++i) {
-				slices[i] = host.createSlice(type, jobID);
-				assertNotNull(slices[i]);
-				assertEquals(numComputeUnits, slices[i].getType().getNumberOfComputeUnits());
-				assertEquals(numCores, slices[i].getType().getNumberOfCores());
-				assertEquals(memorySize, slices[i].getType().getMemorySize());
-				assertEquals(diskCapacity, slices[i].getType().getDiskCapacity());
+				try {
+					allocatedResources[i] = host.allocateSlot(jobID);
+				}catch (InstanceException ex){
+					fail(ex.getMessage());
+				}
+
+				assertNotNull(allocatedResources[i]);
 			}
+
+			boolean instanceException = false;
 			// now no resources should be left
-			assertNull(host.createSlice(InstanceTypeFactory.construct("dummy", 1, 0, 0, 0, 0), jobID));
-			assertNull(host.createSlice(InstanceTypeFactory.construct("dummy", 0, 1, 0, 0, 0), jobID));
-			assertNull(host.createSlice(InstanceTypeFactory.construct("dummy", 0, 0, 1, 0, 0), jobID));
-			assertNull(host.createSlice(InstanceTypeFactory.construct("dummy", 0, 0, 0, 1, 0), jobID));
-			List<AllocatedSlice> removedSlices = host.removeAllAllocatedSlices();
-
-			final Set<AllocatedSlice> slicesSet = new HashSet<AllocatedSlice>();
-			for(int i = 0; i < slices.length; ++i) {
-				slicesSet.add(slices[i]);
+			try {
+				host.allocateSlot(jobID);
+			} catch (InstanceException ex){
+				instanceException = true;
+			}
+
+			assertTrue(instanceException);
+			Collection<AllocatedSlot> allocatedSlots = host.removeAllocatedSlots();
+			Set<AllocationID> removedAllocationIDs = new HashSet<AllocationID>();
+
+			for(AllocatedSlot slot: allocatedSlots){
+				removedAllocationIDs.add(slot.getAllocationID());
+			}
+
+			final Set<AllocationID> allocationIDs = new HashSet<AllocationID>();
+			for(int i = 0; i < allocatedResources.length; ++i) {
+				allocationIDs.add(allocatedResources[i].getAllocationID());
 			}
 			
-			final Set<AllocatedSlice> removedSlicesSet = new HashSet<AllocatedSlice>(removedSlices);
-			
+
 			//Check if both sets are equal
-			assertEquals(slicesSet.size(), removedSlices.size());
-			final Iterator<AllocatedSlice> it = slicesSet.iterator();
+			assertEquals(allocationIDs.size(), removedAllocationIDs.size());
+			final Iterator<AllocationID> it = allocationIDs.iterator();
 			while(it.hasNext()) {
-				assertTrue(removedSlicesSet.remove(it.next()));
+				assertTrue(removedAllocationIDs.remove(it.next()));
 			}
 			
-			assertEquals(0, removedSlicesSet.size());
+			assertEquals(0, removedAllocationIDs.size());
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/cluster/PendingRequestsMapTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/cluster/PendingRequestsMapTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/cluster/PendingRequestsMapTest.java
deleted file mode 100644
index 974283d..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/cluster/PendingRequestsMapTest.java
+++ /dev/null
@@ -1,91 +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.instance.cluster;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.Iterator;
-import java.util.Map;
-
-import org.junit.Test;
-
-import eu.stratosphere.nephele.instance.InstanceType;
-import eu.stratosphere.nephele.instance.InstanceTypeFactory;
-
-/**
- * This class checks the {@link PendingRequestsMap} data structure.
- * 
- */
-public class PendingRequestsMapTest {
-
-	/**
-	 * The first instance type used in the tests.
-	 */
-	private static final InstanceType INSTANCE_TYPE1 = InstanceTypeFactory.construct("test1", 1, 1, 2, 2, 0);
-
-	/**
-	 * The second instance type used in the tests.
-	 */
-	private static final InstanceType INSTANCE_TYPE2 = InstanceTypeFactory.construct("test2", 2, 2, 4, 4, 0);
-
-	/**
-	 * Checks the correctness of the {@link PendingRequestsMap} data structure.
-	 */
-	@Test
-	public void testPendingRequestsMap() {
-
-		final PendingRequestsMap prm = new PendingRequestsMap();
-
-		assertFalse(prm.hasPendingRequests());
-
-		prm.addRequest(INSTANCE_TYPE1, 1);
-		prm.addRequest(INSTANCE_TYPE2, 2);
-		prm.addRequest(INSTANCE_TYPE2, 2);
-
-		assertTrue(prm.hasPendingRequests());
-
-		final Iterator<Map.Entry<InstanceType, Integer>> it = prm.iterator();
-		int iterationCounter = 0;
-		while (it.hasNext()) {
-
-			final Map.Entry<InstanceType, Integer> entry = it.next();
-			++iterationCounter;
-
-			if (entry.getKey().equals(INSTANCE_TYPE1)) {
-				assertEquals(1, entry.getValue().intValue());
-			}
-
-			if (entry.getKey().equals(INSTANCE_TYPE2)) {
-				assertEquals(4, entry.getValue().intValue());
-			}
-		}
-
-		assertEquals(2, iterationCounter);
-
-		prm.decreaseNumberOfPendingInstances(INSTANCE_TYPE1);
-		prm.decreaseNumberOfPendingInstances(INSTANCE_TYPE1);
-		prm.decreaseNumberOfPendingInstances(INSTANCE_TYPE1); // This call is actually superfluous
-
-		assertTrue(prm.hasPendingRequests());
-
-		prm.decreaseNumberOfPendingInstances(INSTANCE_TYPE2);
-		prm.decreaseNumberOfPendingInstances(INSTANCE_TYPE2);
-		prm.decreaseNumberOfPendingInstances(INSTANCE_TYPE2);
-		prm.decreaseNumberOfPendingInstances(INSTANCE_TYPE2);
-
-		assertFalse(prm.hasPendingRequests());
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/local/LocalInstanceManagerTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/local/LocalInstanceManagerTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/local/LocalInstanceManagerTest.java
index 92ea5ab..a8f1331 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/local/LocalInstanceManagerTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/local/LocalInstanceManagerTest.java
@@ -13,16 +13,15 @@
 
 package eu.stratosphere.nephele.instance.local;
 
-import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
-import eu.stratosphere.nephele.ExecutionMode;
+import eu.stratosphere.nephele.instance.InstanceManager;
 import junit.framework.Assert;
 
 import org.junit.Test;
 
+import eu.stratosphere.nephele.ExecutionMode;
 import eu.stratosphere.configuration.GlobalConfiguration;
-import eu.stratosphere.nephele.instance.InstanceType;
 import eu.stratosphere.nephele.jobmanager.JobManager;
 import eu.stratosphere.nephele.util.ServerTestUtils;
 
@@ -53,17 +52,9 @@ public class LocalInstanceManagerTest {
 
 			final TestInstanceListener testInstanceListener = new TestInstanceListener();
 	
-			LocalInstanceManager lm = (LocalInstanceManager) jm.getInstanceManager(); // this is for sure, because I chose the local strategy
+			InstanceManager im = jm.getInstanceManager();
 			try {
-				lm.setInstanceListener(testInstanceListener);
-	
-				final InstanceType defaultInstanceType = lm.getDefaultInstanceType();
-				assertEquals("test", defaultInstanceType.getIdentifier());
-				assertEquals(4, defaultInstanceType.getNumberOfComputeUnits());
-				assertEquals(4, defaultInstanceType.getNumberOfCores());
-				assertEquals(1024, defaultInstanceType.getMemorySize());
-				assertEquals(160, defaultInstanceType.getDiskCapacity());
-				assertEquals(0, defaultInstanceType.getPricePerHour());
+				im.setInstanceListener(testInstanceListener);
 	
 			} catch (Exception e) {
 				e.printStackTrace();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java
index 063b827..fa4fbfa 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java
@@ -34,11 +34,8 @@ import eu.stratosphere.nephele.util.FileLineWriter;
 import eu.stratosphere.nephele.util.JarFileCreator;
 import eu.stratosphere.nephele.util.ServerTestUtils;
 import eu.stratosphere.util.LogUtils;
-import eu.stratosphere.util.StringUtils;
-import org.apache.log4j.ConsoleAppender;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
-import org.apache.log4j.PatternLayout;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -166,19 +163,23 @@ public class JobManagerITCase {
 			final JobFileInputVertex i1 = new JobFileInputVertex("Input 1", jg);
 			i1.setFileInputClass(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.setTaskClass(ForwardTask.class);
+			t1.setNumberOfSubtasks(1);
 
 			// task vertex 2
 			final JobTaskVertex t2 = new JobTaskVertex("Task 2", jg);
 			t2.setTaskClass(ForwardTask.class);
+			t2.setNumberOfSubtasks(1);
 
 			// output vertex
 			JobFileOutputVertex o1 = new JobFileOutputVertex("Output 1", jg);
 			o1.setFileOutputClass(FileLineWriter.class);
 			o1.setFilePath(new Path(outputFile.toURI()));
+			o1.setNumberOfSubtasks(1);
 
 			t1.setVertexToShareInstancesWith(i1);
 			t2.setVertexToShareInstancesWith(i1);
@@ -473,19 +474,23 @@ public class JobManagerITCase {
 			final JobFileInputVertex i1 = new JobFileInputVertex("Input 1", jg);
 			i1.setFileInputClass(FileLineReader.class);
 			i1.setFilePath(new Path(inputFile.toURI()));
+			i1.setNumberOfSubtasks(1);
 
 			// task vertex 1
 			final JobTaskVertex t1 = new JobTaskVertex("Task 1", jg);
 			t1.setTaskClass(ForwardTask.class);
+			t1.setNumberOfSubtasks(1);
 
 			// task vertex 2
 			final JobTaskVertex t2 = new JobTaskVertex("Task 2", jg);
 			t2.setTaskClass(ForwardTask.class);
+			t2.setNumberOfSubtasks(1);
 
 			// output vertex
 			JobFileOutputVertex o1 = new JobFileOutputVertex("Output 1", jg);
 			o1.setFileOutputClass(FileLineWriter.class);
 			o1.setFilePath(new Path(outputFile.toURI()));
+			o1.setNumberOfSubtasks(1);
 
 			t1.setVertexToShareInstancesWith(i1);
 			t2.setVertexToShareInstancesWith(i1);
@@ -747,6 +752,7 @@ public class JobManagerITCase {
 			JobFileOutputVertex o1 = new JobFileOutputVertex("Output", jg);
 			o1.setFileOutputClass(FileLineWriter.class);
 			o1.setFilePath(new Path(outputFile.toURI()));
+			o1.setNumberOfSubtasks(1);
 
 			i1.setVertexToShareInstancesWith(o1);
 			i2.setVertexToShareInstancesWith(o1);
@@ -877,27 +883,23 @@ public class JobManagerITCase {
 			i1.setFileInputClass(FileLineReader.class);
 			i1.setFilePath(new Path(inputFile1.toURI()));
 			i1.setNumberOfSubtasks(numberOfSubtasks);
-			i1.setNumberOfSubtasksPerInstance(numberOfSubtasks);
 
 			// input vertex 2
 			final JobFileInputVertex i2 = new JobFileInputVertex("Input 2", jg);
 			i2.setFileInputClass(FileLineReader.class);
 			i2.setFilePath(new Path(inputFile2.toURI()));
 			i2.setNumberOfSubtasks(numberOfSubtasks);
-			i2.setNumberOfSubtasksPerInstance(numberOfSubtasks);
 
 			// union task
 			final JobTaskVertex f1 = new JobTaskVertex("Forward 1", jg);
 			f1.setTaskClass(DoubleTargetTask.class);
 			f1.setNumberOfSubtasks(numberOfSubtasks);
-			f1.setNumberOfSubtasksPerInstance(numberOfSubtasks);
 
 			// output vertex
 			JobFileOutputVertex o1 = new JobFileOutputVertex("Output", jg);
 			o1.setFileOutputClass(FileLineWriter.class);
 			o1.setFilePath(new Path(outputFile.toURI()));
 			o1.setNumberOfSubtasks(numberOfSubtasks);
-			o1.setNumberOfSubtasksPerInstance(numberOfSubtasks);
 
 			i1.setVertexToShareInstancesWith(o1);
 			i2.setVertexToShareInstancesWith(o1);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/DefaultSchedulerTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/DefaultSchedulerTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/DefaultSchedulerTest.java
new file mode 100644
index 0000000..c8bcddc
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/DefaultSchedulerTest.java
@@ -0,0 +1,185 @@
+/***********************************************************************************************************************
+ * 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.jobmanager.scheduler.queue;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.List;
+
+import eu.stratosphere.nephele.jobmanager.scheduler.DefaultScheduler;
+
+import org.junit.Test;
+
+import eu.stratosphere.core.io.StringRecord;
+import eu.stratosphere.nephele.execution.ExecutionState;
+import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
+import eu.stratosphere.nephele.executiongraph.ExecutionGraph;
+import eu.stratosphere.nephele.executiongraph.ExecutionVertex;
+import eu.stratosphere.nephele.executiongraph.GraphConversionException;
+import eu.stratosphere.nephele.jobgraph.JobGraph;
+import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException;
+import eu.stratosphere.nephele.jobgraph.JobInputVertex;
+import eu.stratosphere.nephele.jobgraph.JobOutputVertex;
+import eu.stratosphere.nephele.jobmanager.scheduler.SchedulingException;
+import eu.stratosphere.nephele.template.AbstractGenericInputTask;
+import eu.stratosphere.nephele.template.AbstractOutputTask;
+import eu.stratosphere.runtime.io.api.RecordReader;
+import eu.stratosphere.runtime.io.api.RecordWriter;
+import eu.stratosphere.runtime.io.channels.ChannelType;
+import eu.stratosphere.util.StringUtils;
+
+/**
+ *         This class checks the functionality of the {@link eu.stratosphere.nephele.jobmanager.scheduler.DefaultScheduler} class
+ */
+public class DefaultSchedulerTest {
+
+	/**
+	 * Test input task.
+	 * 
+	 */
+	public static final class InputTask extends AbstractGenericInputTask {
+
+		/**
+		 * {@inheritDoc}
+		 */
+		@Override
+		public void registerInputOutput() {
+			new RecordWriter<StringRecord>(this);
+		}
+
+		/**
+		 * {@inheritDoc}
+		 */
+		@Override
+		public void invoke() throws Exception {
+			// Nothing to do here
+		}
+
+	}
+
+	/**
+	 * Test output task.
+	 * 
+	 */
+	public static final class OutputTask extends AbstractOutputTask {
+
+		/**
+		 * {@inheritDoc}
+		 */
+		@Override
+		public void registerInputOutput() {
+			new RecordReader<StringRecord>(this, StringRecord.class);
+		}
+
+		/**
+		 * {@inheritDoc}
+		 */
+		@Override
+		public void invoke() throws Exception {
+			// Nothing to do here
+		}
+
+	}
+
+	/**
+	 * 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.setInputClass(InputTask.class);
+		inputVertex.setNumberOfSubtasks(1);
+
+		final JobOutputVertex outputVertex = new JobOutputVertex("Output 1", jobGraph);
+		outputVertex.setOutputClass(OutputTask.class);
+		outputVertex.setNumberOfSubtasks(1);
+
+		try {
+			inputVertex.connectTo(outputVertex, channelType);
+		} catch (JobGraphDefinitionException e) {
+			fail(StringUtils.stringifyException(e));
+		}
+
+		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));
+		}
+
+		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);
+
+		try {
+			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);
+			}
+
+			assertEquals(1, tim.getNumberOfReleaseMethodCalls());
+		} finally {
+			try {
+				LibraryCacheManager.unregister(executionGraph.getJobID());
+			} catch (IOException ioe) {
+				// Ignore exception here
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueSchedulerTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueSchedulerTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueSchedulerTest.java
deleted file mode 100644
index f1e3191..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueSchedulerTest.java
+++ /dev/null
@@ -1,186 +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.jobmanager.scheduler.queue;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.util.List;
-
-import eu.stratosphere.runtime.io.api.RecordWriter;
-import org.junit.Test;
-
-import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.execution.ExecutionState;
-import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
-import eu.stratosphere.nephele.executiongraph.ExecutionGraph;
-import eu.stratosphere.nephele.executiongraph.ExecutionVertex;
-import eu.stratosphere.nephele.executiongraph.GraphConversionException;
-import eu.stratosphere.nephele.instance.InstanceManager;
-import eu.stratosphere.runtime.io.api.RecordReader;
-import eu.stratosphere.runtime.io.channels.ChannelType;
-import eu.stratosphere.nephele.jobgraph.JobGraph;
-import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException;
-import eu.stratosphere.nephele.jobgraph.JobInputVertex;
-import eu.stratosphere.nephele.jobgraph.JobOutputVertex;
-import eu.stratosphere.nephele.jobmanager.scheduler.SchedulingException;
-import eu.stratosphere.nephele.template.AbstractGenericInputTask;
-import eu.stratosphere.nephele.template.AbstractOutputTask;
-import eu.stratosphere.util.StringUtils;
-
-/**
- *         This class checks the functionality of the {@link QueueScheduler} class
- */
-public class QueueSchedulerTest {
-
-	/**
-	 * Test input task.
-	 * 
-	 */
-	public static final class InputTask extends AbstractGenericInputTask {
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public void registerInputOutput() {
-			new RecordWriter<StringRecord>(this);
-		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public void invoke() throws Exception {
-			// Nothing to do here
-		}
-
-	}
-
-	/**
-	 * Test output task.
-	 * 
-	 */
-	public static final class OutputTask extends AbstractOutputTask {
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public void registerInputOutput() {
-			new RecordReader<StringRecord>(this, StringRecord.class);
-		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public void invoke() throws Exception {
-			// Nothing to do here
-		}
-
-	}
-
-	/**
-	 * 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
-	 * @param instanceManager
-	 *        the instance manager that shall be used during the creation of the execution graph
-	 * @return a sample execution graph
-	 */
-	private ExecutionGraph createExecutionGraph(final ChannelType channelType, final InstanceManager instanceManager) {
-
-		final JobGraph jobGraph = new JobGraph("Job Graph");
-
-		final JobInputVertex inputVertex = new JobInputVertex("Input 1", jobGraph);
-		inputVertex.setInputClass(InputTask.class);
-		inputVertex.setNumberOfSubtasks(1);
-
-		final JobOutputVertex outputVertex = new JobOutputVertex("Output 1", jobGraph);
-		outputVertex.setOutputClass(OutputTask.class);
-		outputVertex.setNumberOfSubtasks(1);
-
-		try {
-			inputVertex.connectTo(outputVertex, channelType);
-		} catch (JobGraphDefinitionException e) {
-			fail(StringUtils.stringifyException(e));
-		}
-
-		try {
-			LibraryCacheManager.register(jobGraph.getJobID(), new String[0]);
-			return new ExecutionGraph(jobGraph, instanceManager);
-
-		} catch (GraphConversionException e) {
-			fail(StringUtils.stringifyException(e));
-		} catch (IOException e) {
-			fail(StringUtils.stringifyException(e));
-		}
-
-		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 testSchedulJobWithInMemoryChannel() {
-
-		final TestInstanceManager tim = new TestInstanceManager();
-		final TestDeploymentManager tdm = new TestDeploymentManager();
-		final QueueScheduler scheduler = new QueueScheduler(tdm, tim);
-
-		final ExecutionGraph executionGraph = createExecutionGraph(ChannelType.IN_MEMORY, tim);
-
-		try {
-			try {
-				scheduler.schedulJob(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);
-			}
-
-			assertEquals(1, tim.getNumberOfReleaseMethodCalls());
-		} finally {
-			try {
-				LibraryCacheManager.unregister(executionGraph.getJobID());
-			} catch (IOException ioe) {
-				// Ignore exception here
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/TestDeploymentManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/TestDeploymentManager.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/TestDeploymentManager.java
index 9f3c190..a118455 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/TestDeploymentManager.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/TestDeploymentManager.java
@@ -16,7 +16,7 @@ package eu.stratosphere.nephele.jobmanager.scheduler.queue;
 import java.util.List;
 
 import eu.stratosphere.nephele.executiongraph.ExecutionVertex;
-import eu.stratosphere.nephele.instance.AbstractInstance;
+import eu.stratosphere.nephele.instance.Instance;
 import eu.stratosphere.nephele.jobgraph.JobID;
 import eu.stratosphere.nephele.jobmanager.DeploymentManager;
 
@@ -46,7 +46,7 @@ public class TestDeploymentManager implements DeploymentManager {
 
 
 	@Override
-	public void deploy(final JobID jobID, final AbstractInstance instance,
+	public void deploy(final JobID jobID, final Instance instance,
 			final List<ExecutionVertex> verticesToBeDeployed) {
 
 		this.jobID = jobID;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/TestInstanceManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/TestInstanceManager.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/TestInstanceManager.java
index 955d3a0..5a3977a 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/TestInstanceManager.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/TestInstanceManager.java
@@ -16,32 +16,18 @@ package eu.stratosphere.nephele.jobmanager.scheduler.queue;
 import java.net.Inet4Address;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 
 import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.nephele.instance.AbstractInstance;
-import eu.stratosphere.nephele.instance.AllocatedResource;
-import eu.stratosphere.nephele.instance.AllocationID;
-import eu.stratosphere.nephele.instance.HardwareDescription;
-import eu.stratosphere.nephele.instance.HardwareDescriptionFactory;
-import eu.stratosphere.nephele.instance.InstanceConnectionInfo;
-import eu.stratosphere.nephele.instance.InstanceException;
-import eu.stratosphere.nephele.instance.InstanceListener;
-import eu.stratosphere.nephele.instance.InstanceManager;
-import eu.stratosphere.nephele.instance.InstanceRequestMap;
-import eu.stratosphere.nephele.instance.InstanceType;
-import eu.stratosphere.nephele.instance.InstanceTypeDescription;
-import eu.stratosphere.nephele.instance.InstanceTypeDescriptionFactory;
-import eu.stratosphere.nephele.instance.InstanceTypeFactory;
+import eu.stratosphere.nephele.instance.*;
+import eu.stratosphere.nephele.instance.Instance;
 import eu.stratosphere.nephele.jobgraph.JobID;
 import eu.stratosphere.nephele.topology.NetworkNode;
 import eu.stratosphere.nephele.topology.NetworkTopology;
 import eu.stratosphere.util.StringUtils;
 
 /**
- * A dummy implementation of an {@link InstanceManager} used for the {@link QueueScheduler} unit tests.
+ * A dummy implementation of an {@link eu.stratosphere.nephele.instance.InstanceManager} used for the {@link QueueScheduler} unit tests.
  * <p>
  * This class is thread-safe.
  * 
@@ -49,16 +35,6 @@ import eu.stratosphere.util.StringUtils;
 public final class TestInstanceManager implements InstanceManager {
 
 	/**
-	 * The default instance type to be used during the tests.
-	 */
-	private static final InstanceType INSTANCE_TYPE = InstanceTypeFactory.construct("test", 1, 1, 1024, 1024, 10);
-
-	/**
-	 * The instances this instance manager is responsible of.
-	 */
-	private final Map<InstanceType, InstanceTypeDescription> instanceMap = new HashMap<InstanceType, InstanceTypeDescription>();
-
-	/**
 	 * Counts the number of times the method releaseAllocatedResource is called.
 	 */
 	private volatile int numberOfReleaseCalls = 0;
@@ -74,16 +50,19 @@ public final class TestInstanceManager implements InstanceManager {
 	private final List<AllocatedResource> allocatedResources;
 
 	/**
-	 * Test implementation of {@link AbstractInstance}.
+	 * The test instance
+	 */
+	private final TestInstance testInstance;
+
+	/**
+	 * Test implementation of {@link eu.stratosphere.nephele.instance.Instance}.
 	 * 
 	 */
-	private static final class TestInstance extends AbstractInstance {
+	private static final class TestInstance extends Instance {
 
 		/**
 		 * Constructs a new test instance.
 		 * 
-		 * @param instanceType
-		 *        the instance type
 		 * @param instanceConnectionInfo
 		 *        the instance connection information
 		 * @param parentNode
@@ -92,11 +71,13 @@ public final class TestInstanceManager implements InstanceManager {
 		 *        the network topology
 		 * @param hardwareDescription
 		 *        the hardware description
+		 * @param numberSlots
+		 * 		  the number of slots available on the instance
 		 */
-		public TestInstance(final InstanceType instanceType, final InstanceConnectionInfo instanceConnectionInfo,
+		public TestInstance(final InstanceConnectionInfo instanceConnectionInfo,
 				final NetworkNode parentNode, final NetworkTopology networkTopology,
-				final HardwareDescription hardwareDescription) {
-			super(instanceType, instanceConnectionInfo, parentNode, networkTopology, hardwareDescription);
+				final HardwareDescription hardwareDescription, int numberSlots) {
+			super(instanceConnectionInfo, parentNode, networkTopology, hardwareDescription, numberSlots);
 		}
 	}
 
@@ -106,15 +87,13 @@ public final class TestInstanceManager implements InstanceManager {
 	public TestInstanceManager() {
 
 		final HardwareDescription hd = HardwareDescriptionFactory.construct(1, 1L, 1L);
-		final InstanceTypeDescription itd = InstanceTypeDescriptionFactory.construct(INSTANCE_TYPE, hd, 1);
-		instanceMap.put(INSTANCE_TYPE, itd);
 
 		this.allocatedResources = new ArrayList<AllocatedResource>();
 		try {
 			final InstanceConnectionInfo ici = new InstanceConnectionInfo(Inet4Address.getLocalHost(), 1, 1);
 			final NetworkTopology nt = new NetworkTopology();
-			final TestInstance ti = new TestInstance(INSTANCE_TYPE, ici, nt.getRootNode(), nt, hd);
-			this.allocatedResources.add(new AllocatedResource(ti, INSTANCE_TYPE, new AllocationID()));
+			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));
 		}
@@ -123,18 +102,7 @@ public final class TestInstanceManager implements InstanceManager {
 
 	@Override
 	public void requestInstance(final JobID jobID, final Configuration conf,
-			final InstanceRequestMap instanceRequestMap, final List<String> splitAffinityList) throws InstanceException {
-
-		if (instanceRequestMap.size() != 1) {
-			throw new InstanceException(
-				"requestInstance of TestInstanceManager expected to receive request for a single instance type");
-		}
-
-		if (instanceRequestMap.getMinimumNumberOfInstances(INSTANCE_TYPE) != 1) {
-			throw new InstanceException(
-				"requestInstance of TestInstanceManager expected to receive request for one instance of type "
-					+ INSTANCE_TYPE.getIdentifier());
-		}
+								int requiredSlots) throws InstanceException {
 
 		if (this.instanceListener == null) {
 			throw new InstanceException("instanceListener not registered with TestInstanceManager");
@@ -158,8 +126,7 @@ public final class TestInstanceManager implements InstanceManager {
 
 
 	@Override
-	public void releaseAllocatedResource(final JobID jobID, final Configuration conf,
-			final AllocatedResource allocatedResource) throws InstanceException {
+	public void releaseAllocatedResource(final AllocatedResource allocatedResource) throws InstanceException {
 
 		++this.numberOfReleaseCalls;
 	}
@@ -176,32 +143,16 @@ public final class TestInstanceManager implements InstanceManager {
 
 
 	@Override
-	public InstanceType getSuitableInstanceType(final int minNumComputeUnits, final int minNumCPUCores,
-			final int minMemorySize, final int minDiskCapacity, final int maxPricePerHour) {
-		throw new IllegalStateException("getSuitableInstanceType called on TestInstanceManager");
-	}
-
-
-	@Override
-	public void reportHeartBeat(final InstanceConnectionInfo instanceConnectionInfo,
-			final HardwareDescription hardwareDescription) {
+	public void reportHeartBeat(final InstanceConnectionInfo instanceConnectionInfo) {
 		throw new IllegalStateException("reportHeartBeat called on TestInstanceManager");
 	}
 
-
 	@Override
-	public InstanceType getInstanceTypeByName(final String instanceTypeName) {
-		throw new IllegalStateException("getInstanceTypeByName called on TestInstanceManager");
+	public void registerTaskManager(final InstanceConnectionInfo instanceConnectionInfo,
+									final HardwareDescription hardwareDescription, int numberSlots){
+		throw new IllegalStateException("registerTaskManager called on TestInstanceManager.");
 	}
 
-
-	@Override
-	public InstanceType getDefaultInstanceType() {
-
-		return INSTANCE_TYPE;
-	}
-
-
 	@Override
 	public NetworkTopology getNetworkTopology(final JobID jobID) {
 		throw new IllegalStateException("getNetworkTopology called on TestInstanceManager");
@@ -214,27 +165,11 @@ public final class TestInstanceManager implements InstanceManager {
 		this.instanceListener = instanceListener;
 	}
 
-
-	@Override
-	public Map<InstanceType, InstanceTypeDescription> getMapOfAvailableInstanceTypes() {
-
-		return this.instanceMap;
-	}
-
-
 	@Override
-	public AbstractInstance getInstanceByName(final String name) {
+	public Instance getInstanceByName(final String name) {
 		throw new IllegalStateException("getInstanceByName called on TestInstanceManager");
 	}
 
-
-	@Override
-	public void cancelPendingRequests(final JobID jobID) {
-		throw new IllegalStateException("cancelPendingRequests called on TestInstanceManager");
-
-	}
-
-
 	@Override
 	public void shutdown() {
 		throw new IllegalStateException("shutdown called on TestInstanceManager");
@@ -244,4 +179,9 @@ public final class TestInstanceManager implements InstanceManager {
 	public int getNumberOfTaskTrackers() {
 		throw new IllegalStateException("getNumberOfTaskTrackers called on TestInstanceManager");
 	}
+
+	@Override
+	public int getNumberOfSlots() {
+		return this.testInstance.getNumberOfSlots();
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/managementgraph/ManagementGraphTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/managementgraph/ManagementGraphTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/managementgraph/ManagementGraphTest.java
index 630f365..f1d164a 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/managementgraph/ManagementGraphTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/managementgraph/ManagementGraphTest.java
@@ -131,7 +131,6 @@ public class ManagementGraphTest {
 		assertEquals(origVertex.getExecutionState(), copyVertex.getExecutionState());
 		assertEquals(origVertex.getIndexInGroup(), copyVertex.getIndexInGroup());
 		assertEquals(origVertex.getInstanceName(), copyVertex.getInstanceName());
-		assertEquals(origVertex.getInstanceType(), copyVertex.getInstanceType());
 		assertEquals(origVertex.getNumberOfInputGates(), copyVertex.getNumberOfInputGates());
 		assertEquals(origVertex.getNumberOfOutputGates(), copyVertex.getNumberOfOutputGates());
 
@@ -248,15 +247,15 @@ public class ManagementGraphTest {
 
 		// Vertices
 		final ManagementVertex vertex1_1 = new ManagementVertex(groupVertex1, new ManagementVertexID(), "Host 1",
-			"small", 0);
+			0);
 		final ManagementVertex vertex2_1 = new ManagementVertex(groupVertex2, new ManagementVertexID(), "Host 2",
-			"medium", 0);
+			0);
 		final ManagementVertex vertex2_2 = new ManagementVertex(groupVertex2, new ManagementVertexID(), "Host 2",
-			"medium", 1);
+			1);
 		final ManagementVertex vertex3_1 = new ManagementVertex(groupVertex3, new ManagementVertexID(), "Host 2",
-			"medium", 0);
+			0);
 		final ManagementVertex vertex4_1 = new ManagementVertex(groupVertex4, new ManagementVertexID(), "Host 2",
-			"medium", 0);
+			0);
 
 		// Input/output gates
 		final ManagementGate outputGate1_1 = new ManagementGate(vertex1_1, new ManagementGateID(), 0, false);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/services/iomanager/IOManagerITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/services/iomanager/IOManagerITCase.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/services/iomanager/IOManagerITCase.java
index 47de74f..02190ca 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/services/iomanager/IOManagerITCase.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/services/iomanager/IOManagerITCase.java
@@ -62,7 +62,7 @@ public class IOManagerITCase {
 
 	@Before
 	public void beforeTest() {
-		memoryManager = new DefaultMemoryManager(NUMBER_OF_SEGMENTS * SEGMENT_SIZE);
+		memoryManager = new DefaultMemoryManager(NUMBER_OF_SEGMENTS * SEGMENT_SIZE, 1);
 		ioManager = new IOManager();
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/services/iomanager/IOManagerPerformanceBenchmark.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/services/iomanager/IOManagerPerformanceBenchmark.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/services/iomanager/IOManagerPerformanceBenchmark.java
index 1f0c509..7936a95 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/services/iomanager/IOManagerPerformanceBenchmark.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/services/iomanager/IOManagerPerformanceBenchmark.java
@@ -69,7 +69,7 @@ public class IOManagerPerformanceBenchmark
 	@Before
 	public void startup()
 	{
-		memManager = new DefaultMemoryManager(MEMORY_SIZE);
+		memManager = new DefaultMemoryManager(MEMORY_SIZE,1);
 		ioManager = new IOManager();
 	}
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/services/iomanager/IOManagerTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/services/iomanager/IOManagerTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/services/iomanager/IOManagerTest.java
index a4d92f1..460d546 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/services/iomanager/IOManagerTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/services/iomanager/IOManagerTest.java
@@ -44,7 +44,7 @@ public class IOManagerTest
 	@Before
 	public void beforeTest()
 	{
-		this.memoryManager = new DefaultMemoryManager(32 * 1024 * 1024);
+		this.memoryManager = new DefaultMemoryManager(32 * 1024 * 1024, 1);
 		this.ioManager = new IOManager();
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/services/memorymanager/MemorySegmentTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/services/memorymanager/MemorySegmentTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/services/memorymanager/MemorySegmentTest.java
index 0d12582..fdd6448 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/services/memorymanager/MemorySegmentTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/services/memorymanager/MemorySegmentTest.java
@@ -46,7 +46,7 @@ public class MemorySegmentTest {
 	@Before
 	public void setUp() throws Exception{
 		try {
-			this.manager = new DefaultMemoryManager(MANAGED_MEMORY_SIZE, PAGE_SIZE);
+			this.manager = new DefaultMemoryManager(MANAGED_MEMORY_SIZE, 1, PAGE_SIZE);
 			this.segment = manager.allocatePages(new DefaultMemoryManagerTest.DummyInvokable(), 1).get(0);
 			this.random = new Random(RANDOM_SEED);
 		} catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/ServerTestUtils.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/ServerTestUtils.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/ServerTestUtils.java
index d6cb9b0..4202880 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/ServerTestUtils.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/ServerTestUtils.java
@@ -24,14 +24,11 @@ import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.FileWriter;
 import java.io.IOException;
-import java.util.Map;
 import java.util.jar.JarEntry;
 import java.util.jar.JarOutputStream;
 import java.util.jar.Manifest;
 
 import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.instance.InstanceType;
-import eu.stratosphere.nephele.instance.InstanceTypeDescription;
 import eu.stratosphere.nephele.jobmanager.JobManagerITCase;
 import eu.stratosphere.nephele.protocols.ExtendedManagementProtocol;
 
@@ -55,6 +52,8 @@ public final class ServerTestUtils {
 	 */
 	private static final String ECLIPSE_PATH_EXTENSION = "/src/test/resources";
 
+	private static final String INTELLIJ_PATH_EXTENSION = "/stratosphere-runtime/src/test/resources";
+
 	/**
 	 * Private constructor.
 	 */
@@ -201,6 +200,12 @@ public final class ServerTestUtils {
 			return configDir;
 		}
 
+		configDir = System.getProperty(USER_DIR_KEY) + INTELLIJ_PATH_EXTENSION + CORRECT_CONF_DIR;
+
+		if(new File(configDir).exists()){
+			return configDir;
+		}
+
 		return null;
 	}
 
@@ -217,12 +222,8 @@ public final class ServerTestUtils {
 	public static void waitForJobManagerToBecomeReady(final ExtendedManagementProtocol jobManager) throws IOException,
 			InterruptedException {
 
-		Map<InstanceType, InstanceTypeDescription> instanceMap = jobManager.getMapOfAvailableInstanceTypes();
-
-		while (instanceMap.isEmpty()) {
-
+		while (jobManager.getAvailableSlots() == 0) {
 			Thread.sleep(100);
-			instanceMap = jobManager.getMapOfAvailableInstanceTypes();
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/hash/HashMatchIteratorITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/hash/HashMatchIteratorITCase.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/hash/HashMatchIteratorITCase.java
index ab57a18..a28ba38 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/hash/HashMatchIteratorITCase.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/hash/HashMatchIteratorITCase.java
@@ -98,7 +98,7 @@ public class HashMatchIteratorITCase {
 		this.pairRecordPairComparator = new IntPairRecordPairComparator();
 		this.recordPairPairComparator = new RecordIntPairPairComparator();
 		
-		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE);
+		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1);
 		this.ioManager = new IOManager();
 	}
 
@@ -150,7 +150,7 @@ public class HashMatchIteratorITCase {
 					new BuildFirstHashMatchIterator<Record, Record, Record>(
 						input1, input2, this.recordSerializer, this.record1Comparator, 
 						this.recordSerializer, this.record2Comparator, this.recordPairComparator,
-						this.memoryManager, ioManager, this.parentTask, MEMORY_SIZE);
+						this.memoryManager, ioManager, this.parentTask, 1.0);
 			
 			iterator.open();
 			
@@ -237,7 +237,7 @@ public class HashMatchIteratorITCase {
 					new BuildFirstHashMatchIterator<Record, Record, Record>(
 						input1, input2, this.recordSerializer, this.record1Comparator, 
 						this.recordSerializer, this.record2Comparator, this.recordPairComparator,
-						this.memoryManager, ioManager, this.parentTask, MEMORY_SIZE);
+						this.memoryManager, ioManager, this.parentTask, 1.0);
 
 			iterator.open();
 			
@@ -286,7 +286,7 @@ public class HashMatchIteratorITCase {
 				new BuildSecondHashMatchIterator<Record, Record, Record>(
 					input1, input2, this.recordSerializer, this.record1Comparator, 
 					this.recordSerializer, this.record2Comparator, this.recordPairComparator,
-					this.memoryManager, ioManager, this.parentTask, MEMORY_SIZE);
+					this.memoryManager, ioManager, this.parentTask, 1.0);
 
 			iterator.open();
 			
@@ -373,7 +373,7 @@ public class HashMatchIteratorITCase {
 				new BuildSecondHashMatchIterator<Record, Record, Record>(
 					input1, input2, this.recordSerializer, this.record1Comparator, 
 					this.recordSerializer, this.record2Comparator, this.recordPairComparator,
-					this.memoryManager, ioManager, this.parentTask, MEMORY_SIZE);
+					this.memoryManager, ioManager, this.parentTask, 1.0);
 			
 			iterator.open();
 			
@@ -420,7 +420,7 @@ public class HashMatchIteratorITCase {
 					new BuildSecondHashMatchIterator<IntPair, Record, Record>(
 						input1, input2, this.pairSerializer, this.pairComparator, 
 						this.recordSerializer, this.record2Comparator, this.pairRecordPairComparator,
-						this.memoryManager, this.ioManager, this.parentTask, MEMORY_SIZE);
+						this.memoryManager, this.ioManager, this.parentTask, 1.0);
 			
 			iterator.open();
 			
@@ -467,7 +467,7 @@ public class HashMatchIteratorITCase {
 					new BuildFirstHashMatchIterator<IntPair, Record, Record>(
 						input1, input2, this.pairSerializer, this.pairComparator, 
 						this.recordSerializer, this.record2Comparator, this.recordPairPairComparator,
-						this.memoryManager, this.ioManager, this.parentTask, MEMORY_SIZE);
+						this.memoryManager, this.ioManager, this.parentTask, 1.0);
 			
 			iterator.open();
 			

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/hash/HashTableITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/hash/HashTableITCase.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/hash/HashTableITCase.java
index a845318..4a2fd7d 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/hash/HashTableITCase.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/hash/HashTableITCase.java
@@ -91,7 +91,7 @@ public class HashTableITCase {
 		this.pairProbeSideComparator = new IntPairComparator();
 		this.pairComparator = new IntPairPairComparator();
 		
-		this.memManager = new DefaultMemoryManager(32 * 1024 * 1024);
+		this.memManager = new DefaultMemoryManager(32 * 1024 * 1024,1);
 		this.ioManager = new IOManager();
 	}
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/hash/ReOpenableHashTableITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/hash/ReOpenableHashTableITCase.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/hash/ReOpenableHashTableITCase.java
index 4a4e13a..d9c8b08 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/hash/ReOpenableHashTableITCase.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/hash/ReOpenableHashTableITCase.java
@@ -116,7 +116,7 @@ public class ReOpenableHashTableITCase {
 		this.recordProbeSideComparator = new RecordComparator(keyPos, keyType);
 		this.pactRecordComparator = new HashTableITCase.RecordPairComparatorFirstInt();
 		
-		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, PAGE_SIZE);
+		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE,1, PAGE_SIZE);
 		this.ioManager = new IOManager();
 	}
 
@@ -238,7 +238,7 @@ public class ReOpenableHashTableITCase {
 				new BuildFirstReOpenableHashMatchIterator<Record, Record, Record>(
 						buildInput, probeInput, this.recordSerializer, this.record1Comparator, 
 					this.recordSerializer, this.record2Comparator, this.recordPairComparator,
-					this.memoryManager, ioManager, this.parentTask, MEMORY_SIZE);
+					this.memoryManager, ioManager, this.parentTask, 1.0);
 		
 		iterator.open();
 		// do first join with both inputs

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/io/ChannelViewsTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/io/ChannelViewsTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/io/ChannelViewsTest.java
index ffc4ae7..fbe4f5b 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/io/ChannelViewsTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/io/ChannelViewsTest.java
@@ -73,7 +73,7 @@ public class ChannelViewsTest
 
 	@Before
 	public void beforeTest() {
-		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, MEMORY_PAGE_SIZE);
+		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1, MEMORY_PAGE_SIZE);
 		this.ioManager = new IOManager();
 	}
 
@@ -189,7 +189,7 @@ public class ChannelViewsTest
 		List<MemorySegment> memory = this.memoryManager.allocatePages(this.parentTask, NUM_MEMORY_SEGMENTS);
 		final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channel);
 		final ChannelWriterOutputView outView = new ChannelWriterOutputView(writer, memory, MEMORY_PAGE_SIZE);
-		
+
 		// write a number of pairs
 		final Record rec = new Record();
 		for (int i = 0; i < NUM_PAIRS_SHORT; i++) {
@@ -197,13 +197,13 @@ public class ChannelViewsTest
 			rec.write(outView);
 		}
 		this.memoryManager.release(outView.close());
-		
+
 		// create the reader input view
 		memory = this.memoryManager.allocatePages(this.parentTask, NUM_MEMORY_SEGMENTS);
 		final BlockChannelReader reader = this.ioManager.createBlockChannelReader(channel);
 		final ChannelReaderInputView inView = new ChannelReaderInputView(reader, memory, outView.getBlockCount(), true);
 		generator.reset();
-		
+
 		// read and re-generate all records and compare them
 		try {
 			final Record readRec = new Record();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/io/SpillingBufferTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/io/SpillingBufferTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/io/SpillingBufferTest.java
index fbf363d..1809540 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/io/SpillingBufferTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/io/SpillingBufferTest.java
@@ -64,7 +64,7 @@ public class SpillingBufferTest {
 
 	@Before
 	public void beforeTest() {
-		memoryManager = new DefaultMemoryManager(MEMORY_SIZE);
+		memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1);
 		ioManager = new IOManager();
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/iterative/event/EventWithAggregatorsTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/iterative/event/EventWithAggregatorsTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/iterative/event/EventWithAggregatorsTest.java
index dcda405..8204eed 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/iterative/event/EventWithAggregatorsTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/iterative/event/EventWithAggregatorsTest.java
@@ -117,6 +117,8 @@ public class EventWithAggregatorsTest {
 	
 	private static class TestAggregator<T extends Value> implements Aggregator<T> {
 
+		private static final long serialVersionUID = 1L;
+		
 		private final T val;
 		
 		

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/resettable/BlockResettableIteratorTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/resettable/BlockResettableIteratorTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/resettable/BlockResettableIteratorTest.java
index c7b4644..49afc3a 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/resettable/BlockResettableIteratorTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/resettable/BlockResettableIteratorTest.java
@@ -50,7 +50,7 @@ public class BlockResettableIteratorTest
 	@Before
 	public void startup() {
 		// set up IO and memory manager
-		this.memman = new DefaultMemoryManager(MEMORY_CAPACITY);
+		this.memman = new DefaultMemoryManager(MEMORY_CAPACITY, 1);
 		
 		// create test objects
 		this.objects = new ArrayList<Record>(20000);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/resettable/BlockResettableMutableObjectIteratorTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/resettable/BlockResettableMutableObjectIteratorTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/resettable/BlockResettableMutableObjectIteratorTest.java
index 4b562bf..e3349fd 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/resettable/BlockResettableMutableObjectIteratorTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/resettable/BlockResettableMutableObjectIteratorTest.java
@@ -52,7 +52,7 @@ public class BlockResettableMutableObjectIteratorTest
 	@Before
 	public void startup() {
 		// set up IO and memory manager
-		this.memman = new DefaultMemoryManager(MEMORY_CAPACITY);
+		this.memman = new DefaultMemoryManager(MEMORY_CAPACITY, 1);
 		
 		// create test objects
 		this.objects = new ArrayList<Record>(20000);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/AsynchonousPartialSorterITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/AsynchonousPartialSorterITCase.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/AsynchonousPartialSorterITCase.java
index b416b73..26ce081 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/AsynchonousPartialSorterITCase.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/AsynchonousPartialSorterITCase.java
@@ -72,7 +72,7 @@ public class AsynchonousPartialSorterITCase
 	@Before
 	public void beforeTest()
 	{
-		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE);
+		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE,1);
 		this.ioManager = new IOManager();
 		this.serializer = RecordSerializerFactory.get();
 		this.comparator = new RecordComparator(new int[] {0}, new Class[] {TestData.Key.class});
@@ -107,7 +107,7 @@ public class AsynchonousPartialSorterITCase
 			// merge iterator
 			LOG.debug("Initializing sortmerger...");
 			Sorter<Record> sorter = new AsynchronousPartialSorter<Record>(this.memoryManager, source,
-				this.parentTask, this.serializer, this.comparator, 32 * 1024 * 1024);
+				this.parentTask, this.serializer, this.comparator, 1.0);
 	
 			runPartialSorter(sorter, NUM_RECORDS, 0);
 		}
@@ -130,7 +130,7 @@ public class AsynchonousPartialSorterITCase
 			// merge iterator
 			LOG.debug("Initializing sortmerger...");
 			Sorter<Record> sorter = new AsynchronousPartialSorter<Record>(this.memoryManager, source,
-				this.parentTask, this.serializer, this.comparator, 32 * 1024 * 1024);
+				this.parentTask, this.serializer, this.comparator, 1.0);
 	
 			runPartialSorter(sorter, NUM_RECORDS, 2);
 		}
@@ -153,7 +153,7 @@ public class AsynchonousPartialSorterITCase
 			// merge iterator
 			LOG.debug("Initializing sortmerger...");
 			Sorter<Record> sorter = new AsynchronousPartialSorter<Record>(this.memoryManager, source,
-				this.parentTask, this.serializer, this.comparator, 32 * 1024 * 1024);
+				this.parentTask, this.serializer, this.comparator, 1.0);
 	
 			runPartialSorter(sorter, NUM_RECORDS, 28);
 		}
@@ -178,7 +178,7 @@ public class AsynchonousPartialSorterITCase
 				// merge iterator
 				LOG.debug("Initializing sortmerger...");
 				sorter = new ExceptionThrowingAsynchronousPartialSorter<Record>(this.memoryManager, source,
-						this.parentTask, this.serializer, this.comparator, 32 * 1024 * 1024);
+						this.parentTask, this.serializer, this.comparator, 1.0);
 		
 				runPartialSorter(sorter, NUM_RECORDS, 0);
 				
@@ -283,10 +283,10 @@ public class AsynchonousPartialSorterITCase
 		public ExceptionThrowingAsynchronousPartialSorter(MemoryManager memoryManager,
 				MutableObjectIterator<E> input, AbstractInvokable parentTask, 
 				TypeSerializerFactory<E> serializer, TypeComparator<E> comparator,
-				long totalMemory)
+				double memoryFraction)
 		throws IOException, MemoryAllocationException
 		{
-			super(memoryManager, input, parentTask, serializer, comparator, totalMemory);
+			super(memoryManager, input, parentTask, serializer, comparator, memoryFraction);
 		}
 
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/CombiningUnilateralSortMergerITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/CombiningUnilateralSortMergerITCase.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/CombiningUnilateralSortMergerITCase.java
index 09777a3..1851480 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/CombiningUnilateralSortMergerITCase.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/CombiningUnilateralSortMergerITCase.java
@@ -85,7 +85,7 @@ public class CombiningUnilateralSortMergerITCase {
 	@SuppressWarnings("unchecked")
 	@Before
 	public void beforeTest() {
-		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE);
+		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1);
 		this.ioManager = new IOManager();
 		
 		this.serializerFactory = RecordSerializerFactory.get();
@@ -121,7 +121,7 @@ public class CombiningUnilateralSortMergerITCase {
 		
 		Sorter<Record> merger = new CombiningUnilateralSortMerger<Record>(comb, 
 				this.memoryManager, this.ioManager, reader, this.parentTask, this.serializerFactory, this.comparator,
-				64 * 1024 * 1024, 64, 0.7f);
+				0.25, 64, 0.7f);
 
 		final Record rec = new Record();
 		rec.setField(1, new IntValue(1));
@@ -162,7 +162,7 @@ public class CombiningUnilateralSortMergerITCase {
 		
 		Sorter<Record> merger = new CombiningUnilateralSortMerger<Record>(comb, 
 				this.memoryManager, this.ioManager, reader, this.parentTask, this.serializerFactory, this.comparator,
-				3 * 1024 * 1024, 64, 0.005f);
+				0.01, 64, 0.005f);
 
 		final Record rec = new Record();
 		rec.setField(1, new IntValue(1));
@@ -211,7 +211,7 @@ public class CombiningUnilateralSortMergerITCase {
 		
 		Sorter<Record> merger = new CombiningUnilateralSortMerger<Record>(comb, 
 				this.memoryManager, this.ioManager, reader, this.parentTask, this.serializerFactory, this.comparator,
-				64 * 1024 * 1024, 2, 0.7f);
+				0.25, 2, 0.7f);
 
 		// emit data
 		LOG.debug("emitting data");

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/ExternalSortITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/ExternalSortITCase.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/ExternalSortITCase.java
index bd68382..7ba42b9 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/ExternalSortITCase.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/ExternalSortITCase.java
@@ -76,7 +76,7 @@ public class ExternalSortITCase {
 	@SuppressWarnings("unchecked")
 	@Before
 	public void beforeTest() {
-		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE);
+		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1);
 		this.ioManager = new IOManager();
 		
 		this.pactRecordSerializer = RecordSerializerFactory.get();
@@ -113,7 +113,7 @@ public class ExternalSortITCase {
 		
 		Sorter<Record> merger = new UnilateralSortMerger<Record>(this.memoryManager, this.ioManager, 
 			source, this.parentTask, this.pactRecordSerializer, this.pactRecordComparator,
-			64 * 1024 * 1024, 2, 0.9f);
+				(double)64/78, 2, 0.9f);
 
 		// emit data
 		LOG.debug("Reading and sorting data...");
@@ -159,7 +159,7 @@ public class ExternalSortITCase {
 		
 		Sorter<Record> merger = new UnilateralSortMerger<Record>(this.memoryManager, this.ioManager, 
 				source, this.parentTask, this.pactRecordSerializer, this.pactRecordComparator,
-				64 * 1024 * 1024, 10, 2, 0.9f);
+				(double)64/78, 10, 2, 0.9f);
 
 		// emit data
 		LOG.debug("Reading and sorting data...");
@@ -205,7 +205,7 @@ public class ExternalSortITCase {
 		
 		Sorter<Record> merger = new UnilateralSortMerger<Record>(this.memoryManager, this.ioManager, 
 				source, this.parentTask, this.pactRecordSerializer, this.pactRecordComparator,
-				16 * 1024 * 1024, 64, 0.7f);
+				(double)16/78, 64, 0.7f);
 
 		// emit data
 		LOG.debug("Reading and sorting data...");
@@ -254,7 +254,7 @@ public class ExternalSortITCase {
 		
 		Sorter<Record> merger = new UnilateralSortMerger<Record>(this.memoryManager, this.ioManager, 
 				source, this.parentTask, this.pactRecordSerializer, this.pactRecordComparator,
-				64 * 1024 * 1024, 16, 0.7f);
+				(double)64/78, 16, 0.7f);
 		
 		// emit data
 		LOG.debug("Emitting data...");
@@ -307,7 +307,7 @@ public class ExternalSortITCase {
 		LOG.debug("Initializing sortmerger...");
 		
 		Sorter<IntPair> merger = new UnilateralSortMerger<IntPair>(this.memoryManager, this.ioManager, 
-				generator, this.parentTask, serializerFactory, comparator, 64 * 1024 * 1024, 4, 0.7f);
+				generator, this.parentTask, serializerFactory, comparator, (double)64/78, 4, 0.7f);
 
 		// emit data
 		LOG.debug("Emitting data...");

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/MassiveStringSortingITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/MassiveStringSortingITCase.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/MassiveStringSortingITCase.java
index cb570c4..f76b802 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/MassiveStringSortingITCase.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/MassiveStringSortingITCase.java
@@ -80,7 +80,7 @@ public class MassiveStringSortingITCase {
 			BufferedReader verifyReader = null;
 			
 			try {
-				MemoryManager mm = new DefaultMemoryManager(1024 * 1024);
+				MemoryManager mm = new DefaultMemoryManager(1024 * 1024, 1);
 				IOManager ioMan = new IOManager();
 					
 				TypeSerializer<String> serializer = StringSerializer.INSTANCE;
@@ -170,7 +170,7 @@ public class MassiveStringSortingITCase {
 			BufferedReader verifyReader = null;
 			
 			try {
-				MemoryManager mm = new DefaultMemoryManager(1024 * 1024);
+				MemoryManager mm = new DefaultMemoryManager(1024 * 1024, 1);
 				IOManager ioMan = new IOManager();
 					
 				TupleTypeInfo<Tuple2<String, String[]>> typeInfo = (TupleTypeInfo<Tuple2<String, String[]>>) (TupleTypeInfo<?>) TypeInfoParser.parse("Tuple2<String, String[]>");

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/SortMergeMatchIteratorITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/SortMergeMatchIteratorITCase.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/SortMergeMatchIteratorITCase.java
index 8437c7e..0f3f558 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/SortMergeMatchIteratorITCase.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/SortMergeMatchIteratorITCase.java
@@ -88,7 +88,7 @@ public class SortMergeMatchIteratorITCase
 		this.comparator2 = new RecordComparator(new int[] {0}, new Class[]{TestData.Key.class});
 		this.pairComparator = new RecordPairComparator(new int[] {0}, new int[] {0}, new Class[]{TestData.Key.class});
 		
-		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE);
+		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1);
 		this.ioManager = new IOManager();
 	}
 


[18/53] [abbrv] git commit: Cleanup of merge with slot-based scheduler branch.

Posted by rm...@apache.org.
Cleanup of merge with slot-based scheduler branch.


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

Branch: refs/heads/travis_test
Commit: 429493d027700c3635c8045ad1087511e456d04f
Parents: 86d206c
Author: Stephan Ewen <se...@apache.org>
Authored: Thu Jun 19 00:15:46 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sun Jun 22 21:07:20 2014 +0200

----------------------------------------------------------------------
 .../eu/stratosphere/client/program/Client.java  |  17 +---
 .../client/program/PackagedProgram.java         |   1 +
 .../stratosphere/client/program/ClientTest.java |   3 +-
 .../compiler/costs/DefaultCostEstimator.java    |   6 +-
 .../compiler/dag/DataSourceNode.java            |   2 +-
 .../pact/compiler/DOPChangeTest.java            |   8 +-
 .../pact/compiler/IterationsCompilerTest.java   |   4 +-
 .../nephele/execution/RuntimeEnvironment.java   |  72 +++++++++++--
 .../nephele/protocols/JobManagerProtocol.java   |   2 +-
 .../stratosphere/nephele/taskmanager/Task.java  |  70 +++----------
 .../nephele/taskmanager/TaskManager.java        |  10 +-
 .../local/LocalInstanceManagerTest.java         |  19 ++--
 .../nephele/jobmanager/JobManagerITCase.java    |  19 ++--
 .../nephele/util/ServerTestUtils.java           |  48 +--------
 .../netty/InboundEnvelopeDecoderTest.java       |   2 +-
 .../confs/jobmanager/nephele-default.xml        |  51 ----------
 .../test/javaApiOperators/SumMinMaxITCase.java  |   4 +-
 .../PackagedProgramEndToEndITCase.java          |  48 +++++----
 .../test/util/testjar/KMeansForTest.java        | 102 +++++--------------
 19 files changed, 183 insertions(+), 305 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/429493d0/stratosphere-clients/src/main/java/eu/stratosphere/client/program/Client.java
----------------------------------------------------------------------
diff --git a/stratosphere-clients/src/main/java/eu/stratosphere/client/program/Client.java b/stratosphere-clients/src/main/java/eu/stratosphere/client/program/Client.java
index 31138f6..ec66f4a 100644
--- a/stratosphere-clients/src/main/java/eu/stratosphere/client/program/Client.java
+++ b/stratosphere-clients/src/main/java/eu/stratosphere/client/program/Client.java
@@ -78,9 +78,6 @@ public class Client {
 		configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerAddress.getPort());
 		
 		this.compiler = new PactCompiler(new DataStatistics(), new DefaultCostEstimator());
-		
-		//  Disable Local Execution when using a Client
-		ContextEnvironment.disableLocalExecution();
 	}
 
 	/**
@@ -105,9 +102,6 @@ public class Client {
 		}
 
 		this.compiler = new PactCompiler(new DataStatistics(), new DefaultCostEstimator());
-		
-		//  Disable Local Execution when using a Client
-		ContextEnvironment.disableLocalExecution();
 	}
 	
 	public void setPrintStatusDuringExecution(boolean print) {
@@ -152,20 +146,13 @@ public class Client {
 			ByteArrayOutputStream baes = new ByteArrayOutputStream();
 			System.setErr(new PrintStream(baes));
 			try {
+				ContextEnvironment.disableLocalExecution();
 				prog.invokeInteractiveModeForExecution();
 			}
 			catch (ProgramInvocationException e) {
-				System.setOut(originalOut);
-				System.setErr(originalErr);
-				System.err.println(baes);
-				System.out.println(baos);
 				throw e;
 			}
 			catch (Throwable t) {
-				System.setOut(originalOut);
-				System.setErr(originalErr);
-				System.err.println(baes);
-				System.out.println(baos);
 				// the invocation gets aborted with the preview plan
 				if (env.optimizerPlan != null) {
 					return env.optimizerPlan;
@@ -240,6 +227,8 @@ public class Client {
 			}
 			env.setAsContext();
 			
+			ContextEnvironment.disableLocalExecution();
+			
 			if (wait) {
 				// invoke here
 				prog.invokeInteractiveModeForExecution();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/429493d0/stratosphere-clients/src/main/java/eu/stratosphere/client/program/PackagedProgram.java
----------------------------------------------------------------------
diff --git a/stratosphere-clients/src/main/java/eu/stratosphere/client/program/PackagedProgram.java b/stratosphere-clients/src/main/java/eu/stratosphere/client/program/PackagedProgram.java
index 51d2e34..edf36b3 100644
--- a/stratosphere-clients/src/main/java/eu/stratosphere/client/program/PackagedProgram.java
+++ b/stratosphere-clients/src/main/java/eu/stratosphere/client/program/PackagedProgram.java
@@ -215,6 +215,7 @@ public class PackagedProgram {
 			PreviewPlanEnvironment env = new PreviewPlanEnvironment();
 			env.setAsContext();
 			try {
+				ContextEnvironment.disableLocalExecution();
 				invokeInteractiveModeForExecution();
 			}
 			catch (ProgramInvocationException e) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/429493d0/stratosphere-clients/src/test/java/eu/stratosphere/client/program/ClientTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-clients/src/test/java/eu/stratosphere/client/program/ClientTest.java b/stratosphere-clients/src/test/java/eu/stratosphere/client/program/ClientTest.java
index 244ec4a..b3f8159 100644
--- a/stratosphere-clients/src/test/java/eu/stratosphere/client/program/ClientTest.java
+++ b/stratosphere-clients/src/test/java/eu/stratosphere/client/program/ClientTest.java
@@ -20,7 +20,6 @@ import static org.mockito.MockitoAnnotations.initMocks;
 import static org.powermock.api.mockito.PowerMockito.whenNew;
 
 import java.io.IOException;
-import java.net.InetSocketAddress;
 
 import org.junit.Before;
 import org.junit.Test;
@@ -95,7 +94,7 @@ public class ClientTest {
 		when(program.getPlanWithJars()).thenReturn(planWithJarsMock);
 		when(planWithJarsMock.getPlan()).thenReturn(planMock);
 		
-		whenNew(PactCompiler.class).withArguments(any(DataStatistics.class), any(CostEstimator.class), any(InetSocketAddress.class)).thenReturn(this.compilerMock);
+		whenNew(PactCompiler.class).withArguments(any(DataStatistics.class), any(CostEstimator.class)).thenReturn(this.compilerMock);
 		when(compilerMock.compile(planMock)).thenReturn(optimizedPlanMock);
 		
 		whenNew(NepheleJobGraphGenerator.class).withNoArguments().thenReturn(generatorMock);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/429493d0/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/costs/DefaultCostEstimator.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/costs/DefaultCostEstimator.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/costs/DefaultCostEstimator.java
index fde5970..3c52f6a 100644
--- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/costs/DefaultCostEstimator.java
+++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/costs/DefaultCostEstimator.java
@@ -34,7 +34,7 @@ public class DefaultCostEstimator extends CostEstimator {
 	 * The case of the estimation for all relative costs. We heuristically pick a very large data volume, which
 	 * will favor strategies that are less expensive on large data volumes. This is robust and 
 	 */
-	private static final long HEURISTIC_COST_BASE = 10000000000l;
+	private static final long HEURISTIC_COST_BASE = 1000000000L;
 	
 	// The numbers for the CPU effort are rather magic at the moment and should be seen rather ordinal
 	
@@ -105,9 +105,9 @@ public class DefaultCostEstimator extends CostEstimator {
 			} else {
 				costs.addNetworkCost(replicationFactor * estOutShipSize);
 			}
-			costs.addHeuristicNetworkCost(HEURISTIC_COST_BASE * replicationFactor);
+			costs.addHeuristicNetworkCost(HEURISTIC_COST_BASE * 10 * replicationFactor);
 		} else {
-			costs.addHeuristicNetworkCost(HEURISTIC_COST_BASE * 200);
+			costs.addHeuristicNetworkCost(HEURISTIC_COST_BASE * 1000);
 		}
 	}
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/429493d0/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/DataSourceNode.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/DataSourceNode.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/DataSourceNode.java
index 7234420..b6d6b71 100644
--- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/DataSourceNode.java
+++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/DataSourceNode.java
@@ -168,7 +168,7 @@ public class DataSourceNode extends OptimizerNode {
 			return this.cachedPlans;
 		}
 		
-		SourcePlanNode candidate = new SourcePlanNode(this, "DataSource("+this.getPactContract().getName()+")");
+		SourcePlanNode candidate = new SourcePlanNode(this, "DataSource ("+this.getPactContract().getName()+")");
 		candidate.updatePropertiesWithUniqueSets(getUniqueFields());
 		
 		final Costs costs = new Costs();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/429493d0/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/DOPChangeTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/DOPChangeTest.java b/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/DOPChangeTest.java
index 273c42c..605f197 100644
--- a/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/DOPChangeTest.java
+++ b/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/DOPChangeTest.java
@@ -209,15 +209,15 @@ public class DOPChangeTest extends CompilerTestBase {
 		ShipStrategyType mapIn = map2Node.getInput().getShipStrategy();
 		ShipStrategyType reduceIn = red2Node.getInput().getShipStrategy();
 		
-		Assert.assertEquals("Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, mapIn);
-		Assert.assertEquals("Invalid ship strategy for an operator.", ShipStrategyType.PARTITION_HASH, reduceIn);
+		Assert.assertTrue("Invalid ship strategy for an operator.", 
+				(ShipStrategyType.PARTITION_RANDOM ==  mapIn && ShipStrategyType.PARTITION_HASH == reduceIn) || 
+				(ShipStrategyType.PARTITION_HASH == mapIn && ShipStrategyType.FORWARD == reduceIn));
 	}
 	
 	
 	
 	@Test
-	public void checkPropertyHandlingWithDecreasingDegreeOfParallelism()
-	{
+	public void checkPropertyHandlingWithDecreasingDegreeOfParallelism() {
 		final int degOfPar = DEFAULT_PARALLELISM;
 		
 		// construct the plan

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/429493d0/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/IterationsCompilerTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/IterationsCompilerTest.java b/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/IterationsCompilerTest.java
index 05a863c..c6ebf50 100644
--- a/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/IterationsCompilerTest.java
+++ b/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/IterationsCompilerTest.java
@@ -153,6 +153,7 @@ public class IterationsCompilerTest extends CompilerTestBase {
 	public void testIterationPushingWorkOut() throws Exception {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+			env.setDegreeOfParallelism(8);
 			
 			DataSet<Tuple2<Long, Long>> input1 = env.readCsvFile("/some/file/path").types(Long.class).map(new DuplicateValue());
 			
@@ -168,6 +169,7 @@ public class IterationsCompilerTest extends CompilerTestBase {
 			
 			BulkIterationPlanNode bipn = (BulkIterationPlanNode) op.getDataSinks().iterator().next().getInput().getSource();
 			
+			// check that work has not! been pushed out, as the end of the step function does not produce the necessary properties
 			for (Channel c : bipn.getPartialSolutionPlanNode().getOutgoingChannels()) {
 				assertEquals(ShipStrategyType.PARTITION_HASH, c.getShipStrategy());
 			}
@@ -182,7 +184,7 @@ public class IterationsCompilerTest extends CompilerTestBase {
 	public static DataSet<Tuple2<Long, Long>> doBulkIteration(DataSet<Tuple2<Long, Long>> vertices, DataSet<Tuple2<Long, Long>> edges) {
 		
 		// open a bulk iteration
-		IterativeDataSet<Tuple2<Long, Long>> iteration = vertices.iterate(100);
+		IterativeDataSet<Tuple2<Long, Long>> iteration = vertices.iterate(20);
 		
 		DataSet<Tuple2<Long, Long>> changes = iteration
 				.join(edges).where(0).equalTo(0).with(new Join222())

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/429493d0/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java
index cc542c0..4e07694 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java
@@ -36,6 +36,7 @@ import eu.stratosphere.runtime.io.network.bufferprovider.GlobalBufferPool;
 import eu.stratosphere.runtime.io.network.bufferprovider.LocalBufferPool;
 import eu.stratosphere.runtime.io.network.bufferprovider.LocalBufferPoolOwner;
 import eu.stratosphere.util.StringUtils;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
@@ -108,11 +109,6 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 	private final AbstractInvokable invokable;
 
 	/**
-	 * The thread executing the task in the environment.
-	 */
-	private volatile Thread executingThread = null;
-
-	/**
 	 * The ID of the job this task belongs to.
 	 */
 	private final JobID jobID;
@@ -136,6 +132,11 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 	 * The observer object for the task's execution.
 	 */
 	private volatile ExecutionObserver executionObserver = null;
+	
+	/**
+	 * The thread executing the task in the environment.
+	 */
+	private volatile Thread executingThread;
 
 	/**
 	 * The RPC proxy to report accumulators to JobManager
@@ -159,7 +160,9 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 
 	private LocalBufferPool outputBufferPool;
 
-	private Map<String,FutureTask<Path>> cacheCopyTasks = new HashMap<String, FutureTask<Path>>();
+	private final Map<String,FutureTask<Path>> cacheCopyTasks;
+	
+	private volatile boolean canceled;
 
 	/**
 	 * Creates a new runtime environment object which contains the runtime information for the encapsulated Nephele
@@ -174,8 +177,9 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 	 */
 	public RuntimeEnvironment(final JobID jobID, final String taskName,
 							final Class<? extends AbstractInvokable> invokableClass, final Configuration taskConfiguration,
-							final Configuration jobConfiguration) throws Exception {
-
+							final Configuration jobConfiguration)
+		throws Exception
+	{
 		this.jobID = jobID;
 		this.taskName = taskName;
 		this.invokableClass = invokableClass;
@@ -186,7 +190,8 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 		this.memoryManager = null;
 		this.ioManager = null;
 		this.inputSplitProvider = null;
-
+		this.cacheCopyTasks = new HashMap<String, FutureTask<Path>>();
+		
 		this.invokable = this.invokableClass.newInstance();
 		this.invokable.setEnvironment(this);
 		this.invokable.registerInputOutput();
@@ -433,6 +438,53 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 			return this.executingThread;
 		}
 	}
+	
+	public void cancelExecution() {
+		canceled = true;
+
+		LOG.info("Canceling " + getTaskNameWithIndex());
+
+		// Request user code to shut down
+		if (this.invokable != null) {
+			try {
+				this.invokable.cancel();
+			} catch (Throwable e) {
+				LOG.error("Error while cancelling the task.", e);
+			}
+		}
+		
+		// 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 " + getTaskName() + " 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");
+				
+				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) {}
+		}
+	}
 
 	/**
 	 * Blocks until all output channels are closed.
@@ -459,7 +511,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 	 */
 	private void waitForInputChannelsToBeClosed() throws IOException, InterruptedException {
 		// Wait for disconnection of all output gates
-		while (true) {
+		while (!canceled) {
 
 			// Make sure, we leave this method with an InterruptedException when the task has been canceled
 			if (this.executionObserver.isCanceled()) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/429493d0/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/JobManagerProtocol.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/JobManagerProtocol.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/JobManagerProtocol.java
index 5070b51..4db5e14 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/JobManagerProtocol.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/JobManagerProtocol.java
@@ -51,7 +51,7 @@ public interface JobManagerProtocol extends VersionedProtocol {
 	 * @return whether the task manager was successfully registered
 	 */
 	RegisterTaskManagerResult registerTaskManager(InstanceConnectionInfo instanceConnectionInfo,
-						HardwareDescription hardwareDescription,IntegerRecord numberOfSlots)
+						HardwareDescription hardwareDescription, IntegerRecord numberOfSlots)
 			throws IOException;
 
 	/**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/429493d0/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java
index 825eae1..d1a6275 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java
@@ -24,14 +24,14 @@ import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
 import eu.stratosphere.nephele.jobgraph.JobID;
 import eu.stratosphere.nephele.profiling.TaskManagerProfiler;
 import eu.stratosphere.nephele.services.memorymanager.MemoryManager;
-import eu.stratosphere.nephele.template.AbstractInvokable;
-import eu.stratosphere.util.StringUtils;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
 import java.util.Iterator;
 import java.util.Queue;
 import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 public final class Task implements ExecutionObserver {
 
@@ -49,13 +49,14 @@ public final class Task implements ExecutionObserver {
 	/**
 	 * Stores whether the task has been canceled.
 	 */
-	private volatile boolean isCanceled = false;
+	private final AtomicBoolean canceled = new AtomicBoolean(false);
 
 	/**
 	 * The current execution state of the task
 	 */
 	private volatile ExecutionState executionState = ExecutionState.STARTING;
 
+	
 	private Queue<ExecutionListener> registeredListeners = new ConcurrentLinkedQueue<ExecutionListener>();
 
 	public Task(ExecutionVertexID vertexID, final RuntimeEnvironment environment, TaskManager taskManager) {
@@ -102,11 +103,11 @@ public final class Task implements ExecutionObserver {
 	}
 
 	public void cancelExecution() {
-		cancelOrKillExecution(true);
+		cancelOrKillExecution();
 	}
 
 	public void killExecution() {
-		cancelOrKillExecution(false);
+		cancelOrKillExecution();
 	}
 
 	/**
@@ -114,10 +115,8 @@ public final class Task implements ExecutionObserver {
 	 *
 	 * @param cancel <code>true/code> if the task shall be canceled, <code>false</code> if it shall be killed
 	 */
-	private void cancelOrKillExecution(boolean cancel) {
-		final Thread executingThread = this.environment.getExecutingThread();
-
-		if (executingThread == null) {
+	private void cancelOrKillExecution() {
+		if (!this.canceled.compareAndSet(false, true)) {
 			return;
 		}
 
@@ -125,45 +124,13 @@ public final class Task implements ExecutionObserver {
 			return;
 		}
 
-		LOG.info((cancel ? "Canceling " : "Killing ") + this.environment.getTaskNameWithIndex());
-
-		if (cancel) {
-			this.isCanceled = true;
-			// Change state
-			executionStateChanged(ExecutionState.CANCELING, null);
-
-			// Request user code to shut down
-			try {
-				final AbstractInvokable invokable = this.environment.getInvokable();
-				if (invokable != null) {
-					invokable.cancel();
-				}
-			} catch (Throwable e) {
-				LOG.error(StringUtils.stringifyException(e));
-			}
-		}
-
-		// Continuously interrupt the user thread until it changed to state CANCELED
-		while (true) {
-
-			executingThread.interrupt();
-
-			if (!executingThread.isAlive()) {
-				break;
-			}
-
-			try {
-				executingThread.join(1000);
-			} catch (InterruptedException e) {}
+		executionStateChanged(ExecutionState.CANCELING, null);
 
-			if (!executingThread.isAlive()) {
-				break;
-			}
-
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("Sending repeated " + (cancel == true ? "canceling" : "killing") + " signal to " +
-						this.environment.getTaskName() + " with state " + this.executionState);
-			}
+		// Request user code to shut down
+		try {
+			this.environment.cancelExecution();
+		} catch (Throwable e) {
+			LOG.error("Error while cancelling the task.", e);
 		}
 	}
 
@@ -271,7 +238,6 @@ public final class Task implements ExecutionObserver {
 	 * @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() + ")";
 	}
@@ -279,7 +245,6 @@ public final class Task implements ExecutionObserver {
 
 	@Override
 	public void userThreadStarted(final Thread userThread) {
-
 		// Notify the listeners
 		final Iterator<ExecutionListener> it = this.registeredListeners.iterator();
 		while (it.hasNext()) {
@@ -290,7 +255,6 @@ public final class Task implements ExecutionObserver {
 
 	@Override
 	public void userThreadFinished(final Thread userThread) {
-
 		// Notify the listeners
 		final Iterator<ExecutionListener> it = this.registeredListeners.iterator();
 		while (it.hasNext()) {
@@ -307,7 +271,6 @@ public final class Task implements ExecutionObserver {
 	 */
 
 	public void registerExecutionListener(final ExecutionListener executionListener) {
-
 		this.registeredListeners.add(executionListener);
 	}
 
@@ -320,15 +283,13 @@ public final class Task implements ExecutionObserver {
 	 */
 
 	public void unregisterExecutionListener(final ExecutionListener executionListener) {
-
 		this.registeredListeners.remove(executionListener);
 	}
 
 
 	@Override
 	public boolean isCanceled() {
-
-		return this.isCanceled;
+		return this.canceled.get();
 	}
 
 	/**
@@ -337,7 +298,6 @@ public final class Task implements ExecutionObserver {
 	 * @return the runtime environment associated with this task
 	 */
 	public RuntimeEnvironment getRuntimeEnvironment() {
-
 		return this.environment;
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/429493d0/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
index 5966cf9..f191df3 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
@@ -164,12 +164,15 @@ public class TaskManager implements TaskOperationProtocol {
 	private volatile boolean shutdownComplete;
 	
 	/**
-	 * Constructs a new task manager, starts its IPC service and attempts to discover the job manager to
-	 * receive an initial configuration. All parameters are obtained from the 
+	 * 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.");
+		}
+		
+		
 		LOG.info("TaskManager started as user " + UserGroupInformation.getCurrentUser().getShortUserName());
 		LOG.info("User system property: " + System.getProperty("user.name"));
 		LOG.info("Execution mode: " + executionMode);
@@ -340,6 +343,7 @@ public class TaskManager implements TaskOperationProtocol {
 
 		{
 			HardwareDescription resources = HardwareDescriptionFactory.extractFromSystem();
+			
 			numberOfSlots = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
 					Hardware.getNumberCPUCores());
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/429493d0/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/local/LocalInstanceManagerTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/local/LocalInstanceManagerTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/local/LocalInstanceManagerTest.java
index a8f1331..b491c12 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/local/LocalInstanceManagerTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/local/LocalInstanceManagerTest.java
@@ -13,21 +13,19 @@
 
 package eu.stratosphere.nephele.instance.local;
 
-import static org.junit.Assert.fail;
-
 import eu.stratosphere.nephele.instance.InstanceManager;
 import junit.framework.Assert;
 
 import org.junit.Test;
 
 import eu.stratosphere.nephele.ExecutionMode;
+import eu.stratosphere.configuration.ConfigConstants;
+import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.configuration.GlobalConfiguration;
 import eu.stratosphere.nephele.jobmanager.JobManager;
-import eu.stratosphere.nephele.util.ServerTestUtils;
 
 /**
  * Tests for the {@link LocalInstanceManager}.
- * 
  */
 public class LocalInstanceManagerTest {
 
@@ -39,12 +37,13 @@ public class LocalInstanceManagerTest {
 	public void testInstanceTypeFromConfiguration() {
 
 		try {
-			final String configDir = ServerTestUtils.getConfigDir();
-			if (configDir == null) {
-				fail("Cannot locate configuration directory");
-			}
-
-			GlobalConfiguration.loadConfiguration(configDir);
+			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);
 
 			// start JobManager
 			ExecutionMode executionMode = ExecutionMode.LOCAL;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/429493d0/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java
index fa4fbfa..89f7428 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java
@@ -20,6 +20,7 @@ import eu.stratosphere.core.fs.Path;
 import eu.stratosphere.nephele.ExecutionMode;
 import eu.stratosphere.nephele.client.JobClient;
 import eu.stratosphere.nephele.client.JobExecutionException;
+import eu.stratosphere.nephele.execution.RuntimeEnvironment;
 import eu.stratosphere.nephele.jobgraph.DistributionPattern;
 import eu.stratosphere.runtime.io.channels.ChannelType;
 import eu.stratosphere.nephele.jobgraph.JobFileInputVertex;
@@ -34,6 +35,7 @@ import eu.stratosphere.nephele.util.FileLineWriter;
 import eu.stratosphere.nephele.util.JarFileCreator;
 import eu.stratosphere.nephele.util.ServerTestUtils;
 import eu.stratosphere.util.LogUtils;
+
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.junit.AfterClass;
@@ -57,7 +59,8 @@ import static org.junit.Assert.fail;
 public class JobManagerITCase {
 
 	static {
-		LogUtils.initializeDefaultTestConsoleLogger();
+		// no logging, because the tests create expected exception
+		LogUtils.initializeDefaultConsoleLogger(Level.INFO);
 	}
 	
 	/**
@@ -75,7 +78,13 @@ public class JobManagerITCase {
 	@BeforeClass
 	public static void startNephele() {
 		try {
-			GlobalConfiguration.loadConfiguration(ServerTestUtils.getConfigDir());
+			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 });
 			
@@ -301,8 +310,9 @@ public class JobManagerITCase {
 			
 			// deactivate logging of expected test exceptions
 			Logger rtLogger = Logger.getLogger(Task.class);
-			Level rtLevel = rtLogger.getEffectiveLevel();
 			rtLogger.setLevel(Level.OFF);
+			Logger envLogger = Logger.getLogger(RuntimeEnvironment.class);
+			envLogger.setLevel(Level.DEBUG);
 			
 			try {
 				jobClient.submitJobAndWait();
@@ -317,9 +327,6 @@ public class JobManagerITCase {
 
 				return;
 			}
-			finally {
-				rtLogger.setLevel(rtLevel);
-			}
 
 			fail("Expected exception but did not receive it");
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/429493d0/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/ServerTestUtils.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/ServerTestUtils.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/ServerTestUtils.java
index 4202880..59de8cc 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/ServerTestUtils.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/ServerTestUtils.java
@@ -38,27 +38,9 @@ import eu.stratosphere.nephele.protocols.ExtendedManagementProtocol;
 public final class ServerTestUtils {
 
 	/**
-	 * The system property key to retrieve the user directory.
-	 */
-	private static final String USER_DIR_KEY = "user.dir";
-
-	/**
-	 * The directory containing the correct configuration file to be used during the tests.
-	 */
-	private static final String CORRECT_CONF_DIR = "/confs/jobmanager";
-
-	/**
-	 * The directory the configuration directory is expected in when test are executed using Eclipse.
-	 */
-	private static final String ECLIPSE_PATH_EXTENSION = "/src/test/resources";
-
-	private static final String INTELLIJ_PATH_EXTENSION = "/stratosphere-runtime/src/test/resources";
-
-	/**
 	 * Private constructor.
 	 */
-	private ServerTestUtils() {
-	}
+	private ServerTestUtils() {}
 
 	/**
 	 * Creates a file with a random name in the given sub directory within the directory for temporary files. The
@@ -182,34 +164,6 @@ public final class ServerTestUtils {
 	}
 
 	/**
-	 * Returns the directory containing the configuration files that shall be used for the test.
-	 * 
-	 * @return the directory containing the configuration files or <code>null</code> if the configuration directory
-	 *         could not be located
-	 */
-	public static String getConfigDir() {
-
-		// This is the correct path for Maven-based tests
-		String configDir = System.getProperty(USER_DIR_KEY) + CORRECT_CONF_DIR;
-		if (new File(configDir).exists()) {
-			return configDir;
-		}
-
-		configDir = System.getProperty(USER_DIR_KEY) + ECLIPSE_PATH_EXTENSION + CORRECT_CONF_DIR;
-		if (new File(configDir).exists()) {
-			return configDir;
-		}
-
-		configDir = System.getProperty(USER_DIR_KEY) + INTELLIJ_PATH_EXTENSION + CORRECT_CONF_DIR;
-
-		if(new File(configDir).exists()){
-			return configDir;
-		}
-
-		return null;
-	}
-
-	/**
 	 * Waits until the job manager for the tests has become ready to accept jobs.
 	 * 
 	 * @param jobManager

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/429493d0/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoderTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoderTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoderTest.java
index 1ee9293..1c6270a 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoderTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoderTest.java
@@ -354,7 +354,7 @@ public class InboundEnvelopeDecoderTest {
 		buf.readerIndex(0);
 		ByteBuf[] slices = randomSlices(buf);
 
-		ch.writeInbound((Object) slices);
+		ch.writeInbound((Object[]) slices);
 
 		for (ByteBuf slice : slices) {
 			Assert.assertEquals(1, slice.refCnt());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/429493d0/stratosphere-runtime/src/test/resources/confs/jobmanager/nephele-default.xml
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/resources/confs/jobmanager/nephele-default.xml b/stratosphere-runtime/src/test/resources/confs/jobmanager/nephele-default.xml
deleted file mode 100644
index 5d93d95..0000000
--- a/stratosphere-runtime/src/test/resources/confs/jobmanager/nephele-default.xml
+++ /dev/null
@@ -1,51 +0,0 @@
-<?xml version="1.0"?>
-<configuration>
-    <property>
-        <key>jobmanager.rpc.address</key>
-        <value>127.0.0.1</value>
-    </property>
-    <property>
-        <key>jobmanager.rpc.port</key>
-        <value>6123</value>
-    </property>
-    <property>
-	<key>taskmanager.setup.usediscovery</key>
-	<value>false</value>
-    </property>
-    <property>
-        <key>discoveryservice.magicnumber</key>
-        <value>12300</value>
-    </property>
-    <property>
-        <key>instancemanager.local.classname</key>
-        <value>eu.stratosphere.nephele.instance.local.LocalInstanceManager</value>
-    </property>
-    <property>
-	<key>jobmanager.scheduler.local.classname</key>
-	<value>eu.stratosphere.nephele.jobmanager.scheduler.local.LocalScheduler</value>
-    </property>
-    <property>
-        <key>channel.network.compressor</key>
-        <value>de.tu_berlin.cit.nephele.io.compression.lzo.LzoCompressor</value>
-    </property>
-    <property>
-        <key>channel.network.decompressor</key>
-        <value>de.tu_berlin.cit.nephele.io.compression.lzo.LzoDecompressor</value>
-    </property>
-    <property>
-        <key>channel.file.compressor</key>
-        <value>de.tu_berlin.cit.nephele.io.compression.lzo.LzoCompressor</value>
-    </property>
-    <property>
-        <key>channel.file.decompressor</key>
-        <value>de.tu_berlin.cit.nephele.io.compression.lzo.LzoDecompressor</value>
-    </property>
-    <property>
-        <key>taskmanager.memory.size</key>
-        <value>8</value>
-    </property>
-    <property>
-        <key>instancemanager.local.type</key>
-        <value>test,4,4,1024,160,0</value>
-    </property>
-</configuration>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/429493d0/stratosphere-tests/src/test/java/eu/stratosphere/test/javaApiOperators/SumMinMaxITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/javaApiOperators/SumMinMaxITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/javaApiOperators/SumMinMaxITCase.java
index 8b7dc80..cef9c05 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/javaApiOperators/SumMinMaxITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/javaApiOperators/SumMinMaxITCase.java
@@ -18,13 +18,14 @@ package eu.stratosphere.test.javaApiOperators;
 
 import eu.stratosphere.api.java.DataSet;
 import eu.stratosphere.api.java.ExecutionEnvironment;
-import eu.stratosphere.api.java.aggregation.Aggregations;
 import eu.stratosphere.api.java.tuple.Tuple1;
 import eu.stratosphere.api.java.tuple.Tuple2;
 import eu.stratosphere.api.java.tuple.Tuple3;
 import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.test.javaApiOperators.util.CollectionDataSets;
 import eu.stratosphere.test.util.JavaProgramTestBase;
+
+import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
 import java.io.FileNotFoundException;
@@ -32,6 +33,7 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.LinkedList;
 
+@RunWith(Parameterized.class)
 public class SumMinMaxITCase extends JavaProgramTestBase  {
 
 	private static int NUM_PROGRAMS = 3;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/429493d0/stratosphere-tests/src/test/java/eu/stratosphere/test/localDistributed/PackagedProgramEndToEndITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/localDistributed/PackagedProgramEndToEndITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/localDistributed/PackagedProgramEndToEndITCase.java
index 0e297ec..17f4a29 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/localDistributed/PackagedProgramEndToEndITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/localDistributed/PackagedProgramEndToEndITCase.java
@@ -14,7 +14,6 @@ package eu.stratosphere.test.localDistributed;
 
 import java.io.File;
 import java.io.FileWriter;
-import java.net.URL;
 
 import eu.stratosphere.client.minicluster.NepheleMiniCluster;
 import org.junit.Assert;
@@ -29,23 +28,26 @@ import eu.stratosphere.util.LogUtils;
 
 public class PackagedProgramEndToEndITCase {
 
-	private static final int DOP = 4;
-
 	static {
 		LogUtils.initializeDefaultTestConsoleLogger();
 	}
 	
 	@Test
 	public void testEverything() {
+		final int PORT = 6498;
+		
 		NepheleMiniCluster cluster = new NepheleMiniCluster();
+		
+		File points = null;
+		File clusters = null;
+		File outFile = null;
+		
 		try {
 			// set up the files
-			File points = File.createTempFile("kmeans_points", ".in");
-			File clusters = File.createTempFile("kmeans_clusters", ".in");
-			File outFile = File.createTempFile("kmeans_result", ".out");
-			points.deleteOnExit();
-			clusters.deleteOnExit();
-			outFile.deleteOnExit();
+			points = File.createTempFile("kmeans_points", ".in");
+			clusters = File.createTempFile("kmeans_clusters", ".in");
+			outFile = File.createTempFile("kmeans_result", ".out");
+			
 			outFile.delete();
 
 			FileWriter fwPoints = new FileWriter(points);
@@ -56,31 +58,39 @@ public class PackagedProgramEndToEndITCase {
 			fwClusters.write(KMeansData.INITIAL_CENTERS);
 			fwClusters.close();
 
-			URL jarFileURL = getClass().getResource("/KMeansForTest.jar");
-			String jarPath = jarFileURL.getFile();
+			String jarPath = "target/maven-test-jar.jar";
 
 			// run KMeans
 			cluster.setNumTaskTracker(2);
 			cluster.setTaskManagerNumSlots(2);
+			cluster.setJobManagerRpcPort(PORT);
 			cluster.start();
-			RemoteExecutor ex = new RemoteExecutor("localhost", 6498);
+			
+			RemoteExecutor ex = new RemoteExecutor("localhost", PORT);
 
 			ex.executeJar(jarPath,
-					"eu.stratosphere.examples.scala.testing.KMeansForTest",
-					new String[] {new Integer(DOP).toString(),
+					"eu.stratosphere.test.util.testjar.KMeansForTest",
+					new String[] {
 							points.toURI().toString(),
 							clusters.toURI().toString(),
 							outFile.toURI().toString(),
 							"25"});
 
-			points.delete();
-			clusters.delete();
-			outFile.delete();
-
 		} catch (Exception e) {
 			e.printStackTrace();
 			Assert.fail(e.getMessage());
-		} finally {
+		}
+		finally {
+			if (points != null) {
+				points.delete();
+			}
+			if (cluster != null) {
+				clusters.delete();
+			}
+			if (outFile != null) {
+				outFile.delete();
+			}
+			
 			try {
 				cluster.stop();
 			} catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/429493d0/stratosphere-tests/src/test/java/eu/stratosphere/test/util/testjar/KMeansForTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/util/testjar/KMeansForTest.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/util/testjar/KMeansForTest.java
index d1b249a..8047649 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/util/testjar/KMeansForTest.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/util/testjar/KMeansForTest.java
@@ -23,7 +23,6 @@ import eu.stratosphere.api.common.Program;
 import eu.stratosphere.api.java.DataSet;
 import eu.stratosphere.api.java.ExecutionEnvironment;
 import eu.stratosphere.api.java.IterativeDataSet;
-import eu.stratosphere.api.java.RemoteEnvironment;
 import eu.stratosphere.api.java.functions.MapFunction;
 import eu.stratosphere.api.java.functions.ReduceFunction;
 import eu.stratosphere.api.java.tuple.Tuple2;
@@ -31,25 +30,41 @@ import eu.stratosphere.api.java.tuple.Tuple3;
 import eu.stratosphere.configuration.Configuration;
 
 @SuppressWarnings("serial")
-public class KMeansForTest implements Program{
+public class KMeansForTest implements Program {
 	
 	// *************************************************************************
 	//     PROGRAM
 	// *************************************************************************
 	
 	
+
 	@Override
 	public Plan getPlan(String... args) {
-		if(!parseParameters(args)) {
-			throw new RuntimeException("Unable to parse the arguments");
+		if (args.length < 4) {
+			throw new IllegalArgumentException("Missing parameters");
 		}
-	
-		// set up execution environment
-		ExecutionEnvironment env = new RemoteEnvironment("localhost", 1);
+		
+		final String pointsPath = args[0];
+		final String centersPath = args[1];
+		final String outputPath = args[2];
+		final int numIterations = Integer.parseInt(args[3]);
+		
+		
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setDegreeOfParallelism(4);
 		
 		// get input data
-		DataSet<Point> points = getPointDataSet(env);
-		DataSet<Centroid> centroids = getCentroidDataSet(env);
+		DataSet<Point> points = env.readCsvFile(pointsPath)
+				.fieldDelimiter('|')
+				.includeFields(true, true)
+				.types(Double.class, Double.class)
+				.map(new TuplePointConverter());
+		
+		DataSet<Centroid> centroids = env.readCsvFile(centersPath)
+				.fieldDelimiter('|')
+				.includeFields(true, true, true)
+				.types(Integer.class, Double.class, Double.class)
+				.map(new TupleCentroidConverter());
 		
 		// set number of bulk iterations for KMeans algorithm
 		IterativeDataSet<Centroid> loop = centroids.iterate(numIterations);
@@ -71,11 +86,8 @@ public class KMeansForTest implements Program{
 				.map(new SelectNearestCenter()).withBroadcastSet(finalCentroids, "centroids");
 		
 		// emit result
-		if(fileOutput) {
-			clusteredPoints.writeAsCsv(outputPath, "\n", " ");
-		} else {
-			clusteredPoints.print();
-		}
+		clusteredPoints.writeAsCsv(outputPath, "\n", " ");
+
 		return env.createProgramPlan();
 	}
 	
@@ -229,66 +241,4 @@ public class KMeansForTest implements Program{
 			return new Centroid(value.f0, value.f1.div(value.f2));
 		}
 	}
-	
-	// *************************************************************************
-	//     UTIL METHODS
-	// *************************************************************************
-	
-	private static boolean fileOutput = false;
-	private static String pointsPath = null;
-	private static String centersPath = null;
-	private static String outputPath = null;
-	private static int numIterations = 10;
-	
-	private static boolean parseParameters(String[] programArguments) {
-		
-		if(programArguments.length > 0) {
-			// parse input arguments
-			fileOutput = true;
-			if(programArguments.length == 4) {
-				pointsPath = programArguments[0];
-				centersPath = programArguments[1];
-				outputPath = programArguments[2];
-				numIterations = Integer.parseInt(programArguments[3]);
-			} else {
-				System.err.println("Usage: KMeans <points path> <centers path> <result path> <num iterations>");
-				return false;
-			}
-		} else {
-			System.out.println("Executing K-Means example with default parameters and built-in default data.");
-			System.out.println("  Provide parameters to read input data from files.");
-			System.out.println("  See the documentation for the correct format of input files.");
-			System.out.println("  We provide a data generator to create synthetic input files for this program.");
-			System.out.println("  Usage: KMeans <points path> <centers path> <result path> <num iterations>");
-		}
-		return true;
-	}
-	
-	private static DataSet<Point> getPointDataSet(ExecutionEnvironment env) {
-		if(fileOutput) {
-			// read points from CSV file
-			return env.readCsvFile(pointsPath)
-						.fieldDelimiter('|')
-						.includeFields(true, true)
-						.types(Double.class, Double.class)
-						.map(new TuplePointConverter());
-		} else {
-			throw new UnsupportedOperationException("Use file output");
-		}
-	}
-	
-	private static DataSet<Centroid> getCentroidDataSet(ExecutionEnvironment env) {
-		if(fileOutput) {
-			return env.readCsvFile(centersPath)
-						.fieldDelimiter('|')
-						.includeFields(true, true, true)
-						.types(Integer.class, Double.class, Double.class)
-						.map(new TupleCentroidConverter());
-		} else {
-			throw new UnsupportedOperationException("Use file output");
-		}
-	}
-
-	
-		
 }


[08/53] [abbrv] Rework the Taskmanager to a slot based model and remove legacy cloud code

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/cluster/AllocatedSlice.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/cluster/AllocatedSlice.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/cluster/AllocatedSlice.java
deleted file mode 100644
index 88d71e0..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/cluster/AllocatedSlice.java
+++ /dev/null
@@ -1,120 +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.instance.cluster;
-
-import eu.stratosphere.nephele.instance.AllocationID;
-import eu.stratosphere.nephele.instance.InstanceType;
-import eu.stratosphere.nephele.jobgraph.JobID;
-
-/**
- * An allocated slice is a part of an instance which is assigned to a job.
- * <p>
- * This class is thread-safe.
- * 
- */
-class AllocatedSlice {
-
-	/**
-	 * The allocation ID which identifies the resources occupied by this slice.
-	 */
-	private final AllocationID allocationID;
-
-	/**
-	 * The machine hosting the slice.
-	 */
-	private final ClusterInstance hostingInstance;
-
-	/**
-	 * The type describing the characteristics of the allocated slice.
-	 */
-	private final InstanceType type;
-
-	/**
-	 * The ID of the job this slice belongs to.
-	 */
-	private final JobID jobID;
-
-	/**
-	 * Time when this machine has been allocation in milliseconds, {@see currentTimeMillis()}.
-	 */
-	private final long allocationTime;
-
-	/**
-	 * Creates a new allocated slice on the given hosting instance.
-	 * 
-	 * @param hostingInstance
-	 *        the instance hosting the slice
-	 * @param type
-	 *        the type describing the characteristics of the allocated slice
-	 * @param jobID
-	 *        the ID of the job this slice belongs to
-	 * @param allocationTime
-	 *        the time the instance was allocated
-	 */
-	public AllocatedSlice(final ClusterInstance hostingInstance, final InstanceType type, final JobID jobID,
-			final long allocationTime) {
-
-		this.allocationID = new AllocationID();
-		this.hostingInstance = hostingInstance;
-		this.type = type;
-		this.jobID = jobID;
-		this.allocationTime = allocationTime;
-	}
-
-	/**
-	 * Returns the allocation ID of this slice.
-	 * 
-	 * @return the allocation ID of this slice
-	 */
-	public AllocationID getAllocationID() {
-		return this.allocationID;
-	}
-
-	/**
-	 * The type describing the characteristics of
-	 * this allocated slice.
-	 * 
-	 * @return the type describing the characteristics of the slice
-	 */
-	public InstanceType getType() {
-		return this.type;
-	}
-
-	/**
-	 * Returns the time the instance was allocated.
-	 * 
-	 * @return the time the instance was allocated
-	 */
-	public long getAllocationTime() {
-		return this.allocationTime;
-	}
-
-	/**
-	 * Returns the ID of the job this allocated slice belongs to.
-	 * 
-	 * @return the ID of the job this allocated slice belongs to
-	 */
-	public JobID getJobID() {
-		return this.jobID;
-	}
-
-	/**
-	 * Returns the instance hosting this slice.
-	 * 
-	 * @return the instance hosting this slice
-	 */
-	public ClusterInstance getHostingInstance() {
-		return this.hostingInstance;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/cluster/ClusterInstance.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/cluster/ClusterInstance.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/cluster/ClusterInstance.java
deleted file mode 100644
index 5c50bd3..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/cluster/ClusterInstance.java
+++ /dev/null
@@ -1,181 +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.instance.cluster;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-import eu.stratosphere.nephele.instance.AbstractInstance;
-import eu.stratosphere.nephele.instance.AllocationID;
-import eu.stratosphere.nephele.instance.HardwareDescription;
-import eu.stratosphere.nephele.instance.InstanceConnectionInfo;
-import eu.stratosphere.nephele.instance.InstanceType;
-import eu.stratosphere.nephele.instance.InstanceTypeFactory;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.topology.NetworkNode;
-import eu.stratosphere.nephele.topology.NetworkTopology;
-
-/**
- * Representation of a host of a compute cluster.
- * <p>
- * This class is thread-safe.
- * 
- */
-class ClusterInstance extends AbstractInstance {
-
-	/**
-	 * A map of slices allocated on this host.
-	 */
-	private final Map<AllocationID, AllocatedSlice> allocatedSlices = new HashMap<AllocationID, AllocatedSlice>();
-
-	/**
-	 * The remaining capacity of this host that can be used by instances.
-	 */
-	private InstanceType remainingCapacity;
-
-	/**
-	 * Time when last heat beat has been received from the task manager running on this instance.
-	 */
-	private long lastReceivedHeartBeat = System.currentTimeMillis();
-
-	/**
-	 * Constructs a new cluster instance.
-	 * 
-	 * @param instanceConnectionInfo
-	 *        the instance connection info identifying the host
-	 * @param capacity
-	 *        capacity of this host
-	 * @param parentNode
-	 *        the parent node of this node in the network topology
-	 * @param networkTopology
-	 *        the network topology this node is part of
-	 * @param hardwareDescription
-	 *        the hardware description reported by the instance itself
-	 */
-	public ClusterInstance(final InstanceConnectionInfo instanceConnectionInfo, final InstanceType capacity,
-			final NetworkNode parentNode, final NetworkTopology networkTopology,
-			final HardwareDescription hardwareDescription) {
-
-		super(capacity, instanceConnectionInfo, parentNode, networkTopology, hardwareDescription);
-
-		this.remainingCapacity = capacity;
-	}
-
-	/**
-	 * Updates the time of last received heart beat to the current system time.
-	 */
-	synchronized void reportHeartBeat() {
-		this.lastReceivedHeartBeat = System.currentTimeMillis();
-	}
-
-	/**
-	 * Returns whether the host is still alive.
-	 * 
-	 * @param cleanUpInterval
-	 *        duration (in milliseconds) after which a host is
-	 *        considered dead if it has no received heat-beats.
-	 * @return <code>true</code> if the host has received a heat-beat before the <code>cleanUpInterval</code> duration
-	 *         has expired, <code>false</code> otherwise
-	 */
-	synchronized boolean isStillAlive(final long cleanUpInterval) {
-
-		if (this.lastReceivedHeartBeat + cleanUpInterval < System.currentTimeMillis()) {
-			return false;
-		}
-		return true;
-	}
-
-	/**
-	 * Tries to create a new slice on this instance.
-	 * 
-	 * @param reqType
-	 *        the type describing the hardware characteristics of the slice
-	 * @param jobID
-	 *        the ID of the job the new slice belongs to
-	 * @return a new {@AllocatedSlice} object if a slice with the given hardware characteristics could
-	 *         still be accommodated on this instance or <code>null</code> if the instance's remaining resources
-	 *         were insufficient to host the desired slice
-	 */
-	synchronized AllocatedSlice createSlice(final InstanceType reqType, final JobID jobID) {
-
-		// check whether we can accommodate the instance
-		if (remainingCapacity.getNumberOfComputeUnits() >= reqType.getNumberOfComputeUnits()
-			&& remainingCapacity.getNumberOfCores() >= reqType.getNumberOfCores()
-			&& remainingCapacity.getMemorySize() >= reqType.getMemorySize()
-			&& remainingCapacity.getDiskCapacity() >= reqType.getDiskCapacity()) {
-
-			// reduce available capacity by what has been requested
-			remainingCapacity = InstanceTypeFactory.construct(remainingCapacity.getIdentifier(), remainingCapacity
-				.getNumberOfComputeUnits()
-				- reqType.getNumberOfComputeUnits(), remainingCapacity.getNumberOfCores() - reqType.getNumberOfCores(),
-				remainingCapacity.getMemorySize() - reqType.getMemorySize(), remainingCapacity.getDiskCapacity()
-					- reqType.getDiskCapacity(), remainingCapacity.getPricePerHour());
-
-			final long allocationTime = System.currentTimeMillis();
-
-			final AllocatedSlice slice = new AllocatedSlice(this, reqType, jobID, allocationTime);
-			this.allocatedSlices.put(slice.getAllocationID(), slice);
-			return slice;
-		}
-
-		// we cannot accommodate the instance
-		return null;
-	}
-
-	/**
-	 * Removes the slice identified by the given allocation ID from
-	 * this instance and frees up the allocated resources.
-	 * 
-	 * @param allocationID
-	 *        the allocation ID of the slice to be removed
-	 * @return the slice with has been removed from the instance or <code>null</code> if no slice
-	 *         with the given allocation ID could be found
-	 */
-	synchronized AllocatedSlice removeAllocatedSlice(final AllocationID allocationID) {
-
-		final AllocatedSlice slice = this.allocatedSlices.remove(allocationID);
-		if (slice != null) {
-
-			this.remainingCapacity = InstanceTypeFactory.construct(this.remainingCapacity.getIdentifier(),
-				this.remainingCapacity
-					.getNumberOfComputeUnits()
-					+ slice.getType().getNumberOfComputeUnits(), this.remainingCapacity.getNumberOfCores()
-					+ slice.getType().getNumberOfCores(), this.remainingCapacity.getMemorySize()
-					+ slice.getType().getMemorySize(), this.remainingCapacity.getDiskCapacity()
-					+ slice.getType().getDiskCapacity(), this.remainingCapacity.getPricePerHour());
-		}
-
-		return slice;
-	}
-
-	/**
-	 * Removes all allocated slices on this instance and frees
-	 * up their allocated resources.
-	 * 
-	 * @return a list of all removed slices
-	 */
-	synchronized List<AllocatedSlice> removeAllAllocatedSlices() {
-
-		final List<AllocatedSlice> slices = new ArrayList<AllocatedSlice>(this.allocatedSlices.values());
-		final Iterator<AllocatedSlice> it = slices.iterator();
-		while (it.hasNext()) {
-			removeAllocatedSlice(it.next().getAllocationID());
-		}
-
-		return slices;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/cluster/ClusterInstanceNotifier.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/cluster/ClusterInstanceNotifier.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/cluster/ClusterInstanceNotifier.java
deleted file mode 100644
index 39d2132..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/cluster/ClusterInstanceNotifier.java
+++ /dev/null
@@ -1,71 +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.instance.cluster;
-
-import java.util.List;
-
-import eu.stratosphere.nephele.instance.AllocatedResource;
-import eu.stratosphere.nephele.instance.InstanceListener;
-import eu.stratosphere.nephele.jobgraph.JobID;
-
-/**
- * This class is an auxiliary class to send the notification
- * about the availability of an {@link AbstractInstance} 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 ASSINING could not be guaranteed.
- * This class is thread-safe.
- * 
- */
-public class ClusterInstanceNotifier 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 ClusterInstanceNotifier(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/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/cluster/ClusterManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/cluster/ClusterManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/cluster/ClusterManager.java
deleted file mode 100644
index 480e521..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/cluster/ClusterManager.java
+++ /dev/null
@@ -1,945 +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.instance.cluster;
-
-import java.net.InetAddress;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.List;
-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 eu.stratosphere.configuration.ConfigConstants;
-import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.configuration.GlobalConfiguration;
-import eu.stratosphere.nephele.instance.AbstractInstance;
-import eu.stratosphere.nephele.instance.AllocatedResource;
-import eu.stratosphere.nephele.instance.HardwareDescription;
-import eu.stratosphere.nephele.instance.HardwareDescriptionFactory;
-import eu.stratosphere.nephele.instance.InstanceConnectionInfo;
-import eu.stratosphere.nephele.instance.InstanceException;
-import eu.stratosphere.nephele.instance.InstanceListener;
-import eu.stratosphere.nephele.instance.InstanceManager;
-import eu.stratosphere.nephele.instance.InstanceRequestMap;
-import eu.stratosphere.nephele.instance.InstanceType;
-import eu.stratosphere.nephele.instance.InstanceTypeDescription;
-import eu.stratosphere.nephele.instance.InstanceTypeDescriptionFactory;
-import eu.stratosphere.nephele.instance.InstanceTypeFactory;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.topology.NetworkNode;
-import eu.stratosphere.nephele.topology.NetworkTopology;
-import eu.stratosphere.nephele.util.SerializableHashMap;
-
-/**
- * Instance Manager for a static cluster.
- * <p>
- * The cluster manager can handle heterogeneous instances (compute nodes). Each instance type used in the cluster must
- * be described in the configuration.
- * <p>
- * This is a sample configuration: <code>
- * # definition of instances in format
- * # instancename,numComputeUnits,numCores,memorySize,diskCapacity,pricePerHour
- * instancemanager.cluster.type.1 = m1.small,2,1,2048,10,10
- * instancemanager.cluster.type. = c1.medium,2,1,2048,10,10
- * instancemanager.cluster.type. = m1.large,4,2,2048,10,10
- * instancemanager.cluster.type. = m1.xlarge,8,4,8192,20,20
- * instancemanager.cluster.type. = c1.xlarge,8,4,16384,20,40
- * 
- * # default instance type
- * instancemanager.cluster.defaulttype = 1 (pointing to m1.small)
- * </code> Each instance is expected to run exactly one {@link eu.stratosphere.nephele.taskmanager.TaskManager}. When
- * the {@link eu.stratosphere.nephele.taskmanager.TaskManager} registers with the
- * {@link eu.stratosphere.nephele.jobmanager.JobManager} it sends a {@link HardwareDescription} which describes the
- * actual hardware characteristics of the instance (compute node). The cluster manage will attempt to match the report
- * hardware characteristics with one of the configured instance types. Moreover, the cluster manager is capable of
- * partitioning larger instances (compute nodes) into smaller, less powerful instances.
- */
-public class ClusterManager implements InstanceManager {
-
-	// ------------------------------------------------------------------------
-	// Internal Constants
-	// ------------------------------------------------------------------------
-
-	/**
-	 * The log object used to report debugging and error information.
-	 */
-	private static final Log LOG = LogFactory.getLog(ClusterManager.class);
-
-	/**
-	 * Default duration after which a host is purged in case it did not send
-	 * a heart-beat message.
-	 */
-	private static final int DEFAULT_CLEANUP_INTERVAL = 2 * 60; // 2 min.
-
-	/**
-	 * The key to retrieve the clean up interval from the configuration.
-	 */
-	private static final String CLEANUP_INTERVAL_KEY = "instancemanager.cluster.cleanupinterval";
-
-	// ------------------------------------------------------------------------
-	// Fields
-	// ------------------------------------------------------------------------
-
-	private final Object lock = new Object();
-	
-	/**
-	 * Duration after which a host is purged in case it did not send a
-	 * heart-beat message.
-	 */
-	private final long cleanUpInterval;
-
-	/**
-	 * The default instance type.
-	 */
-	private final InstanceType defaultInstanceType;
-
-	/**
-	 * Set of hosts known to run a task manager that are thus able to execute
-	 * tasks.
-	 */
-	private final Map<InstanceConnectionInfo, ClusterInstance> registeredHosts;
-
-	/**
-	 * Map of a {@link JobID} to all {@link AllocatedSlice}s that belong to this job.
-	 */
-	private final Map<JobID, List<AllocatedSlice>> slicesOfJobs;
-
-	/**
-	 * List of instance types that can be executed on this cluster, sorted by
-	 * price (cheapest to most expensive).
-	 */
-	private final InstanceType[] availableInstanceTypes;
-
-	/**
-	 * Map of instance type descriptions which can be queried by the job manager.
-	 */
-	private final Map<InstanceType, InstanceTypeDescription> instanceTypeDescriptionMap;
-
-	/**
-	 * Map of IP addresses to instance types.
-	 */
-	private final Map<InetAddress, InstanceType> ipToInstanceTypeMapping = new HashMap<InetAddress, InstanceType>();
-
-	/**
-	 * Map of pending requests of a job, i.e. the instance requests that could not be fulfilled during the initial
-	 * instance request.
-	 */
-	private final Map<JobID, PendingRequestsMap> pendingRequestsOfJob = new LinkedHashMap<JobID, PendingRequestsMap>();
-
-	/**
-	 * The network topology of the cluster.
-	 */
-	private final NetworkTopology networkTopology;
-
-	/**
-	 * Object that is notified if instances become available or vanish.
-	 */
-	private InstanceListener instanceListener;
-
-	/**
-	 * Matrix storing how many instances of a particular type and be accommodated in another instance type.
-	 */
-	private final int[][] instanceAccommodationMatrix;
-
-	private boolean shutdown;
-	
-	/**
-	 * 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() {
-
-			synchronized (ClusterManager.this.lock) {
-
-				final List<Map.Entry<InstanceConnectionInfo, ClusterInstance>> hostsToRemove =
-					new ArrayList<Map.Entry<InstanceConnectionInfo, ClusterInstance>>();
-
-				final Map<JobID, List<AllocatedResource>> staleResources = new HashMap<JobID, List<AllocatedResource>>();
-
-				// check all hosts whether they did not send heat-beat messages.
-				for (Map.Entry<InstanceConnectionInfo, ClusterInstance> entry : registeredHosts.entrySet()) {
-
-					final ClusterInstance host = entry.getValue();
-					if (!host.isStillAlive(cleanUpInterval)) {
-
-						// this host has not sent the heat-beat messages
-						// -> we terminate all instances running on this host and notify the jobs
-						final List<AllocatedSlice> removedSlices = host.removeAllAllocatedSlices();
-						for (AllocatedSlice removedSlice : removedSlices) {
-
-							final JobID jobID = removedSlice.getJobID();
-							final List<AllocatedSlice> slicesOfJob = slicesOfJobs.get(jobID);
-							if (slicesOfJob == null) {
-								LOG.error("Cannot find allocated slices for job with ID + " + jobID);
-								continue;
-							}
-
-							slicesOfJob.remove(removedSlice);
-
-							// Clean up
-							if (slicesOfJob.isEmpty()) {
-								slicesOfJobs.remove(jobID);
-							}
-
-							List<AllocatedResource> staleResourcesOfJob = staleResources.get(removedSlice.getJobID());
-							if (staleResourcesOfJob == null) {
-								staleResourcesOfJob = new ArrayList<AllocatedResource>();
-								staleResources.put(removedSlice.getJobID(), staleResourcesOfJob);
-							}
-
-							staleResourcesOfJob.add(new AllocatedResource(removedSlice.getHostingInstance(),
-								removedSlice.getType(),
-								removedSlice.getAllocationID()));
-						}
-
-						hostsToRemove.add(entry);
-					}
-				}
-
-				registeredHosts.entrySet().removeAll(hostsToRemove);
-
-				updateInstaceTypeDescriptionMap();
-
-				final Iterator<Map.Entry<JobID, List<AllocatedResource>>> it = staleResources.entrySet().iterator();
-				while (it.hasNext()) {
-					final Map.Entry<JobID, List<AllocatedResource>> entry = it.next();
-					if (instanceListener != null) {
-						instanceListener.allocatedResourcesDied(entry.getKey(), entry.getValue());
-					}
-				}
-			}
-		}
-	};
-
-	// ------------------------------------------------------------------------
-	// Constructor and set-up
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Constructor.
-	 */
-	public ClusterManager() {
-
-		this.registeredHosts = new HashMap<InstanceConnectionInfo, ClusterInstance>();
-
-		this.slicesOfJobs = new HashMap<JobID, List<AllocatedSlice>>();
-
-		// Load the instance type this cluster can offer
-		this.defaultInstanceType = InstanceTypeFactory.constructFromDescription(ConfigConstants.DEFAULT_INSTANCE_TYPE);
-		
-		this.availableInstanceTypes = new InstanceType[] { this.defaultInstanceType };
-		
-		this.instanceAccommodationMatrix = calculateInstanceAccommodationMatrix();
-
-		this.instanceTypeDescriptionMap = new SerializableHashMap<InstanceType, InstanceTypeDescription>();
-
-		long tmpCleanUpInterval = (long) GlobalConfiguration.getInteger(CLEANUP_INTERVAL_KEY, DEFAULT_CLEANUP_INTERVAL) * 1000;
-
-		if (tmpCleanUpInterval < 10) { // Clean up interval must be at least ten seconds
-			LOG.warn("Invalid clean up interval. Reverting to default cleanup interval of " + DEFAULT_CLEANUP_INTERVAL
-				+ " secs.");
-			tmpCleanUpInterval = DEFAULT_CLEANUP_INTERVAL;
-		}
-
-		this.cleanUpInterval = tmpCleanUpInterval;
-
-		// sort available instances by CPU core
-		sortAvailableInstancesByNumberOfCPUCores();
-
-		this.networkTopology = NetworkTopology.createEmptyTopology();
-
-		// look every BASEINTERVAL milliseconds for crashed hosts
-		final boolean runTimerAsDaemon = true;
-		new Timer(runTimerAsDaemon).schedule(cleanupStaleMachines, 1000, 1000);
-
-		// Load available instance types into the instance description list
-		updateInstaceTypeDescriptionMap();
-	}
-
-	/**
-	 * Sorts the list of available instance types by the number of CPU cores in a descending order.
-	 */
-	private void sortAvailableInstancesByNumberOfCPUCores() {
-
-		if (this.availableInstanceTypes.length < 2) {
-			return;
-		}
-
-		for (int i = 1; i < this.availableInstanceTypes.length; i++) {
-			final InstanceType it = this.availableInstanceTypes[i];
-			int j = i;
-			while (j > 0 && this.availableInstanceTypes[j - 1].getNumberOfCores() < it.getNumberOfCores()) {
-				this.availableInstanceTypes[j] = this.availableInstanceTypes[j - 1];
-				--j;
-			}
-			this.availableInstanceTypes[j] = it;
-		}
-	}
-
-	@Override
-	public void shutdown() {
-		synchronized (this.lock) {
-			if (this.shutdown) {
-				return;
-			}
-			
-			this.cleanupStaleMachines.cancel();
-			
-			Iterator<ClusterInstance> it = this.registeredHosts.values().iterator();
-			while (it.hasNext()) {
-				it.next().destroyProxies();
-			}
-			this.registeredHosts.clear();
-			
-			this.shutdown = true;
-		}
-	}
-
-	@Override
-	public InstanceType getDefaultInstanceType() {
-		return this.defaultInstanceType;
-	}
-
-	@Override
-	public InstanceType getInstanceTypeByName(String instanceTypeName) {
-		synchronized (this.lock) {
-			for (InstanceType it : availableInstanceTypes) {
-				if (it.getIdentifier().equals(instanceTypeName)) {
-					return it;
-				}
-			}
-		}
-
-		return null;
-	}
-
-
-	@Override
-	public InstanceType getSuitableInstanceType(int minNumComputeUnits, int minNumCPUCores,
-			int minMemorySize, int minDiskCapacity, int maxPricePerHour)
-	{
-		// the instances are sorted by price -> the first instance that
-		// fulfills/ the requirements is suitable and the cheapest
-
-		synchronized (this.lock) {
-			for (InstanceType i : availableInstanceTypes) {
-				if (i.getNumberOfComputeUnits() >= minNumComputeUnits && i.getNumberOfCores() >= minNumCPUCores
-					&& i.getMemorySize() >= minMemorySize && i.getDiskCapacity() >= minDiskCapacity
-					&& i.getPricePerHour() <= maxPricePerHour) {
-					return i;
-				}
-			}
-		}
-		return null;
-	}
-
-
-	@Override
-	public void releaseAllocatedResource(JobID jobID, Configuration conf,
-			AllocatedResource allocatedResource) throws InstanceException
-	{
-		synchronized (this.lock) {
-			// release the instance from the host
-			final ClusterInstance clusterInstance = (ClusterInstance) allocatedResource.getInstance();
-			final AllocatedSlice removedSlice = clusterInstance.removeAllocatedSlice(allocatedResource.getAllocationID());
-
-			// remove the local association between instance and job
-			final List<AllocatedSlice> slicesOfJob = this.slicesOfJobs.get(jobID);
-			if (slicesOfJob == null) {
-				LOG.error("Cannot find allocated slice to release allocated slice for job " + jobID);
-				return;
-			}
-
-			slicesOfJob.remove(removedSlice);
-
-			// Clean up
-			if (slicesOfJob.isEmpty()) {
-				this.slicesOfJobs.remove(jobID);
-			}
-
-			// Check pending requests
-			checkPendingRequests();
-		}
-	}
-
-	/**
-	 * Creates a new {@link ClusterInstance} object to manage instances that can
-	 * be executed on that host.
-	 * 
-	 * @param instanceConnectionInfo
-	 *        the connection information for the instance
-	 * @param hardwareDescription
-	 *        the hardware description provided by the new instance
-	 * @return a new {@link ClusterInstance} object or <code>null</code> if the cluster instance could not be created
-	 */
-	private ClusterInstance createNewHost(final InstanceConnectionInfo instanceConnectionInfo,
-			final HardwareDescription hardwareDescription) {
-
-		// Check if there is a user-defined instance type for this IP address
-		InstanceType instanceType = this.ipToInstanceTypeMapping.get(instanceConnectionInfo.address());
-		if (instanceType != null) {
-			LOG.info("Found user-defined instance type for cluster instance with IP "
-				+ instanceConnectionInfo.address() + ": " + instanceType);
-		} else {
-			instanceType = matchHardwareDescriptionWithInstanceType(hardwareDescription);
-			if (instanceType != null) {
-				LOG.info("Hardware profile of cluster instance with IP " + instanceConnectionInfo.address()
-					+ " matches with instance type " + instanceType);
-			} else {
-				LOG.error("No matching instance type, cannot create cluster instance");
-				return null;
-			}
-		}
-
-		// Try to match new host with a stub host from the existing topology
-		String instanceName = instanceConnectionInfo.hostname();
-		NetworkNode parentNode = this.networkTopology.getRootNode();
-		NetworkNode currentStubNode = null;
-
-		// Try to match new host using the host name
-		while (true) {
-
-			currentStubNode = this.networkTopology.getNodeByName(instanceName);
-			if (currentStubNode != null) {
-				break;
-			}
-
-			final int pos = instanceName.lastIndexOf('.');
-			if (pos == -1) {
-				break;
-			}
-
-			/*
-			 * If host name is reported as FQDN, iterative remove parts
-			 * of the domain name until a match occurs or no more dots
-			 * can be found in the host name.
-			 */
-			instanceName = instanceName.substring(0, pos);
-		}
-
-		// Try to match the new host using the IP address
-		if (currentStubNode == null) {
-			instanceName = instanceConnectionInfo.address().toString();
-			instanceName = instanceName.replaceAll("/", ""); // Remove any / characters
-			currentStubNode = this.networkTopology.getNodeByName(instanceName);
-		}
-
-		if (currentStubNode != null) {
-			/*
-			 * The instance name will be the same as the one of the stub node. That way
-			 * the stub now will be removed from the network topology and replaced be
-			 * the new node.
-			 */
-			if (currentStubNode.getParentNode() != null) {
-				parentNode = currentStubNode.getParentNode();
-			}
-			// Remove the stub node from the tree
-			currentStubNode.remove();
-		}
-
-		LOG.info("Creating instance of type " + instanceType + " for " + instanceConnectionInfo + ", parent is "
-			+ parentNode.getName());
-		final ClusterInstance host = new ClusterInstance(instanceConnectionInfo, instanceType, parentNode,
-			this.networkTopology, hardwareDescription);
-
-		return host;
-	}
-
-	/**
-	 * Attempts to match the hardware characteristics provided by the {@link HardwareDescription} object with one
-	 * of the instance types set in the configuration. The matching is pessimistic, i.e. the hardware characteristics of
-	 * the chosen instance type never exceed the actually reported characteristics from the hardware description.
-	 * 
-	 * @param hardwareDescription
-	 *        the hardware description as reported by the instance
-	 * @return the best matching instance type or <code>null</code> if no matching instance type can be found
-	 */
-	private InstanceType matchHardwareDescriptionWithInstanceType(final HardwareDescription hardwareDescription) {
-
-		// Assumes that the available instance types are ordered by number of CPU cores in descending order
-		for (int i = 0; i < this.availableInstanceTypes.length; i++) {
-
-			final InstanceType candidateInstanceType = this.availableInstanceTypes[i];
-			// Check if number of CPU cores match
-			if (candidateInstanceType.getNumberOfCores() > hardwareDescription.getNumberOfCPUCores()) {
-				continue;
-			}
-
-			// Check if size of physical memory matches
-			final int memoryInMB = (int) (hardwareDescription.getSizeOfPhysicalMemory() / (1024L * 1024L));
-			if (candidateInstanceType.getMemorySize() > memoryInMB) {
-				continue;
-			}
-
-			return candidateInstanceType;
-		}
-
-		LOG.error("Cannot find matching instance type for hardware description ("
-			+ hardwareDescription.getNumberOfCPUCores() + " cores, " + hardwareDescription.getSizeOfPhysicalMemory()
-			+ " bytes of memory)");
-
-		return null;
-	}
-
-
-	@Override
-	public void reportHeartBeat(InstanceConnectionInfo instanceConnectionInfo, HardwareDescription hardwareDescription) {
-
-		synchronized (this.lock) {
-			ClusterInstance host = registeredHosts.get(instanceConnectionInfo);
-	
-			// check whether we have discovered a new host
-			if (host == null) {
-				host = createNewHost(instanceConnectionInfo, hardwareDescription);
-	
-				if (host == null) {
-					LOG.error("Could not create a new host object for incoming heart-beat. "
-						+ "Probably the configuration file is lacking some entries.");
-					return;
-				}
-	
-				this.registeredHosts.put(instanceConnectionInfo, host);
-				LOG.info("New number of registered hosts is " + this.registeredHosts.size());
-	
-				// Update the list of instance type descriptions
-				updateInstaceTypeDescriptionMap();
-	
-				// Check if a pending request can be fulfilled by the new host
-				checkPendingRequests();
-			}
-			
-			host.reportHeartBeat();
-		}
-	}
-
-	/**
-	 * Checks if a pending request can be fulfilled.
-	 */
-	private void checkPendingRequests() {
-
-		final Iterator<Map.Entry<JobID, PendingRequestsMap>> it = this.pendingRequestsOfJob.entrySet().iterator();
-		while (it.hasNext()) {
-
-			final List<AllocatedResource> allocatedResources = new ArrayList<AllocatedResource>();
-			final Map.Entry<JobID, PendingRequestsMap> entry = it.next();
-			final JobID jobID = entry.getKey();
-			final PendingRequestsMap pendingRequestsMap = entry.getValue();
-			final Iterator<Map.Entry<InstanceType, Integer>> it2 = pendingRequestsMap.iterator();
-			while (it2.hasNext()) {
-
-				final Map.Entry<InstanceType, Integer> entry2 = it2.next();
-				final InstanceType requestedInstanceType = entry2.getKey();
-				int numberOfPendingInstances = entry2.getValue().intValue();
-
-				// Consistency check
-				if (numberOfPendingInstances <= 0) {
-					LOG.error("Inconsistency: Job " + jobID + " has " + numberOfPendingInstances
-						+ " requests for instance type " + requestedInstanceType.getIdentifier());
-					continue;
-				}
-
-				while (numberOfPendingInstances > 0) {
-
-					if (LOG.isDebugEnabled()) {
-						LOG.debug("Trying to allocate instance of type " + requestedInstanceType.getIdentifier());
-					}
-
-					// TODO: Introduce topology awareness here
-					final AllocatedSlice slice = getSliceOfType(jobID, requestedInstanceType);
-					if (slice == null) {
-						break;
-					} else {
-
-						LOG.info("Allocated instance of type " + requestedInstanceType.getIdentifier()
-							+ " as a result of pending request for job " + jobID);
-
-						// Decrease number of pending instances
-						--numberOfPendingInstances;
-						pendingRequestsMap.decreaseNumberOfPendingInstances(requestedInstanceType);
-
-						List<AllocatedSlice> allocatedSlices = this.slicesOfJobs.get(jobID);
-						if (allocatedSlices == null) {
-							allocatedSlices = new ArrayList<AllocatedSlice>();
-							this.slicesOfJobs.put(jobID, allocatedSlices);
-						}
-						allocatedSlices.add(slice);
-
-						allocatedResources.add(new AllocatedResource(slice.getHostingInstance(), slice.getType(), slice
-							.getAllocationID()));
-					}
-				}
-			}
-
-			if (!allocatedResources.isEmpty() && this.instanceListener != null) {
-
-				final ClusterInstanceNotifier clusterInstanceNotifier = new ClusterInstanceNotifier(
-					this.instanceListener, jobID, allocatedResources);
-
-				clusterInstanceNotifier.start();
-			}
-		}
-	}
-
-	/**
-	 * Attempts to allocate a slice of the given type for the given job. The method first attempts to allocate this
-	 * slice by finding a physical host which exactly matches the given instance type. If this attempt failed, it tries
-	 * to allocate the slice by partitioning the resources of a more powerful host.
-	 * 
-	 * @param jobID
-	 *        the ID of the job the slice shall be allocated for
-	 * @param instanceType
-	 *        the instance type of the requested slice
-	 * @return the allocated slice or <code>null</code> if no such slice could be allocated
-	 */
-	private AllocatedSlice getSliceOfType(final JobID jobID, final InstanceType instanceType) {
-
-		AllocatedSlice slice = null;
-
-		// Try to match the instance type without slicing first
-		for (final ClusterInstance host : this.registeredHosts.values()) {
-			if (host.getType().equals(instanceType)) {
-				slice = host.createSlice(instanceType, jobID);
-				if (slice != null) {
-					break;
-				}
-			}
-		}
-
-		// Use slicing now if necessary
-		if (slice == null) {
-
-			for (final ClusterInstance host : this.registeredHosts.values()) {
-				slice = host.createSlice(instanceType, jobID);
-				if (slice != null) {
-					break;
-				}
-			}
-		}
-
-		return slice;
-	}
-
-
-	@Override
-	public void requestInstance(JobID jobID, Configuration conf,  InstanceRequestMap instanceRequestMap, List<String> splitAffinityList)
-		throws InstanceException
-	{
-		final List<AllocatedSlice> newlyAllocatedSlicesOfJob = new ArrayList<AllocatedSlice>();
-		final Map<InstanceType, Integer> pendingRequests = new HashMap<InstanceType, Integer>();
-
-		synchronized(this.lock) {
-			// Iterate over all instance types
-			for (Iterator<Map.Entry<InstanceType, Integer>> it = instanceRequestMap.getMaximumIterator(); it.hasNext();) {
-	
-				// Iterate over all requested instances of a specific type
-				final Map.Entry<InstanceType, Integer> entry = it.next();
-				final int maximumNumberOfInstances = entry.getValue().intValue();
-	
-				for (int i = 0; i < maximumNumberOfInstances; i++) {
-	
-					LOG.info("Trying to allocate instance of type " + entry.getKey().getIdentifier());
-					
-					final AllocatedSlice slice = getSliceOfType(jobID, entry.getKey());
-	
-					if (slice == null) {
-						if (i < instanceRequestMap.getMinimumNumberOfInstances(entry.getKey())) {
-							// The request cannot be fulfilled, release the slices again and throw an exception
-							for (final AllocatedSlice sliceToRelease : newlyAllocatedSlicesOfJob) {
-								sliceToRelease.getHostingInstance().removeAllocatedSlice(sliceToRelease.getAllocationID());
-							}
-	
-							// TODO: Remove previously allocated slices again
-							throw new InstanceException("Could not find a suitable instance");
-						} else {
-	
-							// Remaining instances are pending
-							final int numberOfRemainingInstances = maximumNumberOfInstances - i;
-							if (numberOfRemainingInstances > 0) {
-	
-								// Store the request for the missing instances
-								Integer val = pendingRequests.get(entry.getKey());
-								if (val == null) {
-									val = Integer.valueOf(0);
-								}
-								val = Integer.valueOf(val.intValue() + numberOfRemainingInstances);
-								pendingRequests.put(entry.getKey(), val);
-							}
-	
-							break;
-						}
-					}
-	
-					newlyAllocatedSlicesOfJob.add(slice);
-				}
-			}
-	
-			// The request could be processed successfully, so update internal bookkeeping.
-			List<AllocatedSlice> allAllocatedSlicesOfJob = this.slicesOfJobs.get(jobID);
-			if (allAllocatedSlicesOfJob == null) {
-				allAllocatedSlicesOfJob = new ArrayList<AllocatedSlice>();
-				this.slicesOfJobs.put(jobID, allAllocatedSlicesOfJob);
-			}
-			allAllocatedSlicesOfJob.addAll(newlyAllocatedSlicesOfJob);
-	
-			PendingRequestsMap allPendingRequestsOfJob = this.pendingRequestsOfJob.get(jobID);
-			if (allPendingRequestsOfJob == null) {
-				allPendingRequestsOfJob = new PendingRequestsMap();
-				this.pendingRequestsOfJob.put(jobID, allPendingRequestsOfJob);
-			}
-			for (final Iterator<Map.Entry<InstanceType, Integer>> it = pendingRequests.entrySet().iterator(); it.hasNext();) {
-				final Map.Entry<InstanceType, Integer> entry = it.next();
-	
-				allPendingRequestsOfJob.addRequest(entry.getKey(), entry.getValue().intValue());
-			}
-	
-			// Finally, create the list of allocated resources for the scheduler
-			final List<AllocatedResource> allocatedResources = new ArrayList<AllocatedResource>();
-			for (final AllocatedSlice slice : newlyAllocatedSlicesOfJob) {
-				allocatedResources.add(new AllocatedResource(slice.getHostingInstance(), slice.getType(), slice
-					.getAllocationID()));
-			}
-	
-			if (this.instanceListener != null) {
-				final ClusterInstanceNotifier clusterInstanceNotifier = new ClusterInstanceNotifier(
-					this.instanceListener, jobID, allocatedResources);
-				clusterInstanceNotifier.start();
-			}
-		}
-	}
-
-
-	@Override
-	public NetworkTopology getNetworkTopology(JobID jobID) {
-		return this.networkTopology;
-	}
-
-
-	@Override
-	public void setInstanceListener(InstanceListener instanceListener) {
-		synchronized (this.lock) {
-			this.instanceListener = instanceListener;
-		}
-	}
-
-
-	@Override
-	public Map<InstanceType, InstanceTypeDescription> getMapOfAvailableInstanceTypes() {
-		Map<InstanceType, InstanceTypeDescription> copyToReturn = new SerializableHashMap<InstanceType, InstanceTypeDescription>();
-		synchronized (this.lock) {
-			copyToReturn.putAll(this.instanceTypeDescriptionMap);
-		}
-		return copyToReturn;
-	}
-
-	/**
-	 * Updates the list of instance type descriptions based on the currently registered hosts.
-	 */
-	private void updateInstaceTypeDescriptionMap() {
-
-		// this.registeredHosts.values().iterator()
-		this.instanceTypeDescriptionMap.clear();
-
-		final List<InstanceTypeDescription> instanceTypeDescriptionList = new ArrayList<InstanceTypeDescription>();
-
-		// initialize array which stores the availability counter for each instance type
-		final int[] numberOfInstances = new int[this.availableInstanceTypes.length];
-		for (int i = 0; i < numberOfInstances.length; i++) {
-			numberOfInstances[i] = 0;
-		}
-
-		// Shuffle through instance types
-		for (int i = 0; i < this.availableInstanceTypes.length; i++) {
-
-			final InstanceType currentInstanceType = this.availableInstanceTypes[i];
-			int numberOfMatchingInstances = 0;
-			int minNumberOfCPUCores = Integer.MAX_VALUE;
-			long minSizeOfPhysicalMemory = Long.MAX_VALUE;
-			long minSizeOfFreeMemory = Long.MAX_VALUE;
-			final Iterator<ClusterInstance> it = this.registeredHosts.values().iterator();
-			while (it.hasNext()) {
-				final ClusterInstance clusterInstance = it.next();
-				if (clusterInstance.getType().equals(currentInstanceType)) {
-					++numberOfMatchingInstances;
-					final HardwareDescription hardwareDescription = clusterInstance.getHardwareDescription();
-					minNumberOfCPUCores = Math.min(minNumberOfCPUCores, hardwareDescription.getNumberOfCPUCores());
-					minSizeOfPhysicalMemory = Math.min(minSizeOfPhysicalMemory,
-						hardwareDescription.getSizeOfPhysicalMemory());
-					minSizeOfFreeMemory = Math.min(minSizeOfFreeMemory, hardwareDescription.getSizeOfFreeMemory());
-				}
-			}
-
-			// Update number of instances
-			int highestAccommodationNumber = -1;
-			int highestAccommodationIndex = -1;
-			for (int j = 0; j < this.availableInstanceTypes.length; j++) {
-				final int accommodationNumber = canBeAccommodated(j, i);
-				// LOG.debug(this.availableInstanceTypes[j].getIdentifier() + " fits into "
-				// + this.availableInstanceTypes[i].getIdentifier() + " " + accommodationNumber + " times");
-				if (accommodationNumber > 0) {
-					numberOfInstances[j] += numberOfMatchingInstances * accommodationNumber;
-					if (accommodationNumber > highestAccommodationNumber) {
-						highestAccommodationNumber = accommodationNumber;
-						highestAccommodationIndex = j;
-					}
-				}
-			}
-
-			// Calculate hardware description
-			HardwareDescription pessimisticHardwareDescription = null;
-			if (minNumberOfCPUCores < Integer.MAX_VALUE && minSizeOfPhysicalMemory < Long.MAX_VALUE
-				&& minSizeOfFreeMemory < Long.MAX_VALUE) {
-
-				pessimisticHardwareDescription = HardwareDescriptionFactory.construct(minNumberOfCPUCores,
-					minSizeOfPhysicalMemory, minSizeOfFreeMemory);
-
-			} else {
-
-				if (highestAccommodationIndex < i) { // Since highestAccommodationIndex smaller than my index, the
-														// target instance must be more powerful
-
-					final InstanceTypeDescription descriptionOfLargerInstanceType = instanceTypeDescriptionList
-						.get(highestAccommodationIndex);
-					if (descriptionOfLargerInstanceType.getHardwareDescription() != null) {
-						final HardwareDescription hardwareDescriptionOfLargerInstanceType = descriptionOfLargerInstanceType
-							.getHardwareDescription();
-
-						final int numCores = hardwareDescriptionOfLargerInstanceType.getNumberOfCPUCores()
-							/ highestAccommodationNumber;
-						final long physMem = hardwareDescriptionOfLargerInstanceType.getSizeOfPhysicalMemory()
-							/ highestAccommodationNumber;
-						final long freeMem = hardwareDescriptionOfLargerInstanceType.getSizeOfFreeMemory()
-							/ highestAccommodationNumber;
-
-						pessimisticHardwareDescription = HardwareDescriptionFactory.construct(numCores, physMem,
-							freeMem);
-					}
-				}
-			}
-
-			instanceTypeDescriptionList.add(InstanceTypeDescriptionFactory.construct(currentInstanceType,
-				pessimisticHardwareDescription, numberOfInstances[i]));
-		}
-
-		final Iterator<InstanceTypeDescription> it = instanceTypeDescriptionList.iterator();
-		while (it.hasNext()) {
-
-			final InstanceTypeDescription itd = it.next();
-			this.instanceTypeDescriptionMap.put(itd.getInstanceType(), itd);
-		}
-	}
-
-	/**
-	 * Calculates the instance accommodation matrix which stores how many times a particular instance type can be
-	 * accommodated inside another instance type based on the list of available instance types.
-	 * 
-	 * @return the instance accommodation matrix
-	 */
-	private int[][] calculateInstanceAccommodationMatrix() {
-
-		if (this.availableInstanceTypes == null) {
-			LOG.error("Cannot compute instance accommodation matrix: availableInstanceTypes is null");
-			return null;
-		}
-
-		final int matrixSize = this.availableInstanceTypes.length;
-		final int[][] am = new int[matrixSize][matrixSize];
-
-		// Populate matrix
-		for (int i = 0; i < matrixSize; i++) {
-			for (int j = 0; j < matrixSize; j++) {
-
-				if (i == j) {
-					am[i][j] = 1;
-				} else {
-
-					final InstanceType sourceType = this.availableInstanceTypes[i];
-					InstanceType targetType = this.availableInstanceTypes[j];
-
-					// How many times can we accommodate source type into target type?
-					final int cores = targetType.getNumberOfCores() / sourceType.getNumberOfCores();
-					final int cu = targetType.getNumberOfComputeUnits() / sourceType.getNumberOfComputeUnits();
-					final int mem = targetType.getMemorySize() / sourceType.getMemorySize();
-					final int disk = targetType.getDiskCapacity() / sourceType.getDiskCapacity();
-
-					am[i][j] = Math.min(cores, Math.min(cu, Math.min(mem, disk)));
-				}
-			}
-		}
-
-		return am;
-	}
-
-	/**
-	 * Returns how many times the instance type stored at index <code>sourceTypeIndex</code> can be accommodated inside
-	 * the instance type stored at index <code>targetTypeIndex</code> in the list of available instance types.
-	 * 
-	 * @param sourceTypeIndex
-	 *        the index of the source instance type in the list of available instance types
-	 * @param targetTypeIndex
-	 *        the index of the target instance type in the list of available instance types
-	 * @return the number of times the source type instance can be accommodated inside the target instance
-	 */
-	private int canBeAccommodated(int sourceTypeIndex, int targetTypeIndex) {
-
-		if (sourceTypeIndex >= this.availableInstanceTypes.length
-			|| targetTypeIndex >= this.availableInstanceTypes.length) {
-			LOG.error("Cannot determine number of instance accomodations: invalid index");
-			return 0;
-		}
-
-		return this.instanceAccommodationMatrix[targetTypeIndex][sourceTypeIndex];
-	}
-
-
-	@Override
-	public AbstractInstance getInstanceByName(String name) {
-		if (name == null) {
-			throw new IllegalArgumentException("Argument name must not be null");
-		}
-
-		synchronized (this.lock) {
-			final Iterator<ClusterInstance> it = this.registeredHosts.values().iterator();
-			while (it.hasNext()) {
-				final AbstractInstance instance = it.next();
-				if (name.equals(instance.getName())) {
-					return instance;
-				}
-			}
-		}
-
-		return null;
-	}
-
-
-	@Override
-	public void cancelPendingRequests(JobID jobID) {
-		synchronized (this.lock) {
-			this.pendingRequestsOfJob.remove(jobID);
-		}
-	}
-
-	@Override
-	public int getNumberOfTaskTrackers() {
-		return this.registeredHosts.size();
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/cluster/PendingRequestsMap.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/cluster/PendingRequestsMap.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/cluster/PendingRequestsMap.java
deleted file mode 100644
index ddc90e9..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/cluster/PendingRequestsMap.java
+++ /dev/null
@@ -1,97 +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.instance.cluster;
-
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-
-import eu.stratosphere.nephele.instance.InstanceType;
-
-/**
- * This class represents a pending request, i.e. a request for a particular type and number of {@link AbstractInstance}
- * objects which could not be fulfilled yet.
- * <p>
- * This class is not thread-safe.
- * 
- */
-public final class PendingRequestsMap {
-
-	/**
-	 * The map storing the pending instance requests for the job this pending request object belongs to.
-	 */
-	private final Map<InstanceType, Integer> pendingRequests = new HashMap<InstanceType, Integer>();
-
-	/**
-	 * Checks if the job this object belongs to has pending instance requests.
-	 * 
-	 * @return <code>true</code> if the job this object belongs to has pending instance requests, <code>false</code>
-	 *         otherwise
-	 */
-	boolean hasPendingRequests() {
-
-		return !(this.pendingRequests.isEmpty());
-	}
-
-	/**
-	 * Adds the a pending request for the given number of instances of the given type to this map.
-	 * 
-	 * @param instanceType
-	 *        the requested instance type
-	 * @param numberOfInstances
-	 *        the requested number of instances of this type
-	 */
-	void addRequest(final InstanceType instanceType, final int numberOfInstances) {
-
-		Integer numberOfRemainingInstances = this.pendingRequests.get(instanceType);
-		if (numberOfRemainingInstances == null) {
-			numberOfRemainingInstances = Integer.valueOf(numberOfInstances);
-		} else {
-			numberOfRemainingInstances = Integer.valueOf(numberOfRemainingInstances.intValue() + numberOfInstances);
-		}
-
-		this.pendingRequests.put(instanceType, numberOfRemainingInstances);
-	}
-
-	/**
-	 * Returns an iterator for the pending requests encapsulated in this map.
-	 * 
-	 * @return an iterator for the pending requests encapsulated in this map
-	 */
-	Iterator<Map.Entry<InstanceType, Integer>> iterator() {
-
-		return this.pendingRequests.entrySet().iterator();
-	}
-
-	/**
-	 * Decreases the number of remaining instances to request of the given type.
-	 * 
-	 * @param instanceType
-	 *        the instance type for which the number of remaining instances shall be decreased
-	 */
-	void decreaseNumberOfPendingInstances(final InstanceType instanceType) {
-
-		Integer numberOfRemainingInstances = this.pendingRequests.get(instanceType);
-		if (numberOfRemainingInstances == null) {
-			return;
-		}
-
-		numberOfRemainingInstances = Integer.valueOf(numberOfRemainingInstances.intValue() - 1);
-		if (numberOfRemainingInstances.intValue() == 0) {
-			this.pendingRequests.remove(instanceType);
-		} else {
-			this.pendingRequests.put(instanceType, numberOfRemainingInstances);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/local/LocalInstance.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/local/LocalInstance.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/local/LocalInstance.java
deleted file mode 100644
index 795889e..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/local/LocalInstance.java
+++ /dev/null
@@ -1,37 +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.instance.local;
-
-import eu.stratosphere.nephele.instance.AbstractInstance;
-import eu.stratosphere.nephele.instance.HardwareDescription;
-import eu.stratosphere.nephele.instance.InstanceConnectionInfo;
-import eu.stratosphere.nephele.instance.InstanceType;
-import eu.stratosphere.nephele.topology.NetworkNode;
-import eu.stratosphere.nephele.topology.NetworkTopology;
-
-public class LocalInstance extends AbstractInstance {
-
-	public LocalInstance(InstanceType instanceType, InstanceConnectionInfo instanceConnectionInfo,
-			NetworkNode parentNode, NetworkTopology networkTopology, HardwareDescription hardwareDescription) {
-		super(instanceType, instanceConnectionInfo, parentNode, networkTopology, hardwareDescription);
-	}
-
-
-	@Override
-	public String toString() {
-
-		return this.getInstanceConnectionInfo().toString();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/local/LocalInstanceManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/local/LocalInstanceManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/local/LocalInstanceManager.java
deleted file mode 100644
index e888b3f..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/local/LocalInstanceManager.java
+++ /dev/null
@@ -1,418 +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.instance.local;
-
-import java.io.File;
-import java.util.Map;
-import java.util.HashMap;
-import java.util.List;
-import java.util.ArrayList;
-import java.util.Iterator;
-
-import eu.stratosphere.nephele.ExecutionMode;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.configuration.ConfigConstants;
-import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.configuration.GlobalConfiguration;
-import eu.stratosphere.nephele.instance.AbstractInstance;
-import eu.stratosphere.nephele.instance.AllocatedResource;
-import eu.stratosphere.nephele.instance.AllocationID;
-import eu.stratosphere.nephele.instance.HardwareDescription;
-import eu.stratosphere.nephele.instance.HardwareDescriptionFactory;
-import eu.stratosphere.nephele.instance.InstanceConnectionInfo;
-import eu.stratosphere.nephele.instance.InstanceException;
-import eu.stratosphere.nephele.instance.InstanceListener;
-import eu.stratosphere.nephele.instance.InstanceManager;
-import eu.stratosphere.nephele.instance.InstanceRequestMap;
-import eu.stratosphere.nephele.instance.InstanceType;
-import eu.stratosphere.nephele.instance.InstanceTypeDescription;
-import eu.stratosphere.nephele.instance.InstanceTypeDescriptionFactory;
-import eu.stratosphere.nephele.instance.InstanceTypeFactory;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.taskmanager.TaskManager;
-import eu.stratosphere.nephele.topology.NetworkTopology;
-import eu.stratosphere.nephele.util.SerializableHashMap;
-
-/**
- * The local instance manager is designed to manage instance allocation/deallocation for a single-node setup. It spans a
- * task manager which is executed within the same process as the job manager. Moreover, it determines the hardware
- * characteristics of the machine it runs on and generates a default instance type with the identifier "default". If
- * desired this default instance type can also be overwritten.
- */
-public class LocalInstanceManager implements InstanceManager {
-
-	/**
-	 * The log object used to report events and errors.
-	 */
-	private static final Log LOG = LogFactory.getLog(LocalInstanceManager.class);
-
-	/**
-	 * The key for the configuration parameter defining the instance type to be used by the local instance manager. If
-	 * the parameter is not set, a default instance type with the identifier "default" is generated from the machine's
-	 * hardware characteristics.
-	 */
-
-	private static final String LOCALINSTANCE_TYPE_KEY = "instancemanager.local.type";
-
-	private static final int SLEEP_TIME = 50;
-
-	private static final int START_STOP_TIMEOUT = 2000;
-
-
-	/**
-	 * The instance listener registered with this instance manager.
-	 */
-	private InstanceListener instanceListener;
-
-	/**
-	 * The default instance type which is either generated from the hardware characteristics of the machine the local
-	 * instance manager runs on or read from the configuration.
-	 */
-	private final InstanceType defaultInstanceType;
-
-	/**
-	 * A synchronization object to protect critical sections.
-	 */
-	private final Object synchronizationObject = new Object();
-
-	/**
-	 * Stores which task manager is currently occupied by a job.
-	 */
-	private Map<LocalInstance, AllocatedResource> allocatedResources = new HashMap<LocalInstance, AllocatedResource>();
-
-	/**
-	 * The local instances encapsulating the task managers
-	 */
-	private Map<InstanceConnectionInfo, LocalInstance> localInstances = new HashMap<InstanceConnectionInfo,
-					LocalInstance>();
-
-	/**
-	 * The threads running the local task managers.
-	 */
-	private final List<TaskManager> taskManagers = new ArrayList<TaskManager>();
-
-	/**
-	 * The network topology the local instance is part of.
-	 */
-	private final NetworkTopology networkTopology;
-
-	/**
-	 * The map of instance type descriptions.
-	 */
-	private final Map<InstanceType, InstanceTypeDescription> instanceTypeDescriptionMap;
-
-	/**
-	 * Number of task managers
-	 */
-	private final int numTaskManagers;
-
-
-
-
-	/**
-	 * Constructs a new local instance manager.
-	 *
-	 */
-	public LocalInstanceManager() throws Exception {
-
-		final Configuration config = GlobalConfiguration.getConfiguration();
-
-		// get the default instance type
-		InstanceType type = null;
-		final String descr = config.getString(LOCALINSTANCE_TYPE_KEY, null);
-		if (descr != null) {
-			LOG.info("Attempting to parse default instance type from string " + descr);
-			type = InstanceTypeFactory.constructFromDescription(descr);
-			if (type == null) {
-				LOG.warn("Unable to parse default instance type from configuration, using hardware profile instead");
-			}
-		}
-
-		this.defaultInstanceType = (type != null) ? type : createDefaultInstanceType();
-
-		LOG.info("Default instance type is " + this.defaultInstanceType.getIdentifier());
-
-		this.networkTopology = NetworkTopology.createEmptyTopology();
-
-		this.instanceTypeDescriptionMap = new SerializableHashMap<InstanceType, InstanceTypeDescription>();
-
-		numTaskManagers = GlobalConfiguration.getInteger(ConfigConstants
-				.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, 1);
-
-		ExecutionMode executionMode = (numTaskManagers > 1) ? ExecutionMode.CLUSTER : ExecutionMode.LOCAL;
-
-		for(int i=0; i< numTaskManagers; i++){
-
-			Configuration tm = new Configuration();
-			int ipcPort = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY,
-					ConfigConstants.DEFAULT_TASK_MANAGER_IPC_PORT);
-			int dataPort = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
-					ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT);
-
-			tm.setInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, ipcPort + i);
-			tm.setInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, dataPort + i);
-
-			GlobalConfiguration.includeConfiguration(tm);
-
-			TaskManager t = new TaskManager(executionMode);
-			taskManagers.add(t);
-		}
-	}
-
-
-	@Override
-	public InstanceType getDefaultInstanceType() {
-		return this.defaultInstanceType;
-	}
-
-
-	@Override
-	public InstanceType getInstanceTypeByName(String instanceTypeName) {
-		if (this.defaultInstanceType.getIdentifier().equals(instanceTypeName)) {
-			return this.defaultInstanceType;
-		}
-
-		return null;
-	}
-
-
-	@Override
-	public InstanceType getSuitableInstanceType(int minNumComputeUnits, int minNumCPUCores,
-			int minMemorySize, int minDiskCapacity, int maxPricePerHour) {
-
-		if (minNumComputeUnits > this.defaultInstanceType.getNumberOfComputeUnits()) {
-			return null;
-		}
-
-		if (minNumCPUCores > this.defaultInstanceType.getNumberOfCores()) {
-			return null;
-		}
-
-		if (minMemorySize > this.defaultInstanceType.getMemorySize()) {
-			return null;
-		}
-
-		if (minDiskCapacity > this.defaultInstanceType.getDiskCapacity()) {
-			return null;
-		}
-
-		if (maxPricePerHour > this.defaultInstanceType.getPricePerHour()) {
-			return null;
-		}
-
-		return this.defaultInstanceType;
-	}
-
-
-	@Override
-	public void releaseAllocatedResource(final JobID jobID, final Configuration conf,
-			final AllocatedResource allocatedResource)
-			throws InstanceException {
-		LocalInstance instance = (LocalInstance) allocatedResource.getInstance();
-
-		synchronized (this.synchronizationObject) {
-			if(allocatedResources.containsKey(allocatedResource.getInstance())){
-				if(allocatedResources.get(instance).equals(allocatedResource)){
-					allocatedResources.remove(instance);
-					return;
-				}
-			}
-			throw new InstanceException("Resource with allocation ID " + allocatedResource.getAllocationID()
-					+ " has not been allocated to job with ID " + jobID
-					+ " according to the local instance manager's internal bookkeeping");
-
-		}
-	}
-
-
-	@Override
-	public void reportHeartBeat(final InstanceConnectionInfo instanceConnectionInfo,
-			final HardwareDescription hardwareDescription) {
-
-		synchronized (this.synchronizationObject) {
-			if(!localInstances.containsKey(instanceConnectionInfo)){
-				LocalInstance localInstance = new LocalInstance(this.defaultInstanceType, instanceConnectionInfo,
-						this.networkTopology.getRootNode(), this.networkTopology, hardwareDescription);
-				localInstances.put(instanceConnectionInfo, localInstance);
-
-				this.instanceTypeDescriptionMap.put(this.defaultInstanceType, InstanceTypeDescriptionFactory
-						.construct(this.defaultInstanceType, hardwareDescription, localInstances.size()));
-			}
-		}
-	}
-
-
-	@Override
-	public void shutdown() {
-		// Stop the task managers
-		for(TaskManager t : taskManagers){
-			t.shutdown();
-		}
-
-		boolean areAllTaskManagerShutdown = false;
-		int timeout = START_STOP_TIMEOUT * this.taskManagers.size();
-
-		for(int sleep = 0; sleep < timeout; sleep += SLEEP_TIME){
-			areAllTaskManagerShutdown = true;
-
-			for(TaskManager t: taskManagers){
-				if(!t.isShutDown()){
-					areAllTaskManagerShutdown = false;
-					break;
-				}
-			}
-
-			if(areAllTaskManagerShutdown){
-				break;
-			}
-
-			try {
-				Thread.sleep(SLEEP_TIME);
-			}catch(InterruptedException e){
-				break;
-			}
-		}
-
-		if(!areAllTaskManagerShutdown){
-			throw new RuntimeException(String.format("TaskManager shut down timed out (%d ms).", timeout));
-		}
-
-		instanceTypeDescriptionMap.clear();
-
-		synchronized(this.synchronizationObject){
-			for(LocalInstance instance: this.localInstances.values()){
-				instance.destroyProxies();
-			}
-
-			localInstances.clear();
-		}
-	}
-
-
-	@Override
-	public NetworkTopology getNetworkTopology(final JobID jobID) {
-		return this.networkTopology;
-	}
-
-
-	@Override
-	public void setInstanceListener(final InstanceListener instanceListener) {
-		this.instanceListener = instanceListener;
-	}
-
-	/**
-	 * Creates a default instance type based on the hardware characteristics of the machine that calls this method. The
-	 * default instance type contains the machine's number of CPU cores and size of physical memory. The disc capacity
-	 * is calculated from the free space in the directory for temporary files.
-	 * 
-	 * @return the default instance type used for the local machine
-	 */
-	public static final InstanceType createDefaultInstanceType() {
-		final HardwareDescription hardwareDescription = HardwareDescriptionFactory.extractFromSystem();
-
-		int diskCapacityInGB = 0;
-		final String[] tempDirs = GlobalConfiguration.getString(ConfigConstants.TASK_MANAGER_TMP_DIR_KEY,
-			ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(File.pathSeparator);
-		
-		for (final String tempDir : tempDirs) {
-			if (tempDir != null) {
-				File f = new File(tempDir);
-				diskCapacityInGB = Math.max(diskCapacityInGB, (int) (f.getFreeSpace() / (1024L * 1024L * 1024L)));
-			}
-		}
-
-		final int physicalMemory = (int) (hardwareDescription.getSizeOfPhysicalMemory() / (1024L * 1024L));
-
-		return InstanceTypeFactory.construct("default", hardwareDescription.getNumberOfCPUCores(),
-			hardwareDescription.getNumberOfCPUCores(), physicalMemory, diskCapacityInGB, 0);
-	}
-
-
-	@Override
-	public Map<InstanceType, InstanceTypeDescription> getMapOfAvailableInstanceTypes() {
-		return this.instanceTypeDescriptionMap;
-	}
-
-	@Override
-	public void requestInstance(final JobID jobID, final Configuration conf,
-			final InstanceRequestMap instanceRequestMap,
-			final List<String> splitAffinityList) throws InstanceException {
-
-		// TODO: This can be implemented way simpler...
-		// Iterate over all instance types
-		final Iterator<Map.Entry<InstanceType, Integer>> it = instanceRequestMap.getMinimumIterator();
-		final List<AllocatedResource> assignedResources = new ArrayList<AllocatedResource>();
-		boolean assignmentSuccessful = true;
-
-		while (it.hasNext()) {
-
-			// Iterate over all requested instances of a specific type
-			final Map.Entry<InstanceType, Integer> entry = it.next();
-
-			for (int i = 0; i < entry.getValue().intValue(); i++) {
-
-				synchronized (this.synchronizationObject) {
-					boolean instanceFound = false;
-					for(LocalInstance instance: localInstances.values()){
-						if(!allocatedResources.containsKey(instance)){
-							AllocatedResource assignedResource = new AllocatedResource(instance, entry.getKey(),
-									new AllocationID());
-							allocatedResources.put(instance, assignedResource);
-							assignedResources.add(assignedResource);
-							instanceFound = true;
-							break;
-						}
-					}
-
-					assignmentSuccessful &= instanceFound;
-				}
-			}
-		}
-
-		if(assignmentSuccessful){
-			new LocalInstanceNotifier(this.instanceListener, jobID, assignedResources).start();
-		}else{
-			throw new InstanceException("Could not satisfy instance request.");
-		}
-	}
-
-	@Override
-	public AbstractInstance getInstanceByName(final String name) {
-		if (name == null) {
-			throw new IllegalArgumentException("Argument name must not be null");
-		}
-
-		synchronized (this.synchronizationObject) {
-			for(LocalInstance instance :localInstances.values()){
-				if(name.equals(instance.getName())){
-					return instance;
-				}
-			}
-		}
-		return null;
-	}
-
-
-	@Override
-	public void cancelPendingRequests(final JobID jobID) {
-		// The local instance manager does not support pending requests, so nothing to do here
-	}
-
-	@Override
-	public int getNumberOfTaskTrackers() {
-		return localInstances.size();
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/local/LocalInstanceNotifier.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/local/LocalInstanceNotifier.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/local/LocalInstanceNotifier.java
deleted file mode 100644
index 52da691..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/local/LocalInstanceNotifier.java
+++ /dev/null
@@ -1,70 +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.instance.local;
-
-import java.util.List;
-
-import eu.stratosphere.nephele.instance.AllocatedResource;
-import eu.stratosphere.nephele.instance.InstanceListener;
-import eu.stratosphere.nephele.jobgraph.JobID;
-
-/**
- * This class is an auxiliary class to send the notification
- * about the availability of an {@link AllocatedResource} 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 ASSINING could not be guaranteed.
- * This class is thread-safe.
- * 
- */
-public class LocalInstanceNotifier extends Thread {
-
-	/**
-	 * The {@link InstanceListener} object to send the notification to.
-	 */
-	private final InstanceListener instanceListener;
-
-	/**
-	 * The ID of the job the new instance belongs to.
-	 */
-	private final JobID jobID;
-
-	/**
-	 * The resources allocated for the job.
-	 */
-	private final List<AllocatedResource> allocatedResources;
-
-	/**
-	 * Constructs a new instance notifier object.
-	 * 
-	 * @param instanceListener
-	 *        the listener object to send the notification to
-	 * @param jobID
-	 *        the ID of the job the newly allocated resources belongs to
-	 * @param allocatedResource
-	 *        the resources allocated for the job
-	 */
-	public LocalInstanceNotifier(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/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java
index b4c51f2..d64c622 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java
@@ -67,16 +67,6 @@ public abstract class AbstractJobVertex implements IOReadableWritable {
 	private int numberOfSubtasks = -1;
 
 	/**
-	 * The type of instance to be assigned to this task at runtime.
-	 */
-	private String instanceType = null;
-
-	/**
-	 * Number of subtasks to share the same instance at runtime.
-	 */
-	private int numberOfSubtasksPerInstance = -1;
-
-	/**
 	 * Number of retries in case of an error before the task represented by this vertex is considered as failed.
 	 */
 	private int numberOfExecutionRetries = -1;
@@ -150,8 +140,6 @@ public abstract class AbstractJobVertex implements IOReadableWritable {
 	 *        the vertex this vertex should connect to
 	 * @param channelType
 	 *        the channel type the two vertices should be connected by at runtime
-	 * @param compressionLevel
-	 *        the compression level the corresponding channel should have at runtime
 	 * @throws JobGraphDefinitionException
 	 *         thrown if the given vertex cannot be connected to <code>vertex</code> in the requested manner
 	 */
@@ -166,8 +154,8 @@ public abstract class AbstractJobVertex implements IOReadableWritable {
 	 *        the vertex this vertex should connect to
 	 * @param channelType
 	 *        the channel type the two vertices should be connected by at runtime
-	 * @param compressionLevel
-	 *        the compression level the corresponding channel should have 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
 	 */
@@ -184,14 +172,14 @@ public abstract class AbstractJobVertex implements IOReadableWritable {
 	 *        the vertex this vertex should connect to
 	 * @param channelType
 	 *        the channel type the two vertices should be connected by at runtime
-	 * @param compressionLevel
-	 *        the compression level the corresponding channel should have 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
 	 */
@@ -274,12 +262,12 @@ public abstract class AbstractJobVertex implements IOReadableWritable {
 	 *        the job vertex to connect to
 	 * @param channelType
 	 *        the channel type the two vertices should be connected by at runtime
-	 * @param compressionLevel
-	 *        the compression level the corresponding channel should have 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
 	 */
 	private void connectBacklink(final AbstractJobVertex vertex, final ChannelType channelType,
 			final int indexOfOutputGate, final int indexOfInputGate,
@@ -364,32 +352,6 @@ public abstract class AbstractJobVertex implements IOReadableWritable {
 	}
 
 	/**
-	 * Returns the index of the edge which is used to connect the given job vertex to this job vertex.
-	 * 
-	 * @param jv
-	 *        the connected job vertex
-	 * @return the index of the edge which is used to connect the given job vertex to this job vertex or -1 if the given
-	 *         vertex is not connected to this job vertex
-	 */
-	/*
-	 * public int getBackwardConnectionIndex(AbstractJobVertex jv) {
-	 * if(jv == null) {
-	 * return -1;
-	 * }
-	 * final Iterator<JobEdge> it = this.backwardEdges.iterator();
-	 * int i = 0;
-	 * while(it.hasNext()) {
-	 * final JobEdge edge = it.next();
-	 * if(edge.getConnectedVertex() == jv) {
-	 * return i;
-	 * }
-	 * i++;
-	 * }
-	 * return -1;
-	 * }
-	 */
-
-	/**
 	 * Returns the ID of this job vertex.
 	 * 
 	 * @return the ID of this job vertex
@@ -407,15 +369,9 @@ public abstract class AbstractJobVertex implements IOReadableWritable {
 			throw new IOException("jobGraph is null, cannot deserialize");
 		}
 
-		// Read instance type
-		this.instanceType = StringRecord.readString(in);
-
 		// Read number of subtasks
 		this.numberOfSubtasks = in.readInt();
 
-		// Read number of subtasks per instance
-		this.numberOfSubtasksPerInstance = in.readInt();
-
 		// Number of execution retries
 		this.numberOfExecutionRetries = in.readInt();
 
@@ -489,15 +445,9 @@ public abstract class AbstractJobVertex implements IOReadableWritable {
 	@Override
 	public void write(final DataOutput out) throws IOException {
 
-		// Instance type
-		StringRecord.writeString(out, this.instanceType);
-
 		// Number of subtasks
 		out.writeInt(this.numberOfSubtasks);
 
-		// Number of subtasks per instance
-		out.writeInt(this.numberOfSubtasksPerInstance);
-
 		// Number of execution retries
 		out.writeInt(this.numberOfExecutionRetries);
 
@@ -595,44 +545,6 @@ public abstract class AbstractJobVertex implements IOReadableWritable {
 	}
 
 	/**
-	 * Sets the instance type the task this vertex represents should run on.
-	 * 
-	 * @param instanceType
-	 *        the instance type the task this vertex represents should run on
-	 */
-	public void setInstanceType(final String instanceType) {
-		this.instanceType = instanceType;
-	}
-
-	/**
-	 * Returns the instance type the task this vertex represents should run on.
-	 * 
-	 * @return the instance type the task this vertex represents should run on, <code>null</code> if unspecified
-	 */
-	public String getInstanceType() {
-		return this.instanceType;
-	}
-
-	/**
-	 * Sets the number of subtasks that should be assigned to the same instance.
-	 * 
-	 * @param numberOfSubtasksPerInstance
-	 *        the number of subtasks that should be assigned to the same instance
-	 */
-	public void setNumberOfSubtasksPerInstance(final int numberOfSubtasksPerInstance) {
-		this.numberOfSubtasksPerInstance = numberOfSubtasksPerInstance;
-	}
-
-	/**
-	 * Returns the number of subtasks that should be assigned to the same instance, <code>-1</code> if undefined.
-	 * 
-	 * @return the number of subtasks that should be assigned to the same instance, <code>-1</code> if undefined
-	 */
-	public int getNumberOfSubtasksPerInstance() {
-		return this.numberOfSubtasksPerInstance;
-	}
-
-	/**
 	 * 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/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/DeploymentManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/DeploymentManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/DeploymentManager.java
index bf017ce..b043ecd 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/DeploymentManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/DeploymentManager.java
@@ -16,19 +16,19 @@ package eu.stratosphere.nephele.jobmanager;
 import java.util.List;
 
 import eu.stratosphere.nephele.executiongraph.ExecutionVertex;
-import eu.stratosphere.nephele.instance.AbstractInstance;
+import eu.stratosphere.nephele.instance.Instance;
 import eu.stratosphere.nephele.jobgraph.JobID;
 
 /**
  * A deployment manager is responsible for deploying a list of {@link ExecutionVertex} objects the given
- * {@link AbstractInstance}. It is called by a {@link AbstractScheduler} implementation whenever at least one
+ * {@link eu.stratosphere.nephele.instance.Instance}. It is called by a {@link eu.stratosphere.nephele.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 AbstractInstance}.
+	 * Deploys the list of vertices on the given {@link eu.stratosphere.nephele.instance.Instance}.
 	 * 
 	 * @param jobID
 	 *        the ID of the job the vertices to be deployed belong to
@@ -37,5 +37,5 @@ public interface DeploymentManager {
 	 * @param verticesToBeDeployed
 	 *        the list of vertices to be deployed
 	 */
-	void deploy(JobID jobID, AbstractInstance instance, List<ExecutionVertex> verticesToBeDeployed);
+	void deploy(JobID jobID, Instance instance, List<ExecutionVertex> verticesToBeDeployed);
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/EventCollector.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/EventCollector.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/EventCollector.java
index a6f9cfe..37f9a43 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/EventCollector.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/EventCollector.java
@@ -38,7 +38,7 @@ import eu.stratosphere.nephele.executiongraph.InternalJobStatus;
 import eu.stratosphere.nephele.executiongraph.JobStatusListener;
 import eu.stratosphere.nephele.executiongraph.ManagementGraphFactory;
 import eu.stratosphere.nephele.executiongraph.VertexAssignmentListener;
-import eu.stratosphere.nephele.instance.AbstractInstance;
+import eu.stratosphere.nephele.instance.Instance;
 import eu.stratosphere.nephele.instance.AllocatedResource;
 import eu.stratosphere.nephele.jobgraph.JobID;
 import eu.stratosphere.nephele.jobgraph.JobStatus;
@@ -266,10 +266,10 @@ public final class EventCollector extends TimerTask implements ProfilingListener
 			final ManagementVertexID managementVertexID = id.toManagementVertexID();
 			final long timestamp = System.currentTimeMillis();
 
-			final AbstractInstance instance = newAllocatedResource.getInstance();
+			final Instance instance = newAllocatedResource.getInstance();
 			VertexAssignmentEvent event;
 			if (instance == null) {
-				event = new VertexAssignmentEvent(timestamp, managementVertexID, "null", "null");
+				event = new VertexAssignmentEvent(timestamp, managementVertexID, "null");
 			} else {
 
 				String instanceName = null;
@@ -279,8 +279,7 @@ public final class EventCollector extends TimerTask implements ProfilingListener
 					instanceName = instance.toString();
 				}
 
-				event = new VertexAssignmentEvent(timestamp, managementVertexID, instanceName, instance.getType()
-					.getIdentifier());
+				event = new VertexAssignmentEvent(timestamp, managementVertexID, instanceName);
 			}
 
 			this.eventCollector.updateManagementGraph(jobID, event);
@@ -609,7 +608,6 @@ public final class EventCollector extends TimerTask implements ProfilingListener
 			}
 
 			vertex.setInstanceName(vertexAssignmentEvent.getInstanceName());
-			vertex.setInstanceType(vertexAssignmentEvent.getInstanceType());
 		}
 	}
 


[32/53] [abbrv] git commit: Add garbage collector stats to output of TaskManager memory usage debug thread

Posted by rm...@apache.org.
Add garbage collector stats to output of TaskManager memory usage debug thread


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

Branch: refs/heads/travis_test
Commit: 1357ea3a40f747e4e51b5b52612323fe55747da7
Parents: 7d0e89d
Author: uce <u....@fu-berlin.de>
Authored: Tue Jun 24 13:43:13 2014 +0200
Committer: uce <u....@fu-berlin.de>
Committed: Tue Jun 24 13:43:13 2014 +0200

----------------------------------------------------------------------
 .../nephele/taskmanager/TaskManager.java        | 35 ++++++++++++++++----
 .../exampleJavaPrograms/WordCountITCase.java    |  4 +++
 2 files changed, 32 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1357ea3a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
index 9b623bd..789955a 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
@@ -15,6 +15,7 @@ package eu.stratosphere.nephele.taskmanager;
 
 import java.io.File;
 import java.io.IOException;
+import java.lang.management.GarbageCollectorMXBean;
 import java.lang.management.ManagementFactory;
 import java.lang.management.MemoryMXBean;
 import java.lang.management.MemoryUsage;
@@ -408,9 +409,10 @@ public class TaskManager implements TaskOperationProtocol {
 		// Memory Usage
 		// --------------------------------------------------------------------
 
-		final MemoryMXBean memoryUsageBean = ManagementFactory.getMemoryMXBean();
+		final MemoryMXBean memoryMXBean = ManagementFactory.getMemoryMXBean();
+		final List<GarbageCollectorMXBean> gcMXBeans = ManagementFactory.getGarbageCollectorMXBeans();
 
-		LOG.info(getMemoryUsageAsString(memoryUsageBean));
+		LOG.info(getMemoryUsageStatsAsString(memoryMXBean));
 
 		boolean startMemoryUsageLogThread = GlobalConfiguration.getBoolean(
 				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
@@ -427,7 +429,10 @@ public class TaskManager implements TaskOperationProtocol {
 					try {
 						while (!isShutDown()) {
 							Thread.sleep(logIntervalMs);
-							LOG.debug(getMemoryUsageAsString(memoryUsageBean));
+
+							LOG.debug(getMemoryUsageStatsAsString(memoryMXBean));
+
+							LOG.debug(getGarbageCollectorStatsAsString(gcMXBeans));
 						}
 					} catch (InterruptedException e) {
 						LOG.warn("Unexpected interruption of memory usage logger thread.");
@@ -1120,9 +1125,9 @@ public class TaskManager implements TaskOperationProtocol {
 		}
 	}
 
-	private String getMemoryUsageAsString(MemoryMXBean memoryUsageBean) {
-		MemoryUsage heap = memoryUsageBean.getHeapMemoryUsage();
-		MemoryUsage nonHeap = memoryUsageBean.getNonHeapMemoryUsage();
+	private String getMemoryUsageStatsAsString(MemoryMXBean memoryMXBean) {
+		MemoryUsage heap = memoryMXBean.getHeapMemoryUsage();
+		MemoryUsage nonHeap = memoryMXBean.getNonHeapMemoryUsage();
 
 		int mb = 1 << 20;
 
@@ -1134,9 +1139,25 @@ public class TaskManager implements TaskOperationProtocol {
 		int nonHeapCommitted = (int) (nonHeap.getCommitted() / mb);
 		int nonHeapMax = (int) (nonHeap.getMax() / mb);
 
-		String msg = String.format("Memory usage HEAP: %d/%d/%d MB, NON HEAP: %d/%d/%d MB (used/comitted/max)",
+		String msg = String.format("Memory usage stats: [HEAP: %d/%d/%d MB, NON HEAP: %d/%d/%d MB (used/comitted/max)]",
 				heapUsed, heapCommitted, heapMax, nonHeapUsed, nonHeapCommitted, nonHeapMax);
 
 		return msg;
 	}
+
+	private String getGarbageCollectorStatsAsString(List<GarbageCollectorMXBean> gcMXBeans) {
+		StringBuilder str = new StringBuilder();
+		str.append("Garbage collector stats: ");
+
+		for (int i = 0; i < gcMXBeans.size(); i++) {
+			GarbageCollectorMXBean bean = gcMXBeans.get(i);
+
+			String msg = String.format("[%s, GC TIME (ms): %d, GC COUNT: %d]",
+					bean.getName(), bean.getCollectionTime(), bean.getCollectionCount());
+			str.append(msg);
+			str.append(i < gcMXBeans.size() - 1 ? ", " : "");
+		}
+
+		return str.toString();
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1357ea3a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/WordCountITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/WordCountITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/WordCountITCase.java
index 272bce6..ec67390 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/WordCountITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/WordCountITCase.java
@@ -17,6 +17,8 @@ package eu.stratosphere.test.exampleJavaPrograms;
 import eu.stratosphere.example.java.wordcount.WordCount;
 import eu.stratosphere.test.testdata.WordCountData;
 import eu.stratosphere.test.util.JavaProgramTestBase;
+import eu.stratosphere.util.LogUtils;
+import org.apache.log4j.Level;
 
 
 public class WordCountITCase extends JavaProgramTestBase {
@@ -25,6 +27,8 @@ public class WordCountITCase extends JavaProgramTestBase {
 	protected String resultPath;
 
 	public WordCountITCase(){
+		LogUtils.initializeDefaultConsoleLogger(Level.DEBUG);
+
 		setDegreeOfParallelism(4);
 		setNumTaskTracker(2);
 		setTaskManagerNumSlots(2);


[51/53] [abbrv] git commit: [FLINK-985] Update the config.md to properly describe the configuration settings.

Posted by rm...@apache.org.
[FLINK-985] Update the config.md to properly describe the configuration settings.

[ci skip]


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

Branch: refs/heads/travis_test
Commit: 47239b282f8fc350a05a4e8050ea931263140ae4
Parents: 9c51834
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Jun 25 19:42:52 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Jun 25 19:44:23 2014 +0200

----------------------------------------------------------------------
 docs/config.md | 222 +++++++++++++++++++++++++++++++++++-----------------
 1 file changed, 151 insertions(+), 71 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/47239b28/docs/config.md
----------------------------------------------------------------------
diff --git a/docs/config.md b/docs/config.md
index c11cc18..ddc579b 100644
--- a/docs/config.md
+++ b/docs/config.md
@@ -4,62 +4,125 @@ title:  "Configuration"
 
 # Overview
 
-This page provides an overview of possible settings for Stratosphere. All
-configuration is done in `conf/stratosphere-conf.yaml`, which is expected to be
+The default configuration parameters allow Flink to run out-of-the-box
+in single node setups.
+
+This page lists the most common options that are typically needed to set
+up a well performing (distributed) installation. In addition a full
+list of all available configuration parameters is listed here.
+
+All configuration is done in `conf/flink-conf.yaml`, which is expected to be
 a flat collection of [YAML key value pairs](http://www.yaml.org/spec/1.2/spec.html)
 with format `key: value`.
 
-The system and run scripts parse the config at startup and override the
-respective default values with the given values for every that has been set.
-This page contains a reference for all configuration keys used in the system.
+The system and run scripts parse the config at startup time. Changes to the configuration
+file require restarting the Flink JobManager and TaskManagers.
+
 
 # Common Options
 
 - `env.java.home`: The path to the Java installation to use (DEFAULT: system's
-default Java installation).
-- `jobmanager.rpc.address`: The IP address of the JobManager (DEFAULT:
-localhost).
+default Java installation, if found). Needs to be specified if the startup
+scipts fail to automatically resolve the java home directory. Can be specified
+to point to a specific java installation or version. If this option is not
+specified, the startup scripts also evaluate the `$JAVA_HOME` environment variable.
+
+- `jobmanager.rpc.address`: The IP address of the JobManager, which is the
+master/coordinator of the distributed system (DEFAULT: localhost).
+
 - `jobmanager.rpc.port`: The port number of the JobManager (DEFAULT: 6123).
+
 - `jobmanager.heap.mb`: JVM heap size (in megabytes) for the JobManager
 (DEFAULT: 256).
-- `taskmanager.heap.mb`: JVM heap size (in megabytes) for the TaskManager. In
-contrast to Hadoop, Stratosphere runs operators and functions inside the
-TaskManager (including sorting/hashing/caching), so this value should be as
-large as possible (DEFAULT: 512).
+
+- `taskmanager.heap.mb`: JVM heap size (in megabytes) for the TaskManagers,
+which are the parallel workers of the system. In
+contrast to Hadoop, Flink runs operators (e.g., join, aggregate) and
+user-defined functions (e.g., Map, Reduce, CoGroup) inside the TaskManager
+(including sorting/hashing/caching), so this value should be as
+large as possible (DEFAULT: 512). On YARN setups, this value is automatically
+configured to the size of the TaskManager's YARN container, minus a
+certain tolerance value.
+
+- `taskmanager.numberOfTaskSlots`: The number of parallel operator or
+UDF instances that a single TaskManager can run (DEFAULT: 1).
+If this value is larger than 1, a single TaskManager takes multiple instances of
+a function or operator. That way, the TaskManager can utilize multiple CPU cores,
+but at the same time, the available memory is divided between the different
+operator or function instances.
+This value is typically proportional to the number of physical CPU cores that
+the TaskManager's machine has (e.g., equal to the number of cores, or half the
+number of cores).
+
+- `parallelization.degree.default`: The default degree of parallelism to use for
+programs that have no degree of parallelism specified. (DEFAULT: 1). For
+setups that have no concurrent jobs running, setting this value to
+NumTaskManagers * NumSlotsPerTaskManager will cause the system to use all
+available execution resources for the program's execution.
+
+- `fs.hdfs.hadoopconf`: The absolute path to the Hadoop File System's (HDFS)
+configuration directory (OPTIONAL VALUE).
+Specifying this value allows programs to reference HDFS files using short URIs
+(`hdfs:///path/to/files`, without including the address and port of the NameNode
+in the file URI). Without this option, HDFS files can be accessed, but require
+fully qualified URIs like `hdfs://address:port/path/to/files`.
+This option also causes file writers to pick up the HDFS's default values for block sizes
+and replication factors. Flink will look for the "core-site.xml" and
+"hdfs-site.xml" files in teh specified directory.
+
+
+# Advanced Options
+
 - `taskmanager.tmp.dirs`: The directory for temporary files, or a list of
 directories separated by the systems directory delimiter (for example ':'
-(colon) on Linux/Unix). If multiple directories are specified then the temporary
-files will be distributed across the directories in a round robin fashion. The
+(colon) on Linux/Unix). If multiple directories are specified, then the temporary
+files will be distributed across the directories in a round-robin fashion. The
 I/O manager component will spawn one reading and one writing thread per
 directory. A directory may be listed multiple times to have the I/O manager use
 multiple threads for it (for example if it is physically stored on a very fast
 disc or RAID) (DEFAULT: The system's tmp dir).
-- `parallelization.degree.default`: The default degree of parallelism to use for
-programs that have no degree of parallelism specified. A value of -1 indicates
-no limit, in which the degree of parallelism is set to the number of available
-instances at the time of compilation (DEFAULT: -1).
-- `parallelization.intra-node.default`: The number of parallel instances of an
-operation that are assigned to each TaskManager. A value of -1 indicates no
-limit (DEFAULT: -1).
+
+- `jobmanager.web.port`: Port of the JobManager's web interface (DEFAULT: 8081).
+
+- `fs.overwrite-files`: Specifies whether file output writers should overwrite
+existing files by default. Set to *true* to overwrite by default, *false* otherwise.
+(DEFAULT: false)
+
+- `fs.output.always-create-directory`: File writers running with a parallelism
+larger than one create a directory for the output file path and put the different
+result files (one per parallel writer task) into that directory. If this option
+is set to *true*, writers with a parallelism of 1 will also create a directory
+and place a single result file into it. If the option is set to *false*, the
+writer will directly create the file directly at the output path, without
+creating a containing directory. (DEFAULT: false)
+
 - `taskmanager.network.numberOfBuffers`: The number of buffers available to the
 network stack. This number determines how many streaming data exchange channels
 a TaskManager can have at the same time and how well buffered the channels are.
 If a job is rejected or you get a warning that the system has not enough buffers
 available, increase this value (DEFAULT: 2048).
+
 - `taskmanager.memory.size`: The amount of memory (in megabytes) that the task
+manager reserves on the JVM's heap space for sorting, hash tables, and caching
+of intermediate results. If unspecified (-1), the memory manager will take a fixed
+ratio of the heap memory available to the JVM, as specified by
+`taskmanager.memory.fraction`. (DEFAULT: -1)
+
+- `taskmanager.memory.fraction`: The relative amount of memory that the task
 manager reserves for sorting, hash tables, and caching of intermediate results.
-If unspecified (-1), the memory manager will take a fixed ratio of the heap
-memory available to the JVM after the allocation of the network buffers (0.8)
-(DEFAULT: -1).
-- `jobmanager.profiling.enable`: Flag to enable job manager's profiling
-component. This collects network/cpu utilization statistics, which are displayed
-as charts in the SWT visualization GUI (DEFAULT: false).
-
-# HDFS
-
-These parameters configure the default HDFS used by Stratosphere. If you don't
-specify a HDFS configuration, you will have to specify the full path to your
-HDFS files like `hdfs://address:port/path/to/files` and filed with be written
+For example, a value of 0.8 means that TaskManagers reserve 80% of the
+JVM's heap space for internal data buffers, leaving 20% of the JVM's heap space
+free for objects created by user-defined functions. (DEFAULT: 0.7)
+This parameter is only evaluated, if `taskmanager.memory.size` is not set.
+
+
+# Full Reference
+
+## HDFS
+
+These parameters configure the default HDFS used by Flink. Setups that do not
+specify a HDFS configuration have to specify the full path to 
+HDFS files (`hdfs://address:port/path/to/files`) Files will also be written
 with default HDFS parameters (block size, replication factor).
 
 - `fs.hdfs.hadoopconf`: The absolute path to the Hadoop configuration directory.
@@ -70,34 +133,38 @@ directory (DEFAULT: null).
 - `fs.hdfs.hdfssite`: The absolute path of Hadoop's own configuration file
 "hdfs-site.xml" (DEFAULT: null).
 
-# JobManager &amp; TaskManager
-
-The following parameters configure Stratosphere's JobManager, TaskManager, and
-runtime channel management.
-
-- `jobmanager.rpc.address`: The hostname or IP address of the JobManager
-(DEFAULT: localhost).
-- `jobmanager.rpc.port`: The port of the JobManager (DEFAULT: 6123).
-- `jobmanager.rpc.numhandler`: The number of RPC threads for the JobManager.
-Increase those for large setups in which many TaskManagers communicate with the
-JobManager simultaneousl (DEFAULT: 8).
-- `jobmanager.profiling.enable`: Flag to enable the profiling component. This
-collects network/cpu utilization statistics, which are displayed as charts in
-the SWT visualization GUI. The profiling may add a small overhead on the
-execution (DEFAULT: false).
-- `jobmanager.web.port`: Port of the JobManager's web interface (DEFAULT: 8081).
-- `jobmanager.heap.mb`: JVM heap size (in megabytes) for the JobManager
-(DEFAULT: 256).
-- `taskmanager.heap.mb`: JVM heap size (in megabytes) for the TaskManager. In
-contrast to Hadoop, Stratosphere runs operators and functions inside the
-TaskManager (including sorting/hashing/caching), so this value should be as
-large as possible (DEFAULT: 512).
+## JobManager &amp; TaskManager
+
+The following parameters configure Flink's JobManager and TaskManagers.
+
+- `jobmanager.rpc.address`: The IP address of the JobManager, which is the
+master/coordinator of the distributed system (DEFAULT: localhost).
+- `jobmanager.rpc.port`: The port number of the JobManager (DEFAULT: 6123).
 - `taskmanager.rpc.port`: The task manager's IPC port (DEFAULT: 6122).
 - `taskmanager.data.port`: The task manager's port used for data exchange
 operations (DEFAULT: 6121).
+- `jobmanager.heap.mb`: JVM heap size (in megabytes) for the JobManager
+(DEFAULT: 256).
+- `taskmanager.heap.mb`: JVM heap size (in megabytes) for the TaskManagers,
+which are the parallel workers of the system. In
+contrast to Hadoop, Flink runs operators (e.g., join, aggregate) and
+user-defined functions (e.g., Map, Reduce, CoGroup) inside the TaskManager
+(including sorting/hashing/caching), so this value should be as
+large as possible (DEFAULT: 512). On YARN setups, this value is automatically
+configured to the size of the TaskManager's YARN container, minus a
+certain tolerance value.
+- `taskmanager.numberOfTaskSlots`: The number of parallel operator or
+UDF instances that a single TaskManager can run (DEFAULT: 1).
+If this value is larger than 1, a single TaskManager takes multiple instances of
+a function or operator. That way, the TaskManager can utilize multiple CPU cores,
+but at the same time, the available memory is divided between the different
+operator or function instances.
+This value is typically proportional to the number of physical CPU cores that
+the TaskManager's machine has (e.g., equal to the number of cores, or half the
+number of cores).
 - `taskmanager.tmp.dirs`: The directory for temporary files, or a list of
 directories separated by the systems directory delimiter (for example ':'
-(colon) on Linux/Unix). If multiple directories are specified then the temporary
+(colon) on Linux/Unix). If multiple directories are specified, then the temporary
 files will be distributed across the directories in a round robin fashion. The
 I/O manager component will spawn one reading and one writing thread per
 directory. A directory may be listed multiple times to have the I/O manager use
@@ -111,27 +178,25 @@ available, increase this value (DEFAULT: 2048).
 - `taskmanager.network.bufferSizeInBytes`: The size of the network buffers, in
 bytes (DEFAULT: 32768 (= 32 KiBytes)).
 - `taskmanager.memory.size`: The amount of memory (in megabytes) that the task
+manager reserves on the JVM's heap space for sorting, hash tables, and caching
+of intermediate results. If unspecified (-1), the memory manager will take a fixed
+ratio of the heap memory available to the JVM, as specified by
+`taskmanager.memory.fraction`. (DEFAULT: -1)
+- `taskmanager.memory.fraction`: The relative amount of memory that the task
 manager reserves for sorting, hash tables, and caching of intermediate results.
-If unspecified (-1), the memory manager will take a relative amount of the heap
-memory available to the JVM after the allocation of the network buffers (0.8)
-(DEFAULT: -1).
-- `taskmanager.memory.fraction`: The fraction of memory (after allocation of the
-network buffers) that the task manager reserves for sorting, hash tables, and
-caching of intermediate results. This value is only used if
-'taskmanager.memory.size' is unspecified (-1) (DEFAULT: 0.8).
+For example, a value of 0.8 means that TaskManagers reserve 80% of the
+JVM's heap space for internal data buffers, leaving 20% of the JVM's heap space
+free for objects created by user-defined functions. (DEFAULT: 0.7)
+This parameter is only evaluated, if `taskmanager.memory.size` is not set.
 - `jobclient.polling.interval`: The interval (in seconds) in which the client
 polls the JobManager for the status of its job (DEFAULT: 2).
 - `taskmanager.runtime.max-fan`: The maximal fan-in for external merge joins and
-fan-out for spilling hash tables. Limits the numer of file handles per operator,
+fan-out for spilling hash tables. Limits the number of file handles per operator,
 but may cause intermediate merging/partitioning, if set too small (DEFAULT: 128).
 - `taskmanager.runtime.sort-spilling-threshold`: A sort operation starts spilling
 when this fraction of its memory budget is full (DEFAULT: 0.8).
-- `taskmanager.runtime.fs_timeout`: The maximal time (in milliseconds) that the
-system waits for a response from the filesystem. Note that for HDFS, this time
-may occasionally be rather long. A value of 0 indicates infinite waiting time
-(DEFAULT: 0).
 
-# JobManager Web Frontend
+## JobManager Web Frontend
 
 - `jobmanager.web.port`: Port of the JobManager's web interface that displays
 status of running jobs and execution time breakdowns of finished jobs
@@ -139,7 +204,7 @@ status of running jobs and execution time breakdowns of finished jobs
 - `jobmanager.web.history`: The number of latest jobs that the JobManager's web
 front-end in its history (DEFAULT: 5).
 
-# Webclient
+## Webclient
 
 These parameters configure the web interface that can be used to submit jobs and
 review the compiler's execution plans.
@@ -154,7 +219,22 @@ uploaded programs (DEFAULT: ${webclient.tempdir}/webclient-jobs/).
 temporary JSON files describing the execution plans
 (DEFAULT: ${webclient.tempdir}/webclient-plans/).
 
-# Compiler/Optimizer
+## File Systems
+
+The parameters define the behavior of tasks that create result files.
+
+- `fs.overwrite-files`: Specifies whether file output writers should overwrite
+existing files by default. Set to *true* to overwrite by default, *false* otherwise.
+(DEFAULT: false)
+- `fs.output.always-create-directory`: File writers running with a parallelism
+larger than one create a directory for the output file path and put the different
+result files (one per parallel writer task) into that directory. If this option
+is set to *true*, writers with a parallelism of 1 will also create a directory
+and place a single result file into it. If the option is set to *false*, the
+writer will directly create the file directly at the output path, without
+creating a containing directory. (DEFAULT: false)
+
+## Compiler/Optimizer
 
 - `compiler.delimited-informat.max-line-samples`: The maximum number of line
 samples taken by the compiler for delimited inputs. The samples are used to


[12/53] [abbrv] git commit: Streamlined job graph algorithms to get rid off linear contains operations.

Posted by rm...@apache.org.
Streamlined job graph algorithms to get rid off linear contains operations.


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

Branch: refs/heads/travis_test
Commit: e52fcf90c37f921f50cd75dfcb7960d2f37c5e74
Parents: fba44a9
Author: Till Rohrmann <ti...@gmail.com>
Authored: Wed Apr 9 16:18:44 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sun Jun 22 21:07:20 2014 +0200

----------------------------------------------------------------------
 .../stratosphere/nephele/jobgraph/JobGraph.java | 120 +++++++------------
 1 file changed, 45 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e52fcf90/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobGraph.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobGraph.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobGraph.java
index 804a258..f048b0d 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobGraph.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobGraph.java
@@ -92,6 +92,11 @@ 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.
 	 */
 	public JobGraph() {
@@ -253,14 +258,51 @@ public class JobGraph implements IOReadableWritable {
 
 	/**
 	 * 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;
+	}
+
+	/**
+	 * 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);
 
-		final Vector<AbstractJobVertex> collector = new Vector<AbstractJobVertex>();
-		collectVertices(null, collector);
-		return collector.toArray(new AbstractJobVertex[0]);
+		for(int i =0; i < jv.getNumberOfForwardConnections(); i++){
+			AbstractJobVertex vertex = jv.getForwardConnection(i).getConnectedVertex();
+
+			if(!visited.contains(vertex.getID())){
+				collectVertices(vertex, visited, collector);
+			}
+		}
 	}
 
 	/**
@@ -293,34 +335,6 @@ public class JobGraph implements IOReadableWritable {
 		return vertices;
 	}
 
-	/**
-	 * 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 List<AbstractJobVertex> collector) {
-
-		if (jv == null) {
-			final Iterator<AbstractJobInputVertex> iter = getInputVertices();
-			while (iter.hasNext()) {
-				collectVertices(iter.next(), collector);
-			}
-		} else {
-
-			if (!collector.contains(jv)) {
-				collector.add(jv);
-			} else {
-				return;
-			}
-
-			for (int i = 0; i < jv.getNumberOfForwardConnections(); i++) {
-				collectVertices(jv.getForwardConnection(i).getConnectedVertex(), collector);
-			}
-		}
-	}
 
 	/**
 	 * Returns the ID of the job.
@@ -356,31 +370,6 @@ public class JobGraph implements IOReadableWritable {
 	}
 
 	/**
-	 * Checks if the job vertex with the given ID is registered with the job graph.
-	 * 
-	 * @param id
-	 *        the ID of the vertex to search for
-	 * @return <code>true</code> if a vertex with the given ID is registered with the job graph, <code>false</code>
-	 *         otherwise.
-	 */
-	private boolean includedInJobGraph(final JobVertexID id) {
-
-		if (this.inputVertices.containsKey(id)) {
-			return true;
-		}
-
-		if (this.outputVertices.containsKey(id)) {
-			return true;
-		}
-
-		if (this.taskVertices.containsKey(id)) {
-			return true;
-		}
-
-		return false;
-	}
-
-	/**
 	 * Checks if the job graph is weakly connected.
 	 * 
 	 * @return <code>true</code> if the job graph is weakly connected, otherwise <code>false</code>
@@ -395,25 +384,6 @@ public class JobGraph implements IOReadableWritable {
 			return false;
 		}
 
-		final HashMap<JobVertexID, AbstractJobVertex> tmp = new HashMap<JobVertexID, AbstractJobVertex>();
-		for (int i = 0; i < reachable.length; i++) {
-			tmp.put(reachable[i].getID(), reachable[i]);
-		}
-
-		// Check if all is subset of reachable
-		for (int i = 0; i < all.length; i++) {
-			if (!tmp.containsKey(all[i].getID())) {
-				return false;
-			}
-		}
-
-		// Check if reachable is a subset of all
-		for (int i = 0; i < reachable.length; i++) {
-			if (!includedInJobGraph(reachable[i].getID())) {
-				return false;
-			}
-		}
-
 		return true;
 	}
 


[17/53] [abbrv] git commit: Standardized creation of input and output channels in InputGate and OutputGate. Removed linear contains check in InputGate for channels.

Posted by rm...@apache.org.
Standardized creation of input and output channels in InputGate and OutputGate. Removed linear contains check in InputGate for channels.


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

Branch: refs/heads/travis_test
Commit: 26926433cbb82ee1789622b6f9baf0638907a69e
Parents: e52fcf9
Author: Till Rohrmann <ti...@gmail.com>
Authored: Wed Apr 9 19:35:57 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sun Jun 22 21:07:20 2014 +0200

----------------------------------------------------------------------
 .../eu/stratosphere/nephele/execution/RuntimeEnvironment.java  | 6 ------
 1 file changed, 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/26926433/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java
index 70718a9..2416b07 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java
@@ -83,12 +83,6 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 	private final List<InputGate<? extends IOReadableWritable>> inputGates = new CopyOnWriteArrayList<InputGate<? extends IOReadableWritable>>();
 
 	/**
-	 * Queue of unbound output gate IDs which are required for deserializing an environment in the course of an RPC
-	 * call.
-	 */
-	private final Queue<GateID> unboundOutputGateIDs = new ArrayDeque<GateID>();
-
-	/**
 	 * Queue of unbound input gate IDs which are required for deserializing an environment in the course of an RPC
 	 * call.
 	 */


[43/53] [abbrv] git commit: Update Quickstarts Java API, Run Example, and Setup Quickstart

Posted by rm...@apache.org.
Update Quickstarts Java API, Run Example, and Setup Quickstart

This closes #39.


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

Branch: refs/heads/travis_test
Commit: f7f1ed2fe879f9f1a11989787f26e23d01603a70
Parents: 515ad3c
Author: Sebastian Kunert <sk...@gmail.com>
Authored: Tue Jun 24 15:42:07 2014 +0200
Committer: uce <u....@fu-berlin.de>
Committed: Wed Jun 25 12:43:34 2014 +0200

----------------------------------------------------------------------
 .../compiler-webclient-new.png                  | Bin 0 -> 192965 bytes
 .../jobmanager-running-new.png                  | Bin 0 -> 143924 bytes
 docs/img/quickstart-example/kmeans003.png       | Bin 0 -> 71309 bytes
 docs/img/quickstart-example/kmeans008.png       | Bin 0 -> 91857 bytes
 docs/img/quickstart-example/kmeans015.png       | Bin 0 -> 95171 bytes
 docs/img/quickstart-example/result003.png       | Bin 0 -> 57838 bytes
 docs/img/quickstart-example/result008.png       | Bin 0 -> 82928 bytes
 docs/img/quickstart-example/result015.png       | Bin 0 -> 88338 bytes
 docs/img/quickstart-example/run-webclient.png   | Bin 0 -> 84682 bytes
 docs/img/quickstart_cluster.png                 | Bin 0 -> 83790 bytes
 docs/java_api_quickstart.md                     | 133 ++++++------
 docs/run_example_quickstart.md                  | 156 ++++++---------
 docs/setup_quickstart.md                        | 200 ++++++++-----------
 13 files changed, 211 insertions(+), 278 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f7f1ed2f/docs/img/quickstart-example/compiler-webclient-new.png
----------------------------------------------------------------------
diff --git a/docs/img/quickstart-example/compiler-webclient-new.png b/docs/img/quickstart-example/compiler-webclient-new.png
new file mode 100644
index 0000000..1141de1
Binary files /dev/null and b/docs/img/quickstart-example/compiler-webclient-new.png differ

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f7f1ed2f/docs/img/quickstart-example/jobmanager-running-new.png
----------------------------------------------------------------------
diff --git a/docs/img/quickstart-example/jobmanager-running-new.png b/docs/img/quickstart-example/jobmanager-running-new.png
new file mode 100644
index 0000000..5bcf9e1
Binary files /dev/null and b/docs/img/quickstart-example/jobmanager-running-new.png differ

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f7f1ed2f/docs/img/quickstart-example/kmeans003.png
----------------------------------------------------------------------
diff --git a/docs/img/quickstart-example/kmeans003.png b/docs/img/quickstart-example/kmeans003.png
new file mode 100644
index 0000000..ab9a61b
Binary files /dev/null and b/docs/img/quickstart-example/kmeans003.png differ

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f7f1ed2f/docs/img/quickstart-example/kmeans008.png
----------------------------------------------------------------------
diff --git a/docs/img/quickstart-example/kmeans008.png b/docs/img/quickstart-example/kmeans008.png
new file mode 100644
index 0000000..c2e2b81
Binary files /dev/null and b/docs/img/quickstart-example/kmeans008.png differ

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f7f1ed2f/docs/img/quickstart-example/kmeans015.png
----------------------------------------------------------------------
diff --git a/docs/img/quickstart-example/kmeans015.png b/docs/img/quickstart-example/kmeans015.png
new file mode 100644
index 0000000..3f0873a
Binary files /dev/null and b/docs/img/quickstart-example/kmeans015.png differ

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f7f1ed2f/docs/img/quickstart-example/result003.png
----------------------------------------------------------------------
diff --git a/docs/img/quickstart-example/result003.png b/docs/img/quickstart-example/result003.png
new file mode 100644
index 0000000..0b3c502
Binary files /dev/null and b/docs/img/quickstart-example/result003.png differ

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f7f1ed2f/docs/img/quickstart-example/result008.png
----------------------------------------------------------------------
diff --git a/docs/img/quickstart-example/result008.png b/docs/img/quickstart-example/result008.png
new file mode 100644
index 0000000..fe215ad
Binary files /dev/null and b/docs/img/quickstart-example/result008.png differ

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f7f1ed2f/docs/img/quickstart-example/result015.png
----------------------------------------------------------------------
diff --git a/docs/img/quickstart-example/result015.png b/docs/img/quickstart-example/result015.png
new file mode 100644
index 0000000..d0428ac
Binary files /dev/null and b/docs/img/quickstart-example/result015.png differ

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f7f1ed2f/docs/img/quickstart-example/run-webclient.png
----------------------------------------------------------------------
diff --git a/docs/img/quickstart-example/run-webclient.png b/docs/img/quickstart-example/run-webclient.png
new file mode 100644
index 0000000..e86bbe4
Binary files /dev/null and b/docs/img/quickstart-example/run-webclient.png differ

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f7f1ed2f/docs/img/quickstart_cluster.png
----------------------------------------------------------------------
diff --git a/docs/img/quickstart_cluster.png b/docs/img/quickstart_cluster.png
new file mode 100644
index 0000000..9e24795
Binary files /dev/null and b/docs/img/quickstart_cluster.png differ

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f7f1ed2f/docs/java_api_quickstart.md
----------------------------------------------------------------------
diff --git a/docs/java_api_quickstart.md b/docs/java_api_quickstart.md
index 75f4c7c..6a4967e 100644
--- a/docs/java_api_quickstart.md
+++ b/docs/java_api_quickstart.md
@@ -2,72 +2,68 @@
 title: "Quickstart: Java API"
 ---
 
-<p class="lead">Start working on your Stratosphere Java program in a few simple steps.</p>
-
-<section id="requirements">
-  <div class="page-header"><h2>Requirements</h2></div>
-  <p class="lead">The only requirements are working <strong>Maven 3.0.4</strong> (or higher) and <strong>Java 6.x</strong> (or higher) installations.</p>
-</section>
-
-<section id="create_project">
-  <div class="page-header"><h2>Create Project</h2></div>
-
-  <p class="lead">Use one of the following commands to <strong>create a project</strong>:</p>
-
-  <ul class="nav nav-tabs" style="border-bottom: none;">
-      <li class="active"><a href="#quickstart-script" data-toggle="tab">Run the <strong>quickstart script</strong></a></li>
-      <li><a href="#maven-archetype" data-toggle="tab">Use <strong>Maven archetypes</strong></a></li>
-  </ul>
-  <div class="tab-content">
-      <div class="tab-pane active" id="quickstart-script">
-{% highlight bash %}
-$ curl https://raw.githubusercontent.com/stratosphere/stratosphere-quickstart/master/quickstart.sh | bash
-{% endhighlight %}
-      </div>
-      <div class="tab-pane" id="maven-archetype">
-{% highlight bash %}
-$ mvn archetype:generate                             \
-    -DarchetypeGroupId=eu.stratosphere               \
-    -DarchetypeArtifactId=quickstart-java            \
-    -DarchetypeVersion={{site.current_stable}}
-{% endhighlight %}
-      This allows you to <strong>name your newly created project</strong>. It will interactively ask you for the groupId, artifactId, and package name.
-      </div>
-  </div>
-</section>
-
-<section id="inspect_project">
-  <div class="page-header"><h2>Inspect Project</h2></div>
-  <p class="lead">There will be a <strong>new directory in your working directory</strong>. If you've used the <em>curl</em> approach, the directory is called <code>quickstart</code>. Otherwise, it has the name of your artifactId.</p>
-  <p class="lead">The sample project is a <strong>Maven project</strong>, which contains two classes. <em>Job</em> is a basic skeleton program and <em>WordCountJob</em> a working example. Please note that the <em>main</em> method of both classes allow you to start Stratosphere in a development/testing mode.</p>
-  <p class="lead">We recommend to <strong>import this project into your IDE</strong> to develop and test it. If you use Eclipse, the <a href="http://www.eclipse.org/m2e/">m2e plugin</a> allows to <a href="http://books.sonatype.com/m2eclipse-book/reference/creating-sect-importing-projects.html#fig-creating-import">import Maven projects</a>. Some Eclipse bundles include that plugin by default, other require you to install it manually. The IntelliJ IDE also supports Maven projects out of the box.</p>
-</section>
-
-<section id="build_project">
-<div class="alert alert-danger">A note to Mac OS X users: The default JVM heapsize for Java is too small for Stratosphere. You have to manually increase it. Choose "Run Configurations" -> Arguments and write into the "VM Arguments" box: "-Xmx800m" in Eclipse.</div>
-  <div class="page-header"><h2>Build Project</h2></div>
-  <p class="lead">If you want to <strong>build your project</strong>, go to your project directory and issue the <code>mvn clean package</code> command. You will <strong>find a jar</strong> that runs on every Stratosphere cluster in <code>target/stratosphere-project-0.1-SNAPSHOT.jar</code>.</p>
-</section>
-
-<section id="next_steps">
-  <div class="page-header"><h2>Next Steps</h2></div>
-  <p class="lead"><strong>Write your application!</strong></p>
-  <p>The quickstart project contains a WordCount implementation, the "Hello World" of Big Data processing systems. The goal of WordCount is to determine the frequencies of words in a text, e.g., how often do the terms "the" or "house" occurs in all Wikipedia texts.</p>
- <br>
-<b>Sample Input:</b> <br>
-{% highlight bash %}
+Start working on your Stratosphere Java program in a few simple steps.
+
+
+# Requirements
+The only requirements are working __Maven 3.0.4__ (or higher) and __Java 6.x__ (or higher) installations.
+
+# Create Project
+Use one of the following commands to __create a project__:
+
+<ul class="nav nav-tabs" style="border-bottom: none;">
+    <li class="active"><a href="#quickstart-script" data-toggle="tab">Run the <strong>quickstart script</strong></a></li>
+    <li><a href="#maven-archetype" data-toggle="tab">Use <strong>Maven archetypes</strong></a></li>
+</ul>
+<div class="tab-content">
+    <div class="tab-pane active" id="quickstart-script">
+    {% highlight bash %}
+    $ curl https://raw.githubusercontent.com/stratosphere/stratosphere-quickstart/master/quickstart.sh | bash
+    {% endhighlight %}
+    </div>
+    <div class="tab-pane" id="maven-archetype">
+    {% highlight bash %}
+    $ mvn archetype:generate                             \
+      -DarchetypeGroupId=eu.stratosphere               \
+      -DarchetypeArtifactId=quickstart-java            \
+      -DarchetypeVersion={{site.current_stable}}
+    {% endhighlight %}
+        This allows you to <strong>name your newly created project</strong>. It will interactively ask you for the groupId, artifactId, and package name.
+    </div>
+</div>
+
+# Inspect Project
+There will be a new directory in your working directory. If you've used the _curl_ approach, the directory is called `quickstart`. Otherwise, it has the name of your artifactId.
+
+The sample project is a __Maven project__, which contains two classes. _Job_ is a basic skeleton program and _WordCountJob_ a working example. Please note that the _main_ method of both classes allow you to start Stratosphere in a development/testing mode.
+
+We recommend to __import this project into your IDE__ to develop and test it. If you use Eclipse, the [m2e plugin](http://www.eclipse.org/m2e/) allows to [import Maven projects](http://books.sonatype.com/m2eclipse-book/reference/creating-sect-importing-projects.html#fig-creating-import). Some Eclipse bundles include that plugin by default, other require you to install it manually. The IntelliJ IDE also supports Maven projects out of the box.
+
+
+A note to Mac OS X users: The default JVM heapsize for Java is too small for Stratosphere. You have to manually increase it. Choose "Run Configurations" -> Arguments and write into the "VM Arguments" box: "-Xmx800m" in Eclipse.
+
+# Build Project
+If you want to __build your project__, go to your project directory and issue the `mvn clean package` command. You will __find a jar__ that runs on every Stratosphere cluster in `target/stratosphere-project-0.1-SNAPSHOT.jar`.
+
+# Next Steps
+Write your application!
+
+The quickstart project contains a WordCount implementation, the "Hello World" of Big Data processing systems. The goal of WordCount is to determine the frequencies of words in a text, e.g., how often do the terms "the" or "house" occurs in all Wikipedia texts.
+
+__Sample Input__:
+```bash
 big data is big
-{% endhighlight %}
-<b>Sample Output:</b> <br>
-{% highlight bash %}
+```
+
+__Sample Output__:
+```bash
 big 2
 data 1
 is 1
-{% endhighlight %}
-
-<p>The following code shows the WordCount implementation from the Quickstart which processes some text lines with two operators (FlatMap and Reduce), and writes the prints the resulting words and counts to std-out.</p>
+```
+The following code shows the WordCount implementation from the Quickstart which processes some text lines with two operators (FlatMap and Reduce), and writes the prints the resulting words and counts to std-out.
 
-{% highlight java %}
+```java
 public class WordCount {
   
   public static void main(String[] args) throws Exception {
@@ -97,11 +93,11 @@ public class WordCount {
     env.execute("WordCount Example");
   }
 }
-{% endhighlight %}
+```
 
-<p>The operations are defined by specialized classes, here the LineSplitter class.</p>
+The operations are defined by specialized classes, here the LineSplitter class.
 
-{% highlight java %}
+```java
 public class LineSplitter extends FlatMapFunction<String, Tuple2<String, Integer>> {
 
   @Override
@@ -117,10 +113,7 @@ public class LineSplitter extends FlatMapFunction<String, Tuple2<String, Integer
     }
   }
 }
+```
+[Check GitHub](https://github.com/apache/incubator-flink/blob/master/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCount.java) for the full example code.
 
-{% endhighlight %}
-
-<p><a href="https://github.com/stratosphere/stratosphere/blob/release-{{site.current_stable}}/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCount.java">Check GitHub</a> for the full example code.</p>
-
-<p class="lead">For a complete overview over our Java API, have a look at the <a href="{{ site.baseurl }}/docs/{{site.current_stable_documentation}}/programming_guides/java.html">Stratosphere Documentation</a> and <a href="{{ site.baseurl }}/docs/{{site.current_stable_documentation}}/programming_guides/examples_java.html">further example programs</a>. If you have any trouble, ask on our <a href="https://groups.google.com/forum/#!forum/stratosphere-dev">Mailing list</a>. We are happy to provide help.</p>
-</section>
+For a complete overview over our Java API, have a look at the [API Documentation](java_api_guide.html) and [further example programs](java_api_examples.html). If you have any trouble, ask on our [Mailing List](http://mail-archives.apache.org/mod_mbox/incubator-flink-dev/). We are happy to provide help.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f7f1ed2f/docs/run_example_quickstart.md
----------------------------------------------------------------------
diff --git a/docs/run_example_quickstart.md b/docs/run_example_quickstart.md
index 600e3fd..0be198c 100644
--- a/docs/run_example_quickstart.md
+++ b/docs/run_example_quickstart.md
@@ -2,81 +2,61 @@
 title: "Quick Start: Run K-Means Example"
 ---
 
+This guide will Peter demonstrate Stratosphere's features by example. You will see how you can leverage Stratosphere's Iteration-feature to find clusters in a dataset using [K-Means clustering](http://en.wikipedia.org/wiki/K-means_clustering). 
+On the way, you will see the compiler, the status interface and the result of the algorithm.
+
+
+#  Generate Input Data
+Stratosphere contains a data generator for K-Means.
+
+	# Download Stratosphere
+	wget {{ site.current_stable_dl }}
+	tar xzf stratosphere-*.tgz 
+	cd stratosphere-*
+	mkdir kmeans
+	cd kmeans
+	# Run data generator
+	java -cp  ../examples/stratosphere-java-examples-{{ site.current_stable }}-KMeans.jar eu.stratosphere.example.java.clustering.util.KMeansDataGenerator 500 10 0.08
+	cp /tmp/points .
+	cp /tmp/centers .
 
-<p class="lead">
-	This guide will demonstrate Stratosphere's features by example. You will see how you can leverage Stratosphere's Iteration-feature to find clusters in a dataset using <a href="http://en.wikipedia.org/wiki/K-means_clustering">K-Means clustering</a>. 
-	On the way, you will see the compiler, the status interface and the result of the algorithm.
-</p>
-
-
-<section id="data">
-  <div class="page-header">
-  	<h2>Generate Input Data</h2>
-  </div>
-  <p>Stratosphere contains a data generator for K-Means.</p>
-  {% highlight bash %}
-# Download Stratosphere
-wget {{ site.current_stable_dl }}
-tar xzf stratosphere-*.tgz 
-cd stratosphere-*
-mkdir kmeans
-cd kmeans
-# run data generator
-java -cp  ../examples/stratosphere-java-examples-{{ site.current_stable }}-KMeans.jar eu.stratosphere.example.java.clustering.util.KMeansDataGenerator 500 10 0.08
-cp /tmp/points .
-cp /tmp/centers .
-  {% endhighlight %}
 The generator has the following arguments:
-{% highlight bash %}
-KMeansDataGenerator <numberOfDataPoints> <numberOfClusterCenters> [<relative stddev>] [<centroid range>] [<seed>]
-{% endhighlight %}
-The <i>relative standard deviation</i> is an interesting tuning parameter: it determines the closeness of the points to the centers.
-<p>The <code>kmeans/</code> directory should now contain two files: <code>centers</code> and <code>points</code>.</p>
 
+	KMeansDataGenerator <numberOfDataPoints> <numberOfClusterCenters> [<relative stddev>] [<centroid range>] [<seed>]
+
+The _relative standard deviation_ is an interesting tuning parameter: it determines the closeness of the points to the centers.
+
+The `kmeans/` directory should now contain two files: `centers` and `points`.
 
-<h2>Review Input Data</h2>
-Use the <code>plotPoints.py</code> tool to review the result of the data generator. <a href="{{site.baseurl}}/quickstart/example-data/plotPoints.py">Download Python Script</a>
-{% highlight bash %}
+
+# Review Input Data
+Use the `plotPoints.py` tool to review the result of the data generator. [Download Python Script](quickstart/plotPoints.py)
+```bash
 python2.7 plotPoints.py points points input
-{% endhighlight %}
+```
 
 
-Note: You might have to install <a href="http://matplotlib.org/">matplotlib</a> (<code>python-matplotlib</code> package on Ubuntu) to use the Python script.
+Note: You might have to install [matplotlib](http://matplotlib.org/) (`python-matplotlib` package on Ubuntu) to use the Python script.
 
 
 The following overview presents the impact of the different standard deviations on the input data.
-<div class="row" style="padding-top:15px">
-	<div class="col-md-4">
-		<div class="text-center" style="font-weight:bold;">relative stddev = 0.03</div>
-		<a data-lightbox="inputs" href="{{site.baseurl}}/img/quickstart-example/kmeans003.png" data-lightbox="example-1"><img class="img-responsive" src="{{site.baseurl}}/img/quickstart-example/kmeans003.png" /></a>
-	</div>
-	<div class="col-md-4">
-		<div class="text-center" style="font-weight:bold;padding-bottom:2px">relative stddev = 0.08</div>
-		<a data-lightbox="inputs" href="{{site.baseurl}}/img/quickstart-example/kmeans008.png" data-lightbox="example-1"><img class="img-responsive" src="{{site.baseurl}}/img/quickstart-example/kmeans008.png" /></a>
-	</div>
-	<div class="col-md-4">
-		<div class="text-center" style="font-weight:bold;">relative stddev = 0.15</div>
-		<a data-lightbox="inputs" href="{{site.baseurl}}/img/quickstart-example/kmeans015.png" data-lightbox="example-1"><img class="img-responsive" src="{{site.baseurl}}/img/quickstart-example/kmeans015.png" /></a>
-	</div>
-</div>
-</section>
 
-<section id="run">
- <div class="page-header">
-  	<h2>Run Clustering</h2>
-  </div>
+|relative stddev = 0.03|relative stddev = 0.08|relative stddev = 0.15|
+|:--------------------:|:--------------------:|:--------------------:|
+|<img src="img/quickstart-example/kmeans003.png" alt="example1" style="width: 275px;"/>|<img src="img/quickstart-example/kmeans008.png" alt="example2" style="width: 275px;"/>|<img src="img/quickstart-example/kmeans015.png" alt="example3" style="width: 275px;"/>|
+
+
+# Run Clustering
 We are using the generated input data to run the clustering using a Stratosphere job.
-{% highlight bash %}
-# go to the Stratosphere-root directory
-cd stratosphere
-# start Stratosphere (use ./bin/start-cluster.sh if you're on a cluster)
-./bin/start-local.sh
-# Start Stratosphere web client
-./bin/start-webclient.sh
-{% endhighlight %}
 
-<h2>Review Stratosphere Compiler</h2>
+	# go to the Stratosphere-root directory
+	cd stratosphere
+	# start Stratosphere (use ./bin/start-cluster.sh if you're on a cluster)
+	./bin/start-local.sh
+	# Start Stratosphere web client
+	./bin/start-webclient.sh
 
+# Review Stratosphere Compiler
 The Stratosphere webclient allows to submit Stratosphere programs using a graphical user interface.
 
 <div class="row" style="padding-top:15px">
@@ -85,19 +65,19 @@ The Stratosphere webclient allows to submit Stratosphere programs using a graphi
 	</div>
 	<div class="col-md-6">
 		1. <a href="http://localhost:8080/launch.html">Open webclient on localhost:8080</a> <br>
-		2. Upload the 
-{% highlight bash %}
-examples/stratosphere-java-examples-0.5-SNAPSHOT-KMeansIterative.jar
-{% endhighlight %} file.<br>
+		2. Upload the file. 
+			{% highlight bash %}
+			examples/stratosphere-java-examples-0.5-SNAPSHOT-KMeansIterative.jar
+			{% endhighlight %} </br>
 		3. Select it in the left box to see how the operators in the plan are connected to each other. <br>
 		4. Enter the arguments in the lower left box:
-{% highlight bash %}
-file://<pathToGenerated>points file://<pathToGenerated>centers file://<pathToGenerated>result 10
-{% endhighlight %}
-For example:
-{% highlight bash %}
-file:///tmp/stratosphere/kmeans/points file:///tmp/stratosphere/kmeans/centers file:///tmp/stratosphere/kmeans/result 20
-{% endhighlight %}
+			{% highlight bash %}
+			file://<pathToGenerated>points file://<pathToGenerated>centers file://<pathToGenerated>result 10
+			{% endhighlight %}
+			For example:
+			{% highlight bash %}
+			file:///tmp/stratosphere/kmeans/points file:///tmp/stratosphere/kmeans/centers file:///tmp/stratosphere/kmeans/result 20
+			{% endhighlight %}
 	</div>
 </div>
 <hr>
@@ -122,33 +102,19 @@ file:///tmp/stratosphere/kmeans/points file:///tmp/stratosphere/kmeans/centers f
 		3. Once the job has finished, you can analyize the runtime of the individual operators.
 	</div>
 </div>
-</section>
 
-<section id="result">
- <div class="page-header">
-  	<h2>Analyze the Result</h2>
-  </div>
-Use the <a href="{{site.baseurl}}/quickstart/example-data/plotPoints.py">Python Script</a> again to visualize the result
 
-{% highlight bash %}
+# Analyze the Result
+
+Use the [Python Script]({{site.baseurl}}/quickstart/plotPoints.py) again to visualize the result
+
+```bash
 python2.7 plotPoints.py result result result-pdf
-{% endhighlight %}
+```
 
 The following three pictures show the results for the sample input above. Play around with the parameters (number of iterations, number of clusters) to see how they affect the result.
 
-<div class="row" style="padding-top:15px">
-	<div class="col-md-4">
-		<div class="text-center" style="font-weight:bold;">relative stddev = 0.03</div>
-		<a data-lightbox="results" href="{{site.baseurl}}/img/quickstart-example/result003.png" data-lightbox="example-1"><img class="img-responsive" src="{{site.baseurl}}/img/quickstart-example/result003.png" /></a>
-	</div>
-	<div class="col-md-4">
-		<div class="text-center" style="font-weight:bold;">relative stddev = 0.08</div>
-		<a data-lightbox="results" href="{{site.baseurl}}/img/quickstart-example/result008.png" data-lightbox="example-1"><img class="img-responsive" src="{{site.baseurl}}/img/quickstart-example/result008.png" /></a>
-	</div>
-	<div class="col-md-4">
-		<div class="text-center" style="font-weight:bold;">relative stddev = 0.15</div>
-		<a data-lightbox="results" href="{{site.baseurl}}/img/quickstart-example/result015.png" data-lightbox="example-1"><img class="img-responsive" src="{{site.baseurl}}/img/quickstart-example/result015.png" /></a>
-	</div>
-</div>
 
-</section>
+|relative stddev = 0.03|relative stddev = 0.08|relative stddev = 0.15|
+|:--------------------:|:--------------------:|:--------------------:|
+|<img src="img/quickstart-example/result003.png" alt="example1" style="width: 275px;"/>|<img src="img/quickstart-example/result008.png" alt="example2" style="width: 275px;"/>|<img src="img/quickstart-example/result015.png" alt="example3" style="width: 275px;"/>|
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f7f1ed2f/docs/setup_quickstart.md
----------------------------------------------------------------------
diff --git a/docs/setup_quickstart.md b/docs/setup_quickstart.md
index debe21c..9c9b6e3 100644
--- a/docs/setup_quickstart.md
+++ b/docs/setup_quickstart.md
@@ -2,131 +2,105 @@
 title: "Quickstart: Setup"
 ---
 
-<p class="lead">Get Stratosphere up and running in a few simple steps.</p>
-
-<section id="requirements">
-  <div class="page-header"><h2>Requirements</h2></div>
-  <p class="lead">Stratosphere runs on all <em>UNIX-like</em> environments: <strong>Linux</strong>, <strong>Mac OS X</strong>, <strong>Cygwin</strong>. The only requirement is to have a working <strong>Java 6.x</strong> (or higher) installation.</p>
-</section>
-
-<section id="download">
-  <div class="page-header"><h2>Download</h2></div>
-  <p class="lead">Download the ready to run binary package. Choose the Stratosphere distribution that <strong>matches your Hadoop version</strong>. If you are unsure which version to choose or you just want to run locally, pick the package for Hadoop 1.2.</p>
-  <p>
-  	<ul class="nav nav-tabs">
-  		<li class="active"><a href="#bin-hadoop1" data-toggle="tab">Hadoop 1.2</a></li>
-      <li><a href="#bin-hadoop2" data-toggle="tab">Hadoop 2 (YARN)</a></li>
-		</ul>
-		<div class="tab-content text-center">
-			<div class="tab-pane active" id="bin-hadoop1">
-				<a class="btn btn-info btn-lg" onclick="_gaq.push(['_trackEvent','Action','download-quickstart-setup-1',this.href]);" href="{{site.current_stable_dl}}"><i class="icon-download"> </i> Download Stratosphere for Hadoop 1.2</a>
-	    </div>
-			<div class="tab-pane" id="bin-hadoop2">
-	      <a class="btn btn-info btn-lg" onclick="_gaq.push(['_trackEvent','Action','download-quickstart-setup-2',this.href]);" href="{{site.current_stable_dl_yarn}}"><i class="icon-download"> </i> Download Stratosphere for Hadoop 2 (YARN)</a>
-	    </div>
-	  </div>
-	</p>
-</section>
-
-<section id="start">
-  <div class="page-header"><h2>Start</h2></div> 
-  <p class="lead">You are almost done.</p>
-  <ol>
-  	<li class="lead"><strong>Go to the download directory</strong>,</li>
-  	<li class="lead"><strong>Unpack the downloaded archive</strong>, and</li>
-  	<li class="lead"><strong>Start Stratosphere</strong>.</li>
-  </ol>
-
-{% highlight bash %}
+Get Stratosphere up and running in a few simple steps.
+
+# Requirements
+Stratosphere runs on all __UNIX-like__ environments: __Linux__, __Mac OS X__, __Cygwin__. The only requirement is to have a working __Java 6.x__ (or higher) installation.
+
+# Download
+Download the ready to run binary package. Choose the Stratosphere distribution that __matches your Hadoop version__. If you are unsure which version to choose or you just want to run locally, pick the package for Hadoop 1.2.
+
+<ul class="nav nav-tabs">
+   <li class="active"><a href="#bin-hadoop1" data-toggle="tab">Hadoop 1.2</a></li>
+   <li><a href="#bin-hadoop2" data-toggle="tab">Hadoop 2 (YARN)</a></li>
+ </ul>
+ <div class="tab-content text-center">
+   <div class="tab-pane active" id="bin-hadoop1">
+     <a class="btn btn-info btn-lg" onclick="_gaq.push(['_trackEvent','Action','download-quickstart-setup-1',this.href]);" href="{{site.current_stable_dl}}"><i class="icon-download"> </i> Download Stratosphere for Hadoop 1.2</a>
+   </div>
+   <div class="tab-pane" id="bin-hadoop2">
+     <a class="btn btn-info btn-lg" onclick="_gaq.push(['_trackEvent','Action','download-quickstart-setup-2',this.href]);" href="{{site.current_stable_dl_yarn}}"><i class="icon-download"> </i> Download Stratosphere for Hadoop 2 (YARN)</a>
+   </div>
+ </div>
+</p>
+
+
+# Start
+You are almost done.
+  
+1. Go to the download directory.
+2. Unpack the downloaded archive.
+3. Start Stratosphere.
+
+
+```bash
 $ cd ~/Downloads              # Go to download directory
 $ tar xzf stratosphere-*.tgz  # Unpack the downloaded archive
 $ cd stratosphere
 $ bin/start-local.sh          # Start Stratosphere
-{% endhighlight %}
+```
 
-  <p class="lead">Check the <strong>JobManager's web frontend</strong> at <a href="http://localhost:8081">http://localhost:8081</a> and make sure everything is up and running.</p>
-</section>
+Check the __JobManager's web frontend__ at [http://localhost:8081](http://localhost:8081) and make sure everything is up and running.
 
-<section id="example">
-  <div class="page-header"><h2>Run Example</h2></div>
-  <p class="lead">Run the <strong>Word Count example</strong> to see Stratosphere at work.</p>
+# Run Example
 
-  <ol>
-  	<li class="lead"><strong>Download test data:</strong>
-{% highlight bash %}
+Run the __Word Count example__ to see Stratosphere at work.
+
+* __Download test data__:
+```bash
 $ wget -O hamlet.txt http://www.gutenberg.org/cache/epub/1787/pg1787.txt
-{% endhighlight %}
-		  You now have a text file called <em>hamlet.txt</em> in your working directory.
-		</li>
-  	<li class="lead"><strong>Start the example program</strong>:
-{% highlight bash %}
+```
+* You now have a text file called _hamlet.txt_ in your working directory.
+* __Start the example program__:
+```bash
 $ bin/stratosphere run \
     --jarfile ./examples/stratosphere-java-examples-{{site.current_stable}}-WordCount.jar \
     --arguments file://`pwd`/hamlet.txt file://`pwd`/wordcount-result.txt
-{% endhighlight %}
-      You will find a file called <strong>wordcount-result.txt</strong> in your current directory.
-  	</li>
-  </ol>
-
-</section>
-
-<section id="cluster">
-  <div class="page-header"><h2>Cluster Setup</h2></div>
-  <p class="lead"><strong>Running Stratosphere on a cluster</strong> is as easy as running it locally. Having <strong>passwordless SSH</strong> and <strong>the same directory structure</strong> on all your cluster nodes lets you use our scripts to control everything.</p>
-  <ol>
-  	<li class="lead">Copy the unpacked <strong>stratosphere</strong> directory from the downloaded archive to the same file system path on each node of your setup.</li>
-  	<li class="lead">Choose a <strong>master node</strong> (JobManager) and set the <code>jobmanager.rpc.address</code> key in <code>conf/stratosphere-conf.yaml</code> to its IP or hostname. Make sure that all nodes in your cluster have the same <code>jobmanager.rpc.address</code> configured.</li>
-  	<li class="lead">Add the IPs or hostnames (one per line) of all <strong>worker nodes</strong> (TaskManager) to the slaves files in <code>conf/slaves</code>.</li>
-  </ol>
-  <p class="lead">You can now <strong>start the cluster</strong> at your master node with <code>bin/start-cluster.sh</code>.</p>
-  <p class="lead">
-    The following <strong>example</strong> illustrates the setup with three nodes (with IP addresses from <em>10.0.0.1</em> to <em>10.0.0.3</em> and hostnames <em>master</em>, <em>worker1</em>, <em>worker2</em>) and shows the contents of the configuration files, which need to be accessible at the same path on all machines:
-  </p>
+```
+* You will find a file called __wordcount-result.txt__ in your current directory.
+  
+
+# Cluster Setup
+  
+__Running Stratosphere on a cluster__ is as easy as running it locally. Having __passwordless SSH__ and __the same directory structure__ on all your cluster nodes lets you use our scripts to control everything.
+
+1. Copy the unpacked __stratosphere__ directory from the downloaded archive to the same file system path on each node of your setup.
+2. Choose a __master node__ (JobManager) and set the `jobmanager.rpc.address` key in `conf/stratosphere-conf.yaml` to its IP or hostname. Make sure that all nodes in your cluster have the same `jobmanager.rpc.address` configured.
+3. Add the IPs or hostnames (one per line) of all __worker nodes__ (TaskManager) to the slaves files in `conf/slaves`.
+
+You can now __start the cluster__ at your master node with `bin/start-cluster.sh`.
+
+
+The following __example__ illustrates the setup with three nodes (with IP addresses from _10.0.0.1_ to _10.0.0.3_ and hostnames _master_, _worker1_, _worker2_) and shows the contents of the configuration files, which need to be accessible at the same path on all machines:
+
+<div class="row">
+  <div class="col-md-6 text-center">
+    <img src="{{ site.baseurl }}/img/quickstart_cluster.png" style="width: 85%">
+  </div>
+<div class="col-md-6">
   <div class="row">
-    <div class="col-md-6 text-center">
-      <img src="{{ site.baseurl }}/img/quickstart_cluster.png" style="width: 85%">
-    </div>
-    <div class="col-md-6">
-      <div class="row">
-        <p class="lead text-center">
-        /path/to/<strong>stratosphere/conf/<br>stratosphere-conf.yaml</strong>
-<pre>
-jobmanager.rpc.address: 10.0.0.1
-</pre>
-        </p>
-      </div>
-      <div class="row" style="margin-top: 1em;">
-        <p class="lead text-center">
-        /path/to/<strong>stratosphere/<br>conf/slaves</strong>
-<pre>
-10.0.0.2
-10.0.0.3
-</pre>
-        </p>
-      </div>
-    </div>
+    <p class="lead text-center">
+      /path/to/<strong>stratosphere/conf/<br>stratosphere-conf.yaml</strong>
+    <pre>jobmanager.rpc.address: 10.0.0.1</pre>
+    </p>
   </div>
-</section>
-
-<section id="yarn">
-  <div class="page-header"><h2>Stratosphere on YARN</h2></div>
-  <p class="lead">You can easily deploy Stratosphere on your existing <strong>YARN cluster</strong>. 
-    <ol>
-    <li class="lead">Download the <strong>Stratosphere YARN package</strong> with the YARN client:
-      <div class="text-center" style="padding: 1em;">
-      <a style="padding-left:10px" onclick="_gaq.push(['_trackEvent','Action','download-quickstart-yarn',this.href]);" class="btn btn-info btn-lg" href="{{site.current_stable_uberjar}}"><i class="icon-download"> </i> Stratosphere {{ site.current_stable }} for YARN</a>
-      </div>
-    </li>
-    <li class="lead">Make sure your <strong>HADOOP_HOME</strong> (or <em>YARN_CONF_DIR</em> or <em>HADOOP_CONF_DIR</em>) <strong>environment variable</strong> is set to read your YARN and HDFS configuration.</li>
-    <li class="lead">Run the <strong>YARN client</strong> with:
-      <div class="text-center" style="padding:1em;">
-        <code>./bin/yarn-session.sh</code>
-      </div>
-      
-      You can run the client with options <code>-n 10 -tm 8192</code> to allocate 10 TaskManagers with 8GB of memory each.</li>
-  </ol>
+<div class="row" style="margin-top: 1em;">
+  <p class="lead text-center">
+    /path/to/<strong>stratosphere/<br>conf/slaves</strong>
+  <pre>
+    10.0.0.2
+    10.0.0.3
+  </pre>
   </p>
-</section>
+</div>
+</div>
+</div>
+
+# Stratosphere on YARN
+You can easily deploy Stratosphere on your existing __YARN cluster__. 
+
+1. Download the __Stratosphere YARN package__ with the YARN client: [Stratosphere for YARN]({{site.current_stable_uberjar}})
+2. Make sure your __HADOOP_HOME__ (or _YARN_CONF_DIR_ or _HADOOP_CONF_DIR_) __environment variable__ is set to read your YARN and HDFS configuration.
+3. Run the __YARN client__ with: `./bin/yarn-session.sh`. You can run the client with options `-n 10 -tm 8192` to allocate 10 TaskManagers with 8GB of memory each.
 
-<hr />
-<p class="lead">For <strong>more detailed instructions</strong>, check out the <a href="{{site.baseurl}}/docs/{{site.current_stable_documentation}}">Documentation</a>.</p>
\ No newline at end of file
+For __more detailed instructions__, check out the programming Guides and examples.
\ No newline at end of file


[36/53] [abbrv] git commit: Changed default number of task slots per machine to one. Add logging for task slots.

Posted by rm...@apache.org.
Changed default number of task slots per machine to one.
Add logging for task slots.


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

Branch: refs/heads/travis_test
Commit: fe91d4bea4b1b68477ae52ed0746d5be3f7c709f
Parents: f13ad5b
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Jun 24 16:06:35 2014 +0200
Committer: uce <u....@fu-berlin.de>
Committed: Wed Jun 25 10:57:56 2014 +0200

----------------------------------------------------------------------
 .../eu/stratosphere/nephele/taskmanager/TaskManager.java     | 8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/fe91d4be/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
index 789955a..d866c64 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
@@ -46,7 +46,6 @@ import eu.stratosphere.nephele.ExecutionMode;
 import eu.stratosphere.runtime.io.network.LocalConnectionManager;
 import eu.stratosphere.runtime.io.network.NetworkConnectionManager;
 import eu.stratosphere.runtime.io.network.netty.NettyConnectionManager;
-import eu.stratosphere.nephele.instance.Hardware;
 import eu.stratosphere.nephele.taskmanager.transferenvelope.RegisterTaskManagerResult;
 import eu.stratosphere.nephele.types.IntegerRecord;
 
@@ -357,10 +356,13 @@ public class TaskManager implements TaskOperationProtocol {
 			HardwareDescription resources = HardwareDescriptionFactory.extractFromSystem();
 			
 			int slots = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, -1);
-			if (slots == -1) { 
-				slots = Hardware.getNumberCPUCores();
+			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;
 			


[44/53] [abbrv] git commit: [FLINK-971] Configure PooledByteBufAllocator in NettyConnectionManager instead of using the default allocator

Posted by rm...@apache.org.
[FLINK-971] Configure PooledByteBufAllocator in NettyConnectionManager instead of using the default allocator

Configuration:
    - 0 heap arenas,
    - n direct arenas (where n = num incoming + num outgoing network IO threads), and
    - bufferSize << 1 bytes page size.

Additionally, OutboundEnvelopeEncoder directly implements ChannelOutboundHandlerAdapter instead of the
MessageToByteEncoder<Envelope> wrapper to have tighter control of memory allocations.

This closes #38.


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

Branch: refs/heads/travis_test
Commit: 6c827fb932efff77c8abcdbb407385898f9d8d40
Parents: f7f1ed2
Author: uce <u....@fu-berlin.de>
Authored: Tue Jun 17 16:18:02 2014 +0200
Committer: uce <u....@fu-berlin.de>
Committed: Wed Jun 25 13:53:51 2014 +0200

----------------------------------------------------------------------
 .../network/netty/NettyConnectionManager.java   | 22 ++++++++++++--
 .../network/netty/OutboundConnectionQueue.java  |  2 +-
 .../network/netty/OutboundEnvelopeEncoder.java  | 32 +++++++++++++++-----
 3 files changed, 45 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6c827fb9/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManager.java
index 73afcbc..4b54641 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManager.java
@@ -92,6 +92,22 @@ public class NettyConnectionManager implements NetworkConnectionManager {
 		final BufferProviderBroker bufferProviderBroker = channelManager;
 		final EnvelopeDispatcher envelopeDispatcher = channelManager;
 
+		int numHeapArenas = 0;
+		int numDirectArenas = numInThreads + numOutThreads;
+		int pageSize = bufferSize << 1;
+		int chunkSize = 16 * 1 << 20; // 16 MB
+
+		// shift pageSize maxOrder times to get to chunkSize
+		int maxOrder = (int) (Math.log(chunkSize/pageSize) / Math.log(2));
+
+		PooledByteBufAllocator pooledByteBufAllocator =
+				new PooledByteBufAllocator(true, numHeapArenas, numDirectArenas, pageSize, maxOrder);
+
+		String msg = String.format("Instantiated PooledByteBufAllocator with direct arenas: %d, heap arenas: %d, " +
+				"page size (bytes): %d, chunk size (bytes): %d.",
+				numDirectArenas, numHeapArenas, pageSize, (pageSize << maxOrder));
+		LOG.info(msg);
+
 		// --------------------------------------------------------------------
 		// server bootstrap (incoming connections)
 		// --------------------------------------------------------------------
@@ -107,8 +123,8 @@ public class NettyConnectionManager implements NetworkConnectionManager {
 								.addLast(new InboundEnvelopeDispatcher(envelopeDispatcher));
 					}
 				})
-				.option(ChannelOption.RCVBUF_ALLOCATOR, new FixedRecvByteBufAllocator(bufferSize))
-				.option(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT);
+				.option(ChannelOption.RCVBUF_ALLOCATOR, new FixedRecvByteBufAllocator(pageSize))
+				.option(ChannelOption.ALLOCATOR, pooledByteBufAllocator);
 
 		// --------------------------------------------------------------------
 		// client bootstrap (outgoing connections)
@@ -125,7 +141,7 @@ public class NettyConnectionManager implements NetworkConnectionManager {
 				})
 				.option(ChannelOption.WRITE_BUFFER_LOW_WATER_MARK, lowWaterMark)
 				.option(ChannelOption.WRITE_BUFFER_HIGH_WATER_MARK, highWaterMark)
-				.option(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT)
+				.option(ChannelOption.ALLOCATOR, pooledByteBufAllocator)
 				.option(ChannelOption.TCP_NODELAY, false)
 				.option(ChannelOption.SO_KEEPALIVE, true);
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6c827fb9/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/OutboundConnectionQueue.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/OutboundConnectionQueue.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/OutboundConnectionQueue.java
index 8fef3c1..ff6c694 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/OutboundConnectionQueue.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/OutboundConnectionQueue.java
@@ -42,7 +42,7 @@ public class OutboundConnectionQueue extends ChannelInboundHandlerAdapter implem
 	}
 
 	/**
-	 * Enqueues an envelope so be sent later.
+	 * Enqueues an envelope to be sent later.
 	 * <p/>
 	 * This method is always invoked by the task thread that wants the envelope sent.
 	 *

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6c827fb9/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/OutboundEnvelopeEncoder.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/OutboundEnvelopeEncoder.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/OutboundEnvelopeEncoder.java
index 424f2c0..dad690c 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/OutboundEnvelopeEncoder.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/OutboundEnvelopeEncoder.java
@@ -16,25 +16,43 @@ package eu.stratosphere.runtime.io.network.netty;
 import eu.stratosphere.runtime.io.Buffer;
 import eu.stratosphere.runtime.io.network.Envelope;
 import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
 import io.netty.channel.ChannelHandler;
 import io.netty.channel.ChannelHandlerContext;
-import io.netty.handler.codec.MessageToByteEncoder;
+import io.netty.channel.ChannelOutboundHandlerAdapter;
+import io.netty.channel.ChannelPromise;
 
 @ChannelHandler.Sharable
-public class OutboundEnvelopeEncoder extends MessageToByteEncoder<Envelope> {
+public class OutboundEnvelopeEncoder extends ChannelOutboundHandlerAdapter {
 
 	public static final int HEADER_SIZE = 48;
 
 	public static final int MAGIC_NUMBER = 0xBADC0FFE;
 
 	@Override
-	protected void encode(ChannelHandlerContext ctx, Envelope env, ByteBuf out) throws Exception {
+	public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) throws Exception {
+		Envelope env = (Envelope) msg;
+
+		ByteBuf buf = ctx.alloc().directBuffer();
+
+		encode(env, buf);
+
+		if (buf.isReadable()) {
+			ctx.write(buf, promise);
+		}
+		else {
+			buf.release();
+			ctx.write(Unpooled.EMPTY_BUFFER, promise);
+		}
+	}
+
+	private void encode(Envelope env, ByteBuf out) {
 		// --------------------------------------------------------------------
 		// (1) header (48 bytes)
 		// --------------------------------------------------------------------
 		out.writeInt(MAGIC_NUMBER); // 4 bytes
 
-		if (out.getInt(out.writerIndex()-4) != MAGIC_NUMBER) {
+		if (out.getInt(out.writerIndex() - 4) != MAGIC_NUMBER) {
 			throw new RuntimeException();
 		}
 
@@ -54,12 +72,12 @@ public class OutboundEnvelopeEncoder extends MessageToByteEncoder<Envelope> {
 		// (3) buffer (var length)
 		// --------------------------------------------------------------------
 		if (env.getBuffer() != null) {
-			Buffer buffer = env.getBuffer();
-			out.writeBytes(buffer.getMemorySegment().wrap(0, buffer.size()));
+			Buffer envBuffer = env.getBuffer();
+			out.writeBytes(envBuffer.getMemorySegment().wrap(0, envBuffer.size()));
 
 			// Recycle the buffer from OUR buffer pool after everything has been
 			// copied to Nettys buffer space.
-			buffer.recycleBuffer();
+			envBuffer.recycleBuffer();
 		}
 	}
 }


[39/53] [abbrv] [FLINK-926] Add shallow copy, deep equality, and hashCode to Tuple classes

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/96e76a58/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/TupleGenerator.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/TupleGenerator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/TupleGenerator.java
index fa11812..08e5cd3 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/TupleGenerator.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/TupleGenerator.java
@@ -28,6 +28,7 @@ import com.google.common.io.Files;
 class TupleGenerator {
 
 	// Parameters for tuple classes	
+
 	private static final String ROOT_DIRECTORY = "./src/main/java";
 
 	private static final String PACKAGE = "eu.stratosphere.api.java.tuple";
@@ -498,7 +499,7 @@ class TupleGenerator {
 
 		for (int i = FIRST; i <= LAST; i++) {
 			File tupleFile = new File(dir, "Tuple" + i + ".java");
-			PrintWriter writer = new PrintWriter(tupleFile);
+				PrintWriter writer = new PrintWriter(tupleFile);
 			writeTupleClass(writer, i);
 			writer.flush();
 			writer.close();
@@ -579,6 +580,7 @@ class TupleGenerator {
 		w.println("\t}");
 		w.println();
 
+
 		// arity accessor
 		w.println("\t@Override");
 		w.println("\tpublic int getArity() { return " + numFields + "; }");
@@ -634,7 +636,7 @@ class TupleGenerator {
 		w.println("\t}");
 		w.println();
 
-		// standard utilities (toString, equals, hashCode)
+		// standard utilities (toString, equals, hashCode, copy)
 		w.println();
 		w.println("\t// -------------------------------------------------------------------------------------------------");
 		w.println("\t// standard utilities");
@@ -659,6 +661,72 @@ class TupleGenerator {
 		w.println("\t\t\t+ \")\";");
 		w.println("\t}");
 
+
+
+
+		w.println();
+		w.println("\t/**");
+		w.println("\t * Deep equality for tuples by calling equals() on the tuple members");
+		w.println("\t * @param o the object checked for equality");
+		w.println("\t * @return true if this is equal to o.");
+		w.println("\t */");
+		w.println("\t@Override");
+		w.println("\tpublic boolean equals(Object o) {");
+		w.println("\t\tif(this == o) { return true; }");
+		w.println("\t\tif (!(o instanceof " + className + ")) { return false; }");
+		w.println("\t\t" + className + " tuple = (" + className + ") o;");
+		for (int i = 0; i < numFields; i++) {
+			String field = "f" + i;
+			w.println("\t\tif (" + field + " != null ? !" + field +".equals(tuple." +
+					field + ") : tuple." + field + " != null) { return false; }");
+		}
+		w.println("\t\treturn true;");
+		w.println("\t}");
+
+		w.println();
+		w.println("\t@Override");
+		w.println("\tpublic int hashCode() {");
+		w.println("\t\tint result = f0 != null ? f0.hashCode() : 0;");
+		for (int i = 1; i < numFields; i++) {
+			String field = "f" + i;
+			w.println("\t\tresult = 31 * result + (" + field + " != null ? " + field + ".hashCode() : 0);");
+		}
+		w.println("\t\treturn result;");
+		w.println("\t}");
+
+
+		String tupleTypes = "<";
+		for (int i = 0; i < numFields; i++) {
+			tupleTypes += "T" + i;
+			if (i < numFields - 1) {
+				tupleTypes += ",";
+			}
+		}
+		tupleTypes += ">";
+
+		w.println("\t/**");
+		w.println("\t* Shallow tuple copy.");
+		w.println("\t* @returns A new Tuple with the same fields as this.");
+		w.println("\t */");
+		w.println("\tpublic " + className + tupleTypes + " copy(){ ");
+
+		w.print("\t\treturn new " + className + tupleTypes + "(this.f0");
+		if (numFields > 1) {
+			w.println(",");
+		}
+		for (int i = 1; i < numFields; i++) {
+			String field = "f" + i;
+			w.print("\t\t\tthis." + field);
+			if (i < numFields - 1) {
+				w.println(",");
+			}
+		}
+		w.println(");");
+		w.println("\t}");
+
+		w.println();
+
+
 		// foot
 		w.println("}");
 	}


[22/53] [abbrv] git commit: Add some options for slot-based scheduling and changed default parallelism to one.

Posted by rm...@apache.org.
Add some options for slot-based scheduling and changed default parallelism to one.


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

Branch: refs/heads/travis_test
Commit: b4b633eab9a70e14d2e0dd5252f4b092a3689093
Parents: 8c1d82a
Author: Stephan Ewen <se...@apache.org>
Authored: Sun Jun 22 18:19:57 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sun Jun 22 21:09:13 2014 +0200

----------------------------------------------------------------------
 .../configuration/ConfigConstants.java          |  5 --
 .../memory/OutputViewDataOutputWrapper.java     |  2 +-
 .../util/InstantiationUtilsTest.java            |  4 +-
 .../conf/stratosphere-conf.yaml                 |  2 +
 .../api/java/io/CollectionInputFormat.java      | 15 ++--
 .../api/java/io/CollectionInputFormatTest.java  | 79 ++++++++++++++++++++
 .../nephele/executiongraph/ExecutionGraph.java  |  4 +-
 .../nephele/jobmanager/JobManager.java          |  4 +-
 .../nephele/taskmanager/TaskManager.java        | 19 +++--
 .../executiongraph/ExecutionGraphTest.java      |  4 +-
 .../nephele/jobmanager/JobManagerITCase.java    |  2 +-
 11 files changed, 107 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b4b633ea/stratosphere-core/src/main/java/eu/stratosphere/configuration/ConfigConstants.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/configuration/ConfigConstants.java b/stratosphere-core/src/main/java/eu/stratosphere/configuration/ConfigConstants.java
index eff48cc..b4699b3 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/configuration/ConfigConstants.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/configuration/ConfigConstants.java
@@ -29,11 +29,6 @@ public final class ConfigConstants {
 	 * The config parameter defining the default degree of parallelism for jobs.
 	 */
 	public static final String DEFAULT_PARALLELIZATION_DEGREE_KEY = "parallelization.degree.default";
-
-	/**
-	 * The config parameter defining the maximal intra-node parallelism for jobs.
-	 */
-	public static final String PARALLELIZATION_MAX_INTRA_NODE_DEGREE_KEY = "parallelization.intra-node.default";
 	
 	// -------------------------------- Runtime -------------------------------
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b4b633ea/stratosphere-core/src/main/java/eu/stratosphere/core/memory/OutputViewDataOutputWrapper.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/core/memory/OutputViewDataOutputWrapper.java b/stratosphere-core/src/main/java/eu/stratosphere/core/memory/OutputViewDataOutputWrapper.java
index 7bb8f8c..cb636ce 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/core/memory/OutputViewDataOutputWrapper.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/core/memory/OutputViewDataOutputWrapper.java
@@ -110,7 +110,7 @@ public class OutputViewDataOutputWrapper implements DataOutputView {
 	@Override
 	public void write(DataInputView source, int numBytes) throws IOException {
 		for (int i = 0; i < numBytes; i++) {
-			this.delegate.writeByte(source.readByte());
+			this.delegate.writeByte(source.readUnsignedByte());
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b4b633ea/stratosphere-core/src/test/java/eu/stratosphere/util/InstantiationUtilsTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/test/java/eu/stratosphere/util/InstantiationUtilsTest.java b/stratosphere-core/src/test/java/eu/stratosphere/util/InstantiationUtilsTest.java
index 50c8703..8b55635 100644
--- a/stratosphere-core/src/test/java/eu/stratosphere/util/InstantiationUtilsTest.java
+++ b/stratosphere-core/src/test/java/eu/stratosphere/util/InstantiationUtilsTest.java
@@ -60,7 +60,5 @@ public class InstantiationUtilsTest {
 		InstantiationUtil.checkForInstantiation(TestClass.class);
 	}
 
-	private class TestClass {
-
-	}
+	private class TestClass {}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b4b633ea/stratosphere-dist/src/main/stratosphere-bin/conf/stratosphere-conf.yaml
----------------------------------------------------------------------
diff --git a/stratosphere-dist/src/main/stratosphere-bin/conf/stratosphere-conf.yaml b/stratosphere-dist/src/main/stratosphere-bin/conf/stratosphere-conf.yaml
index ccf1c21..ccc67c2 100644
--- a/stratosphere-dist/src/main/stratosphere-bin/conf/stratosphere-conf.yaml
+++ b/stratosphere-dist/src/main/stratosphere-bin/conf/stratosphere-conf.yaml
@@ -25,6 +25,8 @@ jobmanager.heap.mb: 256
 
 taskmanager.heap.mb: 512
 
+taskmanager.numberOfTaskSlots: -1
+
 parallelization.degree.default: 1
 
 #==============================================================================

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b4b633ea/stratosphere-java/src/main/java/eu/stratosphere/api/java/io/CollectionInputFormat.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/io/CollectionInputFormat.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/io/CollectionInputFormat.java
index fd5ae36..82f2755 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/io/CollectionInputFormat.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/io/CollectionInputFormat.java
@@ -37,10 +37,10 @@ public class CollectionInputFormat<T> extends GenericInputFormat<T> implements N
 
 	private static final long serialVersionUID = 1L;
 
-	private Collection<T> dataSet; // input data as collection
-
 	private TypeSerializer<T> serializer;
 
+	private transient Collection<T> dataSet; // input data as collection. transient, because it will be serialized in a custom way
+	
 	private transient Iterator<T> iterator;
 
 	
@@ -75,7 +75,7 @@ public class CollectionInputFormat<T> extends GenericInputFormat<T> implements N
 	// --------------------------------------------------------------------------------------------
 
 	private void writeObject(ObjectOutputStream out) throws IOException {
-		out.writeObject(serializer);
+		out.defaultWriteObject();
 		out.writeInt(dataSet.size());
 		
 		OutputViewDataOutputWrapper outWrapper = new OutputViewDataOutputWrapper();
@@ -86,13 +86,8 @@ public class CollectionInputFormat<T> extends GenericInputFormat<T> implements N
 		}
 	}
 
-	@SuppressWarnings("unchecked")
-	private void readObject(ObjectInputStream in) throws IOException {
-		try {
-			this.serializer = (TypeSerializer<T>) in.readObject();
-		} catch (ClassNotFoundException ex){
-			throw new IOException("Could not load the serializer class.", ex);
-		}
+	private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+		in.defaultReadObject();
 
 		int collectionLength = in.readInt();
 		List<T> list = new ArrayList<T>(collectionLength);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b4b633ea/stratosphere-java/src/test/java/eu/stratosphere/api/java/io/CollectionInputFormatTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/test/java/eu/stratosphere/api/java/io/CollectionInputFormatTest.java b/stratosphere-java/src/test/java/eu/stratosphere/api/java/io/CollectionInputFormatTest.java
index 4388c9c..f734540 100644
--- a/stratosphere-java/src/test/java/eu/stratosphere/api/java/io/CollectionInputFormatTest.java
+++ b/stratosphere-java/src/test/java/eu/stratosphere/api/java/io/CollectionInputFormatTest.java
@@ -17,6 +17,7 @@ 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 eu.stratosphere.api.java.typeutils.BasicTypeInfo;
 import eu.stratosphere.api.java.typeutils.TypeExtractor;
 import eu.stratosphere.core.io.GenericInputSplit;
 import eu.stratosphere.types.TypeInformation;
@@ -29,7 +30,9 @@ import java.io.IOException;
 import java.io.ObjectInputStream;
 import java.io.ObjectOutputStream;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
+import java.util.List;
 
 public class CollectionInputFormatTest {
 	public static class ElementType{
@@ -105,4 +108,80 @@ public class CollectionInputFormatTest {
 			fail(ex.toString());
 		}
 	}
+	
+	@Test
+	public void testSerializabilityStrings() {
+		
+		final String[] data = new String[] {
+				"To be, or not to be,--that is the question:--",
+				"Whether 'tis nobler in the mind to suffer",
+				"The slings and arrows of outrageous fortune",
+				"Or to take arms against a sea of troubles,",
+				"And by opposing end them?--To die,--to sleep,--",
+				"No more; and by a sleep to say we end",
+				"The heartache, and the thousand natural shocks",
+				"That flesh is heir to,--'tis a consummation",
+				"Devoutly to be wish'd. To die,--to sleep;--",
+				"To sleep! perchance to dream:--ay, there's the rub;",
+				"For in that sleep of death what dreams may come,",
+				"When we have shuffled off this mortal coil,",
+				"Must give us pause: there's the respect",
+				"That makes calamity of so long life;",
+				"For who would bear the whips and scorns of time,",
+				"The oppressor's wrong, the proud man's contumely,",
+				"The pangs of despis'd love, the law's delay,",
+				"The insolence of office, and the spurns",
+				"That patient merit of the unworthy takes,",
+				"When he himself might his quietus make",
+				"With a bare bodkin? who would these fardels bear,",
+				"To grunt and sweat under a weary life,",
+				"But that the dread of something after death,--",
+				"The undiscover'd country, from whose bourn",
+				"No traveller returns,--puzzles the will,",
+				"And makes us rather bear those ills we have",
+				"Than fly to others that we know not of?",
+				"Thus conscience does make cowards of us all;",
+				"And thus the native hue of resolution",
+				"Is sicklied o'er with the pale cast of thought;",
+				"And enterprises of great pith and moment,",
+				"With this regard, their currents turn awry,",
+				"And lose the name of action.--Soft you now!",
+				"The fair Ophelia!--Nymph, in thy orisons",
+				"Be all my sins remember'd."
+		};
+		
+		try {
+			
+			List<String> inputCollection = Arrays.asList(data);
+			CollectionInputFormat<String> inputFormat = new CollectionInputFormat<String>(inputCollection, BasicTypeInfo.STRING_TYPE_INFO.createSerializer());
+			
+			// serialize
+			ByteArrayOutputStream baos = new ByteArrayOutputStream();
+			ObjectOutputStream oos = new ObjectOutputStream(baos);
+			oos.writeObject(inputFormat);
+			oos.close();
+			
+			// deserialize
+			ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
+			ObjectInputStream ois = new ObjectInputStream(bais);
+			Object result = ois.readObject();
+			
+			assertTrue(result instanceof CollectionInputFormat);
+			
+			int i = 0;
+			@SuppressWarnings("unchecked")
+			CollectionInputFormat<String> in = (CollectionInputFormat<String>) result;
+			in.open(new GenericInputSplit());
+			
+			while (!in.reachedEnd()) {
+				assertEquals(data[i++], in.nextRecord(""));
+			}
+			
+			assertEquals(data.length, i);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b4b633ea/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java
index 18395fb..5886650 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java
@@ -145,7 +145,6 @@ public class ExecutionGraph implements ExecutionListener {
 	 *        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");
 		}
@@ -165,8 +164,7 @@ public class ExecutionGraph implements ExecutionListener {
 	 * @throws GraphConversionException
 	 *         thrown if the job graph is not valid and no execution graph can be constructed from it
 	 */
-	public ExecutionGraph(final JobGraph job, final int defaultParallelism)
-																					throws GraphConversionException {
+	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

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b4b633ea/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java
index f3cf3a3..40e2a0b 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java
@@ -16,14 +16,12 @@ package eu.stratosphere.nephele.jobmanager;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.io.InputStream;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -468,7 +466,7 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 			ExecutionGraph eg;
 	
 			try {
-				eg = new ExecutionGraph(job, this.getAvailableSlots());
+				eg = new ExecutionGraph(job, 1);
 			} catch (GraphConversionException e) {
 				if (e.getCause() == null) {
 					return new JobSubmissionResult(AbstractJobResult.ReturnCode.ERROR, StringUtils.stringifyException(e));

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b4b633ea/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
index 575454f..3225ab7 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
@@ -172,8 +172,12 @@ public class TaskManager implements TaskOperationProtocol {
 			throw new NullPointerException("Execution mode must not be null.");
 		}
 		
-		
-//		LOG.info("TaskManager started as user " + UserGroupInformation.getCurrentUser().getShortUserName());
+		try {
+			LOG.info("TaskManager started as user " + UserGroupInformation.getCurrentUser().getShortUserName());
+		} catch (Throwable t) {
+			LOG.error("Cannot determine user group information.", t);
+		}
+			
 		LOG.info("User system property: " + System.getProperty("user.name"));
 		LOG.info("Execution mode: " + executionMode);
 
@@ -344,9 +348,14 @@ public class TaskManager implements TaskOperationProtocol {
 		{
 			HardwareDescription resources = HardwareDescriptionFactory.extractFromSystem();
 			
-			numberOfSlots = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
-					Hardware.getNumberCPUCores());
-
+			int slots = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, -1);
+			if (slots == -1) { 
+				slots = Hardware.getNumberCPUCores();
+			} else if (slots <= 0) {
+				throw new Exception("Illegal value for the number of task slots: " + slots);
+			}
+			this.numberOfSlots = slots;
+			
 			// Check whether the memory size has been explicitly configured. if so that overrides the default mechanism
 			// of taking as much as is mentioned in the hardware description
 			long memorySize = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b4b633ea/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ExecutionGraphTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ExecutionGraphTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ExecutionGraphTest.java
index 2e75305..9d8700d 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ExecutionGraphTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ExecutionGraphTest.java
@@ -104,7 +104,7 @@ public class ExecutionGraphTest {
 
 			LibraryCacheManager.register(jobID, new String[0]);
 
-			final ExecutionGraph eg = new ExecutionGraph(jg, -1);
+			final ExecutionGraph eg = new ExecutionGraph(jg, 1);
 
 			// test all methods of ExecutionGraph
 			final ExecutionStage executionStage = eg.getCurrentExecutionStage();
@@ -215,7 +215,7 @@ public class ExecutionGraphTest {
 			assertEquals(1, egv2.getNumberOfBackwardLinks());
 			assertEquals(1, egv2.getNumberOfForwardLinks());
 			assertEquals(0, egv2.getStageNumber());
-			assertEquals(-1, egv2.getUserDefinedNumberOfMembers());
+			assertEquals(1, egv2.getUserDefinedNumberOfMembers());
 			assertNull(egv2.getVertexToShareInstancesWith());
 
 			// test all methods of ExecutionVertex

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b4b633ea/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java
index 2549d4f..ffb958a 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java
@@ -66,7 +66,7 @@ public class JobManagerITCase {
 
 	static {
 		// no logging, because the tests create expected exception
-		LogUtils.initializeDefaultConsoleLogger(Level.INFO);
+		LogUtils.initializeDefaultConsoleLogger(Level.WARN);
 	}
 	
 	/**


[52/53] [abbrv] git commit: Fixed Bug Flink-978

Posted by rm...@apache.org.
Fixed Bug Flink-978

https://issues.apache.org/jira/browse/FLINK-978

Author: Jonathan <Jo...@Hasenburg.de>

Closes #46 from JonathanH5/fixWebI and squashes the following commits:

13a7d41 [Jonathan] Fixed Bug Flink-978


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

Branch: refs/heads/travis_test
Commit: 69589fa90ee08bb1c87a56bf57102f5a99d8af79
Parents: 47239b2
Author: Jonathan <Jo...@Hasenburg.de>
Authored: Thu Jun 26 10:09:00 2014 +0200
Committer: Robert Metzger <rm...@apache.org>
Committed: Thu Jun 26 10:09:00 2014 +0200

----------------------------------------------------------------------
 .../eu/stratosphere/nephele/managementgraph/ManagementVertex.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/69589fa9/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertex.java
index eaececc..a270700 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertex.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertex.java
@@ -318,7 +318,7 @@ public final class ManagementVertex extends ManagementAttachment implements IORe
 		json.append("\"vertexid\": \"" + this.getID() + "\",");
 		json.append("\"vertexname\": \"" + StringUtils.escapeHtml(this.toString()) + "\",");
 		json.append("\"vertexstatus\": \"" + this.getExecutionState() + "\",");
-		json.append("\"vertexinstancename\": \"" + this.getInstanceName() + "\",");
+		json.append("\"vertexinstancename\": \"" + this.getInstanceName() + "\"");
 		json.append("}");
 		return json.toString();
 	}


[14/53] [abbrv] git commit: Added failing behavior to JobManager if it is still in scheduled or created state New test case: Exception during ExecutionGraph construction in the JobManager Removed checkConfiguration method from AbstractJobVertex because i

Posted by rm...@apache.org.
Added failing behavior to JobManager if it is still in scheduled or created state
New test case: Exception during ExecutionGraph construction in the JobManager
Removed checkConfiguration method from AbstractJobVertex because it is not needed anymore


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

Branch: refs/heads/travis_test
Commit: 93bc0b9bf281490610d51f086eb0df781f207876
Parents: ea79186
Author: Till Rohrmann <ti...@mailbox.tu-berlin.de>
Authored: Thu Mar 27 03:01:22 2014 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Sun Jun 22 21:07:20 2014 +0200

----------------------------------------------------------------------
 .../plantranslate/NepheleJobGraphGenerator.java |   3 +-
 .../api/common/io/FileOutputFormat.java         |   5 +
 .../api/common/io/OutputFormat.java             |   3 +-
 .../configuration/Configuration.java            |   4 +-
 .../nephele/executiongraph/ExecutionGraph.java  |  11 +-
 .../jobgraph/AbstractJobInputVertex.java        |  13 +++
 .../nephele/jobgraph/AbstractJobVertex.java     |  12 --
 .../nephele/jobgraph/JobInputVertex.java        |  45 +++++++-
 .../nephele/jobgraph/JobOutputVertex.java       |  31 +++++
 .../jobmanager/ExceptionOutputFormat.java       |  53 +++++++++
 .../nephele/jobmanager/JobManagerITCase.java    | 114 ++++++++++++++++++-
 11 files changed, 268 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/93bc0b9b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plantranslate/NepheleJobGraphGenerator.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plantranslate/NepheleJobGraphGenerator.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plantranslate/NepheleJobGraphGenerator.java
index 200ef7c..3089cdb 100644
--- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plantranslate/NepheleJobGraphGenerator.java
+++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plantranslate/NepheleJobGraphGenerator.java
@@ -834,8 +834,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 		vertex.getConfiguration().setInteger(DataSinkTask.DEGREE_OF_PARALLELISM_KEY, node.getDegreeOfParallelism());
 
 		// set user code
-		vertex.setOutputFormat((UserCodeWrapper<? extends OutputFormat<?>>)node.getPactContract().getUserCodeWrapper
-				());
+		vertex.setOutputFormat((UserCodeWrapper<? extends OutputFormat<?>>)node.getPactContract().getUserCodeWrapper());
 		vertex.setOutputFormatParameters(node.getPactContract().getParameters());
 		
 		return vertex;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/93bc0b9b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/FileOutputFormat.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/FileOutputFormat.java b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/FileOutputFormat.java
index 7733c71..c4e1d5a 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/FileOutputFormat.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/FileOutputFormat.java
@@ -438,6 +438,11 @@ public abstract class FileOutputFormat<IT> implements OutputFormat<IT> {
 		}
 	}
 
+	/**
+	 * Initialization of the distributed file system if it is used.
+	 *
+	 * @param configuration The task configuration
+	 */
 	@Override
 	public void initialize(Configuration configuration){
 		final Path path = this.getOutputFilePath();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/93bc0b9b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/OutputFormat.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/OutputFormat.java b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/OutputFormat.java
index 3b66902..bdc59e4 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/OutputFormat.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/OutputFormat.java
@@ -81,7 +81,8 @@ public interface OutputFormat<IT> extends Serializable {
 	void close() throws IOException;
 
 	/**
-	 * Method which is called on the master node prior to execution. It can be used to set up the output format.
+	 * Method which is called on the JobManager node prior to execution. It can be used to set up output format
+	 * related tasks.
 	 *
 	 * @param configuration The task configuration
 	 */

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/93bc0b9b/stratosphere-core/src/main/java/eu/stratosphere/configuration/Configuration.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/configuration/Configuration.java b/stratosphere-core/src/main/java/eu/stratosphere/configuration/Configuration.java
index 0271b59..451577f 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/configuration/Configuration.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/configuration/Configuration.java
@@ -407,10 +407,10 @@ public class Configuration implements IOReadableWritable {
 	}
 
 	/**
-	 * Checks whether there is an entry with key
+	 * Checks whether there is an entry with the specified key
 	 *
 	 * @param key key of entry
-	 * @return true if entry with key is stored in the configuration, otherwise false
+	 * @return true if the key is stored, false otherwise
 	 */
 	public boolean containsKey(String key){
 		synchronized (this.confData){

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/93bc0b9b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java
index 93e0a25..1c4a820 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java
@@ -465,13 +465,6 @@ public class ExecutionGraph implements ExecutionListener {
 			throw new GraphConversionException(StringUtils.stringifyException(t));
 		}
 
-		// Run the configuration check the user has provided for the vertex
-		try {
-			jobVertex.checkConfiguration(groupVertex.getEnvironment().getInvokable());
-		} catch (IllegalConfigurationException e) {
-			throw new GraphConversionException(StringUtils.stringifyException(e));
-		}
-
 		// Register input and output vertices separately
 		if (jobVertex instanceof AbstractJobInputVertex) {
 
@@ -1043,6 +1036,8 @@ public class ExecutionGraph implements ExecutionListener {
 				if (eg.jobHasFailedOrCanceledStatus()) {
 					return InternalJobStatus.CANCELED;
 				}
+			}else if(latestStateChange == ExecutionState.FAILED){
+				return InternalJobStatus.FAILING;
 			}
 			break;
 		case SCHEDULED:
@@ -1052,6 +1047,8 @@ public class ExecutionGraph implements ExecutionListener {
 				if (eg.jobHasFailedOrCanceledStatus()) {
 					return InternalJobStatus.CANCELED;
 				}
+			}else if(latestStateChange == ExecutionState.FAILED){
+				return InternalJobStatus.FAILING;
 			}
 			break;
 		case RUNNING:

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/93bc0b9b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobInputVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobInputVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobInputVertex.java
index 22b4d7c..e4d3b9d 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobInputVertex.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobInputVertex.java
@@ -39,6 +39,19 @@ public abstract class AbstractJobInputVertex extends AbstractJobVertex {
 		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/93bc0b9b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java
index 7cec46a..cdadd3c 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java
@@ -573,18 +573,6 @@ public abstract class AbstractJobVertex implements IOReadableWritable {
 	}
 
 	/**
-	 * Performs check whether the vertex has been properly configured
-	 *
-	 * @param configuration
-	 *        configuration of this vertex
-	 * @throws IllegalConfigurationException
-	 *         thrown if the respective tasks is not configured properly
-	 */
-	public void checkConfiguration(final Configuration configuration) throws IllegalConfigurationException {
-		//default configuration check
-	}
-
-	/**
 	 * Returns the invokable class which represents the task of this vertex
 	 * 
 	 * @return the invokable class, <code>null</code> if it is not set

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/93bc0b9b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobInputVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobInputVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobInputVertex.java
index 9e5f6c7..29f98d9 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobInputVertex.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobInputVertex.java
@@ -27,6 +27,11 @@ import java.io.DataInput;
 import java.io.IOException;
 
 public class JobInputVertex extends AbstractJobInputVertex {
+	/**
+	 * Input format associated to this JobInputVertex. It is either directly set or reconstructed from the task
+	 * configuration. Every job input vertex requires an input format to compute the input splits and the input split
+	 * type.
+	 */
 	private volatile InputFormat<?, ? extends InputSplit> inputFormat = null;
 
 	/**
@@ -85,6 +90,11 @@ public class JobInputVertex extends AbstractJobInputVertex {
 		return (Class<? extends AbstractInputTask<?>>) this.invokableClass;
 	}
 
+	/**
+	 * Sets the input format and writes it to the task configuration. It extracts it from the UserCodeWrapper.
+	 *
+	 * @param inputFormatWrapper Wrapped input format
+	 */
 	public void setInputFormat(UserCodeWrapper<? extends InputFormat<?, ? extends InputSplit>> inputFormatWrapper) {
 		TaskConfig config = new TaskConfig(this.getConfiguration());
 		config.setStubWrapper(inputFormatWrapper);
@@ -92,6 +102,11 @@ public class JobInputVertex extends AbstractJobInputVertex {
 		inputFormat = inputFormatWrapper.getUserCodeObject();
 	}
 
+	/**
+	 * Sets the input format and writes it to the task configuration.
+	 *
+	 * @param inputFormat Input format
+	 */
 	public void setInputFormat(InputFormat<?, ? extends InputSplit> inputFormat) {
 		this.inputFormat = inputFormat;
 
@@ -101,6 +116,11 @@ public class JobInputVertex extends AbstractJobInputVertex {
 		config.setStubWrapper(wrapper);
 	}
 
+	/**
+	 * Sets the input format parameters.
+	 *
+	 * @param inputFormatParameters Input format parameters
+	 */
 	public void setInputFormatParameters(Configuration inputFormatParameters){
 		TaskConfig config = new TaskConfig(this.getConfiguration());
 		config.setStubParameters(inputFormatParameters);
@@ -112,12 +132,23 @@ public class JobInputVertex extends AbstractJobInputVertex {
 		inputFormat.configure(inputFormatParameters);
 	}
 
+	/**
+	 * Sets the output serializer for the task associated to this vertex.
+	 *
+	 * @param factory Type serializer factory
+	 */
 	public void setOutputSerializer(TypeSerializerFactory<?> factory){
 		TaskConfig config = new TaskConfig(this.getConfiguration());
 		config.setOutputSerializer(factory);
 	}
 
-
+	/**
+	 * Deserializes the input format from the deserialized task configuration. It then configures the input format by
+	 * calling the configure method with the current configuration.
+	 *
+	 * @param input
+	 * @throws IOException
+	 */
 	@Override
 	public void read(final DataInput input) throws IOException{
 		super.read(input);
@@ -143,6 +174,11 @@ public class JobInputVertex extends AbstractJobInputVertex {
 		inputFormat.configure(taskConfig.getStubParameters());
 	}
 
+	/**
+	 * Gets the input split type class
+	 *
+	 * @return Input split type class
+	 */
 	@Override
 	public Class<? extends InputSplit> getInputSplitType() {
 		if(inputFormat == null){
@@ -152,6 +188,13 @@ public class JobInputVertex extends AbstractJobInputVertex {
 		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){

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/93bc0b9b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobOutputVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobOutputVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobOutputVertex.java
index 154e639..cf937a0 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobOutputVertex.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobOutputVertex.java
@@ -31,6 +31,9 @@ import java.io.IOException;
  * 
  */
 public class JobOutputVertex extends AbstractJobOutputVertex {
+	/**
+	 * Contains the output format associated to this output vertex. It can be <pre>null</pre>.
+	 */
 	private volatile OutputFormat<?> outputFormat = null;
 
 	/**
@@ -89,12 +92,22 @@ public class JobOutputVertex extends AbstractJobOutputVertex {
 		return (Class<? extends AbstractOutputTask>) this.invokableClass;
 	}
 
+	/**
+	 * Sets the output format and writes it to the task configuration.
+	 *
+	 * @param outputFormatWrapper Wrapped output format
+	 */
 	public void setOutputFormat(UserCodeWrapper<? extends OutputFormat<?>> outputFormatWrapper){
 		TaskConfig config = new TaskConfig(this.getConfiguration());
 		config.setStubWrapper(outputFormatWrapper);
 		outputFormat = outputFormatWrapper.getUserCodeObject();
 	}
 
+	/**
+	 * Sets the output format and writes it to the task configuration.
+	 *
+	 * @param outputFormat Output format
+	 */
 	public void setOutputFormat(OutputFormat<?> outputFormat){
 		this.outputFormat = outputFormat;
 		UserCodeWrapper<? extends OutputFormat<?>> wrapper = new UserCodeObjectWrapper<OutputFormat<?>>
@@ -103,6 +116,11 @@ public class JobOutputVertex extends AbstractJobOutputVertex {
 		config.setStubWrapper(wrapper);
 	}
 
+	/**
+	 * Sets the output format parameters for the output format by writing it to the task configuration.
+	 *
+	 * @param parameters Output format parameters
+	 */
 	public void setOutputFormatParameters(Configuration parameters){
 		TaskConfig config = new TaskConfig(this.getConfiguration());
 		config.setStubParameters(parameters);
@@ -110,6 +128,14 @@ public class JobOutputVertex extends AbstractJobOutputVertex {
 		outputFormat.configure(parameters);
 	}
 
+	/**
+	 * Deserializes the output format from the deserialized configuration if it contains an output format. The output
+	 * format is always stored in the stub wrapper. If the task configuration contains an output format,
+	 * then it is configured after deserialization.
+	 *
+	 * @param input
+	 * @throws IOException
+	 */
 	@Override
 	public void read(final DataInput input) throws IOException{
 		super.read(input);
@@ -133,5 +159,10 @@ public class JobOutputVertex extends AbstractJobOutputVertex {
 		}
 	}
 
+	/**
+	 * 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/93bc0b9b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ExceptionOutputFormat.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ExceptionOutputFormat.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ExceptionOutputFormat.java
new file mode 100644
index 0000000..ffc4b42
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ExceptionOutputFormat.java
@@ -0,0 +1,53 @@
+/***********************************************************************************************************************
+ * 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.jobmanager;
+
+import eu.stratosphere.api.common.io.OutputFormat;
+import eu.stratosphere.configuration.Configuration;
+
+import java.io.IOException;
+
+
+public class ExceptionOutputFormat implements OutputFormat<Object> {
+	/**
+	 * 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) throws IOException {
+
+	}
+
+	@Override
+	public void writeRecord(Object record) throws IOException {
+
+	}
+
+	@Override
+	public void close() throws IOException {
+
+	}
+
+	@Override
+	public void initialize(Configuration configuration) {
+		throw new RuntimeException(RUNTIME_EXCEPTION_MESSAGE);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/93bc0b9b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java
index 89f7428..db2d9af 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java
@@ -367,7 +367,7 @@ public class JobManagerITCase {
 
 		try {
 
-			inputFile = ServerTestUtils.createInputFile(0);
+			inputFile = ServerTestUtils.createInputFile(100);
 			outputFile = new File(ServerTestUtils.getTempDir() + File.separator + ServerTestUtils.getRandomFilename());
 			jarFile = ServerTestUtils.createJarFile(runtimeExceptionClassName);
 
@@ -451,6 +451,118 @@ public class JobManagerITCase {
 	}
 
 	/**
+	 * 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 JobInputVertex i1 = new JobInputVertex("Input 1", jg);
+			i1.setNumberOfSubtasks(1);
+			Class<AbstractInputTask<?>> clazz = (Class<AbstractInputTask<?>>)(Class<?>)DataSourceTask
+					.class;
+			i1.setInputClass(clazz);
+			TextInputFormat inputFormat = new TextInputFormat();
+			inputFormat.setFilePath(new Path(inputFile.toURI()));
+			i1.setInputFormat(inputFormat);
+			i1.setInputFormat(inputFormat);
+			i1.setOutputSerializer(RecordSerializerFactory.get());
+			TaskConfig config= new TaskConfig(i1.getConfiguration());
+			config.addOutputShipStrategy(ShipStrategyType.FORWARD);
+
+			// task vertex 1
+			final JobTaskVertex t1 = new JobTaskVertex("Task with Exception", jg);
+			t1.setTaskClass(ForwardTask.class);
+
+			// output vertex
+			JobOutputVertex o1 = new JobOutputVertex("Output 1", jg);
+			o1.setNumberOfSubtasks(1);
+			o1.setOutputClass(DataSinkTask.class);
+			ExceptionOutputFormat outputFormat = new ExceptionOutputFormat();
+			o1.setOutputFormat(outputFormat);
+			TaskConfig outputConfig = new TaskConfig(o1.getConfiguration());
+			outputConfig.addInputToGroup(0);
+			outputConfig.setInputSerializer(RecordSerializerFactory.get(), 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);
+
+			// deactivate logging of expected test exceptions
+			Logger jcLogger = Logger.getLogger(JobClient.class);
+			Level jcLevel = jcLogger.getEffectiveLevel();
+			jcLogger.setLevel(Level.OFF);
+			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());
+				}
+
+				// Check if the correct error message is encapsulated in the exception
+				return;
+			}
+			finally {
+				jcLogger.setLevel(jcLevel);
+			}
+
+			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


[38/53] [abbrv] git commit: [FLINK-973] [FLINK-969] Unify, clean up, and extend all environment logging at JobManager/TaskManager startup

Posted by rm...@apache.org.
[FLINK-973] [FLINK-969] Unify, clean up, and extend all environment logging at JobManager/TaskManager startup

This closes #40.


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

Branch: refs/heads/travis_test
Commit: 28863ee089bd0f81d6541a631c2d8699eaa71471
Parents: d55ba70
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Jun 24 18:32:25 2014 +0200
Committer: uce <u....@fu-berlin.de>
Committed: Wed Jun 25 11:04:50 2014 +0200

----------------------------------------------------------------------
 .../nephele/jobmanager/JobManager.java          |  19 +--
 .../nephele/jobmanager/JobManagerUtils.java     |  48 +-----
 .../jobmanager/web/JobmanagerInfoServlet.java   |   6 +-
 .../nephele/taskmanager/TaskManager.java        |  25 +--
 .../runtime/util/EnvironmentInformation.java    | 157 +++++++++++++++++++
 5 files changed, 170 insertions(+), 85 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/28863ee0/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java
index 6401407..877288c 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java
@@ -79,7 +79,6 @@ import eu.stratosphere.nephele.ipc.Server;
 import eu.stratosphere.nephele.jobgraph.AbstractJobVertex;
 import eu.stratosphere.nephele.jobgraph.JobGraph;
 import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.jobmanager.JobManagerUtils.RevisionInformation;
 import eu.stratosphere.nephele.jobmanager.accumulators.AccumulatorManager;
 import eu.stratosphere.nephele.jobmanager.archive.ArchiveListener;
 import eu.stratosphere.nephele.jobmanager.archive.MemoryArchivist;
@@ -103,6 +102,7 @@ import eu.stratosphere.nephele.taskmanager.TaskExecutionState;
 import eu.stratosphere.nephele.taskmanager.TaskSubmissionResult;
 import eu.stratosphere.runtime.io.network.ConnectionInfoLookupResponse;
 import eu.stratosphere.runtime.io.network.RemoteReceiver;
+import eu.stratosphere.runtime.util.EnvironmentInformation;
 import eu.stratosphere.nephele.taskmanager.ExecutorThreadFactory;
 import eu.stratosphere.nephele.taskmanager.transferenvelope.RegisterTaskManagerResult;
 import eu.stratosphere.nephele.topology.NetworkTopology;
@@ -287,17 +287,6 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 		this.isShutDown = true;
 		LOG.debug("Shutdown of job manager completed");
 	}
-
-	/**
-	 * Log Stratosphere version information.
-	 */
-	private static void logVersionInformation() {
-		RevisionInformation rev = JobManagerUtils.getRevisionInformation();
-		LOG.info("Starting Stratosphere JobManager "
-				+ "(Version: " + JobManagerUtils.getVersion() + ", "
-					+ "Rev:" + rev.commitId + ", "
-					+ "Date:" + rev.commitDate + ")");
-	}
 	
 	/**
 	 * Entry point for the program
@@ -340,9 +329,6 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 	
 	@SuppressWarnings("static-access")
 	public static JobManager initialize(String[] args) throws Exception {
-		// output the version and revision information to the log
-		logVersionInformation();
-		
 		final Option configDirOpt = OptionBuilder.withArgName("config directory").hasArg()
 			.withDescription("Specify configuration directory.").create("configDir");
 
@@ -375,6 +361,9 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 			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);
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/28863ee0/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManagerUtils.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManagerUtils.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManagerUtils.java
index f2e1d33..31879b8 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManagerUtils.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManagerUtils.java
@@ -13,18 +13,15 @@
 
 package eu.stratosphere.nephele.jobmanager;
 
-import java.io.IOException;
-import java.io.InputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
-import java.util.Properties;
 
-import eu.stratosphere.nephele.ExecutionMode;
-import eu.stratosphere.nephele.instance.InstanceManager;
-import eu.stratosphere.nephele.jobmanager.scheduler.DefaultScheduler;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
+import eu.stratosphere.nephele.ExecutionMode;
+import eu.stratosphere.nephele.instance.InstanceManager;
+import eu.stratosphere.nephele.jobmanager.scheduler.DefaultScheduler;
 import eu.stratosphere.util.StringUtils;
 
 /**
@@ -167,43 +164,4 @@ public class JobManagerUtils {
 			throw new RuntimeException("Unrecognized Execution Mode.");
 		}
 	}
-	
-	/**
-	 * Returns the version of Stratosphere as String.
-	 * If version == null, then the JobManager runs from inside the IDE (or somehow not from the maven build jar)
-	 * @return String
-	 */
-	public static String getVersion() {
-		String version = JobManagerUtils.class.getPackage().getImplementationVersion();
-		return version;
-	}
-
-	/**
-	 * Returns the revision of Stratosphere as String.
-	 * @return String
-	 */
-	public static RevisionInformation getRevisionInformation() {
-		RevisionInformation info = new RevisionInformation();
-		String revision = "<unknown>";
-		String commitDate = "<unknown>";
-		try {
-			Properties properties = new Properties();
-			InputStream propFile = JobManagerUtils.class.getClassLoader().getResourceAsStream(".version.properties");
-			if (propFile != null) {
-				properties.load(propFile);
-				revision = properties.getProperty("git.commit.id.abbrev");
-				commitDate = properties.getProperty("git.commit.time");
-			}
-		} catch (IOException e) {
-			LOG.info("Cannot determine code revision. Unable ro read version property file.");
-		}
-		info.commitId = revision;
-		info.commitDate = commitDate;
-		return info;
-	}
-	
-	public static class RevisionInformation {
-		public String commitId;
-		public String commitDate;
-	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/28863ee0/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/web/JobmanagerInfoServlet.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/web/JobmanagerInfoServlet.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/web/JobmanagerInfoServlet.java
index 8ffbd57..4b23399 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/web/JobmanagerInfoServlet.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/web/JobmanagerInfoServlet.java
@@ -40,13 +40,13 @@ import eu.stratosphere.nephele.execution.ExecutionState;
 import eu.stratosphere.nephele.jobgraph.JobID;
 import eu.stratosphere.nephele.jobgraph.JobStatus;
 import eu.stratosphere.nephele.jobmanager.JobManager;
-import eu.stratosphere.nephele.jobmanager.JobManagerUtils;
 import eu.stratosphere.nephele.managementgraph.ManagementGraph;
 import eu.stratosphere.nephele.managementgraph.ManagementGraphIterator;
 import eu.stratosphere.nephele.managementgraph.ManagementGroupVertex;
 import eu.stratosphere.nephele.managementgraph.ManagementGroupVertexID;
 import eu.stratosphere.nephele.managementgraph.ManagementVertex;
 import eu.stratosphere.nephele.services.accumulators.AccumulatorEvent;
+import eu.stratosphere.runtime.util.EnvironmentInformation;
 import eu.stratosphere.util.StringUtils;
 
 
@@ -516,8 +516,8 @@ public class JobmanagerInfoServlet extends HttpServlet {
 	 */
 	private void writeJsonForVersion(PrintWriter wrt) {
 		wrt.write("{");
-		wrt.write("\"version\": \"" + JobManagerUtils.getVersion() + "\",");
-		wrt.write("\"revision\": \"" + JobManagerUtils.getRevisionInformation().commitId + "\"");
+		wrt.write("\"version\": \"" + EnvironmentInformation.getVersion() + "\",");
+		wrt.write("\"revision\": \"" + EnvironmentInformation.getRevisionInformation().commitId + "\"");
 		wrt.write("}");
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/28863ee0/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
index d866c64..bedafaf 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
@@ -58,7 +58,6 @@ import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.security.UserGroupInformation;
 
 import eu.stratosphere.api.common.cache.DistributedCache;
 import eu.stratosphere.api.common.cache.DistributedCache.DistributedCacheEntry;
@@ -80,8 +79,6 @@ import eu.stratosphere.nephele.instance.InstanceConnectionInfo;
 import eu.stratosphere.nephele.ipc.RPC;
 import eu.stratosphere.nephele.ipc.Server;
 import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.jobmanager.JobManagerUtils;
-import eu.stratosphere.nephele.jobmanager.JobManagerUtils.RevisionInformation;
 import eu.stratosphere.nephele.net.NetUtils;
 import eu.stratosphere.nephele.profiling.ProfilingUtils;
 import eu.stratosphere.nephele.profiling.TaskManagerProfiler;
@@ -98,6 +95,7 @@ import eu.stratosphere.pact.runtime.cache.FileCache;
 import eu.stratosphere.runtime.io.channels.ChannelID;
 import eu.stratosphere.runtime.io.network.ChannelManager;
 import eu.stratosphere.runtime.io.network.InsufficientResourcesException;
+import eu.stratosphere.runtime.util.EnvironmentInformation;
 import eu.stratosphere.util.StringUtils;
 
 /**
@@ -173,18 +171,6 @@ public class TaskManager implements TaskOperationProtocol {
 		if (executionMode == null) {
 			throw new NullPointerException("Execution mode must not be null.");
 		}
-		
-		RevisionInformation rev = JobManagerUtils.getRevisionInformation();
-		LOG.info("Starting Stratosphere TaskManager "
-				+ "(Version: " + JobManagerUtils.getVersion() + ", "
-					+ "Rev:" + rev.commitId + ", "
-					+ "Date:" + rev.commitDate + ")");
-		
-		try {
-			LOG.info("TaskManager started as user " + UserGroupInformation.getCurrentUser().getShortUserName());
-		} catch (Throwable t) {
-			LOG.error("Cannot determine user group information.", t);
-		}
 			
 		LOG.info("Execution mode: " + executionMode);
 
@@ -513,14 +499,9 @@ public class TaskManager implements TaskOperationProtocol {
 			LOG.info("Setting temporary directory to "+tempDirVal);
 			GlobalConfiguration.includeConfiguration(c);
 		}
-		System.err.println("Configuration "+GlobalConfiguration.getConfiguration());
-		LOG.info("Current user "+UserGroupInformation.getCurrentUser().getShortUserName());
 		
-		{
-			// log the available JVM memory
-			long maxMemoryMiBytes = Runtime.getRuntime().maxMemory() >>> 20;
-			LOG.info("Starting TaskManager in a JVM with " + maxMemoryMiBytes + " MiBytes maximum heap size.");
-		}
+		// print some startup environment info, like user, code revision, etc
+		EnvironmentInformation.logEnvironmentInfo(LOG, "TaskManager");
 		
 		// Create a new task manager object
 		try {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/28863ee0/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/util/EnvironmentInformation.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/util/EnvironmentInformation.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/util/EnvironmentInformation.java
new file mode 100644
index 0000000..29d0804
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/util/EnvironmentInformation.java
@@ -0,0 +1,157 @@
+/***********************************************************************************************************************
+ *
+ * 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.runtime.util;
+
+import java.io.InputStream;
+import java.lang.management.ManagementFactory;
+import java.lang.management.RuntimeMXBean;
+import java.util.Properties;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.security.UserGroupInformation;
+
+public class EnvironmentInformation {
+
+	private static final Log LOG = LogFactory.getLog(EnvironmentInformation.class);
+
+	private static final String UNKNOWN = "<unknown>";
+
+	private static final String LOG_FILE_OPTION = "-Dlog.file";
+
+	private static final String LOG_CONFIGURAION_OPTION = "-Dlog4j.configuration";
+
+	/**
+	 * Returns the version of the code as String. If version == null, then the JobManager does not run from a
+	 * maven build. An example is a source code checkout, compile, and run from inside an IDE.
+	 * 
+	 * @return The version string.
+	 */
+	public static String getVersion() {
+		return EnvironmentInformation.class.getPackage().getImplementationVersion();
+	}
+
+	/**
+	 * Returns the code revision (commit and commit date) of Stratosphere.
+	 * 
+	 * @return The code revision.
+	 */
+	public static RevisionInformation getRevisionInformation() {
+		RevisionInformation info = new RevisionInformation();
+		String revision = UNKNOWN;
+		String commitDate = UNKNOWN;
+		try {
+			Properties properties = new Properties();
+			InputStream propFile = EnvironmentInformation.class.getClassLoader().getResourceAsStream(".version.properties");
+			if (propFile != null) {
+				properties.load(propFile);
+				revision = properties.getProperty("git.commit.id.abbrev");
+				commitDate = properties.getProperty("git.commit.time");
+			}
+		} catch (Throwable t) {
+			if (LOG.isDebugEnabled()) {
+				LOG.debug("Cannot determine code revision: Unable ro read version property file.", t);
+			} else {
+				LOG.info("Cannot determine code revision: Unable ro read version property file.");
+			}
+		}
+		info.commitId = revision;
+		info.commitDate = commitDate;
+		return info;
+	}
+
+	public static class RevisionInformation {
+		public String commitId;
+		public String commitDate;
+	}
+
+	public static String getUserRunning() {
+		try {
+			return UserGroupInformation.getCurrentUser().getShortUserName();
+		} catch (Throwable t) {
+			if (LOG.isDebugEnabled()) {
+				LOG.debug("Cannot determine user/group information for the current user.", t);
+			} else {
+				LOG.info("Cannot determine user/group information for the current user.");
+			}
+			return UNKNOWN;
+		}
+	}
+
+	public static long getMaxJvmMemory() {
+		return Runtime.getRuntime().maxMemory() >>> 20;
+	}
+
+	public static String getJvmVersion() {
+		try {
+			final RuntimeMXBean bean = ManagementFactory.getRuntimeMXBean();
+			return bean.getVmName() + " - " + bean.getVmVendor() + " - " + bean.getSpecVersion() + '/' + bean.getVmVersion();
+		}
+		catch (Throwable t) {
+			return UNKNOWN;
+		}
+	}
+
+	public static String getJvmStartupOptions() {
+		try {
+			final RuntimeMXBean bean = ManagementFactory.getRuntimeMXBean();
+			final StringBuilder bld = new StringBuilder();
+			for (String s : bean.getInputArguments()) {
+				if (!s.startsWith(LOG_FILE_OPTION) && !s.startsWith(LOG_CONFIGURAION_OPTION)) {
+					bld.append(s).append(' ');
+				}
+			}
+			return bld.toString();
+		}
+		catch (Throwable t) {
+			return UNKNOWN;
+		}
+	}
+
+	public static void logEnvironmentInfo(Log log, String componentName) {
+		if (log.isInfoEnabled()) {
+			RevisionInformation rev = getRevisionInformation();
+			String version = getVersion();
+			
+			String user = getUserRunning();
+			
+			String jvmVersion = getJvmVersion();
+			String options = getJvmStartupOptions();
+			
+			String javaHome = System.getenv("JAVA_HOME");
+			
+			long memory = getMaxJvmMemory();
+			
+			log.info("-------------------------------------------------------");
+			log.info(" Starting " + componentName + " (Version: " + version + ", "
+					+ "Rev:" + rev.commitId + ", " + "Date:" + rev.commitDate + ")");
+			log.info(" Current user: " + user);
+			log.info(" JVM: " + jvmVersion);
+			log.info(" Startup Options: " + options);
+			log.info(" Maximum heap size: " + memory + " MiBytes");
+			log.info(" JAVA_HOME: " + (javaHome == null ? "not set" : javaHome));
+			log.info("-------------------------------------------------------");
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+	
+	private EnvironmentInformation() {}
+	
+	public static void main(String[] args) {
+		logEnvironmentInfo(LOG, "Test");
+	}
+}


[35/53] [abbrv] git commit: [FLINK-980] Fix buffer leak in OutputChannel

Posted by rm...@apache.org.
[FLINK-980] Fix buffer leak in OutputChannel


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

Branch: refs/heads/travis_test
Commit: f13ad5b415a57e7d1c97319935a04f076cc1776b
Parents: 2f0bd8f
Author: uce <u....@fu-berlin.de>
Authored: Wed Jun 25 10:30:32 2014 +0200
Committer: uce <u....@fu-berlin.de>
Committed: Wed Jun 25 10:30:32 2014 +0200

----------------------------------------------------------------------
 .../java/eu/stratosphere/runtime/io/channels/OutputChannel.java     | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f13ad5b4/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/OutputChannel.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/OutputChannel.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/OutputChannel.java
index f4fed65..b620f78 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/OutputChannel.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/OutputChannel.java
@@ -66,6 +66,7 @@ public class OutputChannel extends Channel {
 
 		// discard empty buffers
 		if (buffer.size() == 0) {
+			buffer.recycleBuffer();
 			return;
 		}
 


[41/53] [abbrv] git commit: [FLINK-949] Properly report GlobalBufferPool OutOfMemoryError to TaskManager

Posted by rm...@apache.org.
[FLINK-949] Properly report GlobalBufferPool OutOfMemoryError to TaskManager

This closes #28.


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

Branch: refs/heads/travis_test
Commit: 3d6cc5f48dc5a0336f8afdd35f59f1ee25357766
Parents: 96e76a5
Author: uce <u....@fu-berlin.de>
Authored: Thu Jun 19 02:42:03 2014 +0200
Committer: uce <u....@fu-berlin.de>
Committed: Wed Jun 25 12:07:16 2014 +0200

----------------------------------------------------------------------
 .../nephele/taskmanager/TaskManager.java        |  9 ++++----
 .../runtime/io/network/ChannelManager.java      |  6 ++++-
 .../bufferprovider/GlobalBufferPool.java        | 24 ++++++++++++++++----
 3 files changed, 28 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/3d6cc5f4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
index bedafaf..cb06d08 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
@@ -288,11 +288,7 @@ public class TaskManager implements TaskOperationProtocol {
 				ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator);
 
 		checkTempDirs(tmpDirPaths);
-		
-		final int pageSize = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY,
-			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_BUFFER_SIZE);
 
-		// Initialize network buffer pool
 		int numBuffers = GlobalConfiguration.getInteger(
 				ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
 				ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS);
@@ -335,7 +331,7 @@ public class TaskManager implements TaskOperationProtocol {
 			channelManager = new ChannelManager(lookupService, localInstanceConnectionInfo, numBuffers, bufferSize, networkConnectionManager);
 		} catch (IOException ioe) {
 			LOG.error(StringUtils.stringifyException(ioe));
-			throw new Exception("Failed to instantiate channel manager. " + ioe.getMessage(), ioe);
+			throw new Exception("Failed to instantiate ChannelManager.", ioe);
 		}
 
 		{
@@ -363,6 +359,9 @@ public class TaskManager implements TaskOperationProtocol {
 			}
 			this.hardwareDescription = resources;
 
+			final int pageSize = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY,
+					ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_BUFFER_SIZE);
+
 			// Initialize the memory manager
 			LOG.info("Initializing memory manager with " + (resources.getSizeOfFreeMemory() >>> 20) + " megabytes of memory. " +
 					"Page size is " + pageSize + " bytes.");

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/3d6cc5f4/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/ChannelManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/ChannelManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/ChannelManager.java
index 34d3501..10a0cfb 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/ChannelManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/ChannelManager.java
@@ -79,7 +79,11 @@ public class ChannelManager implements EnvelopeDispatcher, BufferProviderBroker
 		this.channelLookupService = channelLookupService;
 		this.connectionInfo = connectionInfo;
 
-		this.globalBufferPool = new GlobalBufferPool(numNetworkBuffers, networkBufferSize);
+		try {
+			this.globalBufferPool = new GlobalBufferPool(numNetworkBuffers, networkBufferSize);
+		} catch (Throwable e) {
+			throw new IOException("Failed to instantiate GlobalBufferPool.", e);
+		}
 
 		this.networkConnectionManager = networkConnectionManager;
 		networkConnectionManager.start(this);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/3d6cc5f4/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/GlobalBufferPool.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/GlobalBufferPool.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/GlobalBufferPool.java
index 2141017..aca1a9f 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/GlobalBufferPool.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/GlobalBufferPool.java
@@ -52,13 +52,27 @@ public final class GlobalBufferPool {
 		this.numBuffers = numBuffers;
 		this.bufferSize = bufferSize;
 
-		this.buffers = new ArrayBlockingQueue<MemorySegment>(this.numBuffers);
-		for (int i = 0; i < this.numBuffers; i++) {
-			this.buffers.add(new MemorySegment(new byte[this.bufferSize]));
+		buffers = new ArrayBlockingQueue<MemorySegment>(numBuffers);
+
+		final int mb = 1 << 20;
+		final int memRequiredMb = (numBuffers * bufferSize) / mb;
+
+		for (int i = 0; i < numBuffers; i++) {
+			try {
+				byte[] buf = new byte[bufferSize];
+				buffers.add(new MemorySegment(buf));
+			} catch (OutOfMemoryError err) {
+				int memAllocatedMb = ((i + 1) * bufferSize) / mb;
+
+				String msg = String.format("Tried to allocate %d buffers of size %d bytes each (total: %d MB) " +
+						"and ran out of memory after %d buffers (%d MB).",
+						numBuffers, bufferSize, memRequiredMb, i + 1, memAllocatedMb);
+				throw new OutOfMemoryError(msg);
+			}
 		}
 
-		LOG.info(String.format("Initialized global buffer pool with %d buffers (%d bytes each).",
-				this.numBuffers, this.bufferSize));
+		LOG.info(String.format("Allocated %d buffers of size %d bytes each (total: %d MB).",
+				numBuffers, bufferSize, memRequiredMb));
 	}
 
 	// -----------------------------------------------------------------------------------------------------------------


[37/53] [abbrv] git commit: LocalExecutor automatically picks up the maximum DOP for plans as its number of slots.

Posted by rm...@apache.org.
LocalExecutor automatically picks up the maximum DOP for plans as its number of slots.


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

Branch: refs/heads/travis_test
Commit: d55ba70eb0d09bf4f62da390f19f1b10a0a4d94d
Parents: fe91d4b
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Jun 24 17:52:26 2014 +0200
Committer: uce <u....@fu-berlin.de>
Committed: Wed Jun 25 10:57:57 2014 +0200

----------------------------------------------------------------------
 .../eu/stratosphere/client/LocalExecutor.java   | 11 ++++-
 .../java/eu/stratosphere/api/common/Plan.java   | 49 +++++++++-----------
 .../jobmanager/scheduler/DefaultScheduler.java  |  2 +-
 3 files changed, 33 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d55ba70e/stratosphere-clients/src/main/java/eu/stratosphere/client/LocalExecutor.java
----------------------------------------------------------------------
diff --git a/stratosphere-clients/src/main/java/eu/stratosphere/client/LocalExecutor.java b/stratosphere-clients/src/main/java/eu/stratosphere/client/LocalExecutor.java
index b017220..d73f893 100644
--- a/stratosphere-clients/src/main/java/eu/stratosphere/client/LocalExecutor.java
+++ b/stratosphere-clients/src/main/java/eu/stratosphere/client/LocalExecutor.java
@@ -42,7 +42,7 @@ public class LocalExecutor extends PlanExecutor {
 	
 	private static boolean DEFAULT_OVERWRITE = false;
 
-	private static final int DEFAULT_TASK_MANAGER_NUM_SLOTS = 1;
+	private static final int DEFAULT_TASK_MANAGER_NUM_SLOTS = -1;
 
 	private final Object lock = new Object();	// we lock to ensure singleton execution
 	
@@ -214,6 +214,15 @@ public class LocalExecutor extends PlanExecutor {
 			if (this.nephele == null) {
 				// we start a session just for us now
 				shutDownAtEnd = true;
+				
+				// configure the number of local slots equal to the parallelism of the local plan
+				if (this.taskManagerNumSlots == DEFAULT_TASK_MANAGER_NUM_SLOTS) {
+					int maxParallelism = plan.getMaximumParallelism();
+					if (maxParallelism > 0) {
+						this.taskManagerNumSlots = maxParallelism;
+					}
+				}
+				
 				start();
 			} else {
 				// we use the existing session

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d55ba70e/stratosphere-core/src/main/java/eu/stratosphere/api/common/Plan.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/api/common/Plan.java b/stratosphere-core/src/main/java/eu/stratosphere/api/common/Plan.java
index b6f5385..8824107 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/api/common/Plan.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/api/common/Plan.java
@@ -62,11 +62,6 @@ public class Plan implements Visitable<Operator<?>> {
 	 * The default parallelism to use for nodes that have no explicitly specified parallelism.
 	 */
 	protected int defaultParallelism = DEFAULT_PARALELLISM;
-	
-	/**
-	 * The maximal number of machines to use in the job.
-	 */
-	protected int maxNumberMachines;
 
 	/**
 	 * Hash map for files in the distributed cache: registered name to cache entry.
@@ -234,28 +229,6 @@ public class Plan implements Visitable<Operator<?>> {
 		checkNotNull(jobName, "The job name must not be null.");
 		this.jobName = jobName;
 	}
-
-	/**
-	 * Gets the maximum number of machines to be used for this job.
-	 * 
-	 * @return The maximum number of machines to be used for this job.
-	 */
-	public int getMaxNumberMachines() {
-		return this.maxNumberMachines;
-	}
-
-	/**
-	 * Sets the maximum number of machines to be used for this job.
-	 * 
-	 * @param maxNumberMachines The the maximum number to set.
-	 */
-	public void setMaxNumberMachines(int maxNumberMachines) {
-		if (maxNumberMachines == 0 || maxNumberMachines < -1) {
-			throw new IllegalArgumentException("The maximum number of machines must be positive, or -1 if no limit is imposed.");
-		}
-		
-		this.maxNumberMachines = maxNumberMachines;
-	}
 	
 	/**
 	 * Gets the default degree of parallelism for this job. That degree is always used when an operator
@@ -338,4 +311,26 @@ public class Plan implements Visitable<Operator<?>> {
 	public Set<Entry<String,DistributedCacheEntry>> getCachedFiles() {
 		return this.cacheFile.entrySet();
 	}
+	
+	public int getMaximumParallelism() {
+		MaxDopVisitor visitor = new MaxDopVisitor();
+		accept(visitor);
+		return Math.max(visitor.maxDop, this.defaultParallelism);
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	private static final class MaxDopVisitor implements Visitor<Operator<?>> {
+
+		private int maxDop = -1;
+		
+		@Override
+		public boolean preVisit(Operator<?> visitable) {
+			this.maxDop = Math.max(this.maxDop, visitable.getDegreeOfParallelism());
+			return true;
+		}
+
+		@Override
+		public void postVisit(Operator<?> visitable) {}
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d55ba70e/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/DefaultScheduler.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/DefaultScheduler.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/DefaultScheduler.java
index 745b199..48c9143 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/DefaultScheduler.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/DefaultScheduler.java
@@ -269,7 +269,7 @@ public class DefaultScheduler implements InstanceListener, JobStatusListener, Ex
 
 			final int requiredSlots = executionStage.getRequiredSlots();
 
-			LOG.info("Requesting " + requiredSlots + " for job " + executionGraph.getJobID());
+			LOG.info("Requesting " + requiredSlots + " slots for job " + executionGraph.getJobID());
 
 			this.instanceManager.requestInstance(executionGraph.getJobID(), executionGraph.getJobConfiguration(),
 				requiredSlots);


[11/53] [abbrv] git commit: Rework the Taskmanager to a slot based model and remove legacy cloud code

Posted by rm...@apache.org.
Rework the Taskmanager to a slot based model and remove legacy cloud code

Squashed commit of the following:

  - Post merge cleanup
  - Renamed fractionMemory into memoryFraction.
  - Removed Local and QueueScheduler and replaced it instead with an unified DefaultScheduler.
  - Removed Local and ClusterManager and inserted instead an unified DefaultInstanceManager.
  - Removed connection IDs from execution edges
  - Removed InstanceType, InstanceRequestMap, InstanceTypeDescription, InstanceTypeDescriptionTypeFactory, PendingRequestsMap
  - Fixed problems with test cases.
  - introduced simple slot system for scheduling.
  - Removed subtasks per instance
  - Added registerTaskManager to the JobManager RPC calls. RegisterTaskManager is called only once where the hardware description information is sent.

Add: Merging cloudmodel remove with new network stack


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

Branch: refs/heads/travis_test
Commit: 86d206c41922a1b7b8c2839b65d3568f9be55e0c
Parents: 7b6b5a2
Author: Till Rohrmann <ti...@gmail.com>
Authored: Sun Jun 1 16:03:27 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sun Jun 22 21:07:10 2014 +0200

----------------------------------------------------------------------
 .../api/avro/AvroExternalJarProgramITCase.java  |   1 +
 .../eu/stratosphere/client/LocalExecutor.java   |   9 +
 .../client/minicluster/NepheleMiniCluster.java  |  23 +-
 .../eu/stratosphere/client/program/Client.java  |   5 +-
 .../client/CliFrontendListCancelTest.java       |  11 +-
 .../stratosphere/client/testjar/WordCount.java  |   3 +-
 .../eu/stratosphere/compiler/PactCompiler.java  | 473 +---------
 .../compiler/costs/DefaultCostEstimator.java    |  18 +-
 .../dag/AbstractPartialSolutionNode.java        |   5 -
 .../compiler/dag/BinaryUnionNode.java           |  48 +-
 .../compiler/dag/BulkIterationNode.java         |  20 +-
 .../stratosphere/compiler/dag/DataSinkNode.java |  18 +-
 .../compiler/dag/DataSourceNode.java            |  16 -
 .../compiler/dag/GroupReduceNode.java           |   1 -
 .../compiler/dag/OptimizerNode.java             |  51 +-
 .../stratosphere/compiler/dag/ReduceNode.java   |   1 -
 .../compiler/dag/SingleInputNode.java           |  41 +-
 .../stratosphere/compiler/dag/SinkJoiner.java   |   1 -
 .../stratosphere/compiler/dag/TwoInputNode.java |  56 +-
 .../compiler/dag/WorksetIterationNode.java      |  12 +-
 .../RequestedGlobalProperties.java              |   8 +-
 .../AllGroupWithPartialPreGroupProperties.java  |   3 +-
 .../compiler/operators/AllReduceProperties.java |   3 +-
 .../GroupReduceWithCombineProperties.java       |   6 +-
 .../operators/PartialGroupProperties.java       |   6 +-
 .../compiler/operators/ReduceProperties.java    |   3 +-
 .../eu/stratosphere/compiler/plan/Channel.java  |  59 +-
 .../eu/stratosphere/compiler/plan/PlanNode.java |  25 +-
 .../plandump/PlanJSONDumpGenerator.java         |   3 -
 .../plantranslate/NepheleJobGraphGenerator.java |  80 +-
 .../pact/compiler/CompilerTestBase.java         |  24 +-
 .../configuration/ConfigConstants.java          |  12 +-
 .../java/eu/stratosphere/util/ClassUtils.java   |   1 +
 .../event/job/VertexAssignmentEvent.java        |  32 +-
 .../nephele/executiongraph/ExecutionEdge.java   |   9 -
 .../nephele/executiongraph/ExecutionGraph.java  | 181 ++--
 .../executiongraph/ExecutionGroupVertex.java    | 184 +---
 .../nephele/executiongraph/ExecutionStage.java  | 112 +--
 .../nephele/executiongraph/ExecutionVertex.java |   1 -
 .../executiongraph/InternalJobStatus.java       |   1 +
 .../executiongraph/ManagementGraphFactory.java  |   7 +-
 .../nephele/instance/AbstractInstance.java      | 297 ------
 .../nephele/instance/AllocatedResource.java     |  38 +-
 .../nephele/instance/AllocatedSlot.java         |  65 ++
 .../nephele/instance/AllocationID.java          |   4 +-
 .../instance/DefaultInstanceManager.java        | 393 ++++++++
 .../nephele/instance/DummyInstance.java         |  14 +-
 .../stratosphere/nephele/instance/Hardware.java |  24 +
 .../stratosphere/nephele/instance/Instance.java | 362 +++++++
 .../nephele/instance/InstanceManager.java       | 145 +--
 .../nephele/instance/InstanceNotifier.java      |  71 ++
 .../nephele/instance/InstanceRequestMap.java    | 184 ----
 .../nephele/instance/InstanceType.java          | 199 ----
 .../instance/InstanceTypeDescription.java       | 137 ---
 .../InstanceTypeDescriptionFactory.java         |  46 -
 .../nephele/instance/InstanceTypeFactory.java   |  91 --
 .../nephele/instance/LocalInstanceManager.java  |  60 ++
 .../instance/cluster/AllocatedSlice.java        | 120 ---
 .../instance/cluster/ClusterInstance.java       | 181 ----
 .../cluster/ClusterInstanceNotifier.java        |  71 --
 .../instance/cluster/ClusterManager.java        | 945 -------------------
 .../instance/cluster/PendingRequestsMap.java    |  97 --
 .../nephele/instance/local/LocalInstance.java   |  37 -
 .../instance/local/LocalInstanceManager.java    | 418 --------
 .../instance/local/LocalInstanceNotifier.java   |  70 --
 .../nephele/jobgraph/AbstractJobVertex.java     | 100 +-
 .../nephele/jobmanager/DeploymentManager.java   |   8 +-
 .../nephele/jobmanager/EventCollector.java      |  10 +-
 .../nephele/jobmanager/JobManager.java          |  98 +-
 .../nephele/jobmanager/JobManagerUtils.java     |  54 +-
 .../scheduler/AbstractExecutionListener.java    | 166 ----
 .../jobmanager/scheduler/AbstractScheduler.java | 662 -------------
 .../scheduler/DefaultExecutionListener.java     | 127 +++
 .../jobmanager/scheduler/DefaultScheduler.java  | 762 +++++++++++++++
 .../jobmanager/scheduler/RecoveryLogic.java     | 248 -----
 .../scheduler/local/LocalExecutionListener.java |  33 -
 .../scheduler/local/LocalScheduler.java         | 213 -----
 .../scheduler/queue/QueueExecutionListener.java |  40 -
 .../scheduler/queue/QueueScheduler.java         | 216 -----
 .../splitassigner/InputSplitManager.java        |   2 +-
 .../LocatableInputSplitAssigner.java            |   4 +-
 .../splitassigner/LocatableInputSplitList.java  |  20 +-
 .../file/FileInputSplitAssigner.java            |   4 +-
 .../splitassigner/file/FileInputSplitList.java  |  20 +-
 .../managementgraph/ManagementGraph.java        |   4 +-
 .../managementgraph/ManagementVertex.java       |  35 +-
 .../eu/stratosphere/nephele/net/NetUtils.java   |   2 +
 .../profiling/impl/JobProfilingData.java        |   6 +-
 .../protocols/ExtendedManagementProtocol.java   |  23 +-
 .../nephele/protocols/JobManagerProtocol.java   |  19 +-
 .../services/iomanager/ChannelAccess.java       |   1 +
 .../services/memorymanager/MemoryManager.java   |  17 +-
 .../memorymanager/spi/DefaultMemoryManager.java |  39 +-
 .../nephele/taskmanager/TaskManager.java        | 123 ++-
 .../RegisterTaskManagerResult.java              |  50 +
 .../nephele/topology/NetworkNode.java           |  10 -
 .../eu/stratosphere/nephele/util/IOUtils.java   |   1 +
 .../pact/runtime/cache/FileCache.java           |   9 +-
 .../hash/BuildFirstHashMatchIterator.java       |   8 +-
 .../BuildFirstReOpenableHashMatchIterator.java  |   8 +-
 .../hash/BuildSecondHashMatchIterator.java      |   8 +-
 .../pact/runtime/hash/InMemoryPartition.java    |   2 +
 .../iterative/task/IterationHeadPactTask.java   |   5 +-
 .../pact/runtime/shipping/ShipStrategyType.java |  23 +-
 .../runtime/sort/AsynchronousPartialSorter.java |  11 +-
 .../AsynchronousPartialSorterCollector.java     |   7 +-
 .../sort/CombiningUnilateralSortMerger.java     |  18 +-
 .../pact/runtime/sort/UnilateralSortMerger.java |  18 +-
 .../AbstractCachedBuildSideMatchDriver.java     |   2 +-
 .../pact/runtime/task/CrossDriver.java          |   3 +-
 .../pact/runtime/task/DataSinkTask.java         |   2 +-
 .../runtime/task/GroupReduceCombineDriver.java  |   4 +-
 .../pact/runtime/task/MatchDriver.java          |  38 +-
 .../pact/runtime/task/ReduceCombineDriver.java  |   3 +-
 .../pact/runtime/task/RegularPactTask.java      |  12 +-
 .../SynchronousChainedCombineDriver.java        |   2 +-
 .../pact/runtime/task/util/TaskConfig.java      |  68 +-
 .../runtime/io/channels/InputChannel.java       |   9 +-
 .../runtime/io/gates/InputGate.java             |   2 +
 .../runtime/io/network/RemoteReceiver.java      |  20 +-
 .../nephele/event/job/ManagementEventTest.java  |   4 +-
 .../executiongraph/ExecutionGraphTest.java      | 258 +----
 .../instance/cluster/ClusterManagerTest.java    | 273 ------
 .../cluster/ClusterManagerTestUtils.java        |  66 --
 .../cluster/DefaultInstanceManagerTest.java     | 232 +++++
 .../DefaultInstanceManagerTestUtils.java        |  66 ++
 .../instance/cluster/HostInClusterTest.java     | 130 ++-
 .../cluster/PendingRequestsMapTest.java         |  91 --
 .../local/LocalInstanceManagerTest.java         |  17 +-
 .../nephele/jobmanager/JobManagerITCase.java    |  16 +-
 .../scheduler/queue/DefaultSchedulerTest.java   | 185 ++++
 .../scheduler/queue/QueueSchedulerTest.java     | 186 ----
 .../scheduler/queue/TestDeploymentManager.java  |   4 +-
 .../scheduler/queue/TestInstanceManager.java    | 118 +--
 .../managementgraph/ManagementGraphTest.java    |  11 +-
 .../services/iomanager/IOManagerITCase.java     |   2 +-
 .../IOManagerPerformanceBenchmark.java          |   2 +-
 .../services/iomanager/IOManagerTest.java       |   2 +-
 .../memorymanager/MemorySegmentTest.java        |   2 +-
 .../nephele/util/ServerTestUtils.java           |  17 +-
 .../runtime/hash/HashMatchIteratorITCase.java   |  14 +-
 .../pact/runtime/hash/HashTableITCase.java      |   2 +-
 .../runtime/hash/ReOpenableHashTableITCase.java |   4 +-
 .../pact/runtime/io/ChannelViewsTest.java       |   8 +-
 .../pact/runtime/io/SpillingBufferTest.java     |   2 +-
 .../event/EventWithAggregatorsTest.java         |   2 +
 .../resettable/BlockResettableIteratorTest.java |   2 +-
 ...lockResettableMutableObjectIteratorTest.java |   2 +-
 .../sort/AsynchonousPartialSorterITCase.java    |  14 +-
 .../CombiningUnilateralSortMergerITCase.java    |   8 +-
 .../pact/runtime/sort/ExternalSortITCase.java   |  12 +-
 .../sort/MassiveStringSortingITCase.java        |   4 +-
 .../sort/SortMergeMatchIteratorITCase.java      |   2 +-
 .../runtime/task/CombineTaskExternalITCase.java |   8 +-
 .../pact/runtime/task/CombineTaskTest.java      |  10 +-
 .../runtime/task/CrossTaskExternalITCase.java   |   7 +-
 .../pact/runtime/task/CrossTaskTest.java        |  36 +-
 .../pact/runtime/task/DataSinkTaskTest.java     |  47 +-
 .../runtime/task/MatchTaskExternalITCase.java   |  14 +-
 .../pact/runtime/task/MatchTaskTest.java        |  56 +-
 .../runtime/task/ReduceTaskExternalITCase.java  |   8 +-
 .../pact/runtime/task/ReduceTaskTest.java       |   3 +-
 .../runtime/task/chaining/ChainTaskTest.java    |  19 +-
 .../task/drivers/ReduceCombineDriverTest.java   |  10 +-
 .../runtime/task/drivers/TestTaskContext.java   |   2 +-
 .../pact/runtime/test/util/DriverTestBase.java  |   8 +-
 .../pact/runtime/test/util/MockEnvironment.java |   9 +-
 .../netty/InboundEnvelopeDecoderTest.java       |   2 +-
 .../test/compiler/util/CompilerTestBase.java    |  26 +-
 .../test/util/AbstractTestBase.java             |  48 +-
 .../test/util/JavaProgramTestBase.java          |   2 +
 .../test/util/RecordAPITestBase.java            |   3 +
 .../test/accumulators/AccumulatorITCase.java    |   7 +-
 .../BroadcastVarsNepheleITCase.java             |  16 +-
 .../KMeansIterativeNepheleITCase.java           |  30 +-
 .../test/cancelling/CancellingTestBase.java     |  10 +-
 .../test/cancelling/MapCancelingITCase.java     |  13 +-
 .../cancelling/MatchJoinCancelingITCase.java    |  17 +-
 .../clients/examples/LocalExecutorITCase.java   |  10 +-
 .../exampleJavaPrograms/WordCountITCase.java    |   4 +-
 .../ComputeEdgeDegreesITCase.java               |   2 +-
 .../ConnectedComponentsITCase.java              |   2 +-
 .../EnumTrianglesOnEdgesWithDegreesITCase.java  |   2 +-
 .../TransitiveClosureNaiveITCase.java           |   2 +-
 .../WebLogAnalysisITCase.java                   |   2 +-
 .../exampleScalaPrograms/WordCountITCase.java   |   2 +-
 .../WordCountPactValueITCase.java               |   2 +-
 .../WordCountWithCountFunctionITCase.java       |   2 +-
 .../test/failingPrograms/TaskFailureITCase.java |   8 +-
 .../CoGroupConnectedComponentsITCase.java       |   6 +-
 .../iterative/ConnectedComponentsITCase.java    |   6 +-
 ...ectedComponentsWithDeferredUpdateITCase.java |   3 +-
 ...tedComponentsWithSolutionSetFirstITCase.java |   7 +-
 .../test/iterative/DanglingPageRankITCase.java  |   3 +-
 .../test/iterative/DeltaPageRankITCase.java     |   3 +-
 .../DependencyConnectedComponentsITCase.java    |   5 +-
 ...IterationTerminationWithTerminationTail.java |   6 +-
 .../IterationTerminationWithTwoTails.java       |   6 +-
 .../IterationWithAllReducerITCase.java          |   6 +-
 .../iterative/IterationWithChainingITCase.java  |   3 +-
 .../iterative/IterationWithUnionITCase.java     |   3 +-
 .../test/iterative/IterativeKMeansITCase.java   |   6 +-
 .../test/iterative/KMeansITCase.java            |   8 +-
 .../test/iterative/LineRankITCase.java          |   5 +-
 .../test/iterative/PageRankITCase.java          |   3 +-
 .../ConnectedComponentsNepheleITCase.java       |  54 +-
 .../nephele/DanglingPageRankNepheleITCase.java  |   7 +-
 ...nglingPageRankWithCombinerNepheleITCase.java |   7 +-
 .../IterationWithChainingNepheleITCase.java     |  17 +-
 .../test/iterative/nephele/JobGraphUtils.java   |  20 +-
 .../CustomCompensatableDanglingPageRank.java    |  57 +-
 ...mpensatableDanglingPageRankWithCombiner.java |  59 +-
 .../CompensatableDanglingPageRank.java          |  55 +-
 .../PackagedProgramEndToEndITCase.java          |  15 +-
 .../test/operators/UnionSinkITCase.java         |   3 +-
 .../recordJobTests/CollectionSourceTest.java    |   8 +-
 .../ComputeEdgeDegreesITCase.java               |   3 +-
 .../EnumTrianglesOnEdgesWithDegreesITCase.java  |   3 +-
 .../recordJobTests/EnumTrianglesRDFITCase.java  |   4 +-
 .../recordJobTests/GlobalSortingITCase.java     |   5 +-
 .../GlobalSortingMixedOrderITCase.java          |  62 +-
 .../recordJobTests/GroupOrderReduceITCase.java  |   3 +-
 .../recordJobTests/MergeOnlyJoinITCase.java     |   1 +
 .../test/recordJobTests/PairwiseSPITCase.java   |   4 +-
 .../test/recordJobTests/TPCHQuery10ITCase.java  |   2 +-
 .../test/recordJobTests/TPCHQuery3ITCase.java   |   3 +-
 .../TPCHQuery3WithUnionITCase.java              |   6 +-
 .../test/recordJobTests/TPCHQuery4ITCase.java   |   6 +-
 .../test/recordJobTests/TPCHQuery9ITCase.java   |   6 +-
 .../recordJobTests/TPCHQueryAsterixITCase.java  |   6 +-
 .../test/recordJobTests/TeraSortITCase.java     |   7 +-
 .../recordJobTests/WebLogAnalysisITCase.java    |   6 +-
 .../test/recordJobTests/WordCountITCase.java    |   6 +-
 .../WordCountUnionReduceITCase.java             |   6 +-
 .../test/runtime/NetworkStackThroughput.java    |  49 +-
 235 files changed, 3917 insertions(+), 7900 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-addons/avro/src/test/java/eu/stratosphere/api/avro/AvroExternalJarProgramITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-addons/avro/src/test/java/eu/stratosphere/api/avro/AvroExternalJarProgramITCase.java b/stratosphere-addons/avro/src/test/java/eu/stratosphere/api/avro/AvroExternalJarProgramITCase.java
index a766fcb..e398acf 100644
--- a/stratosphere-addons/avro/src/test/java/eu/stratosphere/api/avro/AvroExternalJarProgramITCase.java
+++ b/stratosphere-addons/avro/src/test/java/eu/stratosphere/api/avro/AvroExternalJarProgramITCase.java
@@ -47,6 +47,7 @@ public class AvroExternalJarProgramITCase {
 		try {
 			testMiniCluster = new NepheleMiniCluster();
 			testMiniCluster.setJobManagerRpcPort(TEST_JM_PORT);
+			testMiniCluster.setTaskManagerNumSlots(4);
 			testMiniCluster.start();
 			
 			String jarFile = JAR_FILE;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-clients/src/main/java/eu/stratosphere/client/LocalExecutor.java
----------------------------------------------------------------------
diff --git a/stratosphere-clients/src/main/java/eu/stratosphere/client/LocalExecutor.java b/stratosphere-clients/src/main/java/eu/stratosphere/client/LocalExecutor.java
index 89f996a..b017220 100644
--- a/stratosphere-clients/src/main/java/eu/stratosphere/client/LocalExecutor.java
+++ b/stratosphere-clients/src/main/java/eu/stratosphere/client/LocalExecutor.java
@@ -42,6 +42,8 @@ public class LocalExecutor extends PlanExecutor {
 	
 	private static boolean DEFAULT_OVERWRITE = false;
 
+	private static final int DEFAULT_TASK_MANAGER_NUM_SLOTS = 1;
+
 	private final Object lock = new Object();	// we lock to ensure singleton execution
 	
 	private NepheleMiniCluster nephele;
@@ -54,6 +56,8 @@ public class LocalExecutor extends PlanExecutor {
 	
 	private int taskManagerDataPort = -1;
 
+	private int taskManagerNumSlots = DEFAULT_TASK_MANAGER_NUM_SLOTS;
+
 	private String configDir;
 
 	private String hdfsConfigFile;
@@ -129,6 +133,10 @@ public class LocalExecutor extends PlanExecutor {
 	public void setDefaultAlwaysCreateDirectory(boolean defaultAlwaysCreateDirectory) {
 		this.defaultAlwaysCreateDirectory = defaultAlwaysCreateDirectory;
 	}
+
+	public void setTaskManagerNumSlots(int taskManagerNumSlots) { this.taskManagerNumSlots = taskManagerNumSlots; }
+
+	public int getTaskManagerNumSlots() { return this.taskManagerNumSlots; }
 	
 	// --------------------------------------------------------------------------------------------
 	
@@ -157,6 +165,7 @@ public class LocalExecutor extends PlanExecutor {
 				}
 				nephele.setDefaultOverwriteFiles(defaultOverwriteFiles);
 				nephele.setDefaultAlwaysCreateDirectory(defaultAlwaysCreateDirectory);
+				nephele.setTaskManagerNumSlots(taskManagerNumSlots);
 				
 				// start it up
 				this.nephele.start();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-clients/src/main/java/eu/stratosphere/client/minicluster/NepheleMiniCluster.java
----------------------------------------------------------------------
diff --git a/stratosphere-clients/src/main/java/eu/stratosphere/client/minicluster/NepheleMiniCluster.java b/stratosphere-clients/src/main/java/eu/stratosphere/client/minicluster/NepheleMiniCluster.java
index 79e5c64..4daca26 100644
--- a/stratosphere-clients/src/main/java/eu/stratosphere/client/minicluster/NepheleMiniCluster.java
+++ b/stratosphere-clients/src/main/java/eu/stratosphere/client/minicluster/NepheleMiniCluster.java
@@ -46,6 +46,8 @@ public class NepheleMiniCluster {
 
 	private static final boolean DEFAULT_LAZY_MEMORY_ALLOCATION = true;
 
+	private static final int DEFAULT_TASK_MANAGER_NUM_SLOTS = -1;
+
 	// --------------------------------------------------------------------------------------------
 	
 	private final Object startStopLock = new Object();
@@ -56,7 +58,9 @@ public class NepheleMiniCluster {
 	
 	private int taskManagerDataPort = DEFAULT_TM_DATA_PORT;
 
-	private int numTaskManager = DEFAULT_NUM_TASK_MANAGER;
+	private int numTaskTracker = DEFAULT_NUM_TASK_MANAGER;
+
+	private int taskManagerNumSlots = DEFAULT_TASK_MANAGER_NUM_SLOTS;
 	
 	private long memorySize = DEFAULT_MEMORY_SIZE;
 	
@@ -149,9 +153,13 @@ public class NepheleMiniCluster {
 		this.defaultAlwaysCreateDirectory = defaultAlwaysCreateDirectory;
 	}
 
-	public void setNumTaskManager(int numTaskManager) { this.numTaskManager = numTaskManager; }
+	public void setNumTaskTracker(int numTaskTracker) { this.numTaskTracker = numTaskTracker; }
+
+	public int getNumTaskTracker() { return numTaskTracker; }
 
-	public int getNumTaskManager() { return numTaskManager; }
+	public void setTaskManagerNumSlots(int taskManagerNumSlots) { this.taskManagerNumSlots = taskManagerNumSlots; }
+
+	public int getTaskManagerNumSlots() { return taskManagerNumSlots; }
 
 	// ------------------------------------------------------------------------
 	// Life cycle and Job Submission
@@ -172,7 +180,7 @@ public class NepheleMiniCluster {
 			} else {
 				Configuration conf = getMiniclusterDefaultConfig(jobManagerRpcPort, taskManagerRpcPort,
 					taskManagerDataPort, memorySize, hdfsConfigFile, lazyMemoryAllocation, defaultOverwriteFiles,
-						defaultAlwaysCreateDirectory, numTaskManager);
+						defaultAlwaysCreateDirectory, taskManagerNumSlots, numTaskTracker);
 				GlobalConfiguration.includeConfiguration(conf);
 			}
 
@@ -196,7 +204,7 @@ public class NepheleMiniCluster {
 			// start the job manager
 			jobManager = new JobManager(ExecutionMode.LOCAL);
 	
-			waitForJobManagerToBecomeReady(numTaskManager);
+			waitForJobManagerToBecomeReady(numTaskTracker);
 		}
 	}
 
@@ -236,7 +244,8 @@ public class NepheleMiniCluster {
 	
 	public static Configuration getMiniclusterDefaultConfig(int jobManagerRpcPort, int taskManagerRpcPort,
 			int taskManagerDataPort, long memorySize, String hdfsConfigFile, boolean lazyMemory,
-			boolean defaultOverwriteFiles, boolean defaultAlwaysCreateDirectory, int numTaskManager)
+			boolean defaultOverwriteFiles, boolean defaultAlwaysCreateDirectory,
+			int taskManagerNumSlots, int numTaskManager)
 	{
 		final Configuration config = new Configuration();
 		
@@ -284,6 +293,8 @@ public class NepheleMiniCluster {
 		config.setLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, memorySize/numTaskManager);
 
 		config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, numTaskManager);
+
+		config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, taskManagerNumSlots);
 		
 		return config;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-clients/src/main/java/eu/stratosphere/client/program/Client.java
----------------------------------------------------------------------
diff --git a/stratosphere-clients/src/main/java/eu/stratosphere/client/program/Client.java b/stratosphere-clients/src/main/java/eu/stratosphere/client/program/Client.java
index 00790f4..31138f6 100644
--- a/stratosphere-clients/src/main/java/eu/stratosphere/client/program/Client.java
+++ b/stratosphere-clients/src/main/java/eu/stratosphere/client/program/Client.java
@@ -77,7 +77,7 @@ public class Client {
 		configuration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, jobManagerAddress.getAddress().getHostAddress());
 		configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerAddress.getPort());
 		
-		this.compiler = new PactCompiler(new DataStatistics(), new DefaultCostEstimator(), jobManagerAddress);
+		this.compiler = new PactCompiler(new DataStatistics(), new DefaultCostEstimator());
 		
 		//  Disable Local Execution when using a Client
 		ContextEnvironment.disableLocalExecution();
@@ -104,8 +104,7 @@ public class Client {
 			throw new CompilerException("Cannot find port to job manager's RPC service in the global configuration.");
 		}
 
-		final InetSocketAddress jobManagerAddress = new InetSocketAddress(address, port);
-		this.compiler = new PactCompiler(new DataStatistics(), new DefaultCostEstimator(), jobManagerAddress);
+		this.compiler = new PactCompiler(new DataStatistics(), new DefaultCostEstimator());
 		
 		//  Disable Local Execution when using a Client
 		ContextEnvironment.disableLocalExecution();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-clients/src/test/java/eu/stratosphere/client/CliFrontendListCancelTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-clients/src/test/java/eu/stratosphere/client/CliFrontendListCancelTest.java b/stratosphere-clients/src/test/java/eu/stratosphere/client/CliFrontendListCancelTest.java
index 7ccd420..ba02fa9 100644
--- a/stratosphere-clients/src/test/java/eu/stratosphere/client/CliFrontendListCancelTest.java
+++ b/stratosphere-clients/src/test/java/eu/stratosphere/client/CliFrontendListCancelTest.java
@@ -21,7 +21,6 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.commons.cli.CommandLine;
 import org.junit.Assert;
@@ -34,8 +33,6 @@ import eu.stratosphere.nephele.client.JobProgressResult;
 import eu.stratosphere.nephele.client.JobSubmissionResult;
 import eu.stratosphere.nephele.event.job.AbstractEvent;
 import eu.stratosphere.nephele.event.job.RecentJobEvent;
-import eu.stratosphere.nephele.instance.InstanceType;
-import eu.stratosphere.nephele.instance.InstanceTypeDescription;
 import eu.stratosphere.nephele.jobgraph.JobGraph;
 import eu.stratosphere.nephele.jobgraph.JobID;
 import eu.stratosphere.nephele.managementgraph.ManagementGraph;
@@ -202,18 +199,18 @@ public class CliFrontendListCancelTest {
 		}
 
 		@Override
-		public Map<InstanceType, InstanceTypeDescription> getMapOfAvailableInstanceTypes() throws IOException {
+		public void logBufferUtilization(JobID jobID) throws IOException {
 			throw new UnsupportedOperationException();
 		}
 
 		@Override
-		public void logBufferUtilization(JobID jobID) throws IOException {
+		public NetworkTopology getNetworkTopology(JobID jobID) throws IOException {
 			throw new UnsupportedOperationException();
 		}
 
 		@Override
-		public NetworkTopology getNetworkTopology(JobID jobID) throws IOException {
-			throw new UnsupportedOperationException();
+		public int getAvailableSlots() {
+			return 1;
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-clients/src/test/java/eu/stratosphere/client/testjar/WordCount.java
----------------------------------------------------------------------
diff --git a/stratosphere-clients/src/test/java/eu/stratosphere/client/testjar/WordCount.java b/stratosphere-clients/src/test/java/eu/stratosphere/client/testjar/WordCount.java
index e827805..5218dc2 100644
--- a/stratosphere-clients/src/test/java/eu/stratosphere/client/testjar/WordCount.java
+++ b/stratosphere-clients/src/test/java/eu/stratosphere/client/testjar/WordCount.java
@@ -70,9 +70,10 @@ public class WordCount {
 	 * FlatMapFunction. The function takes a line (String) and splits it into 
 	 * multiple pairs in the form of "(word,1)" (Tuple2<String, Integer>).
 	 */
-	@SuppressWarnings("serial")
 	public static final class Tokenizer extends FlatMapFunction<String, Tuple2<String, Integer>> {
 
+		private static final long serialVersionUID = 1L;
+
 		@Override
 		public void flatMap(String value, Collector<Tuple2<String, Integer>> out) {
 			// normalize and split the line

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/PactCompiler.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/PactCompiler.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/PactCompiler.java
index 2076902..bf3d6af 100644
--- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/PactCompiler.java
+++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/PactCompiler.java
@@ -13,8 +13,6 @@
 
 package eu.stratosphere.compiler;
 
-import java.io.IOException;
-import java.net.InetSocketAddress;
 import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Deque;
@@ -90,11 +88,6 @@ import eu.stratosphere.compiler.postpass.OptimizerPostPass;
 import eu.stratosphere.configuration.ConfigConstants;
 import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.configuration.GlobalConfiguration;
-import eu.stratosphere.nephele.instance.InstanceType;
-import eu.stratosphere.nephele.instance.InstanceTypeDescription;
-import eu.stratosphere.nephele.ipc.RPC;
-import eu.stratosphere.nephele.net.NetUtils;
-import eu.stratosphere.nephele.protocols.ExtendedManagementProtocol;
 import eu.stratosphere.pact.runtime.shipping.ShipStrategyType;
 import eu.stratosphere.pact.runtime.task.util.LocalStrategy;
 import eu.stratosphere.util.InstantiationUtil;
@@ -340,24 +333,10 @@ public class PactCompiler {
 	private final CostEstimator costEstimator;
 
 	/**
-	 * The connection used to connect to the job-manager.
-	 */
-	private final InetSocketAddress jobManagerAddress;
-
-	/**
-	 * The maximum number of machines (instances) to use, per the configuration.
-	 */
-	private int maxMachines;
-
-	/**
 	 * The default degree of parallelism for jobs compiled by this compiler.
 	 */
 	private int defaultDegreeOfParallelism;
 
-	/**
-	 * The maximum number of subtasks that should share an instance.
-	 */
-	private int maxIntraNodeParallelism;
 
 	// ------------------------------------------------------------------------
 	// Constructor & Setup
@@ -420,106 +399,29 @@ public class PactCompiler {
 	 *        The <tt>CostEstimator</tt> to use to cost the individual operations.
 	 */
 	public PactCompiler(DataStatistics stats, CostEstimator estimator) {
-		this(stats, estimator, null);
-	}
-
-	/**
-	 * Creates a new compiler instance that uses the statistics object to determine properties about the input.
-	 * Given those statistics, the compiler can make better choices for the execution strategies.
-	 * as if no filesystem was given. It uses the given cost estimator to compute the costs of the individual
-	 * operations.
-	 * <p>
-	 * The given socket-address is used to connect to the job manager to obtain system characteristics, like available
-	 * memory. If that parameter is null, then the address is obtained from the global configuration.
-	 * 
-	 * @param stats
-	 *        The statistics to be used to determine the input properties.
-	 * @param estimator
-	 *        The <tt>CostEstimator</tt> to use to cost the individual operations.
-	 * @param jobManagerConnection
-	 *        The address of the job manager that is queried for system characteristics.
-	 */
-	public PactCompiler(DataStatistics stats, CostEstimator estimator, InetSocketAddress jobManagerConnection) {
 		this.statistics = stats;
 		this.costEstimator = estimator;
 
 		Configuration config = GlobalConfiguration.getConfiguration();
 
-		// determine the maximum number of instances to use
-		this.maxMachines = -1;
-
 		// determine the default parallelization degree
 		this.defaultDegreeOfParallelism = config.getInteger(ConfigConstants.DEFAULT_PARALLELIZATION_DEGREE_KEY,
 			ConfigConstants.DEFAULT_PARALLELIZATION_DEGREE);
-
-		// determine the default intra-node parallelism
-		int maxInNodePar = config.getInteger(ConfigConstants.PARALLELIZATION_MAX_INTRA_NODE_DEGREE_KEY,
-			ConfigConstants.DEFAULT_MAX_INTRA_NODE_PARALLELIZATION_DEGREE);
-		if (maxInNodePar == 0 || maxInNodePar < -1) {
-			LOG.error("Invalid maximum degree of intra-node parallelism: " + maxInNodePar +
-				". Ignoring parameter.");
-			maxInNodePar = ConfigConstants.DEFAULT_MAX_INTRA_NODE_PARALLELIZATION_DEGREE;
-		}
-		this.maxIntraNodeParallelism = maxInNodePar;
-
-		// assign the connection to the job-manager
-		if (jobManagerConnection != null) {
-			this.jobManagerAddress = jobManagerConnection;
-		} else {
-			final String address = config.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null);
-			if (address == null) {
-				throw new CompilerException(
-					"Cannot find address to job manager's RPC service in the global configuration.");
-			}
-
-			final int port = GlobalConfiguration.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY,
-				ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT);
-			if (port < 0) {
-				throw new CompilerException(
-					"Cannot find port to job manager's RPC service in the global configuration.");
-			}
-
-			this.jobManagerAddress = new InetSocketAddress(address, port);
-		}
 	}
 	
 	// ------------------------------------------------------------------------
 	//                             Getters / Setters
 	// ------------------------------------------------------------------------
 	
-	public int getMaxMachines() {
-		return maxMachines;
-	}
-	
-	public void setMaxMachines(int maxMachines) {
-		if (maxMachines == -1 || maxMachines > 0) {
-			this.maxMachines = maxMachines;
-		} else {
-			throw new IllegalArgumentException();
-		}
-	}
-	
 	public int getDefaultDegreeOfParallelism() {
 		return defaultDegreeOfParallelism;
 	}
 	
 	public void setDefaultDegreeOfParallelism(int defaultDegreeOfParallelism) {
-		if (defaultDegreeOfParallelism == -1 || defaultDegreeOfParallelism > 0) {
+		if (defaultDegreeOfParallelism > 0) {
 			this.defaultDegreeOfParallelism = defaultDegreeOfParallelism;
 		} else {
-			throw new IllegalArgumentException();
-		}
-	}
-	
-	public int getMaxIntraNodeParallelism() {
-		return maxIntraNodeParallelism;
-	}
-	
-	public void setMaxIntraNodeParallelism(int maxIntraNodeParallelism) {
-		if (maxIntraNodeParallelism == -1 || maxIntraNodeParallelism > 0) {
-			this.maxIntraNodeParallelism = maxIntraNodeParallelism;
-		} else {
-			throw new IllegalArgumentException();
+			throw new IllegalArgumentException("Default parallelism cannot be zero or negative.");
 		}
 	}
 	
@@ -550,14 +452,9 @@ public class PactCompiler {
 		// -------------------- try to get the connection to the job manager ----------------------
 		// --------------------------to obtain instance information --------------------------------
 		final OptimizerPostPass postPasser = getPostPassFromPlan(program);
-		return compile(program, getInstanceTypeInfo(), postPasser);
-	}
-	
-	public OptimizedPlan compile(Plan program, InstanceTypeDescription type) throws CompilerException {
-		final OptimizerPostPass postPasser = getPostPassFromPlan(program);
-		return compile(program, type, postPasser);
+		return compile(program, postPasser);
 	}
-	
+
 	/**
 	 * Translates the given pact plan in to an OptimizedPlan, where all nodes have their local strategy assigned
 	 * and all channels have a shipping strategy assigned. The process goes through several phases:
@@ -569,8 +466,6 @@ public class PactCompiler {
 	 * </ol>
 	 * 
 	 * @param program The program to be translated.
-	 * @param type The instance type to schedule the execution on. Used also to determine the amount of memory
-	 *             available to the tasks.
 	 * @param postPasser The function to be used for post passing the optimizer's plan and setting the
 	 *                   data type specific serialization routines.
 	 * @return The optimized plan.
@@ -579,8 +474,8 @@ public class PactCompiler {
 	 *         Thrown, if the plan is invalid or the optimizer encountered an inconsistent
 	 *         situation during the compilation process.
 	 */
-	private OptimizedPlan compile(Plan program, InstanceTypeDescription type, OptimizerPostPass postPasser) throws CompilerException {
-		if (program == null || type == null || postPasser == null) {
+	private OptimizedPlan compile(Plan program, OptimizerPostPass postPasser) throws CompilerException {
+		if (program == null || postPasser == null) {
 			throw new NullPointerException();
 		}
 		
@@ -588,73 +483,14 @@ public class PactCompiler {
 		if (LOG.isDebugEnabled()) {
 			LOG.debug("Beginning compilation of program '" + program.getJobName() + '\'');
 		}
-		
-		final String instanceName = type.getInstanceType().getIdentifier();
-		
-		// we subtract some percentage of the memory to accommodate for rounding errors
-		final long memoryPerInstance = (long) (type.getHardwareDescription().getSizeOfFreeMemory() * 0.96f);
-		final int numInstances = type.getMaximumNumberOfAvailableInstances();
-		
-		// determine the maximum number of machines to use
-		int maxMachinesJob = program.getMaxNumberMachines();
-
-		if (maxMachinesJob < 1) {
-			maxMachinesJob = this.maxMachines;
-		} else if (this.maxMachines >= 1) {
-			// check if the program requested more than the global config allowed
-			if (maxMachinesJob > this.maxMachines && LOG.isWarnEnabled()) {
-				LOG.warn("Maximal number of machines specified in program (" + maxMachinesJob
-					+ ") exceeds the maximum number in the global configuration (" + this.maxMachines
-					+ "). Using the global configuration value.");
-			}
-
-			maxMachinesJob = Math.min(maxMachinesJob, this.maxMachines);
-		}
-
-		// adjust the maximum number of machines the the number of available instances
-		if (maxMachinesJob < 1) {
-			maxMachinesJob = numInstances;
-		} else if (maxMachinesJob > numInstances) {
-			maxMachinesJob = numInstances;
-			if (LOG.isInfoEnabled()) {
-				LOG.info("Maximal number of machines decreased to " + maxMachinesJob +
-					" because no more instances are available.");
-			}
-		}
 
 		// set the default degree of parallelism
 		int defaultParallelism = program.getDefaultParallelism() > 0 ?
 			program.getDefaultParallelism() : this.defaultDegreeOfParallelism;
-		
-		if (this.maxIntraNodeParallelism > 0) {
-			if (defaultParallelism < 1) {
-				defaultParallelism = maxMachinesJob * this.maxIntraNodeParallelism;
-			}
-			else if (defaultParallelism > maxMachinesJob * this.maxIntraNodeParallelism) {
-				int oldParallelism = defaultParallelism;
-				defaultParallelism = maxMachinesJob * this.maxIntraNodeParallelism;
-
-				if (LOG.isInfoEnabled()) {
-					LOG.info("Decreasing default degree of parallelism from " + oldParallelism +
-						" to " + defaultParallelism + " to fit a maximum number of " + maxMachinesJob +
-						" instances with a intra-parallelism of " + this.maxIntraNodeParallelism);
-				}
-			}
-		} else if (defaultParallelism < 1) {
-			defaultParallelism = maxMachinesJob;
-			if (LOG.isInfoEnabled()) {
-				LOG.info("No default parallelism specified. Using default parallelism of " + defaultParallelism + " (One task per instance)");
-			}
-		}
 
 		// log the output
 		if (LOG.isDebugEnabled()) {
-			LOG.debug("Using a default degree of parallelism of " + defaultParallelism +
-				", a maximum intra-node parallelism of " + this.maxIntraNodeParallelism + '.');
-			if (this.maxMachines > 0) {
-				LOG.debug("The execution is limited to a maximum number of " + maxMachinesJob + " machines.");
-			}
-
+			LOG.debug("Using a default degree of parallelism of " + defaultParallelism + '.');
 		}
 
 		// the first step in the compilation is to create the optimizer plan representation
@@ -666,7 +502,7 @@ public class PactCompiler {
 		// 4) It makes estimates about the data volume of the data sources and
 		// propagates those estimates through the plan
 
-		GraphCreatingVisitor graphCreator = new GraphCreatingVisitor(maxMachinesJob, defaultParallelism);
+		GraphCreatingVisitor graphCreator = new GraphCreatingVisitor(defaultParallelism);
 		program.accept(graphCreator);
 
 		// if we have a plan with multiple data sinks, add logical optimizer nodes that have two data-sinks as children
@@ -689,8 +525,7 @@ public class PactCompiler {
 		// now that we have all nodes created and recorded which ones consume memory, tell the nodes their minimal
 		// guaranteed memory, for further cost estimations. we assume an equal distribution of memory among consumer tasks
 		
-		rootNode.accept(new IdAndMemoryAndEstimatesVisitor(this.statistics,
-			graphCreator.getMemoryConsumerCount() == 0 ? 0 : memoryPerInstance / graphCreator.getMemoryConsumerCount()));
+		rootNode.accept(new IdAndEstimatesVisitor(this.statistics));
 		
 		// Now that the previous step is done, the next step is to traverse the graph again for the two
 		// steps that cannot directly be performed during the plan enumeration, because we are dealing with DAGs
@@ -733,9 +568,8 @@ public class PactCompiler {
 		dp.resolveDeadlocks(bestPlanSinks);
 
 		// finalize the plan
-		OptimizedPlan plan = new PlanFinalizer().createFinalPlan(bestPlanSinks, program.getJobName(), program, memoryPerInstance);
-		plan.setInstanceTypeName(instanceName);
-		
+		OptimizedPlan plan = new PlanFinalizer().createFinalPlan(bestPlanSinks, program.getJobName(), program);
+
 		// swap the binary unions for n-ary unions. this changes no strategies or memory consumers whatsoever, so
 		// we can do this after the plan finalization
 		plan.accept(new BinaryUnionReplacer());
@@ -755,7 +589,7 @@ public class PactCompiler {
 	 *         from the plan can be traversed.
 	 */
 	public static List<DataSinkNode> createPreOptimizedPlan(Plan program) {
-		GraphCreatingVisitor graphCreator = new GraphCreatingVisitor(-1, 1);
+		GraphCreatingVisitor graphCreator = new GraphCreatingVisitor(1);
 		program.accept(graphCreator);
 		return graphCreator.sinks;
 	}
@@ -783,22 +617,18 @@ public class PactCompiler {
 
 		private final List<DataSinkNode> sinks; // all data sink nodes in the optimizer plan
 
-		private final int maxMachines; // the maximum number of machines to use
-
 		private final int defaultParallelism; // the default degree of parallelism
 		
-		private int numMemoryConsumers;
-		
 		private final GraphCreatingVisitor parent;	// reference to enclosing creator, in case of a recursive translation
 		
 		private final boolean forceDOP;
 
 		
-		private GraphCreatingVisitor(int maxMachines, int defaultParallelism) {
-			this(null, false, maxMachines, defaultParallelism, null);
+		private GraphCreatingVisitor(int defaultParallelism) {
+			this(null, false, defaultParallelism, null);
 		}
 
-		private GraphCreatingVisitor(GraphCreatingVisitor parent, boolean forceDOP, int maxMachines,
+		private GraphCreatingVisitor(GraphCreatingVisitor parent, boolean forceDOP,
 									int defaultParallelism, HashMap<Operator<?>, OptimizerNode> closure) {
 			if (closure == null){
 				con2node = new HashMap<Operator<?>, OptimizerNode>();
@@ -807,7 +637,6 @@ public class PactCompiler {
 			}
 			this.sources = new ArrayList<DataSourceNode>(4);
 			this.sinks = new ArrayList<DataSinkNode>(2);
-			this.maxMachines = maxMachines;
 			this.defaultParallelism = defaultParallelism;
 			this.parent = parent;
 			this.forceDOP = forceDOP;
@@ -878,7 +707,6 @@ public class PactCompiler {
 				// catch this for the recursive translation of step functions
 				BulkPartialSolutionNode p = new BulkPartialSolutionNode(holder, containingIterationNode);
 				p.setDegreeOfParallelism(containingIterationNode.getDegreeOfParallelism());
-				p.setSubtasksPerInstance(containingIterationNode.getSubtasksPerInstance());
 				n = p;
 			}
 			else if (c instanceof WorksetPlaceHolder) {
@@ -890,7 +718,6 @@ public class PactCompiler {
 				// catch this for the recursive translation of step functions
 				WorksetNode p = new WorksetNode(holder, containingIterationNode);
 				p.setDegreeOfParallelism(containingIterationNode.getDegreeOfParallelism());
-				p.setSubtasksPerInstance(containingIterationNode.getSubtasksPerInstance());
 				n = p;
 			}
 			else if (c instanceof SolutionSetPlaceHolder) {
@@ -902,18 +729,14 @@ public class PactCompiler {
 				// catch this for the recursive translation of step functions
 				SolutionSetNode p = new SolutionSetNode(holder, containingIterationNode);
 				p.setDegreeOfParallelism(containingIterationNode.getDegreeOfParallelism());
-				p.setSubtasksPerInstance(containingIterationNode.getSubtasksPerInstance());
 				n = p;
 			}
 			else {
-				throw new IllegalArgumentException("Unknown operator type: " + c.getClass() + " " + c);
+				throw new IllegalArgumentException("Unknown operator type: " + c);
 			}
 
 			this.con2node.put(c, n);
 			
-			// record the potential memory consumption
-			this.numMemoryConsumers += n.isMemoryConsumer() ? 1 : 0;
-
 			// set the parallelism only if it has not been set before. some nodes have a fixed DOP, such as the
 			// key-less reducer (all-reduce)
 			if (n.getDegreeOfParallelism() < 1) {
@@ -931,19 +754,6 @@ public class PactCompiler {
 				n.setDegreeOfParallelism(par);
 			}
 
-			// check if we need to set the instance sharing accordingly such that
-			// the maximum number of machines is not exceeded
-			if (n.getSubtasksPerInstance() < 1) {
-				int tasksPerInstance = 1;
-				if (this.maxMachines > 0) {
-					int p = n.getDegreeOfParallelism();
-					tasksPerInstance = (p / this.maxMachines) + (p % this.maxMachines == 0 ? 0 : 1);
-				}
-	
-				// we group together n tasks per machine, depending on config and the above computed
-				// value required to obey the maximum number of machines
-				n.setSubtasksPerInstance(tasksPerInstance);
-			}
 			return true;
 		}
 
@@ -966,7 +776,7 @@ public class PactCompiler {
 
 				// first, recursively build the data flow for the step function
 				final GraphCreatingVisitor recursiveCreator = new GraphCreatingVisitor(this, true,
-					this.maxMachines, iterNode.getDegreeOfParallelism(), closure);
+					iterNode.getDegreeOfParallelism(), closure);
 				
 				BulkPartialSolutionNode partialSolution = null;
 				
@@ -994,9 +804,6 @@ public class PactCompiler {
 				iterNode.setNextPartialSolution(rootOfStepFunction, terminationCriterion);
 				iterNode.setPartialSolution(partialSolution);
 				
-				// account for the nested memory consumers
-				this.numMemoryConsumers += recursiveCreator.numMemoryConsumers;
-				
 				// go over the contained data flow and mark the dynamic path nodes
 				StaticDynamicPathIdentifier identifier = new StaticDynamicPathIdentifier(iterNode.getCostWeight());
 				rootOfStepFunction.accept(identifier);
@@ -1013,7 +820,7 @@ public class PactCompiler {
 
 				// first, recursively build the data flow for the step function
 				final GraphCreatingVisitor recursiveCreator = new GraphCreatingVisitor(this, true,
-					this.maxMachines, iterNode.getDegreeOfParallelism(), closure);
+					iterNode.getDegreeOfParallelism(), closure);
 				// descend from the solution set delta. check that it depends on both the workset
 				// and the solution set. If it does depend on both, this descend should create both nodes
 				iter.getSolutionSetDelta().accept(recursiveCreator);
@@ -1067,19 +874,12 @@ public class PactCompiler {
 				iterNode.setPartialSolution(solutionSetNode, worksetNode);
 				iterNode.setNextPartialSolution(solutionSetDeltaNode, nextWorksetNode);
 				
-				// account for the nested memory consumers
-				this.numMemoryConsumers += recursiveCreator.numMemoryConsumers;
-				
 				// go over the contained data flow and mark the dynamic path nodes
 				StaticDynamicPathIdentifier pathIdentifier = new StaticDynamicPathIdentifier(iterNode.getCostWeight());
 				nextWorksetNode.accept(pathIdentifier);
 				iterNode.getSolutionSetDelta().accept(pathIdentifier);
 			}
 		}
-		
-		int getMemoryConsumerCount() {
-			return this.numMemoryConsumers;
-		}
 	};
 	
 	private static final class StaticDynamicPathIdentifier implements Visitor<OptimizerNode> {
@@ -1107,17 +907,14 @@ public class PactCompiler {
 	 * Simple visitor that sets the minimal guaranteed memory per task based on the amount of available memory,
 	 * the number of memory consumers, and on the task's degree of parallelism.
 	 */
-	private static final class IdAndMemoryAndEstimatesVisitor implements Visitor<OptimizerNode> {
+	private static final class IdAndEstimatesVisitor implements Visitor<OptimizerNode> {
 		
 		private final DataStatistics statistics;
-		
-		private final long memoryPerTaskPerInstance;
-		
+
 		private int id = 1;
 		
-		private IdAndMemoryAndEstimatesVisitor(DataStatistics statistics, long memoryPerTaskPerInstance) {
+		private IdAndEstimatesVisitor(DataStatistics statistics) {
 			this.statistics = statistics;
-			this.memoryPerTaskPerInstance = memoryPerTaskPerInstance;
 		}
 
 
@@ -1128,11 +925,6 @@ public class PactCompiler {
 				return false;
 			}
 			
-			// assign minimum memory share, for lower bound estimates
-			final long mem = visitable.isMemoryConsumer() ? 
-					this.memoryPerTaskPerInstance / visitable.getSubtasksPerInstance() : 0;
-			visitable.setMinimalMemoryPerSubTask(mem);
-			
 			return true;
 		}
 
@@ -1234,8 +1026,6 @@ public class PactCompiler {
 		
 		private final Deque<IterationPlanNode> stackOfIterationNodes;
 
-		private long memoryPerInstance; // the amount of memory per instance
-		
 		private int memoryConsumerWeights; // a counter of all memory consumers
 
 		/**
@@ -1248,12 +1038,7 @@ public class PactCompiler {
 			this.stackOfIterationNodes = new ArrayDeque<IterationPlanNode>();
 		}
 
-		private OptimizedPlan createFinalPlan(List<SinkPlanNode> sinks, String jobName, Plan originalPlan, long memPerInstance) {
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("Available memory per instance: " + memPerInstance);
-			}
-			
-			this.memoryPerInstance = memPerInstance;
+		private OptimizedPlan createFinalPlan(List<SinkPlanNode> sinks, String jobName, Plan originalPlan) {
 			this.memoryConsumerWeights = 0;
 			
 			// traverse the graph
@@ -1263,44 +1048,36 @@ public class PactCompiler {
 
 			// assign the memory to each node
 			if (this.memoryConsumerWeights > 0) {
-				final long memoryPerInstanceAndWeight = this.memoryPerInstance / this.memoryConsumerWeights;
-				
-				if (LOG.isDebugEnabled()) {
-					LOG.debug("Memory per consumer weight: " + memoryPerInstanceAndWeight);
-				}
-				
 				for (PlanNode node : this.allNodes) {
 					// assign memory to the driver strategy of the node
 					final int consumerWeight = node.getMemoryConsumerWeight();
 					if (consumerWeight > 0) {
-						final long mem = memoryPerInstanceAndWeight * consumerWeight / node.getSubtasksPerInstance();
-						node.setMemoryPerSubTask(mem);
+						final double relativeMem = (double)consumerWeight / this.memoryConsumerWeights;
+						node.setRelativeMemoryPerSubtask(relativeMem);
 						if (LOG.isDebugEnabled()) {
-							final long mib = mem >> 20;
-							LOG.debug("Assigned " + mib + " MiBytes memory to each subtask of " + 
-								node.getPactContract().getName() + " (" + mib * node.getDegreeOfParallelism() +
-								" MiBytes total.)"); 
+							LOG.debug("Assigned " + relativeMem + " of total memory to each subtask of " +
+								node.getPactContract().getName() + ".");
 						}
 					}
 					
 					// assign memory to the local and global strategies of the channels
 					for (Channel c : node.getInputs()) {
 						if (c.getLocalStrategy().dams()) {
-							final long mem = memoryPerInstanceAndWeight / node.getSubtasksPerInstance();
-							c.setMemoryLocalStrategy(mem);
+							final double relativeMem = 1.0 / this.memoryConsumerWeights;
+							c.setRelativeMemoryLocalStrategy(relativeMem);
 							if (LOG.isDebugEnabled()) {
-								final long mib = mem >> 20;
-								LOG.debug("Assigned " + mib + " MiBytes memory to each local strategy instance of " + 
-									c + " (" + mib * node.getDegreeOfParallelism() + " MiBytes total.)"); 
+								LOG.debug("Assigned " + relativeMem + " of total memory to each local strategy " +
+										"instance of " + c + ".");
 							}
 						}
 						if (c.getTempMode() != TempMode.NONE) {
-							final long mem = memoryPerInstanceAndWeight / node.getSubtasksPerInstance();
-							c.setTempMemory(mem);
+							final double relativeMem = 1.0/ this.memoryConsumerWeights;
+							c.setRelativeTempMemory(relativeMem);
 							if (LOG.isDebugEnabled()) {
-								final long mib = mem >> 20;
-								LOG.debug("Assigned " + mib + " MiBytes memory to each instance of the temp table for " + 
-									c + " (" + mib * node.getDegreeOfParallelism() + " MiBytes total.)"); 
+								LOG.debug("Assigned " + relativeMem + " of total memory to each instance of the temp " +
+										"table" +
+										" " +
+										"for " + c + ".");
 							}
 						}
 					}
@@ -1525,182 +1302,4 @@ public class PactCompiler {
 			throw new CompilerException("Class '" + className + "' is not an optimizer post passer.", ccex);
 		}
 	}
-
-	private InstanceTypeDescription getInstanceTypeInfo() {
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("Connecting compiler to JobManager to dertermine instance information.");
-		}
-		
-		// create the connection in a separate thread, such that this thread
-		// can abort, if an unsuccessful connection occurs.
-		Map<InstanceType, InstanceTypeDescription> instances = null;
-		
-		JobManagerConnector jmc = new JobManagerConnector(this.jobManagerAddress);
-		Thread connectorThread = new Thread(jmc, "Compiler - JobManager connector.");
-		connectorThread.setDaemon(true);
-		connectorThread.start();
-
-		// connect and get the result
-		try {
-			jmc.waitForCompletion();
-			instances = jmc.instances;
-			if (instances == null) {
-				throw new NullPointerException("Returned instance map is <null>");
-			}
-		}
-		catch (IOException e) {
-			throw new CompilerException(e.getMessage());
-		}
-		catch (Throwable t) {
-			throw new CompilerException("Cannot connect to the JobManager to determine the available TaskManagers. "
-					+ "Check if the JobManager is running (using the web interface or log files). Reason: " + 
-				t.getMessage(), t);
-		}
-
-		// determine which type to run on
-		return getType(instances);
-	}
-	
-	/**
-	 * This utility method picks the instance type to be used for executing programs.
-	 * <p>
-	 * 
-	 * @param types The available types.
-	 * @return The type to be used for scheduling.
-	 * 
-	 * @throws CompilerException
-	 * @throws IllegalArgumentException
-	 */
-	private InstanceTypeDescription getType(Map<InstanceType, InstanceTypeDescription> types)
-	throws CompilerException
-	{
-		if (types == null || types.size() < 1) {
-			throw new IllegalArgumentException("No instance type found.");
-		}
-		
-		InstanceTypeDescription retValue = null;
-		long totalMemory = 0;
-		int numInstances = 0;
-		
-		final Iterator<InstanceTypeDescription> it = types.values().iterator();
-		while(it.hasNext())
-		{
-			final InstanceTypeDescription descr = it.next();
-			
-			// skip instances for which no hardware description is available
-			// this means typically that no 
-			if (descr.getHardwareDescription() == null || descr.getInstanceType() == null) {
-				continue;
-			}
-			
-			final int curInstances = descr.getMaximumNumberOfAvailableInstances();
-			final long curMemory = curInstances * descr.getHardwareDescription().getSizeOfFreeMemory();
-			
-			// get, if first, or if it has more instances and not less memory, or if it has significantly more memory
-			// and the same number of cores still
-			if ( (retValue == null) ||
-				(curInstances > numInstances && (int) (curMemory * 1.2f) > totalMemory) ||
-				(curInstances * retValue.getInstanceType().getNumberOfCores() >= numInstances && 
-							(int) (curMemory * 1.5f) > totalMemory)
-				)
-			{
-				retValue = descr;
-				numInstances = curInstances;
-				totalMemory = curMemory;
-			}
-		}
-		
-		if (retValue == null) {
-			throw new CompilerException("No instance currently registered at the job-manager. Retry later.\n" +
-				"If the system has recently started, it may take a few seconds until the instances register.");
-		}
-		
-		return retValue;
-	}
-	
-	/**
-	 * Utility class for an asynchronous connection to the job manager to determine the available instances.
-	 */
-	private static final class JobManagerConnector implements Runnable {
-		
-		private static final long MAX_MILLIS_TO_WAIT = 10000;
-		
-		private final InetSocketAddress jobManagerAddress;
-		
-		private final Object lock = new Object();
-		
-		private volatile Map<InstanceType, InstanceTypeDescription> instances;
-		
-		private volatile Throwable error;
-		
-		
-		private JobManagerConnector(InetSocketAddress jobManagerAddress) {
-			this.jobManagerAddress = jobManagerAddress;
-		}
-		
-		
-		public Map<InstanceType, InstanceTypeDescription> waitForCompletion() throws Throwable {
-			long start = System.currentTimeMillis();
-			long remaining = MAX_MILLIS_TO_WAIT;
-			
-			if (this.error != null) {
-				throw this.error;
-			}
-			if (this.instances != null) {
-				return this.instances;
-			}
-			
-			do {
-				try {
-					synchronized (this.lock) {
-						this.lock.wait(remaining);
-					}
-				} catch (InterruptedException iex) {}
-			}
-			while (this.error == null && this.instances == null &&
-					(remaining = MAX_MILLIS_TO_WAIT + start - System.currentTimeMillis()) > 0);
-			
-			if (this.error != null) {
-				throw this.error;
-			}
-			if (this.instances != null) {
-				return this.instances;
-			}
-			
-			throw new IOException("Could not connect to the JobManager at " + jobManagerAddress + 
-				". Please make sure that the Job Manager is started properly.");
-		}
-		
-
-		@Override
-		public void run() {
-			ExtendedManagementProtocol jobManagerConnection = null;
-
-			try {
-				jobManagerConnection = RPC.getProxy(ExtendedManagementProtocol.class,
-					this.jobManagerAddress, NetUtils.getSocketFactory());
-
-				this.instances = jobManagerConnection.getMapOfAvailableInstanceTypes();
-				if (this.instances == null) {
-					throw new IOException("Returned instance map was <null>");
-				}
-			} catch (Throwable t) {
-				this.error = t;
-			} finally {
-				// first of all, signal completion
-				synchronized (this.lock) {
-					this.lock.notifyAll();
-				}
-				
-				if (jobManagerConnection != null) {
-					try {
-						RPC.stopProxy(jobManagerConnection);
-					} catch (Throwable t) {
-						LOG.error("Could not cleanly shut down connection from compiler to job manager,", t);
-					}
-				}
-				jobManagerConnection = null;
-			}
-		}
-	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/costs/DefaultCostEstimator.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/costs/DefaultCostEstimator.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/costs/DefaultCostEstimator.java
index 058af1a..fde5970 100644
--- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/costs/DefaultCostEstimator.java
+++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/costs/DefaultCostEstimator.java
@@ -95,14 +95,20 @@ public class DefaultCostEstimator extends CostEstimator {
 
 	@Override
 	public void addBroadcastCost(EstimateProvider estimates, int replicationFactor, Costs costs) {
-		// assumption: we need ship the whole data over the network to each node.
-		final long estOutShipSize = estimates.getEstimatedOutputSize();
-		if (estOutShipSize <= 0) {
-			costs.setNetworkCost(Costs.UNKNOWN);
+		// if our replication factor is negative, we cannot calculate broadcast costs
+
+		if (replicationFactor > 0) {
+			// assumption: we need ship the whole data over the network to each node.
+			final long estOutShipSize = estimates.getEstimatedOutputSize();
+			if (estOutShipSize <= 0) {
+				costs.setNetworkCost(Costs.UNKNOWN);
+			} else {
+				costs.addNetworkCost(replicationFactor * estOutShipSize);
+			}
+			costs.addHeuristicNetworkCost(HEURISTIC_COST_BASE * replicationFactor);
 		} else {
-			costs.addNetworkCost(replicationFactor * estOutShipSize);
+			costs.addHeuristicNetworkCost(HEURISTIC_COST_BASE * 200);
 		}
-		costs.addHeuristicNetworkCost(HEURISTIC_COST_BASE * replicationFactor * 100);
 	}
 	
 	// --------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/AbstractPartialSolutionNode.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/AbstractPartialSolutionNode.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/AbstractPartialSolutionNode.java
index 8fd6f79..2f7cb2b 100644
--- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/AbstractPartialSolutionNode.java
+++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/AbstractPartialSolutionNode.java
@@ -42,11 +42,6 @@ public abstract class AbstractPartialSolutionNode extends OptimizerNode {
 	public abstract IterationNode getIterationNode();
 	
 	// --------------------------------------------------------------------------------------------
-
-	@Override
-	public boolean isMemoryConsumer() {
-		return false;
-	}
 	
 	public boolean isOnDynamicPath() {
 		return true;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/BinaryUnionNode.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/BinaryUnionNode.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/BinaryUnionNode.java
index 70752b5..50ec01b 100644
--- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/BinaryUnionNode.java
+++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/BinaryUnionNode.java
@@ -122,20 +122,12 @@ public class BinaryUnionNode extends TwoInputNode {
 		final RequestedLocalProperties noLocalProps = new RequestedLocalProperties();
 		
 		final int dop = getDegreeOfParallelism();
-		final int subPerInstance = getSubtasksPerInstance();
-		final int numInstances = dop / subPerInstance + (dop % subPerInstance == 0 ? 0 : 1);
 		final int inDop1 = getFirstPredecessorNode().getDegreeOfParallelism();
-		final int inSubPerInstance1 = getFirstPredecessorNode().getSubtasksPerInstance();
-		final int inNumInstances1 = inDop1 / inSubPerInstance1 + (inDop1 % inSubPerInstance1 == 0 ? 0 : 1);
 		final int inDop2 = getSecondPredecessorNode().getDegreeOfParallelism();
-		final int inSubPerInstance2 = getSecondPredecessorNode().getSubtasksPerInstance();
-		final int inNumInstances2 = inDop2 / inSubPerInstance2 + (inDop2 % inSubPerInstance2 == 0 ? 0 : 1);
-		
-		final boolean globalDopChange1 = numInstances != inNumInstances1;
-		final boolean globalDopChange2 = numInstances != inNumInstances2;
-		final boolean localDopChange1 = numInstances == inNumInstances1 & subPerInstance != inSubPerInstance1;
-		final boolean localDopChange2 = numInstances == inNumInstances2 & subPerInstance != inSubPerInstance2;
-		
+
+		final boolean dopChange1 = dop != inDop1;
+		final boolean dopChange2 = dop != inDop2;
+
 		// enumerate all pairwise combination of the children's plans together with
 		// all possible operator strategy combination
 		
@@ -154,15 +146,11 @@ public class BinaryUnionNode extends TwoInputNode {
 					Channel c1 = new Channel(child1, this.input1.getMaterializationMode());
 					if (this.input1.getShipStrategy() == null) {
 						// free to choose the ship strategy
-						igps.parameterizeChannel(c1, globalDopChange1, localDopChange1);
+						igps.parameterizeChannel(c1, dopChange1);
 						
 						// if the DOP changed, make sure that we cancel out properties, unless the
 						// ship strategy preserves/establishes them even under changing DOPs
-						if (globalDopChange1 && !c1.getShipStrategy().isNetworkStrategy()) {
-							c1.getGlobalProperties().reset();
-						}
-						if (localDopChange1 && !(c1.getShipStrategy().isNetworkStrategy() || 
-									c1.getShipStrategy().compensatesForLocalDOPChanges())) {
+						if (dopChange1 && !c1.getShipStrategy().isNetworkStrategy()) {
 							c1.getGlobalProperties().reset();
 						}
 					} else {
@@ -173,10 +161,8 @@ public class BinaryUnionNode extends TwoInputNode {
 							c1.setShipStrategy(this.input1.getShipStrategy());
 						}
 						
-						if (globalDopChange1) {
+						if (dopChange1) {
 							c1.adjustGlobalPropertiesForFullParallelismChange();
-						} else if (localDopChange1) {
-							c1.adjustGlobalPropertiesForLocalParallelismChange();
 						}
 					}
 					
@@ -184,15 +170,11 @@ public class BinaryUnionNode extends TwoInputNode {
 					Channel c2 = new Channel(child2, this.input2.getMaterializationMode());
 					if (this.input2.getShipStrategy() == null) {
 						// free to choose the ship strategy
-						igps.parameterizeChannel(c2, globalDopChange2, localDopChange2);
+						igps.parameterizeChannel(c2, dopChange2);
 						
 						// if the DOP changed, make sure that we cancel out properties, unless the
 						// ship strategy preserves/establishes them even under changing DOPs
-						if (globalDopChange2 && !c2.getShipStrategy().isNetworkStrategy()) {
-							c2.getGlobalProperties().reset();
-						}
-						if (localDopChange2 && !(c2.getShipStrategy().isNetworkStrategy() || 
-									c2.getShipStrategy().compensatesForLocalDOPChanges())) {
+						if (dopChange2 && !c2.getShipStrategy().isNetworkStrategy()) {
 							c2.getGlobalProperties().reset();
 						}
 					} else {
@@ -203,10 +185,8 @@ public class BinaryUnionNode extends TwoInputNode {
 							c2.setShipStrategy(this.input2.getShipStrategy());
 						}
 						
-						if (globalDopChange2) {
+						if (dopChange2) {
 							c2.adjustGlobalPropertiesForFullParallelismChange();
-						} else if (localDopChange2) {
-							c2.adjustGlobalPropertiesForLocalParallelismChange();
 						}
 					}
 					
@@ -224,20 +204,20 @@ public class BinaryUnionNode extends TwoInputNode {
 						if (c1.getShipStrategy() == ShipStrategyType.FORWARD && c2.getShipStrategy() != ShipStrategyType.FORWARD) {
 							// adjust c2 to c1
 							c2 = c2.clone();
-							p1.parameterizeChannel(c2,globalDopChange2);
+							p1.parameterizeChannel(c2,dopChange2);
 						} else if (c2.getShipStrategy() == ShipStrategyType.FORWARD && c1.getShipStrategy() != ShipStrategyType.FORWARD) {
 							// adjust c1 to c2
 							c1 = c1.clone();
-							p2.parameterizeChannel(c1,globalDopChange1);
+							p2.parameterizeChannel(c1,dopChange1);
 						} else if (c1.getShipStrategy() == ShipStrategyType.FORWARD && c2.getShipStrategy() == ShipStrategyType.FORWARD) {
 							boolean adjustC1 = c1.getEstimatedOutputSize() <= 0 || c2.getEstimatedOutputSize() <= 0 ||
 									c1.getEstimatedOutputSize() <= c2.getEstimatedOutputSize();
 							if (adjustC1) {
 								c2 = c2.clone();
-								p1.parameterizeChannel(c2, globalDopChange2);
+								p1.parameterizeChannel(c2, dopChange2);
 							} else {
 								c1 = c1.clone();
-								p2.parameterizeChannel(c1, globalDopChange1);
+								p2.parameterizeChannel(c1, dopChange1);
 							}
 						} else {
 							// this should never happen, as it implies both realize a different strategy, which is

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/BulkIterationNode.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/BulkIterationNode.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/BulkIterationNode.java
index f6720ea..bfbca15 100644
--- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/BulkIterationNode.java
+++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/BulkIterationNode.java
@@ -65,9 +65,9 @@ public class BulkIterationNode extends SingleInputNode implements IterationNode
 	// --------------------------------------------------------------------------------------------
 	
 	/**
-	 * Creates a new node with a single input for the optimizer plan.
+	 * Creates a new node for the bulk iteration.
 	 * 
-	 * @param iteration The PACT that the node represents.
+	 * @param iteration The bulk iteration the node represents.
 	 */
 	public BulkIterationNode(BulkIterationBase<?> iteration) {
 		super(iteration);
@@ -124,14 +124,12 @@ public class BulkIterationNode extends SingleInputNode implements IterationNode
 	public void setNextPartialSolution(OptimizerNode nextPartialSolution, OptimizerNode terminationCriterion) {
 		
 		// check if the root of the step function has the same DOP as the iteration
-		if (nextPartialSolution.getDegreeOfParallelism() != getDegreeOfParallelism() ||
-			nextPartialSolution.getSubtasksPerInstance() != getSubtasksPerInstance() )
+		if (nextPartialSolution.getDegreeOfParallelism() != getDegreeOfParallelism())
 		{
 			// add a no-op to the root to express the re-partitioning
 			NoOpNode noop = new NoOpNode();
 			noop.setDegreeOfParallelism(getDegreeOfParallelism());
-			noop.setSubtasksPerInstance(getSubtasksPerInstance());
-			
+
 			PactConnection noOpConn = new PactConnection(nextPartialSolution, noop);
 			noop.setIncomingConnection(noOpConn);
 			nextPartialSolution.addOutgoingConnection(noOpConn);
@@ -198,12 +196,7 @@ public class BulkIterationNode extends SingleInputNode implements IterationNode
 	protected List<OperatorDescriptorSingle> getPossibleProperties() {
 		return Collections.<OperatorDescriptorSingle>singletonList(new NoOpDescriptor());
 	}
-	
-	@Override
-	public boolean isMemoryConsumer() {
-		return true;
-	}
-	
+
 	@Override
 	public void computeInterestingPropertiesForInputs(CostEstimator estimator) {
 		final InterestingProperties intProps = getInterestingProperties().clone();
@@ -306,12 +299,11 @@ public class BulkIterationNode extends SingleInputNode implements IterationNode
 				else if (report == FeedbackPropertiesMeetRequirementsReport.NOT_MET) {
 					// attach a no-op node through which we create the properties of the original input
 					Channel toNoOp = new Channel(candidate);
-					globPropsReq.parameterizeChannel(toNoOp, false, false);
+					globPropsReq.parameterizeChannel(toNoOp, false);
 					locPropsReq.parameterizeChannel(toNoOp);
 					
 					UnaryOperatorNode rebuildPropertiesNode = new UnaryOperatorNode("Rebuild Partial Solution Properties", FieldList.EMPTY_LIST);
 					rebuildPropertiesNode.setDegreeOfParallelism(candidate.getDegreeOfParallelism());
-					rebuildPropertiesNode.setSubtasksPerInstance(candidate.getSubtasksPerInstance());
 					
 					SingleInputPlanNode rebuildPropertiesPlanNode = new SingleInputPlanNode(rebuildPropertiesNode, "Rebuild Partial Solution Properties", toNoOp, DriverStrategy.UNARY_NO_OP);
 					rebuildPropertiesPlanNode.initProperties(toNoOp.getGlobalProperties(), toNoOp.getLocalProperties());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/DataSinkNode.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/DataSinkNode.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/DataSinkNode.java
index fe823d2..d4f9d67 100644
--- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/DataSinkNode.java
+++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/DataSinkNode.java
@@ -87,11 +87,6 @@ public class DataSinkNode extends OptimizerNode {
 	}
 
 	@Override
-	public boolean isMemoryConsumer() {
-		return getPactContract().getPartitionOrdering() != null || getPactContract().getLocalOrder() != null;
-	}
-
-	@Override
 	public List<PactConnection> getIncomingConnections() {
 		return Collections.singletonList(this.input);
 	}
@@ -194,21 +189,16 @@ public class DataSinkNode extends OptimizerNode {
 		List<PlanNode> outputPlans = new ArrayList<PlanNode>();
 		
 		final int dop = getDegreeOfParallelism();
-		final int subPerInstance = getSubtasksPerInstance();
 		final int inDop = getPredecessorNode().getDegreeOfParallelism();
-		final int inSubPerInstance = getPredecessorNode().getSubtasksPerInstance();
-		final int numInstances = dop / subPerInstance + (dop % subPerInstance == 0 ? 0 : 1);
-		final int inNumInstances = inDop / inSubPerInstance + (inDop % inSubPerInstance == 0 ? 0 : 1);
-		
-		final boolean globalDopChange = numInstances != inNumInstances;
-		final boolean localDopChange = numInstances == inNumInstances & subPerInstance != inSubPerInstance;
-		
+
+		final boolean dopChange = dop != inDop;
+
 		InterestingProperties ips = this.input.getInterestingProperties();
 		for (PlanNode p : subPlans) {
 			for (RequestedGlobalProperties gp : ips.getGlobalProperties()) {
 				for (RequestedLocalProperties lp : ips.getLocalProperties()) {
 					Channel c = new Channel(p);
-					gp.parameterizeChannel(c, globalDopChange, localDopChange);
+					gp.parameterizeChannel(c, dopChange);
 					lp.parameterizeChannel(c);
 					c.setRequiredLocalProps(lp);
 					c.setRequiredGlobalProps(gp);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/DataSourceNode.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/DataSourceNode.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/DataSourceNode.java
index 17c11c9..7234420 100644
--- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/DataSourceNode.java
+++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/DataSourceNode.java
@@ -55,7 +55,6 @@ public class DataSourceNode extends OptimizerNode {
 		
 		if (NonParallelInput.class.isAssignableFrom(pactContract.getUserCodeWrapper().getUserCodeClass())) {
 			setDegreeOfParallelism(1);
-			setSubtasksPerInstance(1);
 			this.sequentialInput = true;
 		} else {
 			this.sequentialInput = false;
@@ -78,27 +77,12 @@ public class DataSourceNode extends OptimizerNode {
 	}
 
 	@Override
-	public boolean isMemoryConsumer() {
-		return false;
-	}
-	
-
-	@Override
 	public void setDegreeOfParallelism(int degreeOfParallelism) {
 		// if unsplittable, DOP remains at 1
 		if (!this.sequentialInput) {
 			super.setDegreeOfParallelism(degreeOfParallelism);
 		}
 	}
-	
-
-	@Override
-	public void setSubtasksPerInstance(int instancesPerMachine) {
-		// if unsplittable, DOP remains at 1
-		if (!this.sequentialInput) {
-			super.setSubtasksPerInstance(instancesPerMachine);
-		}
-	}
 
 	@Override
 	public List<PactConnection> getIncomingConnections() {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/GroupReduceNode.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/GroupReduceNode.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/GroupReduceNode.java
index 6eb2903..4d7230e 100644
--- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/GroupReduceNode.java
+++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/GroupReduceNode.java
@@ -46,7 +46,6 @@ public class GroupReduceNode extends SingleInputNode {
 		if (this.keys == null) {
 			// case of a key-less reducer. force a parallelism of 1
 			setDegreeOfParallelism(1);
-			setSubtasksPerInstance(1);
 		}
 	}
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/OptimizerNode.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/OptimizerNode.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/OptimizerNode.java
index b2c9330..85a6568 100644
--- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/OptimizerNode.java
+++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/OptimizerNode.java
@@ -262,13 +262,6 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
 	 */
 	@Override
 	public abstract void accept(Visitor<OptimizerNode> visitor);
-
-	/**
-	 * Checks, whether this node requires memory for its tasks or not.
-	 * 
-	 * @return True, if this node contains logic that requires memory usage, false otherwise.
-	 */
-	public abstract boolean isMemoryConsumer();
 	
 	/**
 	 * Checks whether a field is modified by the user code or whether it is kept unchanged.
@@ -408,7 +401,7 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
 	 * @param degreeOfParallelism
 	 *        The degree of parallelism to set.
 	 * @throws IllegalArgumentException
-	 *         If the degree of parallelism is smaller than one.
+	 *         If the degree of parallelism is smaller than one and not -1.
 	 */
 	public void setDegreeOfParallelism(int degreeOfParallelism) {
 		if (degreeOfParallelism < 1) {
@@ -416,48 +409,6 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
 		}
 		this.degreeOfParallelism = degreeOfParallelism;
 	}
-
-	/**
-	 * Gets the number of parallel instances of the contract that are
-	 * to be executed on the same compute instance (logical machine).
-	 * 
-	 * @return The number of subtask instances per machine.
-	 */
-	public int getSubtasksPerInstance() {
-		return this.subtasksPerInstance;
-	}
-
-	/**
-	 * Sets the number of parallel task instances of the contract that are
-	 * to be executed on the same computing instance (logical machine).
-	 * 
-	 * @param instancesPerMachine The instances per machine.
-	 * @throws IllegalArgumentException If the number of instances per machine is smaller than one.
-	 */
-	public void setSubtasksPerInstance(int instancesPerMachine) {
-		if (instancesPerMachine < 1) {
-			throw new IllegalArgumentException();
-		}
-		this.subtasksPerInstance = instancesPerMachine;
-	}
-	
-	/**
-	 * Gets the minimal guaranteed memory per subtask for tasks represented by this OptimizerNode.
-	 *
-	 * @return The minimal guaranteed memory per subtask, in bytes.
-	 */
-	public long getMinimalMemoryPerSubTask() {
-		return this.minimalMemoryPerSubTask;
-	}
-	
-	/**
-	 * Sets the minimal guaranteed memory per subtask for tasks represented by this OptimizerNode.
-	 *
-	 * @param minimalGuaranteedMemory The minimal guaranteed memory per subtask, in bytes.
-	 */
-	public void setMinimalMemoryPerSubTask(long minimalGuaranteedMemory) {
-		this.minimalMemoryPerSubTask = minimalGuaranteedMemory;
-	}
 	
 	/**
 	 * Gets the amount of memory that all subtasks of this task have jointly available.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/ReduceNode.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/ReduceNode.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/ReduceNode.java
index 2190060..409d027 100644
--- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/ReduceNode.java
+++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/ReduceNode.java
@@ -36,7 +36,6 @@ public class ReduceNode extends SingleInputNode {
 		if (this.keys == null) {
 			// case of a key-less reducer. force a parallelism of 1
 			setDegreeOfParallelism(1);
-			setSubtasksPerInstance(1);
 		}
 	}
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/SingleInputNode.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/SingleInputNode.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/SingleInputNode.java
index 8bf3f16..0b872a7 100644
--- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/SingleInputNode.java
+++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/SingleInputNode.java
@@ -206,22 +206,6 @@ public abstract class SingleInputNode extends OptimizerNode {
 	
 	protected abstract List<OperatorDescriptorSingle> getPossibleProperties();
 	
-
-	@Override
-	public boolean isMemoryConsumer() {
-		for (OperatorDescriptorSingle dps : getPossibleProperties()) {
-			if (dps.getStrategy().firstDam().isMaterializing()) {
-				return true;
-			}
-			for (RequestedLocalProperties rlp : dps.getPossibleLocalProperties()) {
-				if (!rlp.isTrivial()) {
-					return true;
-				}
-			}
-		}
-		return false;
-	}
-
 	@Override
 	public void computeInterestingPropertiesForInputs(CostEstimator estimator) {
 		// get what we inherit and what is preserved by our user code 
@@ -284,30 +268,21 @@ public abstract class SingleInputNode extends OptimizerNode {
 		final ArrayList<PlanNode> outputPlans = new ArrayList<PlanNode>();
 		
 		final int dop = getDegreeOfParallelism();
-		final int subPerInstance = getSubtasksPerInstance();
 		final int inDop = getPredecessorNode().getDegreeOfParallelism();
-		final int inSubPerInstance = getPredecessorNode().getSubtasksPerInstance();
-		final int numInstances = dop / subPerInstance + (dop % subPerInstance == 0 ? 0 : 1);
-		final int inNumInstances = inDop / inSubPerInstance + (inDop % inSubPerInstance == 0 ? 0 : 1);
-		
-		final boolean globalDopChange = numInstances != inNumInstances;
-		final boolean localDopChange = numInstances == inNumInstances & subPerInstance != inSubPerInstance;
-		
+
+		final boolean dopChange = inDop != dop;
+
 		// create all candidates
 		for (PlanNode child : subPlans) {
 			if (this.inConn.getShipStrategy() == null) {
 				// pick the strategy ourselves
 				for (RequestedGlobalProperties igps: intGlobal) {
 					final Channel c = new Channel(child, this.inConn.getMaterializationMode());
-					igps.parameterizeChannel(c, globalDopChange, localDopChange);
+					igps.parameterizeChannel(c, dopChange);
 					
 					// if the DOP changed, make sure that we cancel out properties, unless the
 					// ship strategy preserves/establishes them even under changing DOPs
-					if (globalDopChange && !c.getShipStrategy().isNetworkStrategy()) {
-						c.getGlobalProperties().reset();
-					}
-					if (localDopChange && !(c.getShipStrategy().isNetworkStrategy() || 
-								c.getShipStrategy().compensatesForLocalDOPChanges())) {
+					if (dopChange && !c.getShipStrategy().isNetworkStrategy()) {
 						c.getGlobalProperties().reset();
 					}
 					
@@ -332,12 +307,10 @@ public abstract class SingleInputNode extends OptimizerNode {
 					c.setShipStrategy(this.inConn.getShipStrategy());
 				}
 				
-				if (globalDopChange) {
+				if (dopChange) {
 					c.adjustGlobalPropertiesForFullParallelismChange();
-				} else if (localDopChange) {
-					c.adjustGlobalPropertiesForLocalParallelismChange();
 				}
-				
+
 				// check whether we meet any of the accepted properties
 				for (RequestedGlobalProperties rgps: allValidGlobals) {
 					if (rgps.isMetBy(c.getGlobalProperties())) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/SinkJoiner.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/SinkJoiner.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/SinkJoiner.java
index 2c765a5..a711ac5 100644
--- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/SinkJoiner.java
+++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/SinkJoiner.java
@@ -42,7 +42,6 @@ public class SinkJoiner extends TwoInputNode {
 		this.input2 = conn2;
 		
 		setDegreeOfParallelism(1);
-		setSubtasksPerInstance(1);
 	}
 	
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/TwoInputNode.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/TwoInputNode.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/TwoInputNode.java
index 9898c81..97a92d0 100644
--- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/TwoInputNode.java
+++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/TwoInputNode.java
@@ -251,22 +251,6 @@ public abstract class TwoInputNode extends OptimizerNode {
 	}
 	
 	protected abstract List<OperatorDescriptorDual> getPossibleProperties();
-	
-	@Override
-	public boolean isMemoryConsumer() {
-		for (OperatorDescriptorDual dpd : this.possibleProperties) {
-			if (dpd.getStrategy().firstDam().isMaterializing() ||
-				dpd.getStrategy().secondDam().isMaterializing()) {
-				return true;
-			}
-			for (LocalPropertiesPair prp : dpd.getPossibleLocalProperties()) {
-				if (!(prp.getProperties1().isTrivial() && prp.getProperties2().isTrivial())) {
-					return true;
-				}
-			}
-		}
-		return false;
-	}
 
 	@Override
 	public void computeInterestingPropertiesForInputs(CostEstimator estimator) {
@@ -348,20 +332,12 @@ public abstract class TwoInputNode extends OptimizerNode {
 		final ArrayList<PlanNode> outputPlans = new ArrayList<PlanNode>();
 		
 		final int dop = getDegreeOfParallelism();
-		final int subPerInstance = getSubtasksPerInstance();
-		final int numInstances = dop / subPerInstance + (dop % subPerInstance == 0 ? 0 : 1);
 		final int inDop1 = getFirstPredecessorNode().getDegreeOfParallelism();
-		final int inSubPerInstance1 = getFirstPredecessorNode().getSubtasksPerInstance();
-		final int inNumInstances1 = inDop1 / inSubPerInstance1 + (inDop1 % inSubPerInstance1 == 0 ? 0 : 1);
 		final int inDop2 = getSecondPredecessorNode().getDegreeOfParallelism();
-		final int inSubPerInstance2 = getSecondPredecessorNode().getSubtasksPerInstance();
-		final int inNumInstances2 = inDop2 / inSubPerInstance2 + (inDop2 % inSubPerInstance2 == 0 ? 0 : 1);
-		
-		final boolean globalDopChange1 = numInstances != inNumInstances1;
-		final boolean globalDopChange2 = numInstances != inNumInstances2;
-		final boolean localDopChange1 = numInstances == inNumInstances1 & subPerInstance != inSubPerInstance1;
-		final boolean localDopChange2 = numInstances == inNumInstances2 & subPerInstance != inSubPerInstance2;
-		
+
+		final boolean dopChange1 = dop != inDop1;
+		final boolean dopChange2 = dop != inDop2;
+
 		// enumerate all pairwise combination of the children's plans together with
 		// all possible operator strategy combination
 		
@@ -380,15 +356,11 @@ public abstract class TwoInputNode extends OptimizerNode {
 					final Channel c1 = new Channel(child1, this.input1.getMaterializationMode());
 					if (this.input1.getShipStrategy() == null) {
 						// free to choose the ship strategy
-						igps1.parameterizeChannel(c1, globalDopChange1, localDopChange1);
+						igps1.parameterizeChannel(c1, dopChange1);
 						
 						// if the DOP changed, make sure that we cancel out properties, unless the
 						// ship strategy preserves/establishes them even under changing DOPs
-						if (globalDopChange1 && !c1.getShipStrategy().isNetworkStrategy()) {
-							c1.getGlobalProperties().reset();
-						}
-						if (localDopChange1 && !(c1.getShipStrategy().isNetworkStrategy() || 
-									c1.getShipStrategy().compensatesForLocalDOPChanges())) {
+						if (dopChange1 && !c1.getShipStrategy().isNetworkStrategy()) {
 							c1.getGlobalProperties().reset();
 						}
 					} else {
@@ -399,10 +371,8 @@ public abstract class TwoInputNode extends OptimizerNode {
 							c1.setShipStrategy(this.input1.getShipStrategy());
 						}
 						
-						if (globalDopChange1) {
+						if (dopChange1) {
 							c1.adjustGlobalPropertiesForFullParallelismChange();
-						} else if (localDopChange1) {
-							c1.adjustGlobalPropertiesForLocalParallelismChange();
 						}
 					}
 					
@@ -411,15 +381,11 @@ public abstract class TwoInputNode extends OptimizerNode {
 						final Channel c2 = new Channel(child2, this.input2.getMaterializationMode());
 						if (this.input2.getShipStrategy() == null) {
 							// free to choose the ship strategy
-							igps2.parameterizeChannel(c2, globalDopChange2, localDopChange2);
+							igps2.parameterizeChannel(c2, dopChange2);
 							
 							// if the DOP changed, make sure that we cancel out properties, unless the
 							// ship strategy preserves/establishes them even under changing DOPs
-							if (globalDopChange2 && !c2.getShipStrategy().isNetworkStrategy()) {
-								c2.getGlobalProperties().reset();
-							}
-							if (localDopChange2 && !(c2.getShipStrategy().isNetworkStrategy() || 
-										c2.getShipStrategy().compensatesForLocalDOPChanges())) {
+							if (dopChange2 && !c2.getShipStrategy().isNetworkStrategy()) {
 								c2.getGlobalProperties().reset();
 							}
 						} else {
@@ -430,10 +396,8 @@ public abstract class TwoInputNode extends OptimizerNode {
 								c2.setShipStrategy(this.input2.getShipStrategy());
 							}
 							
-							if (globalDopChange2) {
+							if (dopChange2) {
 								c2.adjustGlobalPropertiesForFullParallelismChange();
-							} else if (localDopChange2) {
-								c2.adjustGlobalPropertiesForLocalParallelismChange();
 							}
 						}
 						


[02/53] [abbrv] Rework the Taskmanager to a slot based model and remove legacy cloud code

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/TransitiveClosureNaiveITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/TransitiveClosureNaiveITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/TransitiveClosureNaiveITCase.java
index feb5f34..95d75fb 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/TransitiveClosureNaiveITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/TransitiveClosureNaiveITCase.java
@@ -39,7 +39,7 @@ public class TransitiveClosureNaiveITCase extends RecordAPITestBase {
 	protected Plan getTestJob() {
 		TransitiveClosureNaive transitiveClosureNaive = new TransitiveClosureNaive();
 		// "2" is the number of iterations here
-		return transitiveClosureNaive.getScalaPlan(4, 2, verticesPath, edgesPath, resultPath);
+		return transitiveClosureNaive.getScalaPlan(DOP, 2, verticesPath, edgesPath, resultPath);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WebLogAnalysisITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WebLogAnalysisITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WebLogAnalysisITCase.java
index 2ddef0c..67c5ce1 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WebLogAnalysisITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WebLogAnalysisITCase.java
@@ -21,6 +21,6 @@ public class WebLogAnalysisITCase extends eu.stratosphere.test.recordJobTests.We
 	@Override
 	protected Plan getTestJob() {
 		WebLogAnalysis webLogAnalysis = new WebLogAnalysis();
-		return webLogAnalysis.getScalaPlan(4, docsPath, ranksPath, visitsPath, resultPath);
+		return webLogAnalysis.getScalaPlan(DOP, docsPath, ranksPath, visitsPath, resultPath);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WordCountITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WordCountITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WordCountITCase.java
index 205828f..42ee31a 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WordCountITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WordCountITCase.java
@@ -22,6 +22,6 @@ public class WordCountITCase extends eu.stratosphere.test.recordJobTests.WordCou
 	@Override
 	protected Plan getTestJob() {
 		WordCount wc = new WordCount();
-		return wc.getScalaPlan(4, textPath, resultPath);
+		return wc.getScalaPlan(DOP, textPath, resultPath);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WordCountPactValueITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WordCountPactValueITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WordCountPactValueITCase.java
index 4627d48..44b37f6 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WordCountPactValueITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WordCountPactValueITCase.java
@@ -22,6 +22,6 @@ public class WordCountPactValueITCase extends eu.stratosphere.test.recordJobTest
 	@Override
 	protected Plan getTestJob() {
 		WordCountWithUserDefinedType wc = new WordCountWithUserDefinedType();
-		return wc.getScalaPlan(4, textPath, resultPath);
+		return wc.getScalaPlan(DOP, textPath, resultPath);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WordCountWithCountFunctionITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WordCountWithCountFunctionITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WordCountWithCountFunctionITCase.java
index 12082ac..a8eba29 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WordCountWithCountFunctionITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WordCountWithCountFunctionITCase.java
@@ -20,6 +20,6 @@ public class WordCountWithCountFunctionITCase extends eu.stratosphere.test.recor
 
 	@Override
 	protected Plan getTestJob() {
-		return new WordCountWithCount().getScalaPlan(4, textPath, resultPath);
+		return new WordCountWithCount().getScalaPlan(DOP, textPath, resultPath);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/failingPrograms/TaskFailureITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/failingPrograms/TaskFailureITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/failingPrograms/TaskFailureITCase.java
index 3f4791e..4f5d38d 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/failingPrograms/TaskFailureITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/failingPrograms/TaskFailureITCase.java
@@ -36,6 +36,8 @@ import eu.stratosphere.util.Collector;
  */
 public class TaskFailureITCase extends FailingTestBase {
 
+	private static final int DOP = 4;
+
 	// input for map tasks
 	private static final String MAP_IN = "1 1\n2 2\n2 8\n4 4\n4 4\n6 6\n7 7\n8 8\n" +
 											"1 1\n2 2\n2 2\n4 4\n4 4\n6 3\n5 9\n8 8\n" +
@@ -47,6 +49,10 @@ public class TaskFailureITCase extends FailingTestBase {
 
 	private String inputPath;
 	private String resultPath;
+
+	public TaskFailureITCase(){
+		setTaskManagerNumSlots(DOP);
+	}
 	
 	@Override
 	protected void preSubmit() throws Exception {
@@ -73,7 +79,7 @@ public class TaskFailureITCase extends FailingTestBase {
 
 		// generate plan
 		Plan plan = new Plan(output);
-		plan.setDefaultParallelism(4);
+		plan.setDefaultParallelism(DOP);
 
 		// optimize and compile plan 
 		PactCompiler pc = new PactCompiler(new DataStatistics());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/CoGroupConnectedComponentsITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/CoGroupConnectedComponentsITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/CoGroupConnectedComponentsITCase.java
index c937435..f8d82ae 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/CoGroupConnectedComponentsITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/CoGroupConnectedComponentsITCase.java
@@ -50,6 +50,10 @@ public class CoGroupConnectedComponentsITCase extends RecordAPITestBase {
 	protected String verticesPath;
 	protected String edgesPath;
 	protected String resultPath;
+
+	public CoGroupConnectedComponentsITCase(){
+		setTaskManagerNumSlots(DOP);
+	}
 	
 	
 	@Override
@@ -61,7 +65,7 @@ public class CoGroupConnectedComponentsITCase extends RecordAPITestBase {
 	
 	@Override
 	protected Plan getTestJob() {
-		return getPlan(4, verticesPath, edgesPath, resultPath, 100);
+		return getPlan(DOP, verticesPath, edgesPath, resultPath, 100);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/ConnectedComponentsITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/ConnectedComponentsITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/ConnectedComponentsITCase.java
index 9c88bb5..53feae6 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/ConnectedComponentsITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/ConnectedComponentsITCase.java
@@ -33,6 +33,10 @@ public class ConnectedComponentsITCase extends RecordAPITestBase {
 	protected String verticesPath;
 	protected String edgesPath;
 	protected String resultPath;
+
+	public ConnectedComponentsITCase(){
+		setTaskManagerNumSlots(DOP);
+	}
 	
 	
 	@Override
@@ -45,7 +49,7 @@ public class ConnectedComponentsITCase extends RecordAPITestBase {
 	@Override
 	protected Plan getTestJob() {
 		WorksetConnectedComponents cc = new WorksetConnectedComponents();
-		return cc.getPlan("4",  verticesPath, edgesPath, resultPath, "100");
+		return cc.getPlan(new Integer(DOP).toString(),  verticesPath, edgesPath, resultPath, "100");
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/ConnectedComponentsWithDeferredUpdateITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/ConnectedComponentsWithDeferredUpdateITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/ConnectedComponentsWithDeferredUpdateITCase.java
index 8de877c..eb2eec0 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/ConnectedComponentsWithDeferredUpdateITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/ConnectedComponentsWithDeferredUpdateITCase.java
@@ -59,6 +59,7 @@ public class ConnectedComponentsWithDeferredUpdateITCase extends RecordAPITestBa
 	
 	public ConnectedComponentsWithDeferredUpdateITCase(Configuration config) {
 		super(config);
+		setTaskManagerNumSlots(DOP);
 	}
 	
 	@Override
@@ -71,7 +72,7 @@ public class ConnectedComponentsWithDeferredUpdateITCase extends RecordAPITestBa
 	@Override
 	protected Plan getTestJob() {
 		boolean extraMapper = config.getBoolean("ExtraMapper", false);
-		return getPlan(4, verticesPath, edgesPath, resultPath, 100, extraMapper);
+		return getPlan(DOP, verticesPath, edgesPath, resultPath, 100, extraMapper);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/ConnectedComponentsWithSolutionSetFirstITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/ConnectedComponentsWithSolutionSetFirstITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/ConnectedComponentsWithSolutionSetFirstITCase.java
index 5c696a2..f040f06 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/ConnectedComponentsWithSolutionSetFirstITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/ConnectedComponentsWithSolutionSetFirstITCase.java
@@ -51,6 +51,10 @@ public class ConnectedComponentsWithSolutionSetFirstITCase extends RecordAPITest
 	protected String verticesPath;
 	protected String edgesPath;
 	protected String resultPath;
+
+	public ConnectedComponentsWithSolutionSetFirstITCase(){
+		setTaskManagerNumSlots(DOP);
+	}
 	
 	
 	@Override
@@ -62,7 +66,8 @@ public class ConnectedComponentsWithSolutionSetFirstITCase extends RecordAPITest
 	
 	@Override
 	protected Plan getTestJob() {
-		return getPlanForWorksetConnectedComponentsWithSolutionSetAsFirstInput(4, verticesPath, edgesPath, resultPath, 100);
+		return getPlanForWorksetConnectedComponentsWithSolutionSetAsFirstInput(DOP, verticesPath, edgesPath,
+				resultPath, 100);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/DanglingPageRankITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/DanglingPageRankITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/DanglingPageRankITCase.java
index e84f21e..5390eed 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/DanglingPageRankITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/DanglingPageRankITCase.java
@@ -35,6 +35,7 @@ public class DanglingPageRankITCase extends RecordAPITestBase {
 	
 	public DanglingPageRankITCase(Configuration config) {
 		super(config);
+		setTaskManagerNumSlots(DOP);
 	}
 	
 	
@@ -63,7 +64,7 @@ public class DanglingPageRankITCase extends RecordAPITestBase {
 	@Parameters
 	public static Collection<Object[]> getConfigurations() {
 		Configuration config1 = new Configuration();
-		config1.setInteger("PageRankITCase#NoSubtasks", 4);
+		config1.setInteger("PageRankITCase#NoSubtasks", DOP);
 		config1.setString("PageRankITCase#NumIterations", "25");
 		return toParameterList(config1);
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/DeltaPageRankITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/DeltaPageRankITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/DeltaPageRankITCase.java
index 8f06929..1eb81ed 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/DeltaPageRankITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/DeltaPageRankITCase.java
@@ -36,6 +36,7 @@ public class DeltaPageRankITCase extends RecordAPITestBase {
 	
 	public DeltaPageRankITCase(Configuration config) {
 		super(config);
+		setTaskManagerNumSlots(DOP);
 	}
 	
 	@Override
@@ -66,7 +67,7 @@ public class DeltaPageRankITCase extends RecordAPITestBase {
 	@Parameters
 	public static Collection<Object[]> getConfigurations() {
 		Configuration config1 = new Configuration();
-		config1.setInteger("NumSubtasks", 4);
+		config1.setInteger("NumSubtasks", DOP);
 		config1.setInteger("NumIterations", 3);
 		return toParameterList(config1);
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/DependencyConnectedComponentsITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/DependencyConnectedComponentsITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/DependencyConnectedComponentsITCase.java
index deda551..e1339e3 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/DependencyConnectedComponentsITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/DependencyConnectedComponentsITCase.java
@@ -28,7 +28,6 @@ import eu.stratosphere.api.java.tuple.Tuple2;
 import eu.stratosphere.test.util.JavaProgramTestBase;
 import eu.stratosphere.util.Collector;
 
-
 /**
  * 
  * Iterative Connected Components test case which recomputes only the elements
@@ -46,6 +45,10 @@ public class DependencyConnectedComponentsITCase extends JavaProgramTestBase {
 	protected static List<Tuple2<Long, Long>> edgesInput = new ArrayList<Tuple2<Long, Long>>();
 	private String resultPath;
 	private String expectedResult;
+
+	public DependencyConnectedComponentsITCase(){
+		setTaskManagerNumSlots(DOP);
+	}
 	
 	
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationTerminationWithTerminationTail.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationTerminationWithTerminationTail.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationTerminationWithTerminationTail.java
index 3c38263..50c1970 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationTerminationWithTerminationTail.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationTerminationWithTerminationTail.java
@@ -40,6 +40,10 @@ public class IterationTerminationWithTerminationTail extends RecordAPITestBase {
 
 	protected String dataPath;
 	protected String resultPath;
+
+	public IterationTerminationWithTerminationTail(){
+		setTaskManagerNumSlots(DOP);
+	}
 	
 	@Override
 	protected void preSubmit() throws Exception {
@@ -54,7 +58,7 @@ public class IterationTerminationWithTerminationTail extends RecordAPITestBase {
 
 	@Override
 	protected Plan getTestJob() {
-		return getTestPlanPlan(4, dataPath, resultPath);
+		return getTestPlanPlan(DOP, dataPath, resultPath);
 	}
 	
 	private static Plan getTestPlanPlan(int numSubTasks, String input, String output) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationTerminationWithTwoTails.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationTerminationWithTwoTails.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationTerminationWithTwoTails.java
index f3c6cbb..cfe0510 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationTerminationWithTwoTails.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationTerminationWithTwoTails.java
@@ -42,6 +42,10 @@ public class IterationTerminationWithTwoTails extends RecordAPITestBase {
 	protected String dataPath;
 	protected String resultPath;
 
+	public IterationTerminationWithTwoTails(){
+		setTaskManagerNumSlots(DOP);
+	}
+
 	@Override
 	protected void preSubmit() throws Exception {
 		dataPath = createTempFile("datapoints.txt", INPUT);
@@ -55,7 +59,7 @@ public class IterationTerminationWithTwoTails extends RecordAPITestBase {
 
 	@Override
 	protected Plan getTestJob() {
-		return getTestPlanPlan(4, dataPath, resultPath);
+		return getTestPlanPlan(DOP, dataPath, resultPath);
 	}
 	
 	private static Plan getTestPlanPlan(int numSubTasks, String input, String output) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationWithAllReducerITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationWithAllReducerITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationWithAllReducerITCase.java
index c7e28e2..7085776 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationWithAllReducerITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationWithAllReducerITCase.java
@@ -40,7 +40,9 @@ public class IterationWithAllReducerITCase extends RecordAPITestBase {
 	protected String dataPath;
 	protected String resultPath;
 
-	
+	public IterationWithAllReducerITCase(){
+		setTaskManagerNumSlots(4);
+	}
 
 	@Override
 	protected void preSubmit() throws Exception {
@@ -55,7 +57,7 @@ public class IterationWithAllReducerITCase extends RecordAPITestBase {
 
 	@Override
 	protected Plan getTestJob() {
-		Plan plan = getTestPlanPlan(4, dataPath, resultPath);
+		Plan plan = getTestPlanPlan(DOP, dataPath, resultPath);
 		return plan;
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationWithChainingITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationWithChainingITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationWithChainingITCase.java
index 55c7a35..dba561d 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationWithChainingITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationWithChainingITCase.java
@@ -46,6 +46,7 @@ public class IterationWithChainingITCase extends RecordAPITestBase {
 
     public IterationWithChainingITCase(Configuration config) {
         super(config);
+		setTaskManagerNumSlots(DOP);
     }
 
     @Override
@@ -69,7 +70,7 @@ public class IterationWithChainingITCase extends RecordAPITestBase {
     @Parameters
     public static Collection<Object[]> getConfigurations() {
         Configuration config1 = new Configuration();
-        config1.setInteger("ChainedMapperITCase#NoSubtasks", 4);
+        config1.setInteger("ChainedMapperITCase#NoSubtasks", DOP);
         return toParameterList(config1);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationWithUnionITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationWithUnionITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationWithUnionITCase.java
index 1272f6e..884dd64 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationWithUnionITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationWithUnionITCase.java
@@ -46,6 +46,7 @@ public class IterationWithUnionITCase extends RecordAPITestBase {
 	
 	public IterationWithUnionITCase(Configuration config) {
 		super(config);
+		setTaskManagerNumSlots(DOP);
 	}
 
 	@Override
@@ -67,7 +68,7 @@ public class IterationWithUnionITCase extends RecordAPITestBase {
 	@Parameters
 	public static Collection<Object[]> getConfigurations() {
 		Configuration config1 = new Configuration();
-		config1.setInteger("IterationWithUnionITCase#NumSubtasks", 4);
+		config1.setInteger("IterationWithUnionITCase#NumSubtasks", DOP);
 
 		return toParameterList(config1);
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterativeKMeansITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterativeKMeansITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterativeKMeansITCase.java
index c8da45f..199f108 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterativeKMeansITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterativeKMeansITCase.java
@@ -27,6 +27,10 @@ public class IterativeKMeansITCase extends RecordAPITestBase {
 	protected String dataPath;
 	protected String clusterPath;
 	protected String resultPath;
+
+	public IterativeKMeansITCase(){
+		setTaskManagerNumSlots(DOP);
+	}
 	
 	@Override
 	protected void preSubmit() throws Exception {
@@ -38,7 +42,7 @@ public class IterativeKMeansITCase extends RecordAPITestBase {
 	@Override
 	protected Plan getTestJob() {
 		KMeansBroadcast kmi = new KMeansBroadcast();
-		return kmi.getPlan("4", dataPath, clusterPath, resultPath, "20");
+		return kmi.getPlan(String.valueOf(DOP), dataPath, clusterPath, resultPath, "20");
 	}
 
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/KMeansITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/KMeansITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/KMeansITCase.java
index 7085c3c..d67da35 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/KMeansITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/KMeansITCase.java
@@ -27,7 +27,11 @@ public class KMeansITCase extends RecordAPITestBase {
 	protected String dataPath;
 	protected String clusterPath;
 	protected String resultPath;
-	
+
+	public KMeansITCase(){
+		setTaskManagerNumSlots(DOP);
+	}
+
 	@Override
 	protected void preSubmit() throws Exception {
 		dataPath = createTempFile("datapoints.txt", KMeansData.DATAPOINTS);
@@ -38,7 +42,7 @@ public class KMeansITCase extends RecordAPITestBase {
 	@Override
 	protected Plan getTestJob() {
 		KMeansBroadcast kmi = new KMeansBroadcast();
-		return kmi.getPlan("4", dataPath, clusterPath, resultPath, "20");
+		return kmi.getPlan(String.valueOf(DOP), dataPath, clusterPath, resultPath, "20");
 	}
 
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/LineRankITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/LineRankITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/LineRankITCase.java
index f38fb7e..7c68605 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/LineRankITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/LineRankITCase.java
@@ -54,6 +54,7 @@ public class LineRankITCase extends RecordAPITestBase {
 	
 	public LineRankITCase(Configuration config) {
 		super(config);
+		setTaskManagerNumSlots(DOP);
 	}
 	
 	@Override
@@ -68,7 +69,7 @@ public class LineRankITCase extends RecordAPITestBase {
 		LineRank lr = new LineRank();
 		
 		Plan plan = lr.getScalaPlan(
-			config.getInteger("NumSubtasks", 1), 
+			config.getInteger("NumSubtasks", 1),
 			sourcesPath,
 			targetsPath,
 			9,
@@ -79,7 +80,7 @@ public class LineRankITCase extends RecordAPITestBase {
 	@Parameters
 	public static Collection<Object[]> getConfigurations() {
 		Configuration config1 = new Configuration();
-		config1.setInteger("NumSubtasks", 4);
+		config1.setInteger("NumSubtasks", DOP);
 		config1.setInteger("NumIterations", 5);
 		return toParameterList(config1);
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/PageRankITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/PageRankITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/PageRankITCase.java
index 8b12b53..58d8170 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/PageRankITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/PageRankITCase.java
@@ -38,6 +38,7 @@ public class PageRankITCase extends RecordAPITestBase {
 	
 	public PageRankITCase(Configuration config) {
 		super(config);
+		setTaskManagerNumSlots(DOP);
 	}
 	
 	@Override
@@ -64,7 +65,7 @@ public class PageRankITCase extends RecordAPITestBase {
 	@Parameters
 	public static Collection<Object[]> getConfigurations() {
 		Configuration config1 = new Configuration();
-		config1.setInteger("NumSubtasks", 4);
+		config1.setInteger("NumSubtasks", DOP);
 		config1.setString("NumIterations", "5");
 		return toParameterList(config1);
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/ConnectedComponentsNepheleITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/ConnectedComponentsNepheleITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/ConnectedComponentsNepheleITCase.java
index 7eff1aa..b2b3df2 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/ConnectedComponentsNepheleITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/ConnectedComponentsNepheleITCase.java
@@ -82,6 +82,10 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 
 	private static final long MEM_PER_CONSUMER = 3;
 
+	private static final int DOP = 4;
+
+	private static final double MEM_FRAC_PER_CONSUMER = (double)MEM_PER_CONSUMER/TASK_MANAGER_MEMORY_SIZE*DOP;
+
 	protected String verticesPath;
 
 	protected String edgesPath;
@@ -90,6 +94,7 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 
 	public ConnectedComponentsNepheleITCase(Configuration config) {
 		super(config);
+		setTaskManagerNumSlots(DOP);
 	}
 
 	@Parameters
@@ -118,20 +123,19 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 
 	@Override
 	protected JobGraph getJobGraph() throws Exception {
-		int dop = 4;
 		int maxIterations = 100;
 
 		int type = config.getInteger("testcase", 0);
 		switch (type) {
 		case 1:
-			return createJobGraphUnifiedTails(verticesPath, edgesPath, resultPath, dop, maxIterations);
+			return createJobGraphUnifiedTails(verticesPath, edgesPath, resultPath, DOP, maxIterations);
 		case 2:
-			return createJobGraphSeparateTails(verticesPath, edgesPath, resultPath, dop, maxIterations);
+			return createJobGraphSeparateTails(verticesPath, edgesPath, resultPath, DOP, maxIterations);
 		case 3:
-			return createJobGraphIntermediateWorksetUpdateAndSolutionSetTail(verticesPath, edgesPath, resultPath, dop,
+			return createJobGraphIntermediateWorksetUpdateAndSolutionSetTail(verticesPath, edgesPath, resultPath, DOP,
 				maxIterations);
 		case 4:
-			return createJobGraphSolutionSetUpdateAndWorksetTail(verticesPath, edgesPath, resultPath, dop,
+			return createJobGraphSolutionSetUpdateAndWorksetTail(verticesPath, edgesPath, resultPath, DOP,
 				maxIterations);
 		default:
 			throw new RuntimeException("Broken test configuration");
@@ -167,7 +171,7 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		@SuppressWarnings("unchecked")
 		CsvInputFormat verticesInFormat = new CsvInputFormat(' ', LongValue.class);
 		JobInputVertex verticesInput = JobGraphUtils.createInput(verticesInFormat, verticesPath, "VerticesInput",
-			jobGraph, numSubTasks, numSubTasks);
+			jobGraph, numSubTasks);
 		TaskConfig verticesInputConfig = new TaskConfig(verticesInput.getConfiguration());
 		{
 			verticesInputConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
@@ -199,7 +203,7 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		@SuppressWarnings("unchecked")
 		CsvInputFormat edgesInFormat = new CsvInputFormat(' ', LongValue.class, LongValue.class);
 		JobInputVertex edgesInput = JobGraphUtils.createInput(edgesInFormat, edgesPath, "EdgesInput", jobGraph,
-			numSubTasks, numSubTasks);
+			numSubTasks);
 		TaskConfig edgesInputConfig = new TaskConfig(edgesInput.getConfiguration());
 		{
 			edgesInputConfig.setOutputSerializer(serializer);
@@ -216,7 +220,7 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 			TypePairComparatorFactory<?, ?> pairComparator) {
 
 		JobTaskVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "Join With Edges (Iteration Head)",
-			jobGraph, numSubTasks, numSubTasks);
+			jobGraph, numSubTasks);
 		TaskConfig headConfig = new TaskConfig(head.getConfiguration());
 		{
 			headConfig.setIterationId(ITERATION_ID);
@@ -234,7 +238,7 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 			headConfig.setInputComparator(comparator, 1);
 			headConfig.setInputLocalStrategy(1, LocalStrategy.NONE);
 			headConfig.setInputCached(1, true);
-			headConfig.setInputMaterializationMemory(1, MEM_PER_CONSUMER * JobGraphUtils.MEGABYTE);
+			headConfig.setRelativeInputMaterializationMemory(1, MEM_FRAC_PER_CONSUMER);
 
 			// initial solution set input
 			headConfig.addInputToGroup(2);
@@ -248,8 +252,8 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 
 			// back channel / iterations
 			headConfig.setIsWorksetIteration();
-			headConfig.setBackChannelMemory(MEM_PER_CONSUMER * JobGraphUtils.MEGABYTE);
-			headConfig.setSolutionSetMemory(MEM_PER_CONSUMER * JobGraphUtils.MEGABYTE);
+			headConfig.setRelativeBackChannelMemory(MEM_FRAC_PER_CONSUMER);
+			headConfig.setRelativeSolutionSetMemory(MEM_FRAC_PER_CONSUMER );
 
 			// output into iteration
 			headConfig.setOutputSerializer(serializer);
@@ -273,7 +277,7 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 			headConfig.setDriverComparator(comparator, 0);
 			headConfig.setDriverComparator(comparator, 1);
 			headConfig.setDriverPairComparator(pairComparator);
-			headConfig.setMemoryDriver(MEM_PER_CONSUMER * JobGraphUtils.MEGABYTE);
+			headConfig.setRelativeMemoryDriver(MEM_FRAC_PER_CONSUMER);
 
 			headConfig.addIterationAggregator(
 				WorksetEmptyConvergenceCriterion.AGGREGATOR_NAME, new LongSumAggregator());
@@ -288,7 +292,7 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 
 		// --------------- the intermediate (reduce to min id) ---------------
 		JobTaskVertex intermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
-			"Find Min Component-ID", jobGraph, numSubTasks, numSubTasks);
+			"Find Min Component-ID", jobGraph, numSubTasks);
 		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
 		{
 			intermediateConfig.setIterationId(ITERATION_ID);
@@ -297,7 +301,7 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 			intermediateConfig.setInputSerializer(serializer, 0);
 			intermediateConfig.setInputComparator(comparator, 0);
 			intermediateConfig.setInputLocalStrategy(0, LocalStrategy.SORT);
-			intermediateConfig.setMemoryInput(0, MEM_PER_CONSUMER * JobGraphUtils.MEGABYTE);
+			intermediateConfig.setRelativeMemoryInput(0, MEM_FRAC_PER_CONSUMER);
 			intermediateConfig.setFilehandlesInput(0, 64);
 			intermediateConfig.setSpillingThresholdInput(0, 0.85f);
 
@@ -316,7 +320,7 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 
 	private static JobOutputVertex createOutput(JobGraph jobGraph, String resultPath, int numSubTasks,
 			TypeSerializerFactory<?> serializer) {
-		JobOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "Final Output", numSubTasks, numSubTasks);
+		JobOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "Final Output", numSubTasks);
 		TaskConfig outputConfig = new TaskConfig(output.getConfiguration());
 		{
 
@@ -341,7 +345,7 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 
 	private static JobOutputVertex createFakeTail(JobGraph jobGraph, int numSubTasks) {
 		JobOutputVertex fakeTailOutput =
-			JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput", numSubTasks, numSubTasks);
+			JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput", numSubTasks);
 		return fakeTailOutput;
 	}
 
@@ -389,7 +393,7 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 
 		// --------------- the tail (solution set join) ---------------
 		JobTaskVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationTail", jobGraph,
-			numSubTasks, numSubTasks);
+			numSubTasks);
 		TaskConfig tailConfig = new TaskConfig(tail.getConfiguration());
 		{
 			tailConfig.setIterationId(ITERATION_ID);
@@ -480,7 +484,7 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 
 		// ------------------ the intermediate (ss join) ----------------------
 		JobTaskVertex ssJoinIntermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
-			"Solution Set Join", jobGraph, numSubTasks, numSubTasks);
+			"Solution Set Join", jobGraph, numSubTasks);
 		TaskConfig ssJoinIntermediateConfig = new TaskConfig(ssJoinIntermediate.getConfiguration());
 		{
 			ssJoinIntermediateConfig.setIterationId(ITERATION_ID);
@@ -509,7 +513,7 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 
 		// -------------------------- ss tail --------------------------------
 		JobTaskVertex ssTail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationSolutionSetTail",
-			jobGraph, numSubTasks, numSubTasks);
+			jobGraph, numSubTasks);
 		TaskConfig ssTailConfig = new TaskConfig(ssTail.getConfiguration());
 		{
 			ssTailConfig.setIterationId(ITERATION_ID);
@@ -520,7 +524,7 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 			ssTailConfig.addInputToGroup(0);
 			ssTailConfig.setInputSerializer(serializer, 0);
 			ssTailConfig.setInputAsynchronouslyMaterialized(0, true);
-			ssTailConfig.setInputMaterializationMemory(0, MEM_PER_CONSUMER * JobGraphUtils.MEGABYTE);
+			ssTailConfig.setRelativeInputMaterializationMemory(0, MEM_FRAC_PER_CONSUMER);
 
 			// output
 			ssTailConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
@@ -534,7 +538,7 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 
 		// -------------------------- ws tail --------------------------------
 		JobTaskVertex wsTail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationWorksetTail",
-			jobGraph, numSubTasks, numSubTasks);
+			jobGraph, numSubTasks);
 		TaskConfig wsTailConfig = new TaskConfig(wsTail.getConfiguration());
 		{
 			wsTailConfig.setIterationId(ITERATION_ID);
@@ -631,7 +635,7 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		// ------------------ the intermediate (ws update) ----------------------
 		JobTaskVertex wsUpdateIntermediate =
 			JobGraphUtils.createTask(IterationIntermediatePactTask.class, "WorksetUpdate", jobGraph,
-				numSubTasks, numSubTasks);
+				numSubTasks);
 		TaskConfig wsUpdateConfig = new TaskConfig(wsUpdateIntermediate.getConfiguration());
 		{
 			wsUpdateConfig.setIterationId(ITERATION_ID);
@@ -661,7 +665,7 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		// -------------------------- ss tail --------------------------------
 		JobTaskVertex ssTail =
 			JobGraphUtils.createTask(IterationTailPactTask.class, "IterationSolutionSetTail", jobGraph,
-				numSubTasks, numSubTasks);
+				numSubTasks);
 		TaskConfig ssTailConfig = new TaskConfig(ssTail.getConfiguration());
 		{
 			ssTailConfig.setIterationId(ITERATION_ID);
@@ -754,7 +758,7 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 
 		// ------------------ the intermediate (ss update) ----------------------
 		JobTaskVertex ssJoinIntermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
-			"Solution Set Update", jobGraph, numSubTasks, numSubTasks);
+			"Solution Set Update", jobGraph, numSubTasks);
 		TaskConfig ssJoinIntermediateConfig = new TaskConfig(ssJoinIntermediate.getConfiguration());
 		{
 			ssJoinIntermediateConfig.setIterationId(ITERATION_ID);
@@ -782,7 +786,7 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 
 		// -------------------------- ws tail --------------------------------
 		JobTaskVertex wsTail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationWorksetTail",
-			jobGraph, numSubTasks, numSubTasks);
+			jobGraph, numSubTasks);
 		TaskConfig wsTailConfig = new TaskConfig(wsTail.getConfiguration());
 		{
 			wsTailConfig.setIterationId(ITERATION_ID);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/DanglingPageRankNepheleITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/DanglingPageRankNepheleITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/DanglingPageRankNepheleITCase.java
index 872cabc..002533a 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/DanglingPageRankNepheleITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/DanglingPageRankNepheleITCase.java
@@ -34,6 +34,10 @@ public class DanglingPageRankNepheleITCase extends RecordAPITestBase {
 	protected String edgesPath;
 	protected String resultPath;
 
+	public DanglingPageRankNepheleITCase(){
+		setTaskManagerNumSlots(DOP);
+	}
+
 	
 	@Override
 	protected void preSubmit() throws Exception {
@@ -45,8 +49,7 @@ public class DanglingPageRankNepheleITCase extends RecordAPITestBase {
 	@Override
 	protected JobGraph getJobGraph() throws Exception {
 		String[] parameters = new String[] {
-			"4",
-			"4",
+			new Integer(DOP).toString(),
 			pagesWithRankPath,
 			edgesPath,
 			resultPath,

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/DanglingPageRankWithCombinerNepheleITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/DanglingPageRankWithCombinerNepheleITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/DanglingPageRankWithCombinerNepheleITCase.java
index 59d5206..e42620e 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/DanglingPageRankWithCombinerNepheleITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/DanglingPageRankWithCombinerNepheleITCase.java
@@ -22,6 +22,10 @@ public class DanglingPageRankWithCombinerNepheleITCase extends RecordAPITestBase
 	protected String pagesWithRankPath;
 	protected String edgesPath;
 	protected String resultPath;
+
+	public DanglingPageRankWithCombinerNepheleITCase(){
+		setTaskManagerNumSlots(DOP);
+	}
 	
 	@Override
 	protected void preSubmit() throws Exception {
@@ -33,8 +37,7 @@ public class DanglingPageRankWithCombinerNepheleITCase extends RecordAPITestBase
 	@Override
 	protected JobGraph getJobGraph() throws Exception {
 		String[] parameters = new String[] {
-			"4",
-			"4",
+			new Integer(DOP).toString(),
 			pagesWithRankPath,
 			edgesPath,
 			resultPath,

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/IterationWithChainingNepheleITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/IterationWithChainingNepheleITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/IterationWithChainingNepheleITCase.java
index ef7c9d2..8a68402 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/IterationWithChainingNepheleITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/IterationWithChainingNepheleITCase.java
@@ -75,6 +75,7 @@ public class IterationWithChainingNepheleITCase extends RecordAPITestBase {
 
 	public IterationWithChainingNepheleITCase(Configuration config) {
 		super(config);
+		setTaskManagerNumSlots(DOP);
 	}
 
 	@Override
@@ -94,7 +95,7 @@ public class IterationWithChainingNepheleITCase extends RecordAPITestBase {
 	@Parameterized.Parameters
 	public static Collection<Object[]> getConfigurations() {
 		Configuration config = new Configuration();
-		config.setInteger("ChainedMapperNepheleITCase#NoSubtasks", 2);
+		config.setInteger("ChainedMapperNepheleITCase#NoSubtasks", DOP);
 		config.setInteger("ChainedMapperNepheleITCase#MaxIterations", 2);
 		return toParameterList(config);
 	}
@@ -118,8 +119,6 @@ public class IterationWithChainingNepheleITCase extends RecordAPITestBase {
 		final TypeComparatorFactory<Record> comparator =
 			new RecordComparatorFactory(new int[] { 0 }, new Class[] { IntValue.class });
 
-		final long MEM_PER_CONSUMER = 2;
-
 		final int ITERATION_ID = 1;
 
 		// --------------------------------------------------------------------------------------------------------------
@@ -128,7 +127,7 @@ public class IterationWithChainingNepheleITCase extends RecordAPITestBase {
 
 		// - input -----------------------------------------------------------------------------------------------------
 		JobInputVertex input = JobGraphUtils.createInput(
-			new PointInFormat(), inputPath, "Input", jobGraph, numSubTasks, numSubTasks);
+			new PointInFormat(), inputPath, "Input", jobGraph, numSubTasks);
 		TaskConfig inputConfig = new TaskConfig(input.getConfiguration());
 		{
 			inputConfig.setOutputSerializer(serializer);
@@ -137,7 +136,7 @@ public class IterationWithChainingNepheleITCase extends RecordAPITestBase {
 
 		// - head ------------------------------------------------------------------------------------------------------
 		JobTaskVertex head = JobGraphUtils.createTask(
-			IterationHeadPactTask.class, "Iteration Head", jobGraph, numSubTasks, numSubTasks);
+			IterationHeadPactTask.class, "Iteration Head", jobGraph, numSubTasks);
 		TaskConfig headConfig = new TaskConfig(head.getConfiguration());
 		{
 			headConfig.setIterationId(ITERATION_ID);
@@ -168,12 +167,12 @@ public class IterationWithChainingNepheleITCase extends RecordAPITestBase {
 			headConfig.setStubWrapper(new UserCodeClassWrapper<DummyMapper>(DummyMapper.class));
 
 			// back channel
-			headConfig.setBackChannelMemory(MEM_PER_CONSUMER * JobGraphUtils.MEGABYTE);
+			headConfig.setRelativeBackChannelMemory(1.0);
 		}
 
 		// - tail ------------------------------------------------------------------------------------------------------
 		JobTaskVertex tail = JobGraphUtils.createTask(
-			IterationTailPactTask.class, "Chained Iteration Tail", jobGraph, numSubTasks, numSubTasks);
+			IterationTailPactTask.class, "Chained Iteration Tail", jobGraph, numSubTasks);
 		TaskConfig tailConfig = new TaskConfig(tail.getConfiguration());
 		{
 			tailConfig.setIterationId(ITERATION_ID);
@@ -210,7 +209,7 @@ public class IterationWithChainingNepheleITCase extends RecordAPITestBase {
 		}
 
 		// - output ----------------------------------------------------------------------------------------------------
-		JobOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "Output", numSubTasks, numSubTasks);
+		JobOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "Output", numSubTasks);
 		TaskConfig outputConfig = new TaskConfig(output.getConfiguration());
 		{
 			outputConfig.addInputToGroup(0);
@@ -221,7 +220,7 @@ public class IterationWithChainingNepheleITCase extends RecordAPITestBase {
 		}
 
 		// - fake tail -------------------------------------------------------------------------------------------------
-		JobOutputVertex fakeTail = JobGraphUtils.createFakeOutput(jobGraph, "Fake Tail", numSubTasks, numSubTasks);
+		JobOutputVertex fakeTail = JobGraphUtils.createFakeOutput(jobGraph, "Fake Tail", numSubTasks);
 
 		// - sync ------------------------------------------------------------------------------------------------------
 		JobOutputVertex sync = JobGraphUtils.createSync(jobGraph, numSubTasks);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/JobGraphUtils.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/JobGraphUtils.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/JobGraphUtils.java
index a229086..109c91a 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/JobGraphUtils.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/JobGraphUtils.java
@@ -51,14 +51,14 @@ public class JobGraphUtils {
 	}
 	
 	public static <T extends FileInputFormat<?>> JobInputVertex createInput(T stub, String path, String name, JobGraph graph,
-			int degreeOfParallelism, int numSubTasksPerInstance)
+			int degreeOfParallelism)
 	{
 		stub.setFilePath(path);
-		return createInput(new UserCodeObjectWrapper<T>(stub), name, graph, degreeOfParallelism, numSubTasksPerInstance);
+		return createInput(new UserCodeObjectWrapper<T>(stub), name, graph, degreeOfParallelism);
 	}
 
 	private static <T extends InputFormat<?,?>> JobInputVertex createInput(UserCodeWrapper<T> stub, String name, JobGraph graph,
-			int degreeOfParallelism, int numSubTasksPerInstance)
+			int degreeOfParallelism)
 	{
 		JobInputVertex inputVertex = new JobInputVertex(name, graph);
 		
@@ -67,8 +67,7 @@ public class JobGraphUtils {
 		inputVertex.setInputClass(clazz);
 		
 		inputVertex.setNumberOfSubtasks(degreeOfParallelism);
-		inputVertex.setNumberOfSubtasksPerInstance(numSubTasksPerInstance);
-		
+
 		TaskConfig inputConfig = new TaskConfig(inputVertex.getConfiguration());
 		inputConfig.setStubWrapper(stub);
 		
@@ -89,12 +88,11 @@ public class JobGraphUtils {
 	}
 
 	public static JobTaskVertex createTask(@SuppressWarnings("rawtypes") Class<? extends RegularPactTask> task, String name, JobGraph graph,
-			int degreeOfParallelism, int numSubtasksPerInstance)
+			int degreeOfParallelism)
 	{
 		JobTaskVertex taskVertex = new JobTaskVertex(name, graph);
 		taskVertex.setTaskClass(task);
 		taskVertex.setNumberOfSubtasks(degreeOfParallelism);
-		taskVertex.setNumberOfSubtasksPerInstance(numSubtasksPerInstance);
 		return taskVertex;
 	}
 
@@ -107,23 +105,19 @@ public class JobGraphUtils {
 		return sync;
 	}
 
-	public static JobOutputVertex createFakeOutput(JobGraph jobGraph, String name, int degreeOfParallelism,
-			int numSubTasksPerInstance)
+	public static JobOutputVertex createFakeOutput(JobGraph jobGraph, String name, int degreeOfParallelism)
 	{
 		JobOutputVertex outputVertex = new JobOutputVertex(name, jobGraph);
 		outputVertex.setOutputClass(FakeOutputTask.class);
 		outputVertex.setNumberOfSubtasks(degreeOfParallelism);
-		outputVertex.setNumberOfSubtasksPerInstance(numSubTasksPerInstance);
 		return outputVertex;
 	}
 
-	public static JobOutputVertex createFileOutput(JobGraph jobGraph, String name, int degreeOfParallelism,
-			int numSubTasksPerInstance)
+	public static JobOutputVertex createFileOutput(JobGraph jobGraph, String name, int degreeOfParallelism)
 	{
 		JobOutputVertex sinkVertex = new JobOutputVertex(name, jobGraph);
 		sinkVertex.setOutputClass(DataSinkTask.class);
 		sinkVertex.setNumberOfSubtasks(degreeOfParallelism);
-		sinkVertex.setNumberOfSubtasksPerInstance(numSubTasksPerInstance);
 		return sinkVertex;
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java
index b112741..5b2433e 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java
@@ -92,7 +92,6 @@ public class CustomCompensatableDanglingPageRank {
 	public static JobGraph getJobGraph(String[] args) throws Exception {
 
 		int degreeOfParallelism = 2;
-		int numSubTasksPerInstance = degreeOfParallelism;
 		String pageWithRankInputPath = ""; //"file://" + PlayConstants.PLAY_DIR + "test-inputs/danglingpagerank/pageWithRank";
 		String adjacencyListInputPath = ""; //"file://" + PlayConstants.PLAY_DIR +
 //			"test-inputs/danglingpagerank/adjacencylists";
@@ -109,31 +108,32 @@ public class CustomCompensatableDanglingPageRank {
 		int failingIteration = 2;
 		double messageLoss = 0.75;
 
-		if (args.length >= 15) {
+		if (args.length >= 14) {
 			degreeOfParallelism = Integer.parseInt(args[0]);
-			numSubTasksPerInstance = Integer.parseInt(args[1]);
-			pageWithRankInputPath = args[2];
-			adjacencyListInputPath = args[3];
-			outputPath = args[4];
-//			confPath = args[5];
-			minorConsumer = Integer.parseInt(args[6]);
-			matchMemory = Integer.parseInt(args[7]);
-			coGroupSortMemory = Integer.parseInt(args[8]);
-			numIterations = Integer.parseInt(args[9]);
-			numVertices = Long.parseLong(args[10]);
-			numDanglingVertices = Long.parseLong(args[11]);
-			failingWorkers = args[12];
-			failingIteration = Integer.parseInt(args[13]);
-			messageLoss = Double.parseDouble(args[14]);
+			pageWithRankInputPath = args[1];
+			adjacencyListInputPath = args[2];
+			outputPath = args[3];
+//			confPath = args[4];
+			minorConsumer = Integer.parseInt(args[5]);
+			matchMemory = Integer.parseInt(args[6]);
+			coGroupSortMemory = Integer.parseInt(args[7]);
+			numIterations = Integer.parseInt(args[8]);
+			numVertices = Long.parseLong(args[9]);
+			numDanglingVertices = Long.parseLong(args[10]);
+			failingWorkers = args[11];
+			failingIteration = Integer.parseInt(args[12]);
+			messageLoss = Double.parseDouble(args[13]);
 		}
 
+		int totalMemoryConsumption = 3*minorConsumer + matchMemory + coGroupSortMemory;
+
 		JobGraph jobGraph = new JobGraph("CompensatableDanglingPageRank");
 		
 		// --------------- the inputs ---------------------
 
 		// page rank input
 		JobInputVertex pageWithRankInput = JobGraphUtils.createInput(new CustomImprovedDanglingPageRankInputFormat(),
-			pageWithRankInputPath, "DanglingPageWithRankInput", jobGraph, degreeOfParallelism, numSubTasksPerInstance);
+			pageWithRankInputPath, "DanglingPageWithRankInput", jobGraph, degreeOfParallelism);
 		TaskConfig pageWithRankInputConfig = new TaskConfig(pageWithRankInput.getConfiguration());
 		pageWithRankInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
 		pageWithRankInputConfig.setOutputComparator(vertexWithRankAndDanglingComparator, 0);
@@ -142,7 +142,7 @@ public class CustomCompensatableDanglingPageRank {
 
 		// edges as adjacency list
 		JobInputVertex adjacencyListInput = JobGraphUtils.createInput(new CustomImprovedAdjacencyListInputFormat(),
-			adjacencyListInputPath, "AdjancencyListInput", jobGraph, degreeOfParallelism, numSubTasksPerInstance);
+			adjacencyListInputPath, "AdjancencyListInput", jobGraph, degreeOfParallelism);
 		TaskConfig adjacencyListInputConfig = new TaskConfig(adjacencyListInput.getConfiguration());
 		adjacencyListInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
 		adjacencyListInputConfig.setOutputSerializer(vertexWithAdjacencyListSerializer);
@@ -150,7 +150,7 @@ public class CustomCompensatableDanglingPageRank {
 
 		// --------------- the head ---------------------
 		JobTaskVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "IterationHead", jobGraph,
-			degreeOfParallelism, numSubTasksPerInstance);
+			degreeOfParallelism);
 		TaskConfig headConfig = new TaskConfig(head.getConfiguration());
 		headConfig.setIterationId(ITERATION_ID);
 		
@@ -160,12 +160,12 @@ public class CustomCompensatableDanglingPageRank {
 		headConfig.setInputSerializer(vertexWithRankAndDanglingSerializer, 0);
 		headConfig.setInputComparator(vertexWithRankAndDanglingComparator, 0);
 		headConfig.setInputLocalStrategy(0, LocalStrategy.SORT);
-		headConfig.setMemoryInput(0, minorConsumer * JobGraphUtils.MEGABYTE);
+		headConfig.setRelativeMemoryInput(0, (double) minorConsumer / totalMemoryConsumption);
 		headConfig.setFilehandlesInput(0, NUM_FILE_HANDLES_PER_SORT);
 		headConfig.setSpillingThresholdInput(0, SORT_SPILL_THRESHOLD);
 		
 		// back channel / iterations
-		headConfig.setBackChannelMemory(minorConsumer * JobGraphUtils.MEGABYTE);
+		headConfig.setRelativeBackChannelMemory((double) minorConsumer / totalMemoryConsumption);
 		
 		// output into iteration
 		headConfig.setOutputSerializer(vertexWithRankAndDanglingSerializer);
@@ -195,13 +195,13 @@ public class CustomCompensatableDanglingPageRank {
 		// --------------- the join ---------------------
 		
 		JobTaskVertex intermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
-			"IterationIntermediate", jobGraph, degreeOfParallelism, numSubTasksPerInstance);
+			"IterationIntermediate", jobGraph, degreeOfParallelism);
 		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
 		intermediateConfig.setIterationId(ITERATION_ID);
 //		intermediateConfig.setDriver(RepeatableHashjoinMatchDriverWithCachedBuildside.class);
 		intermediateConfig.setDriver(BuildSecondCachedMatchDriver.class);
 		intermediateConfig.setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND);
-		intermediateConfig.setMemoryDriver(matchMemory * JobGraphUtils.MEGABYTE);
+		intermediateConfig.setRelativeMemoryDriver((double) matchMemory / totalMemoryConsumption);
 		intermediateConfig.addInputToGroup(0);
 		intermediateConfig.addInputToGroup(1);
 		intermediateConfig.setInputSerializer(vertexWithRankAndDanglingSerializer, 0);
@@ -223,7 +223,7 @@ public class CustomCompensatableDanglingPageRank {
 		// ---------------- the tail (co group) --------------------
 		
 		JobTaskVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationTail", jobGraph,
-			degreeOfParallelism, numSubTasksPerInstance);
+			degreeOfParallelism);
 		TaskConfig tailConfig = new TaskConfig(tail.getConfiguration());
 		tailConfig.setIterationId(ITERATION_ID);
         tailConfig.setIsWorksetUpdate();
@@ -240,10 +240,10 @@ public class CustomCompensatableDanglingPageRank {
 		tailConfig.setDriverComparator(vertexWithRankComparator, 1);
 		tailConfig.setDriverPairComparator(coGroupComparator);
 		tailConfig.setInputAsynchronouslyMaterialized(0, true);
-		tailConfig.setInputMaterializationMemory(0, minorConsumer * JobGraphUtils.MEGABYTE);
+		tailConfig.setRelativeInputMaterializationMemory(0, (double)minorConsumer/totalMemoryConsumption);
 		tailConfig.setInputLocalStrategy(1, LocalStrategy.SORT);
 		tailConfig.setInputComparator(vertexWithRankComparator, 1);
-		tailConfig.setMemoryInput(1, coGroupSortMemory * JobGraphUtils.MEGABYTE);
+		tailConfig.setRelativeMemoryInput(1, (double) coGroupSortMemory / totalMemoryConsumption);
 		tailConfig.setFilehandlesInput(1, NUM_FILE_HANDLES_PER_SORT);
 		tailConfig.setSpillingThresholdInput(1, SORT_SPILL_THRESHOLD);
 		
@@ -261,8 +261,7 @@ public class CustomCompensatableDanglingPageRank {
 		
 		// --------------- the output ---------------------
 
-		JobOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", degreeOfParallelism,
-			numSubTasksPerInstance);
+		JobOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", degreeOfParallelism);
 		TaskConfig outputConfig = new TaskConfig(output.getConfiguration());
 		outputConfig.addInputToGroup(0);
 		outputConfig.setInputSerializer(vertexWithRankAndDanglingSerializer, 0);
@@ -272,7 +271,7 @@ public class CustomCompensatableDanglingPageRank {
 		// --------------- the auxiliaries ---------------------
 		
 		JobOutputVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput",
-			degreeOfParallelism, numSubTasksPerInstance);
+			degreeOfParallelism);
 
 		JobOutputVertex sync = JobGraphUtils.createSync(jobGraph, degreeOfParallelism);
 		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java
index 37fab39..698296a 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java
@@ -93,7 +93,6 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 	public static JobGraph getJobGraph(String[] args) throws Exception {
 
 		int degreeOfParallelism = 2;
-		int numSubTasksPerInstance = degreeOfParallelism;
 		String pageWithRankInputPath = ""; //"file://" + PlayConstants.PLAY_DIR + "test-inputs/danglingpagerank/pageWithRank";
 		String adjacencyListInputPath = ""; //"file://" + PlayConstants.PLAY_DIR +
 //			"test-inputs/danglingpagerank/adjacencylists";
@@ -109,31 +108,32 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 		int failingIteration = 2;
 		double messageLoss = 0.75;
 
-		if (args.length >= 15) {
+		if (args.length >= 14) {
 			degreeOfParallelism = Integer.parseInt(args[0]);
-			numSubTasksPerInstance = Integer.parseInt(args[1]);
-			pageWithRankInputPath = args[2];
-			adjacencyListInputPath = args[3];
-			outputPath = args[4];
-			// [5] is config path
-			minorConsumer = Integer.parseInt(args[6]);
-			matchMemory = Integer.parseInt(args[7]);
-			coGroupSortMemory = Integer.parseInt(args[8]);
-			numIterations = Integer.parseInt(args[9]);
-			numVertices = Long.parseLong(args[10]);
-			numDanglingVertices = Long.parseLong(args[11]);
-			failingWorkers = args[12];
-			failingIteration = Integer.parseInt(args[13]);
-			messageLoss = Double.parseDouble(args[14]);
+			pageWithRankInputPath = args[1];
+			adjacencyListInputPath = args[2];
+			outputPath = args[3];
+			// [4] is config path
+			minorConsumer = Integer.parseInt(args[5]);
+			matchMemory = Integer.parseInt(args[6]);
+			coGroupSortMemory = Integer.parseInt(args[7]);
+			numIterations = Integer.parseInt(args[8]);
+			numVertices = Long.parseLong(args[9]);
+			numDanglingVertices = Long.parseLong(args[10]);
+			failingWorkers = args[11];
+			failingIteration = Integer.parseInt(args[12]);
+			messageLoss = Double.parseDouble(args[13]);
 		}
 
+		int totalMemoryConsumption = 3*minorConsumer + 2*coGroupSortMemory + matchMemory;
+
 		JobGraph jobGraph = new JobGraph("CompensatableDanglingPageRank");
 		
 		// --------------- the inputs ---------------------
 
 		// page rank input
 		JobInputVertex pageWithRankInput = JobGraphUtils.createInput(new CustomImprovedDanglingPageRankInputFormat(),
-			pageWithRankInputPath, "DanglingPageWithRankInput", jobGraph, degreeOfParallelism, numSubTasksPerInstance);
+			pageWithRankInputPath, "DanglingPageWithRankInput", jobGraph, degreeOfParallelism);
 		TaskConfig pageWithRankInputConfig = new TaskConfig(pageWithRankInput.getConfiguration());
 		pageWithRankInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
 		pageWithRankInputConfig.setOutputComparator(vertexWithRankAndDanglingComparator, 0);
@@ -142,7 +142,7 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 
 		// edges as adjacency list
 		JobInputVertex adjacencyListInput = JobGraphUtils.createInput(new CustomImprovedAdjacencyListInputFormat(),
-			adjacencyListInputPath, "AdjancencyListInput", jobGraph, degreeOfParallelism, numSubTasksPerInstance);
+			adjacencyListInputPath, "AdjancencyListInput", jobGraph, degreeOfParallelism);
 		TaskConfig adjacencyListInputConfig = new TaskConfig(adjacencyListInput.getConfiguration());
 		adjacencyListInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
 		adjacencyListInputConfig.setOutputSerializer(vertexWithAdjacencyListSerializer);
@@ -150,7 +150,7 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 
 		// --------------- the head ---------------------
 		JobTaskVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "IterationHead", jobGraph,
-			degreeOfParallelism, numSubTasksPerInstance);
+			degreeOfParallelism);
 		TaskConfig headConfig = new TaskConfig(head.getConfiguration());
 		headConfig.setIterationId(ITERATION_ID);
 		
@@ -160,12 +160,12 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 		headConfig.setInputSerializer(vertexWithRankAndDanglingSerializer, 0);
 		headConfig.setInputComparator(vertexWithRankAndDanglingComparator, 0);
 		headConfig.setInputLocalStrategy(0, LocalStrategy.SORT);
-		headConfig.setMemoryInput(0, minorConsumer * JobGraphUtils.MEGABYTE);
+		headConfig.setRelativeMemoryInput(0, (double)minorConsumer/totalMemoryConsumption);
 		headConfig.setFilehandlesInput(0, NUM_FILE_HANDLES_PER_SORT);
 		headConfig.setSpillingThresholdInput(0, SORT_SPILL_THRESHOLD);
 		
 		// back channel / iterations
-		headConfig.setBackChannelMemory(minorConsumer * JobGraphUtils.MEGABYTE);
+		headConfig.setRelativeBackChannelMemory((double)minorConsumer/totalMemoryConsumption);
 		
 		// output into iteration
 		headConfig.setOutputSerializer(vertexWithRankAndDanglingSerializer);
@@ -195,13 +195,13 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 		// --------------- the join ---------------------
 		
 		JobTaskVertex intermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
-			"IterationIntermediate", jobGraph, degreeOfParallelism, numSubTasksPerInstance);
+			"IterationIntermediate", jobGraph, degreeOfParallelism);
 		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
 		intermediateConfig.setIterationId(ITERATION_ID);
 //		intermediateConfig.setDriver(RepeatableHashjoinMatchDriverWithCachedBuildside.class);
 		intermediateConfig.setDriver(BuildSecondCachedMatchDriver.class);
 		intermediateConfig.setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND);
-		intermediateConfig.setMemoryDriver(matchMemory * JobGraphUtils.MEGABYTE);
+		intermediateConfig.setRelativeMemoryDriver((double)matchMemory/totalMemoryConsumption);
 		intermediateConfig.addInputToGroup(0);
 		intermediateConfig.addInputToGroup(1);
 		intermediateConfig.setInputSerializer(vertexWithRankAndDanglingSerializer, 0);
@@ -225,7 +225,7 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 		combinerConfig.setInputSerializer(vertexWithRankSerializer, 0);
 		combinerConfig.setDriverStrategy(DriverStrategy.SORTED_GROUP_COMBINE);
 		combinerConfig.setDriverComparator(vertexWithRankComparator, 0);
-		combinerConfig.setMemoryDriver(coGroupSortMemory * JobGraphUtils.MEGABYTE);
+		combinerConfig.setRelativeMemoryDriver((double)coGroupSortMemory/totalMemoryConsumption);
 		combinerConfig.setOutputSerializer(vertexWithRankSerializer);
 		combinerConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
 		combinerConfig.setOutputComparator(vertexWithRankComparator, 0);
@@ -235,7 +235,7 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 		// ---------------- the tail (co group) --------------------
 		
 		JobTaskVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationTail", jobGraph,
-			degreeOfParallelism, numSubTasksPerInstance);
+			degreeOfParallelism);
 		TaskConfig tailConfig = new TaskConfig(tail.getConfiguration());
 		tailConfig.setIterationId(ITERATION_ID);
         tailConfig.setIsWorksetUpdate();
@@ -251,10 +251,10 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 		tailConfig.setDriverComparator(vertexWithRankComparator, 1);
 		tailConfig.setDriverPairComparator(coGroupComparator);
 		tailConfig.setInputAsynchronouslyMaterialized(0, true);
-		tailConfig.setInputMaterializationMemory(0, minorConsumer * JobGraphUtils.MEGABYTE);
+		tailConfig.setRelativeInputMaterializationMemory(0, (double)minorConsumer/totalMemoryConsumption);
 		tailConfig.setInputLocalStrategy(1, LocalStrategy.SORT);
 		tailConfig.setInputComparator(vertexWithRankComparator, 1);
-		tailConfig.setMemoryInput(1, coGroupSortMemory * JobGraphUtils.MEGABYTE);
+		tailConfig.setRelativeMemoryInput(1, (double)coGroupSortMemory/totalMemoryConsumption);
 		tailConfig.setFilehandlesInput(1, NUM_FILE_HANDLES_PER_SORT);
 		tailConfig.setSpillingThresholdInput(1, SORT_SPILL_THRESHOLD);
 		tailConfig.addIterationAggregator(CustomCompensatableDotProductCoGroup.AGGREGATOR_NAME, new PageRankStatsAggregator());
@@ -273,8 +273,7 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 		
 		// --------------- the output ---------------------
 
-		JobOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", degreeOfParallelism,
-			numSubTasksPerInstance);
+		JobOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", degreeOfParallelism);
 		TaskConfig outputConfig = new TaskConfig(output.getConfiguration());
 		outputConfig.addInputToGroup(0);
 		outputConfig.setInputSerializer(vertexWithRankAndDanglingSerializer, 0);
@@ -284,7 +283,7 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 		// --------------- the auxiliaries ---------------------
 		
 		JobOutputVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput",
-			degreeOfParallelism, numSubTasksPerInstance);
+			degreeOfParallelism);
 
 		JobOutputVertex sync = JobGraphUtils.createSync(jobGraph, degreeOfParallelism);
 		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java
index 944f13b..f870fe6 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java
@@ -72,7 +72,6 @@ public class CompensatableDanglingPageRank {
 	public static JobGraph getJobGraph(String[] args) throws Exception {
 
 		int degreeOfParallelism = 2;
-		int numSubTasksPerInstance = degreeOfParallelism;
 		String pageWithRankInputPath = ""; // "file://" + PlayConstants.PLAY_DIR + "test-inputs/danglingpagerank/pageWithRank";
 		String adjacencyListInputPath = ""; // "file://" + PlayConstants.PLAY_DIR +
 //			"test-inputs/danglingpagerank/adjacencylists";
@@ -91,29 +90,30 @@ public class CompensatableDanglingPageRank {
 
 		if (args.length >= 15) {
 			degreeOfParallelism = Integer.parseInt(args[0]);
-			numSubTasksPerInstance = Integer.parseInt(args[1]);
-			pageWithRankInputPath = args[2];
-			adjacencyListInputPath = args[3];
-			outputPath = args[4];
-//			confPath = args[5];
-			minorConsumer = Integer.parseInt(args[6]);
-			matchMemory = Integer.parseInt(args[7]);
-			coGroupSortMemory = Integer.parseInt(args[8]);
-			numIterations = Integer.parseInt(args[9]);
-			numVertices = Long.parseLong(args[10]);
-			numDanglingVertices = Long.parseLong(args[11]);
-			failingWorkers = args[12];
-			failingIteration = Integer.parseInt(args[13]);
-			messageLoss = Double.parseDouble(args[14]);
+			pageWithRankInputPath = args[1];
+			adjacencyListInputPath = args[2];
+			outputPath = args[3];
+//			confPath = args[4];
+			minorConsumer = Integer.parseInt(args[5]);
+			matchMemory = Integer.parseInt(args[6]);
+			coGroupSortMemory = Integer.parseInt(args[7]);
+			numIterations = Integer.parseInt(args[8]);
+			numVertices = Long.parseLong(args[9]);
+			numDanglingVertices = Long.parseLong(args[10]);
+			failingWorkers = args[11];
+			failingIteration = Integer.parseInt(args[12]);
+			messageLoss = Double.parseDouble(args[13]);
 		}
 
+		int totalMemoryConsumption = 3*minorConsumer + matchMemory + coGroupSortMemory;
+
 		JobGraph jobGraph = new JobGraph("CompensatableDanglingPageRank");
 		
 		// --------------- the inputs ---------------------
 
 		// page rank input
 		JobInputVertex pageWithRankInput = JobGraphUtils.createInput(new ImprovedDanglingPageRankInputFormat(),
-			pageWithRankInputPath, "DanglingPageWithRankInput", jobGraph, degreeOfParallelism, numSubTasksPerInstance);
+			pageWithRankInputPath, "DanglingPageWithRankInput", jobGraph, degreeOfParallelism);
 		TaskConfig pageWithRankInputConfig = new TaskConfig(pageWithRankInput.getConfiguration());
 		pageWithRankInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
 		pageWithRankInputConfig.setOutputComparator(fieldZeroComparator, 0);
@@ -122,7 +122,7 @@ public class CompensatableDanglingPageRank {
 
 		// edges as adjacency list
 		JobInputVertex adjacencyListInput = JobGraphUtils.createInput(new ImprovedAdjacencyListInputFormat(),
-			adjacencyListInputPath, "AdjancencyListInput", jobGraph, degreeOfParallelism, numSubTasksPerInstance);
+			adjacencyListInputPath, "AdjancencyListInput", jobGraph, degreeOfParallelism);
 		TaskConfig adjacencyListInputConfig = new TaskConfig(adjacencyListInput.getConfiguration());
 		adjacencyListInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
 		adjacencyListInputConfig.setOutputSerializer(recSerializer);
@@ -130,7 +130,7 @@ public class CompensatableDanglingPageRank {
 
 		// --------------- the head ---------------------
 		JobTaskVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "IterationHead", jobGraph,
-			degreeOfParallelism, numSubTasksPerInstance);
+			degreeOfParallelism);
 		TaskConfig headConfig = new TaskConfig(head.getConfiguration());
 		headConfig.setIterationId(ITERATION_ID);
 		
@@ -140,12 +140,12 @@ public class CompensatableDanglingPageRank {
 		headConfig.setInputSerializer(recSerializer, 0);
 		headConfig.setInputComparator(fieldZeroComparator, 0);
 		headConfig.setInputLocalStrategy(0, LocalStrategy.SORT);
-		headConfig.setMemoryInput(0, minorConsumer * JobGraphUtils.MEGABYTE);
+		headConfig.setRelativeMemoryInput(0, (double)minorConsumer/totalMemoryConsumption);
 		headConfig.setFilehandlesInput(0, NUM_FILE_HANDLES_PER_SORT);
 		headConfig.setSpillingThresholdInput(0, SORT_SPILL_THRESHOLD);
 		
 		// back channel / iterations
-		headConfig.setBackChannelMemory(minorConsumer * JobGraphUtils.MEGABYTE);
+		headConfig.setRelativeBackChannelMemory((double)minorConsumer/totalMemoryConsumption);
 		
 		// output into iteration
 		headConfig.setOutputSerializer(recSerializer);
@@ -175,13 +175,13 @@ public class CompensatableDanglingPageRank {
 		// --------------- the join ---------------------
 		
 		JobTaskVertex intermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
-			"IterationIntermediate", jobGraph, degreeOfParallelism, numSubTasksPerInstance);
+			"IterationIntermediate", jobGraph, degreeOfParallelism);
 		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
 		intermediateConfig.setIterationId(ITERATION_ID);
 //		intermediateConfig.setDriver(RepeatableHashjoinMatchDriverWithCachedBuildside.class);
 		intermediateConfig.setDriver(BuildSecondCachedMatchDriver.class);
 		intermediateConfig.setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND);
-		intermediateConfig.setMemoryDriver(matchMemory * JobGraphUtils.MEGABYTE);
+		intermediateConfig.setRelativeMemoryDriver((double)matchMemory/totalMemoryConsumption);
 		intermediateConfig.addInputToGroup(0);
 		intermediateConfig.addInputToGroup(1);
 		intermediateConfig.setInputSerializer(recSerializer, 0);
@@ -203,7 +203,7 @@ public class CompensatableDanglingPageRank {
 		// ---------------- the tail (co group) --------------------
 		
 		JobTaskVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationTail", jobGraph,
-			degreeOfParallelism, numSubTasksPerInstance);
+			degreeOfParallelism);
 		TaskConfig tailConfig = new TaskConfig(tail.getConfiguration());
 		tailConfig.setIterationId(ITERATION_ID);
         tailConfig.setIsWorksetUpdate();
@@ -220,10 +220,10 @@ public class CompensatableDanglingPageRank {
 		tailConfig.setDriverComparator(fieldZeroComparator, 1);
 		tailConfig.setDriverPairComparator(pairComparatorFactory);
 		tailConfig.setInputAsynchronouslyMaterialized(0, true);
-		tailConfig.setInputMaterializationMemory(0, minorConsumer * JobGraphUtils.MEGABYTE);
+		tailConfig.setRelativeInputMaterializationMemory(0, (double)minorConsumer/totalMemoryConsumption);
 		tailConfig.setInputLocalStrategy(1, LocalStrategy.SORT);
 		tailConfig.setInputComparator(fieldZeroComparator, 1);
-		tailConfig.setMemoryInput(1, coGroupSortMemory * JobGraphUtils.MEGABYTE);
+		tailConfig.setRelativeMemoryInput(1, (double)coGroupSortMemory/totalMemoryConsumption);
 		tailConfig.setFilehandlesInput(1, NUM_FILE_HANDLES_PER_SORT);
 		tailConfig.setSpillingThresholdInput(1, SORT_SPILL_THRESHOLD);
 		
@@ -241,8 +241,7 @@ public class CompensatableDanglingPageRank {
 		
 		// --------------- the output ---------------------
 
-		JobOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", degreeOfParallelism,
-			numSubTasksPerInstance);
+		JobOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", degreeOfParallelism);
 		TaskConfig outputConfig = new TaskConfig(output.getConfiguration());
 		outputConfig.addInputToGroup(0);
 		outputConfig.setInputSerializer(recSerializer, 0);
@@ -252,7 +251,7 @@ public class CompensatableDanglingPageRank {
 		// --------------- the auxiliaries ---------------------
 		
 		JobOutputVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput",
-			degreeOfParallelism, numSubTasksPerInstance);
+			degreeOfParallelism);
 
 		JobOutputVertex sync = JobGraphUtils.createSync(jobGraph, degreeOfParallelism);
 		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/localDistributed/PackagedProgramEndToEndITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/localDistributed/PackagedProgramEndToEndITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/localDistributed/PackagedProgramEndToEndITCase.java
index 3626ba7..0e297ec 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/localDistributed/PackagedProgramEndToEndITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/localDistributed/PackagedProgramEndToEndITCase.java
@@ -14,6 +14,7 @@ package eu.stratosphere.test.localDistributed;
 
 import java.io.File;
 import java.io.FileWriter;
+import java.net.URL;
 
 import eu.stratosphere.client.minicluster.NepheleMiniCluster;
 import org.junit.Assert;
@@ -28,6 +29,8 @@ import eu.stratosphere.util.LogUtils;
 
 public class PackagedProgramEndToEndITCase {
 
+	private static final int DOP = 4;
+
 	static {
 		LogUtils.initializeDefaultTestConsoleLogger();
 	}
@@ -53,16 +56,18 @@ public class PackagedProgramEndToEndITCase {
 			fwClusters.write(KMeansData.INITIAL_CENTERS);
 			fwClusters.close();
 
-			String jarPath = "target/maven-test-jar.jar";
+			URL jarFileURL = getClass().getResource("/KMeansForTest.jar");
+			String jarPath = jarFileURL.getFile();
 
 			// run KMeans
-			cluster.setNumTaskManager(2);
+			cluster.setNumTaskTracker(2);
+			cluster.setTaskManagerNumSlots(2);
 			cluster.start();
 			RemoteExecutor ex = new RemoteExecutor("localhost", 6498);
-			
+
 			ex.executeJar(jarPath,
-					"eu.stratosphere.test.util.testjar.KMeansForTest",
-					new String[] {
+					"eu.stratosphere.examples.scala.testing.KMeansForTest",
+					new String[] {new Integer(DOP).toString(),
 							points.toURI().toString(),
 							clusters.toURI().toString(),
 							outFile.toURI().toString(),

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/operators/UnionSinkITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/operators/UnionSinkITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/operators/UnionSinkITCase.java
index b9f0f2c..2637acb 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/operators/UnionSinkITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/operators/UnionSinkITCase.java
@@ -44,6 +44,7 @@ public class UnionSinkITCase extends RecordAPITestBase {
 	
 	public UnionSinkITCase(Configuration testConfig) {
 		super(testConfig);
+		setTaskManagerNumSlots(DOP);
 	}
 
 	private static final String MAP_IN = "1 1\n2 2\n2 8\n4 4\n4 4\n6 6\n7 7\n8 8\n" +
@@ -115,7 +116,7 @@ public class UnionSinkITCase extends RecordAPITestBase {
 		output.addInput(testMapper2);
 		
 		Plan plan = new Plan(output);
-		plan.setDefaultParallelism(4);
+		plan.setDefaultParallelism(DOP);
 
 		PactCompiler pc = new PactCompiler(new DataStatistics());
 		OptimizedPlan op = pc.compile(plan);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/CollectionSourceTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/CollectionSourceTest.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/CollectionSourceTest.java
index 48f703a..1f30075 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/CollectionSourceTest.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/CollectionSourceTest.java
@@ -36,8 +36,14 @@ import eu.stratosphere.util.Collector;
  */
 public class CollectionSourceTest extends RecordAPITestBase {
 
+	private static final int DOP = 4;
+
 	protected String resultPath;
 
+	public CollectionSourceTest(){
+		setTaskManagerNumSlots(DOP);
+	}
+
 	public static class Join extends JoinFunction {
 
 		private static final long serialVersionUID = 1L;
@@ -110,7 +116,7 @@ public class CollectionSourceTest extends RecordAPITestBase {
 
 	@Override
 	protected Plan getTestJob() {
-		return getPlan(4, resultPath);
+		return getPlan(DOP, resultPath);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/ComputeEdgeDegreesITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/ComputeEdgeDegreesITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/ComputeEdgeDegreesITCase.java
index f52e108..7a86112 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/ComputeEdgeDegreesITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/ComputeEdgeDegreesITCase.java
@@ -35,6 +35,7 @@ public class ComputeEdgeDegreesITCase extends RecordAPITestBase {
 	
 	public ComputeEdgeDegreesITCase(Configuration config) {
 		super(config);
+		setTaskManagerNumSlots(DOP);
 	}
 
 	@Override
@@ -58,7 +59,7 @@ public class ComputeEdgeDegreesITCase extends RecordAPITestBase {
 	@Parameters
 	public static Collection<Object[]> getConfigurations() {
 		Configuration config = new Configuration();
-		config.setInteger("ComputeEdgeDegreesTest#NumSubtasks", 4);
+		config.setInteger("ComputeEdgeDegreesTest#NumSubtasks", DOP);
 		return toParameterList(config);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/EnumTrianglesOnEdgesWithDegreesITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/EnumTrianglesOnEdgesWithDegreesITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/EnumTrianglesOnEdgesWithDegreesITCase.java
index b4b0386..68d66a6 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/EnumTrianglesOnEdgesWithDegreesITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/EnumTrianglesOnEdgesWithDegreesITCase.java
@@ -35,6 +35,7 @@ public class EnumTrianglesOnEdgesWithDegreesITCase extends RecordAPITestBase {
 	
 	public EnumTrianglesOnEdgesWithDegreesITCase(Configuration config) {
 		super(config);
+		setTaskManagerNumSlots(DOP);
 	}
 
 	@Override
@@ -60,7 +61,7 @@ public class EnumTrianglesOnEdgesWithDegreesITCase extends RecordAPITestBase {
 	@Parameters
 	public static Collection<Object[]> getConfigurations() {
 		Configuration config = new Configuration();
-		config.setInteger("EnumTrianglesTest#NumSubtasks", 4);
+		config.setInteger("EnumTrianglesTest#NumSubtasks", DOP);
 		return toParameterList(config);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/EnumTrianglesRDFITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/EnumTrianglesRDFITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/EnumTrianglesRDFITCase.java
index 945cc67..93e9bfe 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/EnumTrianglesRDFITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/EnumTrianglesRDFITCase.java
@@ -54,7 +54,7 @@ public class EnumTrianglesRDFITCase extends RecordAPITestBase {
 	protected Plan getTestJob() {
 		EnumTrianglesRdfFoaf enumTriangles = new EnumTrianglesRdfFoaf();
 		return enumTriangles.getPlan(
-				config.getString("EnumTrianglesTest#NoSubtasks", "4"), edgesPath, resultPath);
+				config.getString("EnumTrianglesTest#NoSubtasks", new Integer(DOP).toString()), edgesPath, resultPath);
 	}
 
 	@Override
@@ -65,7 +65,7 @@ public class EnumTrianglesRDFITCase extends RecordAPITestBase {
 	@Parameters
 	public static Collection<Object[]> getConfigurations() {
 		Configuration config = new Configuration();
-		config.setInteger("EnumTrianglesTest#NoSubtasks", 4);
+		config.setInteger("EnumTrianglesTest#NoSubtasks", DOP);
 		return toParameterList(config);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/GlobalSortingITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/GlobalSortingITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/GlobalSortingITCase.java
index 7aa9a78..f628ca5 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/GlobalSortingITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/GlobalSortingITCase.java
@@ -38,6 +38,9 @@ public class GlobalSortingITCase extends RecordAPITestBase {
 
 	private String sortedRecords;
 
+	public GlobalSortingITCase(){
+		setTaskManagerNumSlots(DOP);
+	}
 
 	@Override
 	protected void preSubmit() throws Exception {
@@ -77,7 +80,7 @@ public class GlobalSortingITCase extends RecordAPITestBase {
 	@Override
 	protected Plan getTestJob() {
 		GlobalSort globalSort = new GlobalSort();
-		return globalSort.getPlan("4", recordsPath, resultPath);
+		return globalSort.getPlan(new Integer(DOP).toString(), recordsPath, resultPath);
 	}
 
 	@Override


[42/53] [abbrv] git commit: [FLINK-960] Fix CollectionDataSource bug

Posted by rm...@apache.org.
[FLINK-960] Fix CollectionDataSource bug

This closes #33.


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

Branch: refs/heads/travis_test
Commit: 515ad3c3362af54efc4a36fe90d9b353c1da85c8
Parents: 3d6cc5f
Author: Till Rohrmann <ti...@gmail.com>
Authored: Fri Jun 20 18:17:42 2014 +0200
Committer: uce <u....@fu-berlin.de>
Committed: Wed Jun 25 12:24:57 2014 +0200

----------------------------------------------------------------------
 .../record/operators/CollectionDataSource.java  | 13 +++---
 stratosphere-scala/pom.xml                      |  6 +++
 .../api/scala/CollectionDataSourceTest.scala    | 46 ++++++++++++++++++++
 3 files changed, 59 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/515ad3c3/stratosphere-java/src/main/java/eu/stratosphere/api/java/record/operators/CollectionDataSource.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/record/operators/CollectionDataSource.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/record/operators/CollectionDataSource.java
index adadea2..f4f85d6 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/record/operators/CollectionDataSource.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/record/operators/CollectionDataSource.java
@@ -136,13 +136,14 @@ public class CollectionDataSource extends GenericDataSourceBase<Record, GenericI
 			checkFormat((Collection<Object>) data[0]);
 			f.setData((Collection<Object>) data[0]);
 		}
-
-		Collection<Object> tmp = new ArrayList<Object>();
-		for (Object o : data) {
-			tmp.add(o);
+		else {
+			Collection<Object> tmp = new ArrayList<Object>();
+			for (Object o : data) {
+				tmp.add(o);
+			}
+			checkFormat(tmp);
+			f.setData(tmp);
 		}
-		checkFormat(tmp);
-		f.setData(tmp);
 	}
 
 	// --------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/515ad3c3/stratosphere-scala/pom.xml
----------------------------------------------------------------------
diff --git a/stratosphere-scala/pom.xml b/stratosphere-scala/pom.xml
index 817a63c..86c1925 100644
--- a/stratosphere-scala/pom.xml
+++ b/stratosphere-scala/pom.xml
@@ -56,6 +56,12 @@
 			<artifactId>asm</artifactId>
 			<version>4.0</version>
 		</dependency>
+
+		<dependency>
+			<groupId>org.scalatest</groupId>
+			<artifactId>scalatest_2.10</artifactId>
+			<version>2.2.0</version>
+		</dependency>
 	</dependencies>
 
 	<build>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/515ad3c3/stratosphere-scala/src/test/scala/eu/stratosphere/api/scala/CollectionDataSourceTest.scala
----------------------------------------------------------------------
diff --git a/stratosphere-scala/src/test/scala/eu/stratosphere/api/scala/CollectionDataSourceTest.scala b/stratosphere-scala/src/test/scala/eu/stratosphere/api/scala/CollectionDataSourceTest.scala
new file mode 100644
index 0000000..3c36ed5
--- /dev/null
+++ b/stratosphere-scala/src/test/scala/eu/stratosphere/api/scala/CollectionDataSourceTest.scala
@@ -0,0 +1,46 @@
+/*
+ * 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.api.scala
+
+import eu.stratosphere.api.java.record.operators.{CollectionDataSource => JCollectionDataSource}
+import eu.stratosphere.types.{DoubleValue, Record}
+import org.scalatest.junit.AssertionsForJUnit
+import org.junit.Assert._
+import org.junit.Test
+
+class CollectionDataSourceTest extends AssertionsForJUnit {
+  @Test def testScalaCollectionInput() {
+    val expected = List(1.0, 2.0, 3.0)
+    val datasource = CollectionDataSource(expected)
+
+    val javaCDS = datasource.contract.asInstanceOf[JCollectionDataSource]
+
+    val inputFormat = javaCDS.getFormatWrapper.getUserCodeObject()
+    val splits = inputFormat.createInputSplits(1)
+    inputFormat.open(splits(0))
+
+    val record = new Record()
+    var result = List[Double]()
+
+    while(!inputFormat.reachedEnd()){
+      inputFormat.nextRecord(record)
+      assertTrue(record.getNumFields == 1)
+      val value = record.getField[DoubleValue](0, classOf[DoubleValue])
+      result = value.getValue :: result
+    }
+
+    assertEquals(expected, result.reverse)
+  }
+
+}


[47/53] [abbrv] git commit: Speed up "getOptimizedPlan" by not starting the local embedded runtime.

Posted by rm...@apache.org.
Speed up "getOptimizedPlan" by not starting the local embedded runtime.


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

Branch: refs/heads/travis_test
Commit: 843139562ad14c4a5980a3477b67bc1b53676b18
Parents: e2aabd9
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Jun 25 14:47:01 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Jun 25 16:14:18 2014 +0200

----------------------------------------------------------------------
 .../eu/stratosphere/client/LocalExecutor.java   | 31 ++++----------------
 1 file changed, 5 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/84313956/stratosphere-clients/src/main/java/eu/stratosphere/client/LocalExecutor.java
----------------------------------------------------------------------
diff --git a/stratosphere-clients/src/main/java/eu/stratosphere/client/LocalExecutor.java b/stratosphere-clients/src/main/java/eu/stratosphere/client/LocalExecutor.java
index d73f893..0f50f96 100644
--- a/stratosphere-clients/src/main/java/eu/stratosphere/client/LocalExecutor.java
+++ b/stratosphere-clients/src/main/java/eu/stratosphere/client/LocalExecutor.java
@@ -257,32 +257,11 @@ public class LocalExecutor extends PlanExecutor {
 	 * @throws Exception
 	 */
 	public String getOptimizerPlanAsJSON(Plan plan) throws Exception {
-		synchronized (this.lock) {
-			
-			// check if we start a session dedicated for this execution
-			final boolean shutDownAtEnd;
-			if (this.nephele == null) {
-				// we start a session just for us now
-				shutDownAtEnd = true;
-				start();
-			} else {
-				// we use the existing session
-				shutDownAtEnd = false;
-			}
-
-			try {
-				PactCompiler pc = new PactCompiler(new DataStatistics());
-				OptimizedPlan op = pc.compile(plan);
-				PlanJSONDumpGenerator gen = new PlanJSONDumpGenerator();
-		
-				return gen.getOptimizerPlanAsJSON(op);
-			}
-			finally {
-				if (shutDownAtEnd) {
-					stop();
-				}
-			}
-		}
+		PactCompiler pc = new PactCompiler(new DataStatistics());
+		OptimizedPlan op = pc.compile(plan);
+		PlanJSONDumpGenerator gen = new PlanJSONDumpGenerator();
+	
+		return gen.getOptimizerPlanAsJSON(op);
 	}
 	
 	// --------------------------------------------------------------------------------------------


[48/53] [abbrv] git commit: InputVertices do not require an input format.

Posted by rm...@apache.org.
InputVertices do not require an input format.


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

Branch: refs/heads/travis_test
Commit: 35438ec2f3b0d3babb4f9348a32df89a7376ae25
Parents: ef623e9
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Jun 25 16:52:55 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Jun 25 16:52:55 2014 +0200

----------------------------------------------------------------------
 .../executiongraph/ExecutionGroupVertex.java    |  6 +-
 .../nephele/jobgraph/JobInputVertex.java        |  6 +-
 .../test/runtime/NetworkStackThroughput.java    | 66 +++-----------------
 3 files changed, 14 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/35438ec2/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java
index dceeb90..599c682 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java
@@ -429,8 +429,9 @@ public final class ExecutionGroupVertex {
 	 *
 	 * @param inputSplitType Input split type class
 	 */
-	public void setInputSplitType(final Class<? extends InputSplit> inputSplitType) { this.inputSplitType =
-			inputSplitType; }
+	public void setInputSplitType(final Class<? extends InputSplit> inputSplitType) { 
+		this.inputSplitType = inputSplitType;
+	}
 
 	/**
 	 * Returns the input splits assigned to this group vertex.
@@ -438,7 +439,6 @@ public final class ExecutionGroupVertex {
 	 * @return the input splits, possibly <code>null</code> if the group vertex does not represent an input vertex
 	 */
 	public InputSplit[] getInputSplits() {
-
 		return this.inputSplits;
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/35438ec2/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobInputVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobInputVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobInputVertex.java
index bf8f544..2c8ba86 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobInputVertex.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobInputVertex.java
@@ -72,8 +72,8 @@ public class JobInputVertex extends AbstractJobInputVertex {
 	 */
 	@Override
 	public Class<? extends InputSplit> getInputSplitType() {
-		if(inputFormat == null){
-			throw new RuntimeException("No input format has been set for job vertex: "+ this.getID());
+		if (inputFormat == null){
+			return InputSplit.class;
 		}
 
 		return inputFormat.getInputSplitType();
@@ -89,7 +89,7 @@ public class JobInputVertex extends AbstractJobInputVertex {
 	@Override
 	public InputSplit[] getInputSplits(int minNumSplits) throws IOException {
 		if (inputFormat == null){
-			throw new RuntimeException("No input format has been set for job vertex: "+ this.getID());
+			return null;
 		}
 
 		return inputFormat.createInputSplits(minNumSplits);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/35438ec2/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackThroughput.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackThroughput.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackThroughput.java
index 74d52b9..9b202c6 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackThroughput.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackThroughput.java
@@ -13,12 +13,16 @@
 
 package eu.stratosphere.test.runtime;
 
-import eu.stratosphere.api.common.io.GenericInputFormat;
-import eu.stratosphere.api.common.io.OutputFormat;
-import eu.stratosphere.api.common.operators.util.UserCodeObjectWrapper;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.junit.After;
+
 import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.core.io.InputSplit;
 import eu.stratosphere.nephele.jobgraph.DistributionPattern;
 import eu.stratosphere.nephele.jobgraph.JobGraph;
 import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException;
@@ -26,20 +30,11 @@ import eu.stratosphere.nephele.jobgraph.JobInputVertex;
 import eu.stratosphere.nephele.jobgraph.JobOutputVertex;
 import eu.stratosphere.nephele.jobgraph.JobTaskVertex;
 import eu.stratosphere.nephele.template.AbstractInvokable;
-import eu.stratosphere.pact.runtime.task.util.TaskConfig;
 import eu.stratosphere.runtime.io.api.RecordReader;
 import eu.stratosphere.runtime.io.api.RecordWriter;
 import eu.stratosphere.runtime.io.channels.ChannelType;
 import eu.stratosphere.test.util.RecordAPITestBase;
-import eu.stratosphere.types.Record;
 import eu.stratosphere.util.LogUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.junit.After;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
 
 public class NetworkStackThroughput {
 
@@ -109,9 +104,6 @@ public class NetworkStackThroughput {
 			producer.getConfiguration().setInteger(DATA_VOLUME_GB_CONFIG_KEY, dataVolumeGb);
 			producer.getConfiguration().setBoolean(IS_SLOW_SENDER_CONFIG_KEY, isSlowSender);
 
-			TaskConfig inputConfig = new TaskConfig(producer.getConfiguration());
-			inputConfig.setStubWrapper(new UserCodeObjectWrapper<Object>(new DummyInputFormat()));
-
 			JobTaskVertex forwarder = null;
 			if (useForwarder) {
 				forwarder = new JobTaskVertex("Speed Test Forwarder", jobGraph);
@@ -124,9 +116,6 @@ public class NetworkStackThroughput {
 			consumer.setNumberOfSubtasks(numSubtasks);
 			consumer.getConfiguration().setBoolean(IS_SLOW_RECEIVER_CONFIG_KEY, isSlowReceiver);
 
-			TaskConfig outputConfig = new TaskConfig(consumer.getConfiguration());
-			outputConfig.setStubWrapper(new UserCodeObjectWrapper<Object>(new DummyOutputFormat()));
-
 			if (useForwarder) {
 				producer.connectTo(forwarder, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
 				forwarder.connectTo(consumer, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
@@ -269,45 +258,6 @@ public class NetworkStackThroughput {
 		}
 	}
 
-	public static final class DummyInputFormat extends GenericInputFormat {
-
-		private static final long serialVersionUID = 6891640958330871924L;
-
-		@Override
-		public void open(InputSplit split) throws IOException {
-
-		}
-
-		@Override
-		public boolean reachedEnd() throws IOException {
-			return false;
-		}
-
-		@Override
-		public Object nextRecord(Object reuse) throws IOException {
-			return null;
-		}
-	}
-
-	public static final class DummyOutputFormat implements OutputFormat<Record> {
-
-		@Override
-		public void configure(Configuration parameters) {
-		}
-
-		@Override
-		public void open(int taskNumber, int numTasks) {
-		}
-
-		@Override
-		public void writeRecord(Record record) {
-		}
-
-		@Override
-		public void close() {
-		}
-	}
-
 	// ------------------------------------------------------------------------
 
 	public void testThroughput() throws Exception {


[25/53] [abbrv] [FLINK-962] Initial import of documentation from website into source code (closes #34)

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/java_api_examples.md
----------------------------------------------------------------------
diff --git a/docs/java_api_examples.md b/docs/java_api_examples.md
new file mode 100644
index 0000000..ddb43e6
--- /dev/null
+++ b/docs/java_api_examples.md
@@ -0,0 +1,304 @@
+---
+title:  "Java API Examples"
+---
+
+The following example programs showcase different applications of Stratosphere 
+from simple word counting to graph algorithms. The code samples illustrate the 
+use of **[Stratosphere's Java API]({{site.baseurl}}/docs/{{site current_stable}}/programming_guides/java.html)**. 
+
+The full source code of the following and more examples can be found in the **[stratosphere-java-examples](https://github.com/stratosphere/stratosphere/tree/release-{{site.current_stable}}/stratosphere-examples/stratosphere-java-examples)** module.
+
+# Word Count
+WordCount is the "Hello World" of Big Data processing systems. It computes the frequency of words in a text collection. The algorithm works in two steps: First, the texts are splits the text to individual words. Second, the words are grouped and counted.
+
+```java
+// get input data
+DataSet<String> text = getTextDataSet(env);
+
+DataSet<Tuple2<String, Integer>> counts = 
+        // split up the lines in pairs (2-tuples) containing: (word,1)
+        text.flatMap(new Tokenizer())
+        // group by the tuple field "0" and sum up tuple field "1"
+        .groupBy(0)
+        .aggregate(Aggregations.SUM, 1);
+
+counts.writeAsCsv(outputPath, "\n", " ");
+
+// User-defined functions
+public static final class Tokenizer extends FlatMapFunction<String, Tuple2<String, Integer>> {
+
+    @Override
+    public void flatMap(String value, Collector<Tuple2<String, Integer>> out) {
+        // normalize and split the line
+        String[] tokens = value.toLowerCase().split("\\W+");
+        
+        // emit the pairs
+        for (String token : tokens) {
+            if (token.length() > 0) {
+                out.collect(new Tuple2<String, Integer>(token, 1));
+            }   
+        }
+    }
+}
+```
+
+The [WordCount example](https://github.com/stratosphere/stratosphere/blob/release-{{site.current_stable}}/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCount.java) implements the above described algorithm with input parameters: `<text input path>, <output path>`. As test data, any text file will do.
+
+# Page Rank
+
+The PageRank algorithm computes the "importance" of pages in a graph defined by links, which point from one pages to another page. It is an iterative graph algorithm, which means that it repeatedly applies the same computation. In each iteration, each page distributes its current rank over all its neighbors, and compute its new rank as a taxed sum of the ranks it received from its neighbors. The PageRank algorithm was popularized by the Google search engine which uses the importance of webpages to rank the results of search queries.
+
+In this simple example, PageRank is implemented with a [bulk iteration]({{site.baseurl}}/docs/{{site.current_stable}}/programming_guides/java.html#iterations) and a fixed number of iterations.
+
+```java
+// get input data
+DataSet<Tuple2<Long, Double>> pagesWithRanks = getPagesWithRanksDataSet(env);
+DataSet<Tuple2<Long, Long[]>> pageLinkLists = getLinksDataSet(env);
+
+// set iterative data set
+IterativeDataSet<Tuple2<Long, Double>> iteration = pagesWithRanks.iterate(maxIterations);
+
+DataSet<Tuple2<Long, Double>> newRanks = iteration
+        // join pages with outgoing edges and distribute rank
+        .join(pageLinkLists).where(0).equalTo(0).flatMap(new JoinVertexWithEdgesMatch())
+        // collect and sum ranks
+        .groupBy(0).aggregate(SUM, 1)
+        // apply dampening factor
+        .map(new Dampener(DAMPENING_FACTOR, numPages));
+
+DataSet<Tuple2<Long, Double>> finalPageRanks = iteration.closeWith(
+        newRanks, 
+        newRanks.join(iteration).where(0).equalTo(0)
+        // termination condition
+        .filter(new EpsilonFilter()));
+
+finalPageRanks.writeAsCsv(outputPath, "\n", " ");
+
+// User-defined functions
+
+public static final class JoinVertexWithEdgesMatch 
+                    extends FlatMapFunction<Tuple2<Tuple2<Long, Double>, Tuple2<Long, Long[]>>, 
+                                            Tuple2<Long, Double>> {
+
+    @Override
+    public void flatMap(Tuple2<Tuple2<Long, Double>, Tuple2<Long, Long[]>> value, 
+                        Collector<Tuple2<Long, Double>> out) {
+        Long[] neigbors = value.f1.f1;
+        double rank = value.f0.f1;
+        double rankToDistribute = rank / ((double) neigbors.length);
+            
+        for (int i = 0; i < neigbors.length; i++) {
+            out.collect(new Tuple2<Long, Double>(neigbors[i], rankToDistribute));
+        }
+    }
+}
+
+public static final class Dampener extends MapFunction<Tuple2<Long,Double>, Tuple2<Long,Double>> {
+    private final double dampening, randomJump;
+
+    public Dampener(double dampening, double numVertices) {
+        this.dampening = dampening;
+        this.randomJump = (1 - dampening) / numVertices;
+    }
+
+    @Override
+    public Tuple2<Long, Double> map(Tuple2<Long, Double> value) {
+        value.f1 = (value.f1 * dampening) + randomJump;
+        return value;
+    }
+}
+
+public static final class EpsilonFilter 
+                    extends FilterFunction<Tuple2<Tuple2<Long, Double>, Tuple2<Long, Double>>> {
+
+    @Override
+    public boolean filter(Tuple2<Tuple2<Long, Double>, Tuple2<Long, Double>> value) {
+        return Math.abs(value.f0.f1 - value.f1.f1) > EPSILON;
+    }
+}
+```
+
+The [PageRank program](https://github.com/stratosphere/stratosphere/blob/release-{{site.current_stable}}/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/PageRankBasic.java) implements the above example.
+It requires the following parameters to run: `<pages input path>, <links input path>, <output path>, <num pages>, <num iterations>`.
+
+Input files are plain text files and must be formatted as follows:
+- Pages represented as an (long) ID separated by new-line characters.
+    * For example `"1\n2\n12\n42\n63\n"` gives five pages with IDs 1, 2, 12, 42, and 63.
+- Links are represented as pairs of page IDs which are separated by space characters. Links are separated by new-line characters:
+    * For example `"1 2\n2 12\n1 12\n42 63\n"` gives four (directed) links (1)->(2), (2)->(12), (1)->(12), and (42)->(63).
+
+For this simple implementation it is required that each page has at least one incoming and one outgoing link (a page can point to itself).
+
+# Connected Components
+
+The Connected Components algorithm identifies parts of a larger graph which are connected by assigning all vertices in the same connected part the same component ID. Similar to PageRank, Connected Components is an iterative algorithm. In each step, each vertex propagates its current component ID to all its neighbors. A vertex accepts the component ID from a neighbor, if it is smaller than its own component ID.
+
+This implementation uses a [delta iteration]({{site.baseurl}}/docs/{{site.current_stable}}/programming_guides/java.html#iterations): Vertices that have not changed their component ID do not participate in the next step. This yields much better performance, because the later iterations typically deal only with a few outlier vertices.
+
+```java
+// read vertex and edge data
+DataSet<Long> vertices = getVertexDataSet(env);
+DataSet<Tuple2<Long, Long>> edges = getEdgeDataSet(env).flatMap(new UndirectEdge());
+
+// assign the initial component IDs (equal to the vertex ID)
+DataSet<Tuple2<Long, Long>> verticesWithInitialId = vertices.map(new DuplicateValue<Long>());
+        
+// open a delta iteration
+DeltaIteration<Tuple2<Long, Long>, Tuple2<Long, Long>> iteration =
+        verticesWithInitialId.iterateDelta(verticesWithInitialId, maxIterations, 0);
+
+// apply the step logic: 
+DataSet<Tuple2<Long, Long>> changes = iteration.getWorkset()
+        // join with the edges
+        .join(edges).where(0).equalTo(0).with(new NeighborWithComponentIDJoin())
+        // select the minimum neighbor component ID
+        .groupBy(0).aggregate(Aggregations.MIN, 1)
+        // update if the component ID of the candidate is smaller
+        .join(iteration.getSolutionSet()).where(0).equalTo(0)
+        .flatMap(new ComponentIdFilter());
+
+// close the delta iteration (delta and new workset are identical)
+DataSet<Tuple2<Long, Long>> result = iteration.closeWith(changes, changes);
+
+// emit result
+result.writeAsCsv(outputPath, "\n", " ");
+
+// User-defined functions
+
+public static final class DuplicateValue<T> extends MapFunction<T, Tuple2<T, T>> {
+    
+    @Override
+    public Tuple2<T, T> map(T vertex) {
+        return new Tuple2<T, T>(vertex, vertex);
+    }
+}
+
+public static final class UndirectEdge 
+                    extends FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
+    Tuple2<Long, Long> invertedEdge = new Tuple2<Long, Long>();
+    
+    @Override
+    public void flatMap(Tuple2<Long, Long> edge, Collector<Tuple2<Long, Long>> out) {
+        invertedEdge.f0 = edge.f1;
+        invertedEdge.f1 = edge.f0;
+        out.collect(edge);
+        out.collect(invertedEdge);
+    }
+}
+
+public static final class NeighborWithComponentIDJoin 
+                    extends JoinFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Tuple2<Long, Long>> {
+
+    @Override
+    public Tuple2<Long, Long> join(Tuple2<Long, Long> vertexWithComponent, Tuple2<Long, Long> edge) {
+        return new Tuple2<Long, Long>(edge.f1, vertexWithComponent.f1);
+    }
+}
+
+public static final class ComponentIdFilter 
+                    extends FlatMapFunction<Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>>, 
+                                            Tuple2<Long, Long>> {
+
+    @Override
+    public void flatMap(Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>> value, 
+                        Collector<Tuple2<Long, Long>> out) {
+        if (value.f0.f1 < value.f1.f1) {
+            out.collect(value.f0);
+        }
+    }
+}
+```
+
+The [ConnectedComponents program](https://github.com/stratosphere/stratosphere/blob/release-{{site.current_stable}}/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/ConnectedComponents.java) implements the above example. It requires the following parameters to run: `<vertex input path>, <edge input path>, <output path> <max num iterations>`.
+
+Input files are plain text files and must be formatted as follows:
+- Vertices represented as IDs and separated by new-line characters.
+    * For example `"1\n2\n12\n42\n63\n"` gives five vertices with (1), (2), (12), (42), and (63).
+- Edges are represented as pairs for vertex IDs which are separated by space characters. Edges are separated by new-line characters:
+    * For example `"1 2\n2 12\n1 12\n42 63\n"` gives four (undirected) links (1)-(2), (2)-(12), (1)-(12), and (42)-(63).
+
+# Relational Query
+
+The Relational Query example assumes two tables, one with `orders` and the other with `lineitems` as specified by the [TPC-H decision support benchmark](http://www.tpc.org/tpch/). TPC-H is a standard benchmark in the database industry. See below for instructions how to generate the input data.
+
+The example implements the following SQL query.
+
+```sql
+SELECT l_orderkey, o_shippriority, sum(l_extendedprice) as revenue
+    FROM orders, lineitem
+WHERE l_orderkey = o_orderkey
+    AND o_orderstatus = "F" 
+    AND YEAR(o_orderdate) > 1993
+    AND o_orderpriority LIKE "5%"
+GROUP BY l_orderkey, o_shippriority;
+```
+
+The Stratosphere Java program, which implements the above query looks as follows.
+
+```java
+// get orders data set: (orderkey, orderstatus, orderdate, orderpriority, shippriority)
+DataSet<Tuple5<Integer, String, String, String, Integer>> orders = getOrdersDataSet(env);
+// get lineitem data set: (orderkey, extendedprice)
+DataSet<Tuple2<Integer, Double>> lineitems = getLineitemDataSet(env);
+
+// orders filtered by year: (orderkey, custkey)
+DataSet<Tuple2<Integer, Integer>> ordersFilteredByYear =
+        // filter orders
+        orders.filter(
+            new FilterFunction<Tuple5<Integer, String, String, String, Integer>>() {
+                @Override
+                public boolean filter(Tuple5<Integer, String, String, String, Integer> t) {
+                    // status filter
+                    if(!t.f1.equals(STATUS_FILTER)) {
+                        return false;
+                    // year filter
+                    } else if(Integer.parseInt(t.f2.substring(0, 4)) <= YEAR_FILTER) {
+                        return false;
+                    // order priority filter
+                    } else if(!t.f3.startsWith(OPRIO_FILTER)) {
+                        return false;
+                    }
+                    return true;
+                }
+            })
+        // project fields out that are no longer required
+        .project(0,4).types(Integer.class, Integer.class);
+
+// join orders with lineitems: (orderkey, shippriority, extendedprice)
+DataSet<Tuple3<Integer, Integer, Double>> lineitemsOfOrders = 
+        ordersFilteredByYear.joinWithHuge(lineitems)
+                            .where(0).equalTo(0)
+                            .projectFirst(0,1).projectSecond(1)
+                            .types(Integer.class, Integer.class, Double.class);
+
+// extendedprice sums: (orderkey, shippriority, sum(extendedprice))
+DataSet<Tuple3<Integer, Integer, Double>> priceSums = 
+        // group by order and sum extendedprice
+        lineitemsOfOrders.groupBy(0,1).aggregate(Aggregations.SUM, 2);
+
+// emit result
+priceSums.writeAsCsv(outputPath);
+```
+
+The [Relational Query program](https://github.com/stratosphere/stratosphere/blob/release-{{site.current_stable}}/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/RelationalQuery.java) implements the above query. It requires the following parameters to run: `<orders input path>, <lineitem input path>, <output path>`.
+
+The orders and lineitem files can be generated using the [TPC-H benchmark](http://www.tpc.org/tpch/) suite's data generator tool (DBGEN). 
+Take the following steps to generate arbitrary large input files for the provided Stratosphere programs:
+
+1.  Download and unpack DBGEN
+2.  Make a copy of *makefile.suite* called *Makefile* and perform the following changes:
+
+```bash
+DATABASE = DB2
+MACHINE  = LINUX
+WORKLOAD = TPCH
+CC       = gcc
+```
+
+1.  Build DBGEN using *make*
+2.  Generate lineitem and orders relations using dbgen. A scale factor
+    (-s) of 1 results in a generated data set with about 1 GB size.
+
+```bash
+./dbgen -T o -s 1
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/java_api_guide.md
----------------------------------------------------------------------
diff --git a/docs/java_api_guide.md b/docs/java_api_guide.md
new file mode 100644
index 0000000..c09c26a
--- /dev/null
+++ b/docs/java_api_guide.md
@@ -0,0 +1,1476 @@
+---
+title: "Java API Programming Guide"
+---
+
+Java API
+========
+
+<section id="introduction">
+Introduction
+------------
+
+Analysis programs in Stratosphere are regular Java programs that implement transformations on data sets (e.g., filtering, mapping, joining, grouping). The data sets are initially created from certain sources (e.g., by reading files, or from collections). Results are returned via sinks, which may for example write the data to (distributed) files, or to standard output (for example the command line terminal). Stratosphere programs run in a variety of contexts, standalone, or embedded in other programs. The execution can happen in a local JVM, or on clusters of many machines.
+
+In order to create your own Stratosphere program, we encourage you to start with the [program skeleton](#skeleton) and gradually add your own [transformations](#transformations). The remaining sections act as references for additional operations and advanced features.
+</section>
+
+<section id="toc">
+
+<div id="docs_05_toc">
+  <div class="list-group">
+{% for sublink in page.toc %}
+   <a href="#{{ sublink.anchor }}" class="list-group-item">{{forloop.index}}. <strong>{{ sublink.title }}</strong></a>
+{% endfor %}
+  </div>
+</div>
+
+</section>
+
+<section id="example">
+Example Program
+---------------
+
+The following program is a complete, working example of WordCount. You can copy &amp; paste the code to run it locally. You only have to include Stratosphere's Java API library into your project (see Section [Linking with Stratosphere](#linking)) and specify the imports. Then you are ready to go!
+
+```java
+public class WordCountExample {
+    public static void main(String[] args) throws Exception {
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+        DataSet<String> text = env.fromElements(
+            "Who's there?",
+            "I think I hear them. Stand, ho! Who's there?");
+
+        DataSet<Tuple2<String, Integer>> wordCounts = text
+            .flatMap(new LineSplitter())
+            .groupBy(0)
+            .aggregate(Aggregations.SUM, 1);
+
+        wordCounts.print();
+
+        env.execute("Word Count Example");
+    }
+
+    public static final class LineSplitter extends FlatMapFunction<String, Tuple2<String, Integer>> {
+        @Override
+        public void flatMap(String line, Collector<Tuple2<String, Integer>> out) {
+            for (String word : line.split(" ")) {
+                out.collect(new Tuple2<String, Integer>(word, 1));
+            }
+        }
+    }
+}
+```
+
+<div class="back-to-top"><a href="#toc">Back to top</a></div>
+</section>
+
+<section id="linking">
+Linking with Stratosphere
+-------------------------
+
+To write programs with Stratosphere, you need to include Stratosphere’s Java API library in your project.
+
+The simplest way to do this is to use the [quickstart scripts]({{site.baseurl}}/quickstart/java.html). They create a blank project from a template (a Maven Archetype), which sets up everything for you. To manually create the project, you can use the archetype and create a project by calling:
+
+{% highlight bash %}
+mvn archetype:generate /
+    -DarchetypeGroupId=eu.stratosphere /
+    -DarchetypeArtifactId=quickstart-java /
+    -DarchetypeVersion={{site.docs_05_stable}}
+{% endhighlight %}
+
+If you want to add Stratosphere to an existing Maven project, add the following entry to your *dependencies* section in the *pom.xml* file of your project:
+
+{% highlight xml %}
+<dependency>
+  <groupId>eu.stratosphere</groupId>
+  <artifactId>stratosphere-java</artifactId>
+  <version>{{site.docs_05_stable}}</version>
+</dependency>
+<dependency>
+  <groupId>eu.stratosphere</groupId>
+  <artifactId>stratosphere-clients</artifactId>
+  <version>{{site.docs_05_stable}}</version>
+</dependency>
+{% endhighlight %}
+
+In order to link against the latest SNAPSHOT versions of the code, please follow [this guide]({{site.baseurl}}/downloads/#nightly).
+
+The *stratosphere-clients* dependency is only necessary to invoke the Stratosphere program locally (for example to run it standalone for testing and debugging). 
+If you intend to only export the program as a JAR file and [run it on a cluster]({{site.baseurl}}/docs/0.5/program_execution/cluster_execution.html), you can skip that dependency.
+
+<div class="back-to-top"><a href="#toc">Back to top</a></div>
+</section>
+
+<section id="skeleton">
+Program Skeleton
+----------------
+
+As we already saw in the example, Stratosphere programs look like regular Java
+programs with a `main()` method. Each program consists of the same basic parts:
+
+1. Obtain an `ExecutionEnvironment`,
+2. Load/create the initial data,
+3. Specify transformations on this data,
+4. Specify where to put the results of your computations, and
+5. Execute your program.
+
+We will now give an overview of each of those steps but please refer
+to the respective sections for more details. Note that all [core classes
+of the Java API](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-java/src/main/java/eu/stratosphere/api/java) are found in the package `eu.stratosphere.api.java`.
+
+The `ExecutionEnvironment` is the basis for all Stratosphere programs. You can
+obtain one using these static methods on class `ExecutionEnvironment`:
+
+```java
+getExecutionEnvironment()
+
+createLocalEnvironment()
+createLocalEnvironment(int degreeOfParallelism)
+
+createRemoteEnvironment(String host, int port, String... jarFiles)
+createRemoteEnvironment(String host, int port, int degreeOfParallelism, String... jarFiles)
+```
+
+Typically, you only need to use `getExecutionEnvironment()`, since this
+will do the right thing depending on the context: if you are executing
+your program inside an IDE or as a regular Java program it will create
+a local environment that will execute your program on your local machine. If
+you created a JAR file from you program, and invoke it through the [command line]({{site.baseurl}}/docs/0.5/program_execution/cli_client.html)
+or the [web interface]({{site.baseurl}}/docs/0.5/program_execution/web_interface.html),
+the Stratosphere cluster manager will
+execute your main method and `getExecutionEnvironment()` will return
+an execution environment for executing your program on a cluster.
+
+For specifying data sources the execution environment has several methods
+to read from files using various methods: you can just read them line by line,
+as CSV files, or using completely custom data input formats. To just read
+a text file as a sequence of lines, you could use:
+
+```java
+final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+DataSet<String> text = env.readTextFile("file:///path/to/file");
+```
+
+This will give you a `DataSet` on which you can then apply transformations. For
+more information on data sources and input formats, please refer to
+[Data Sources](#data_sources).
+
+Once you have a `DataSet` you can apply transformations to create a new
+`DataSet` which you can then write to a file, transform again, or
+combine with other `DataSet`s. You apply transformations by calling
+methods on `DataSet` with your own custom transformation function. For example,
+map looks like this:
+
+```java
+DataSet<String> input = ...;
+
+DataSet<Integer> tokenized = text.map(new MapFunction<String, Integer>() {
+    @Override
+    public Integer map(String value) {
+        return Integer.parseInt(value);
+    }
+});
+```
+
+This will create a new `DataSet` by converting every String in the original
+set to an Integer. For more information and a list of all the transformations,
+please refer to [Transformations](#transformations).
+
+Once you have a `DataSet` that needs to be written to disk you call one
+of these methods on `DataSet`:
+
+```java
+writeAsText(String path)
+writeAsCsv(String path)
+write(FileOutputFormat<T> outputFormat, String filePath)
+
+print()
+```
+
+The last method is only useful for developing/debugging on a local machine,
+it will output the contents of the `DataSet` to standard output. (Note that in
+a cluster, the result goes to the standard out stream of the cluster nodes and ends
+up in the *.out* files of the workers).
+The first two do as the name suggests, the third one can be used to specify a
+custom data output format. Keep in mind, that these calls do not actually
+write to a file yet. Only when your program is completely specified and you
+call the `execute` method on your `ExecutionEnvironment` are all the
+transformations executed and is data written to disk. Please refer
+to [Data Sinks](#data_sinks) for more information on writing to files and also
+about custom data output formats.
+
+Once you specified the complete program you need to call `execute` on
+the `ExecutionEnvironment`. This will either execute on your local
+machine or submit your program for execution on a cluster, depending on
+how you created the execution environment.
+<div class="back-to-top"><a href="#toc">Back to top</a></div>
+</section>
+
+<section id="lazyeval">
+Lazy Evaluation
+---------------
+
+All Stratosphere programs are executed lazily: When the program's main method is executed, the data loading and transformations do not happen directly. Rather, each operation is created and added to the program's plan. The operations are actually executed when one of the `execute()` methods is invoked on the ExecutionEnvironment object. Whether the program is executed locally or on a cluster depends on the environment of the program.
+
+The lazy evaluation lets you construct sophisticated programs that Stratosphere executes as one holistically planned unit.
+</section>
+
+<section id="types">
+Data Types
+----------
+
+The Java API is strongly typed: All data sets and transformations accept typed elements. This catches type errors very early and supports safe refactoring of programs. The API supports various different data types for the input and output of operators. Both `DataSet` and functions like `MapFunction`, `ReduceFunction`, etc. are parameterized with data types using Java generics in order to ensure type-safety.
+
+There are four different categories of data types, which are treated slightly different:
+
+1. **Regular Types**
+2. **Tuples**
+3. **Values**
+4. **Hadoop Writables**
+
+
+#### Regular Types
+
+Out of the box, the Java API supports all common basic Java types: `Byte`, `Short`, `Integer`, `Long`, `Float`, `Double`, `Boolean`, `Character`, `String`.
+
+Furthermore, you can use the vast majority of custom Java classes. Restrictions apply to classes containing fields that cannot be serialized, like File pointers, I/O streams, or other native resources. Classes that follow the Java Beans conventions work well in general. The following defines a simple example class to illustrate how you can use custom classes:
+
+```java
+public class WordWithCount {
+
+    public String word;
+    public int count;
+
+    public WordCount() {}
+
+    public WordCount(String word, int count) {
+        this.word = word;
+        this.count = count;
+    }
+}
+```
+
+You can use all of those types to parameterize `DataSet` and function implementations, e.g. `DataSet<String>` for a `String` data set or `MapFunction<String, Integer>` for a mapper from `String` to `Integer`.
+
+```java
+
+// using a basic data type
+DataSet<String> numbers = env.fromElements("1", "2");
+
+numbers.map(new MapFunction<String, Integer>() {
+    @Override
+    public String map(String value) throws Exception {
+        return Integer.parseInt(value);
+    }
+});
+
+// using a custom class
+DataSet<WordCount> wordCounts = env.fromElements(
+    new WordCount("hello", 1),
+    new WordCount("world", 2));
+
+wordCounts.map(new MapFunction<WordCount, Integer>() {
+    @Override
+    public String map(WordCount value) throws Exception {
+        return value.count;
+    }
+});
+```
+
+When working with operators that require a Key for grouping or matching records
+you need to implement a `KeySelector` for your custom type (see
+[Section Data Transformations](#transformations)).
+
+```java
+wordCounts.groupBy(new KeySelector<WordCount, String>() {
+    public String getKey(WordCount v) {
+        return v.word;
+    }
+}).reduce(new MyReduceFunction());
+```
+
+#### Tuples
+
+You can use the `Tuple` classes for composite types. Tuples contain a fix number of fields of various types. The Java API provides classes from `Tuple1` up to `Tuple25`. Every field of a tuple can be an arbitrary Stratosphere type - including further tuples, resulting in nested tuples. Fields of a Tuple can be accessed directly using the fields `tuple.f4`, or using the generic getter method `tuple.getField(int position)`. The field numbering starts with 0. Note that this stands in contrast to the Scala tuples, but it is more consistent with Java's general indexing.
+
+```java
+DataSet<Tuple2<String, Integer>> wordCounts = env.fromElements(
+    new Tuple2<String, Integer>("hello", 1),
+    new Tuple2<String, Integer>("world", 2));
+
+wordCounts.map(new MapFunction<Tuple2<String, Integer>, Integer>() {
+    @Override
+    public String map(Tuple2<String, Integer> value) throws Exception {
+        return value.f1;
+    }
+});
+```
+
+When working with operators that require a Key for grouping or matching records,
+Tuples let you simply specify the positions of the fields to be used as key. You can specify more
+than one position to use composite keys (see [Section Data Transformations](#transformations)).
+
+```java
+wordCounts
+    .groupBy(0)
+    .reduce(new MyReduceFunction());
+```
+
+In order to access fields more intuitively and to generate more readable code, it is also possible to extend a subclass of `Tuple`. You can add getters and setters with custom names that delegate to the field positions. See this [example](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/TPCHQuery3.java) for an illustration how to make use of that mechanism.
+
+
+#### Values
+
+*Value* types describe their serialization and deserialization manually. Instead of going through a general purpose serialization framework, they provide custom code for those operations by means implementing the `eu.stratosphere.types.Value` interface with the methods `read` and `write`. Using a *Value* type is reasonable when general purpose serialization would be highly inefficient. An example would be a data type that implements a sparse vector of elements as an array. Knowing that the array is mostly zero, one can use a special encoding for the non-zero elements, while the general purpose serialization would simply write all array elements.
+
+The `eu.stratosphere.types.CopyableValue` interface supports manual internal cloning logic in a similar way.
+
+Stratosphere comes with pre-defined Value types that correspond to Java's basic data types. (`ByteValue`, `ShortValue`, `IntValue`, `LongValue`, `FloatValue`, `DoubleValue`, `StringValue`, `CharValue`, `BooleanValue`). These Value types act as mutable variants of the basic data types: Their value can be altered, allowing programmers to reuse objects and take pressure off the garbage collector. 
+
+
+#### Hadoop Writables
+
+You can use types that implement the `org.apache.hadoop.Writable` interface. The serialization logic defined in the `write()`and `readFields()` methods will be used for serialization.
+
+
+#### Type Erasure & Type Inferrence
+
+The Java compiler throws away much of the generic type information after the compilation. This is known as *type erasure* in Java. It means that at runtime, an instance of an object does not know its generic type any more. For example, instances of `DataSet<String>` and `DataSet<Long>` look the same to the JVM.
+
+Stratosphere requires type information at the time when it prepares the program for execution (when the main method of the program is called). The Stratosphere Java API tries to reconstruct the type information that was thrown away in various ways and store it explicitly in the data sets and operators. You can retrieve the type via `DataSet.getType()`. The method returns an instance of `TypeInformation`, which is Stratosphere's internal way of representing types.
+
+The type inference has its limits and needs the "cooperation" of the programmer in some cases. Examples for that are methods that create data sets from collections, such as `ExecutionEnvironment.fromCollection(),` where you can pass an argument that describes the type. But also generic functions like `MapFunction<I, O>` may need extra type information.
+
+The [ResultTypeQueryable](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/ResultTypeQueryable.java) interface can be implemented by input formats and functions to tell the API explicitly about their return type. The *input types* that the functions are invoked with can usually be inferred by the result types of the previous operations.
+
+<div class="back-to-top"><a href="#toc">Back to top</a></div>
+</section>
+
+<section id="transformations">
+Data Transformations
+--------------------
+
+A data transformation transforms one or more `DataSet`s into a new `DataSet`. Advanced data analysis programs can be assembled by chaining multiple transformations.
+
+### Map
+
+The Map transformation applies a user-defined `MapFunction` on each element of a DataSet.
+It implements a one-to-one mapping, that is, exactly one element must be returned by
+the function.
+
+The following code transforms a `DataSet` of Integer pairs into a `DataSet` of Integers:
+
+```java
+// MapFunction that adds two integer values
+public class IntAdder extends MapFunction<Tuple2<Integer, Integer>, Integer> {
+  @Override
+  public Integer map(Tuple2<Integer, Integer> in) {
+    return in.f0 + in.f1;
+  }
+}
+
+// [...]
+DataSet<Tuple2<Integer, Integer>> intPairs = // [...]
+DataSet<Integer> intSums = intPairs.map(new IntAdder());
+```
+
+### FlatMap
+
+The FlatMap transformation applies a user-defined `FlatMapFunction` on each element of a `DataSet`.
+This variant of a map function can return arbitrary many result elements (including none) for each input element.
+
+The following code transforms a `DataSet` of text lines into a `DataSet` of words:
+
+```java
+// FlatMapFunction that tokenizes a String by whitespace characters and emits all String tokens.
+public class Tokenizer extends FlatMapFunction<String, String> {
+  @Override
+  public void flatMap(String value, Collector<String> out) {
+    for (String token : value.split("\\W")) {
+      out.collect(token);
+    }
+  }
+}
+
+// [...]
+DataSet<String> textLines = // [...]
+DataSet<String> words = textLines.flatMap(new Tokenizer());
+
+```
+
+### Filter
+
+The Filter transformation applies a user-defined `FilterFunction` on each element of a `DataSet` and retains only those elements for which the function returns `true`.
+
+The following code removes all Integers smaller than zero from a `DataSet`:
+
+```java
+// FilterFunction that filters out all Integers smaller than zero.
+public class NaturalNumberFilter extends FilterFunction<Integer> {
+  @Override
+  public boolean filter(Integer number) {
+    return number >= 0;
+  }
+}
+
+// [...]
+DataSet<Integer> intNumbers = // [...]
+DataSet<Integer> naturalNumbers = intNumbers.filter(new NaturalNumberFilter());
+```
+
+### Project (Tuple DataSets only)
+
+The Project transformation removes or moves `Tuple` fields of a `Tuple` `DataSet`.
+The `project(int...)` method selects `Tuple` fields that should be retained by their index and defines their order in the output `Tuple`.
+The `types(Class<?> ...)`method must give the types of the output `Tuple` fields.
+
+Projections do not require the definition of a user function.
+
+The following code shows different ways to apply a Project transformation on a `DataSet`:
+
+```java
+DataSet<Tuple3<Integer, Double, String>> in = // [...]
+// converts Tuple3<Integer, Double, String> into Tuple2<String, Integer>
+DataSet<Tuple2<String, Integer>> out = in.project(2,0).types(String.class, Integer.class);
+```
+
+### Transformations on grouped DataSet
+
+The reduce operations can operate on grouped data sets. Specifying the key to
+be used for grouping can be done in two ways:
+
+- a `KeySelector` function or
+- one or more field position keys (`Tuple` `DataSet` only).
+
+Please look at the reduce examples to see how the grouping keys are specified.
+
+### Reduce on grouped DataSet
+
+A Reduce transformation that is applied on a grouped `DataSet` reduces each group to a single element using a user-defined `ReduceFunction`.
+For each group of input elements, a `ReduceFunction` successively combines pairs of elements into one element until only a single element for each group remains.
+
+#### Reduce on DataSet grouped by KeySelector Function
+
+A `KeySelector` function extracts a key value from each element of a `DataSet`. The extracted key value is used to group the `DataSet`.
+The following code shows how to group a POJO `DataSet` using a `KeySelector` function and to reduce it with a `ReduceFunction`.
+
+```java
+// some ordinary POJO
+public class WC {
+  public String word;
+  public int count;
+  // [...]
+}
+
+// ReduceFunction that sums Integer attributes of a POJO
+public class WordCounter extends ReduceFunction<WC> {
+  @Override
+  public WC reduce(WC in1, WC in2) {
+    return new WC(in1.word, in1.count + in2.count);
+  }
+}
+
+// [...]
+DataSet<WC> words = // [...]
+DataSet<WC> wordCounts = words
+                         // DataSet grouping with inline-defined KeySelector function
+                         .groupBy(
+                           new KeySelector<WC, String>() {
+                             public String getKey(WC wc) { return wc.word; }
+                           })
+                         // apply ReduceFunction on grouped DataSet
+                         .reduce(new WordCounter());
+```
+
+#### Reduce on DataSet grouped by Field Position Keys (Tuple DataSets only)
+
+Field position keys specify one or more fields of a `Tuple` `DataSet` that are used as grouping keys.
+The following code shows how to use field position keys and apply a `ReduceFunction`.
+
+```java
+DataSet<Tuple3<String, Integer, Double>> tuples = // [...]
+DataSet<Tuple3<String, Integer, Double>> reducedTuples =
+                                         tuples
+                                         // group DataSet on first and second field of Tuple
+                                         .groupBy(0,1)
+                                         // apply ReduceFunction on grouped DataSet
+                                         .reduce(new MyTupleReducer());
+```
+
+### GroupReduce on grouped DataSet
+
+A GroupReduce transformation that is applied on a grouped `DataSet` calls a user-defined `GroupReduceFunction` for each group. The difference
+between this and `Reduce` is that the user defined function gets the whole group at once.
+The function is invoked with an iterator over all elements of a group and can return an arbitrary number of result elements using the collector.
+
+#### GroupReduce on DataSet grouped by Field Position Keys (Tuple DataSets only)
+
+The following code shows how duplicate strings can be removed from a `DataSet` grouped by Integer.
+
+```java
+public class DistinctReduce
+         extends GroupReduceFunction<Tuple2<Integer, String>, Tuple2<Integer, String>> {
+  // Set to hold all unique strings of a group
+  Set<String> uniqStrings = new HashSet<String>();
+
+  @Override
+  public void reduce(Iterator<Tuple2<Integer, String>> in, Collector<Tuple2<Integer, String>> out) {
+    // clear set
+    uniqStrings.clear();
+    // there is at least one element in the iterator
+    Tuple2<Integer, String> first = in.next();
+    Integer key = first.f0;
+    uniqStrings.add(first.f1);
+    // add all strings of the group to the set
+    while(in.hasNext()) {
+      uniqStrings.add(in.next().f1);
+    }
+    // emit all unique strings
+    Tuple2<Integer, String> t = new Tuple2<Integer, String>(key, "");
+    for(String s : uniqStrings) {
+      t.f1 = s;
+      out.collect(t);
+    }
+  }
+}
+
+// [...]
+DataSet<Tuple2<Integer, String>> input = // [...]
+DataSet<Tuple2<Integer, String>> output =
+                                 input
+                                 // group DataSet by the first tuple field
+                                 .groupBy(0)
+                                 // apply GroupReduceFunction on each group and
+                                 //   remove elements with duplicate strings.
+                                 .reduceGroup(new DistinctReduce());
+```
+
+**Note:** Stratosphere internally works a lot with mutable objects. Collecting objects like in the above example only works because Strings are immutable in Java!
+
+#### GroupReduce on DataSet grouped by KeySelector Function
+
+Works analogous to `KeySelector` functions in Reduce transformations.
+
+#### GroupReduce on sorted groups (Tuple DataSets only)
+
+A `GroupReduceFunction` accesses the elements of a group using an iterator. Optionally, the iterator can hand out the elements of a group in a specified order. In many cases this can help to reduce the complexity of a user-defined `GroupReduceFunction` and improve its efficiency.
+Right now, this feature is only available for `Tuple` `DataSet`.
+
+The following code shows another example how to remove duplicate Strings in a `DataSet` grouped by an Integer and sorted by String.
+
+```java
+// GroupReduceFunction that removes consecutive identical elements
+public class DistinctReduce
+         extends GroupReduceFunction<Tuple2<Integer, String>, Tuple2<Integer, String>> {
+  @Override
+  public void reduce(Iterator<Tuple2<Integer, String>> in, Collector<Tuple2<Integer, String>> out) {
+    // there is at least one element in the iterator
+    Tuple2<Integer, String> first = in.next();
+    Integer key = first.f0;
+    String comp = first.f1;
+    // for each element in group
+    while(in.hasNext()) {
+      String next = in.next().f1;
+      // check if strings are different
+      if(!next.equals(comp)) {
+        // emit a new element
+        out.collect(new Tuple2<Integer, String>(key, comp));
+        // update compare string
+        comp = next;
+      }
+    }
+    // emit last element
+    out.collect(new Tuple2<Integer, String>(key, comp));
+  }
+}
+
+// [...]
+DataSet<Tuple2<Integer, String>> input = // [...]
+DataSet<Double> output = input
+                         // group DataSet by the first tuple field
+                         .groupBy(0)
+                         // sort groups on second tuple field
+                         .sortGroup(1, Order.ASCENDING)
+                         // // apply GroupReduceFunction on DataSet with sorted groups
+                         .reduceGroup(new DistinctReduce());
+```
+
+**Note:** A GroupSort often comes for free if the grouping is established using a sort-based execution strategy of an operator before the reduce operation.
+
+#### Combinable GroupReduceFunctions
+
+In contrast to a `ReduceFunction`, a `GroupReduceFunction` is not implicitly combinable. In order to make a `GroupReduceFunction` combinable, you need to implement (override) the ```combine()``` method and annotate the `GroupReduceFunction` with the ```@Combinable``` annotation as shown here:
+
+The following code shows how to compute multiple sums using a combinable `GroupReduceFunction`:
+
+```java
+// Combinable GroupReduceFunction that computes two sums.
+@Combinable
+public class MyCombinableGroupReducer
+         extends GroupReduceFunction<Tuple3<String, Integer, Double>,
+                                     Tuple3<String, Integer, Double>> {
+  @Override
+  public void reduce(Iterator<Tuple3<String, Integer, Double>> in,
+                     Collector<Tuple3<String, Integer, Double>> out) {
+    // one element is always present in iterator
+    Tuple3<String, Integer, Double> curr = in.next();
+    String key = curr.f0;
+    int intSum = curr.f1;
+    double doubleSum = curr.f2;
+    // sum up all ints and doubles
+    while(in.hasNext()) {
+      curr = in.next();
+      intSum += curr.f1;
+      doubleSum += curr.f2;
+    }
+    // emit a tuple with both sums
+    out.collect(new Tuple3<String, Integer, Double>(key, intSum, doubleSum));
+  }
+
+  @Override
+  public void combine(Iterator<Tuple3<String, Integer, Double>> in,
+                      Collector<Tuple3<String, Integer, Double>> out)) {
+    // in some cases combine() calls can simply be forwarded to reduce().
+    this.reduce(in, out);
+  }
+}
+```
+
+### Aggregate on grouped Tuple DataSet
+
+There are some common aggregation operations that are frequently used. The Aggregate transformation provides the following build-in aggregation functions:
+
+- Sum,
+- Min, and
+- Max.
+
+The Aggregate transformation can only be applied on a `Tuple` `DataSet` and supports only field positions keys for grouping.
+
+The following code shows how to apply an Aggregation transformation on a `DataSet` grouped by field position keys:
+
+```java
+DataSet<Tuple3<Integer, String, Double>> input = // [...]
+DataSet<Tuple3<Integer, String, Double>> output = input
+                                          // group DataSet on second field
+                                          .groupBy(1)
+                                          // compute sum of the first field
+                                          .aggregate(SUM, 0)
+                                          // compute minimum of the third field
+                                          .and(MIN, 2);
+```
+
+To apply multiple aggregations on a DataSet it is necessary to use the `.and()` function after the first aggregate, that means `.aggregate(SUM, 0).and(MIN, 2)` produces the sum of field 0 and the minimum of field 2 of the original DataSet. 
+In contrast to that `.aggregate(SUM, 0).aggregate(MIN, 2)` will apply an aggregation on an aggregation. In the given example it would produce the minimum of field 2 after calculating the sum of field 0 grouped by field 1.
+
+**Note:** The set of aggregation functions will be extended in the future.
+
+### Reduce on full DataSet
+
+The Reduce transformation applies a user-defined `ReduceFunction` to all elements of a `DataSet`.
+The `ReduceFunction` subsequently combines pairs of elements into one element until only a single element remains.
+
+The following code shows how to sum all elements of an Integer `DataSet`:
+
+```java
+// ReduceFunction that sums Integers
+public class IntSummer extends ReduceFunction<Integer> {
+  @Override
+  public Integer reduce(Integer num1, Integer num2) {
+    return num1 + num2;
+  }
+}
+
+// [...]
+DataSet<Integer> intNumbers = // [...]
+DataSet<Integer> sum = intNumbers.reduce(new IntSummer());
+```
+
+Reducing a full `DataSet` using the Reduce transformation implies that the final Reduce operation cannot be done in parallel. However, a `ReduceFunction` is automatically combinable such that a Reduce transformation does not limit scalability for most use cases.
+
+### GroupReduce on full DataSet
+
+The GroupReduce transformation applies a user-defined `GroupReduceFunction` on all elements of a `DataSet`.
+A `GroupReduceFunction` can iterate over all elements of `DataSet` and return an arbitrary number of result elements.
+
+The following example shows how to apply a GroupReduce transformation on a full `DataSet`:
+
+```java
+DataSet<Integer> input = // [...]
+// apply a (preferably combinable) GroupReduceFunction to a DataSet
+DataSet<Double> output = input.reduceGroup(new MyGroupReducer());
+```
+
+**Note:** A GroupReduce transformation on a full `DataSet` cannot be done in parallel if the `GroupReduceFunction` is not combinable. Therefore, this can be a very compute intensive operation. See the paragraph on "Combineable `GroupReduceFunction`s" above to learn how to implement a combinable `GroupReduceFunction`.
+
+### Aggregate on full Tuple DataSet
+
+There are some common aggregation operations that are frequently used. The Aggregate transformation provides the following build-in aggregation functions:
+
+- Sum,
+- Min, and
+- Max.
+
+The Aggregate transformation can only be applied on a `Tuple` `DataSet`.
+
+The following code shows how to apply an Aggregation transformation on a full `DataSet`:
+
+```java
+DataSet<Tuple2<Integer, Double>> input = // [...]
+DataSet<Tuple2<Integer, Double>> output = input
+                                          // compute sum of the first field
+                                          .aggregate(SUM, 0)
+                                          // compute minimum of the second field
+                                          .and(MIN, 1);
+```
+
+**Note:** Extending the set of supported aggregation functions is on our roadmap.
+
+### Join
+
+The Join transformation joins two `DataSet`s into one `DataSet`. The elements of both `DataSet`s are joined on one or more keys which can be specified using
+
+- a `KeySelector` function or
+- one or more field position keys (`Tuple` `DataSet` only).
+
+There are a few different ways to perform a Join transformation which are shown in the following.
+
+#### Default Join (Join into Tuple2)
+
+The default Join transformation produces a new `Tuple``DataSet` with two fields. Each tuple holds a joined element of the first input `DataSet` in the first tuple field and a matching element of the second input `DataSet` in the second field.
+
+The following code shows a default Join transformation using field position keys:
+
+```java
+DataSet<Tuple2<Integer, String>> input1 = // [...]
+DataSet<Tuple2<Double, Integer>> input2 = // [...]
+// result dataset is typed as Tuple2
+DataSet<Tuple2<Tuple2<Integer, String>, Tuple2<Double, Integer>>>
+            result =
+            input1.join(input2)
+                  // key definition on first DataSet using a field position key
+                  .where(0)
+                  // key definition of second DataSet using a field position key
+                  .equalTo(1);
+```
+
+#### Join with JoinFunction
+
+A Join transformation can also call a user-defined `JoinFunction` to process joining tuples.
+A `JoinFunction` receives one element of the first input `DataSet` and one element of the second input `DataSet` and returns exactly one element.
+
+The following code performs a join of `DataSet` with custom java objects and a `Tuple` `DataSet` using `KeySelector` functions and shows how to call a user-defined `JoinFunction`:
+
+```java
+// some POJO
+public class Rating {
+  public String name;
+  public String category;
+  public int points;
+}
+
+// Join function that joins a custom POJO with a Tuple
+public class PointWeighter
+         extends JoinFunction<Rating, Tuple2<String, Double>, Tuple2<String, Double>> {
+
+  @Override
+  public Tuple2<String, Double> join(Rating rating, Tuple2<String, Double> weight) {
+    // multiply the points and rating and construct a new output tuple
+    return new Tuple2<String, Double>(rating.name, rating.points * weight.f1);
+  }
+}
+
+DataSet<Rating> ratings = // [...]
+DataSet<Tuple2<String, Double>> weights = // [...]
+DataSet<Tuple2<String, Double>>
+            weightedRatings =
+            ratings.join(weights)
+                   // key definition of first DataSet using a KeySelector function
+                   .where(new KeySelection<Rating, String>() {
+                            public String getKey(Rating r) { return r.category; }
+                          })
+                   // key definition of second DataSet using a KeySelector function
+                   .equalTo(new KeySelection<Tuple2<String, Double>, String>() {
+                              public String getKey(Tuple2<String, Double> t) { return t.f0; }
+                            })
+                   // applying the JoinFunction on joining pairs
+                   .with(new PointWeighter());
+```
+
+#### Join with Projection
+
+A Join transformation can construct result tuples using a projection as shown here:
+
+```java
+DataSet<Tuple3<Integer, Byte, String>> input1 = // [...]
+DataSet<Tuple2<Integer, Double>> input2 = // [...]
+DataSet<Tuple4<Integer, String, Double, Byte>
+            result =
+            input1.join(input2)
+                  // key definition on first DataSet using a field position key
+                  .where(0)
+                  // key definition of second DataSet using a field position key
+                  .equalTo(0)
+                  // select and reorder fields of matching tuples
+                  .projectFirst(0,2).projectSecond(1).projectFirst(1)
+                  .types(Integer.class, String.class, Double.class, Byte.class);
+```
+
+`projectFirst(int...)` and `projectSecond(int...)` select the fields of the first and second joined input that should be assembled into an output `Tuple`. The order of indexes defines the order of fields in the output tuple.
+The join projection works also for non-`Tuple` `DataSet`s. In this case, `projectFirst()` or `projectSecond()` must be called without arguments to add a joined element to the output `Tuple`.
+
+#### Join with DataSet Size Hint
+
+In order to guide the optimizer to pick the right execution strategy, you can hint the size of a `DataSet` to join as shown here:
+
+```java
+DataSet<Tuple2<Integer, String>> input1 = // [...]
+DataSet<Tuple2<Integer, String>> input2 = // [...]
+
+DataSet<Tuple2<Tuple2<Integer, String>, Tuple2<Integer, String>>>
+            result1 =
+            // hint that the second DataSet is very small
+            input1.joinWithTiny(input2)
+                  .where(0)
+                  .equalTo(0);
+
+DataSet<Tuple2<Tuple2<Integer, String>, Tuple2<Integer, String>>>
+            result2 =
+            // hint that the second DataSet is very large
+            input1.joinWithHuge(input2)
+                  .where(0)
+                  .equalTo(0);
+```
+
+### Cross
+
+The Cross transformation combines two `DataSet`s into one `DataSet`. It builds all pairwise combinations of the elements of both input `DataSet`s, i.e., it builds a Cartesian product.
+The Cross transformation either calls a user-defined `CrossFunction` on each pair of elements or applies a projection. Both modes are shown in the following.
+
+**Note:** Cross is potentially a *very* compute-intensive operation which can challenge even large compute clusters!
+
+#### Cross with User-Defined Function
+
+A Cross transformation can call a user-defined `CrossFunction`. A `CrossFunction` receives one element of the first input and one element of the second input and returns exactly one result element.
+
+The following code shows how to apply a Cross transformation on two `DataSet`s using a `CrossFunction`:
+
+```java
+public class Coord {
+  public int id;
+  public int x;
+  public int y;
+}
+
+// CrossFunction computes the Euclidean distance between two Coord objects.
+public class EuclideanDistComputer
+         extends CrossFunction<Coord, Coord, Tuple3<Integer, Integer, Double>> {
+
+  @Override
+  public Tuple3<Integer, Integer, Double> cross(Coord c1, Coord c2) {
+    // compute Euclidean distance of coordinates
+    double dist = Math.sqrt(Math.pow(c1.x - c2.x, 2) + Math.pow(c1.y - c2.y, 2));
+    return new Tuple3<Integer, Integer, Double>(c1.id, c2.id, dist);
+  }
+}
+
+DataSet<Coord> coords1 = // [...]
+DataSet<Coord> coords2 = // [...]
+DataSet<Tuple3<Integer, Integer, Double>>
+            distances =
+            coords1.cross(coords2)
+                   // apply CrossFunction
+                   .with(new EuclideanDistComputer());
+```
+
+#### Cross with Projection
+
+A Cross transformation can also construct result tuples using a projection as shown here:
+
+```java
+DataSet<Tuple3<Integer, Byte, String>> input1 = // [...]
+DataSet<Tuple2<Integer, Double>> input2 = // [...]
+DataSet<Tuple4<Integer, Byte, Integer, Double>
+            result =
+            input1.cross(input2)
+                  // select and reorder fields of matching tuples
+                  .projectSecond(0).projectFirst(1,0).projectSecond(1)
+                  .types(Integer.class, Byte.class, Integer.class, Double.class);
+```
+
+The field selection in a Cross projection works the same way as in the projection of Join results.
+
+#### Cross with DataSet Size Hint
+
+In order to guide the optimizer to pick the right execution strategy, you can hint the size of a `DataSet` to cross as shown here:
+
+```java
+DataSet<Tuple2<Integer, String>> input1 = // [...]
+DataSet<Tuple2<Integer, String>> input2 = // [...]
+
+DataSet<Tuple4<Integer, String, Integer, String>>
+            udfResult =
+                  // hint that the second DataSet is very small
+            input1.crossWithTiny(input2)
+                  // apply any Cross function (or projection)
+                  .with(new MyCrosser());
+
+DataSet<Tuple3<Integer, Integer, String>>
+            projectResult =
+                  // hint that the second DataSet is very large
+            input1.crossWithHuge(input2)
+                  // apply a projection (or any Cross function)
+                  .projectFirst(0,1).projectSecond(1).types(Integer.class, String.class, String.class)
+```
+
+### CoGroup
+
+The CoGroup transformation jointly processes groups of two `DataSet`s. Both `DataSet`s are grouped on a defined key and groups of both `DataSet`s that share the same key are handed together to a user-defined `CoGroupFunction`. If for a specific key only one `DataSet` has a group, the `CoGroupFunction` is called with this group and an empty group.
+A `CoGroupFunction` can separately iterate over the elements of both groups and return an arbitrary number of result elements.
+
+Similar to Reduce, GroupReduce, and Join, keys can be defined using
+
+- a `KeySelector` function or
+- one or more field position keys (`Tuple` `DataSet` only).
+
+#### CoGroup on DataSets grouped by Field Position Keys (Tuple DataSets only)
+
+```java
+// Some CoGroupFunction definition
+class MyCoGrouper
+         extends CoGroupFunction<Tuple2<String, Integer>, Tuple2<String, Double>, Double> {
+  // set to hold unique Integer values
+  Set<Integer> ints = new HashSet<Integer>();
+
+  @Override
+  public void coGroup(Iterator<Tuple2<String, Integer>> iVals,
+                      Iterator<Tuple2<String, Double>> dVals,
+                      Collector<Double> out) {
+    // clear Integer set
+    ints.clear();
+    // add all Integer values in group to set
+    while(iVals.hasNext()) {
+      ints.add(iVals.next().f1);
+    }
+    // multiply each Double value with each unique Integer values of group
+    while(dVals.hasNext()) {
+      for(Integer i : ints) {
+        out.collect(dVals.next().f1 * i));
+      }
+    }
+  }
+}
+
+// [...]
+DataSet<Tuple2<String, Integer>> iVals = // [...]
+DataSet<Tuple2<String, Double>> dVals = // [...]
+DataSet<Double> output = iVals.coGroup(dVals)
+                         // group first DataSet on first tuple field
+                         .where(0)
+                         // group second DataSet on first tuple field
+                         .equalTo(0)
+                         // apply CoGroup function on each pair of groups
+                         .with(new MyCoGrouper());
+```
+
+#### CoGroup on DataSets grouped by Key Selector Function
+
+Works analogous to key selector functions in Join transformations.
+
+### Union
+
+Produces the union of two `DataSet`s, which have to be of the same type. A union of more than two `DataSet`s can be implemented with multiple union calls, as shown here:
+
+```java
+DataSet<Tuple2<String, Integer>> vals1 = // [...]
+DataSet<Tuple2<String, Integer>> vals2 = // [...]
+DataSet<Tuple2<String, Integer>> vals3 = // [...]
+DataSet<Tuple2<String, Integer>> unioned = vals1.union(vals2)
+                    .union(vals3);
+```
+
+
+<div class="back-to-top"><a href="#toc">Back to top</a></div>
+</section>
+
+<section id="data_sources">
+Data Sources
+------------
+
+Data sources create the initial data sets, such as from files or from Java collections. The general mechanism of of creating data sets is abstracted behind an [InputFormat](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/InputFormat.java). Stratosphere comes with several built-in formats to create data sets from common file formats. Many of them have shortcut methods on the *ExecutionEnvironment*.
+
+File-based:
+
+- `readTextFile(path)` / `TextInputFormat` - Reads files line wise and returns them as Strings.
+- `readTextFileWithValue(path)` / `TextValueInputFormat` - Reads files line wise and returns them as StringValues. StringValues are mutable strings.
+- `readCsvFile(path)` / `CsvInputFormat` - Parses files of comma (or another char) delimited fields. Returns a DataSet of tuples. Supports the basic java types and their Value counterparts as field types.
+
+Collection-based:
+
+- `fromCollection(Collection)` - Creates a data set from the Java Java.util.Collection. All elements in the collection must be of the same type.
+- `fromCollection(Iterator, Class)` - Creates a data set from an iterator. The class specifies the data type of the elements returned by the iterator.
+- `fromElements(T ...)` - Creates a data set from the given sequence of objects. All objects must be of the same type.
+- `fromParallelCollection(SplittableIterator, Class)` - Creates a data set from an iterator, in parallel. The class specifies the data type of the elements returned by the iterator.
+- `generateSequence(from, to)` - Generates the squence of numbers in the given interval, in parallel.
+
+Generic:
+
+- `createInput(path)` / `InputFormat` - Accepts a generic input format.
+
+**Examples**
+
+```java
+ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+// read text file from local files system
+DataSet<String> localLines = env.readTextFile("file:///path/to/my/textfile");
+
+// read text file from a HDFS running at nnHost:nnPort
+DataSet<String> hdfsLines = env.readTextFile("hdfs://nnHost:nnPort/path/to/my/textfile");
+
+// read a CSV file with three fields
+DataSet<Tuple3<Integer, String, Double>> csvInput = env.readCsvFile("hdfs:///the/CSV/file")
+	                       .types(Integer.class, String.class, Double.class);
+
+// read a CSV file with five fields, taking only two of them
+DataSet<Tuple2<String, Double>> csvInput = env.readCsvFile("hdfs:///the/CSV/file")
+                               .includeFields("10010")  // take the first and the fourth fild
+	                       .types(String.class, Double.class);
+
+// create a set from some given elements
+DataSet<String> value = env.fromElements("Foo", "bar", "foobar", "fubar");
+
+// generate a number sequence
+DataSet<Long> numbers = env.generateSequence(1, 10000000);
+
+// Read data from a relational database using the JDBC input format
+DataSet<Tuple2<String, Integer> dbData = 
+    env.createInput(
+      // create and configure input format
+      JDBCInputFormat.buildJDBCInputFormat()
+                     .setDrivername("org.apache.derby.jdbc.EmbeddedDriver")
+                     .setDBUrl("jdbc:derby:memory:persons")
+                     .setQuery("select name, age from persons")
+                     .finish(),
+      // specify type information for DataSet
+      new TupleTypeInfo(Tuple2.class, STRING_TYPE_INFO, INT_TYPE_INFO)
+    );
+
+// Note: Stratosphere's program compiler needs to infer the data types of the data items which are returned by an InputFormat. If this information cannot be automatically inferred, it is necessary to manually provide the type information as shown in the examples above.
+```
+
+<div class="back-to-top"><a href="#toc">Back to top</a></div>
+</section>
+
+<section id="data_sinks">
+Data Sinks
+----------
+
+Data sinks consume DataSets and are used to store or return them. Data sink operations are described using an [OutputFormat](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/OutputFormat.java). Stratosphere comes with a variety of built-in output formats that
+are encapsulated behind operations on the DataSet type:
+
+- `writeAsText()` / `TextOuputFormat` - Writes for each element as a String in a line. The String are obtained by calling the *toString()* method.
+- `writeAsCsv` / `CsvOutputFormat` - Writes tuples as comma-separated value files. Row and field delimiters are configurable. The value for each field comes from the *toString()* method of the objects.
+- `print()` / `printToErr()` - Prints the *toString()* value of each element on the standard out / strandard error stream.
+- `write()` / `FileOutputFormat` - Method and base class for custom file outputs. Supports custom object-to-bytes conversion.
+- `output()`/ `OutputFormat` - Most generic output method, for data sinks that are not file based (such as storing the result in a database).
+
+A DataSet can be input to multiple operations. Programs can write or print a data set and at the same time run additional transformations on them.
+
+**Examples**
+
+Standard data sink methods:
+
+```java
+// text data 
+DataSet<String> textData = // [...]
+
+// write DataSet to a file on the local file system
+textData.writeAsText("file:///my/result/on/localFS");
+
+// write DataSet to a file on a HDFS with a namenode running at nnHost:nnPort
+textData.writeAsText("hdfs://nnHost:nnPort/my/result/on/localFS");
+
+// write DataSet to a file and overwrite the file if it exists
+textData.writeAsText("file:///my/result/on/localFS", WriteMode.OVERWRITE);
+
+// tuples as lines with pipe as the separator "a|b|c"
+DataSet<Tuple3<String, Integer, Double>> values = // [...]
+values.writeAsCsv("file:///path/to/the/result/file", "\n", "|");
+
+// this writes tuples in the text formatting "(a, b, c)", rather than as CSV lines
+value.writeAsText("file:///path/to/the/result/file");
+```
+
+Using a custom output format:
+
+```java
+DataSet<Tuple3<String, Integer, Double>> myResult = [...]
+
+// write Tuple DataSet to a relational database
+myResult.output(
+    // build and configure OutputFormat
+    JDBCOutputFormat.buildJDBCOutputFormat()
+                    .setDrivername("org.apache.derby.jdbc.EmbeddedDriver")
+                    .setDBUrl("jdbc:derby:memory:persons")
+                    .setQuery("insert into persons (name, age, height) values (?,?,?)")
+                    .finish()
+    );
+```
+
+<div class="back-to-top"><a href="#toc">Back to top</a></div>
+</section>
+
+<section id="debugging">
+Debugging
+---------
+
+Before running a data analysis program on a large data set in a distributed cluster, it is a good idea to make sure that the implemented algorithm works as desired. Hence, implementing data analysis programs is usually an incremental process of checking results, debugging, and improving. 
+
+<p>
+Stratosphere provides a few nice features to significantly ease the development process of data analysis programs by supporting local debugging from within an IDE, injection of test data, and collection of result data. This section give some hints how to ease the development of Stratosphere programs.
+</p>
+
+### Local Execution Environment
+
+A `LocalEnvironment` starts a Stratosphere system within the same JVM process it was created in. If you start the LocalEnvironement from an IDE, you can set breakpoint in your code and easily debug your program. 
+
+<p>
+A LocalEnvironment is created and used as follows:
+</p>
+
+```java
+final ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
+
+DataSet<String> lines = env.readTextFile(pathToTextFile);
+// build your program
+
+env.execute();
+
+```
+
+### Collection Data Sources and Sinks
+
+Providing input for an analysis program and checking its output is cumbersome done by creating input files and reading output files. Stratosphere features special data sources and sinks which are backed by Java collections to ease testing. Once a program has been tested, the sources and sinks can be easily replaced by sources and sinks that read from / write to external data stores such as HDFS.
+
+<p>
+Collection data sources can be used as follows:
+</p>
+
+```java
+final ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
+
+// Create a DataSet from a list of elements
+DataSet<Integer> myInts = env.fromElements(1, 2, 3, 4, 5);
+
+// Create a DataSet from any Java collection
+List<Tuple2<String, Integer>> data = ...
+DataSet<Tuple2<String, Integer>> myTuples = env.fromCollection(data);
+
+// Create a DataSet from an Iterator
+Iterator<Long> longIt = ...
+DataSet<Long> myLongs = env.fromCollection(longIt, Long.class);
+```
+
+**Note:** Currently, the collection data source requires that data types and iterators implement `Serializable`. Furthermore, collection data sources can not be executed in parallel (degree of parallelism = 1).
+
+<p>
+A collection data sink is specified as follows:
+</p>
+
+```java
+DataSet<Tuple2<String, Integer>> myResult = ...
+
+List<Tuple2<String, Integer>> outData = new ArrayList<Tuple2<String, Integer>>();
+myResult.output(new LocalCollectionOutputFormat(outData));
+```
+
+**Note:** Collection data sources will only work correctly, if the whole program is executed in the same JVM!
+
+<div class="back-to-top"><a href="#toc">Back to top</a></div>
+</section>
+
+<section id="iterations">
+Iteration Operators
+-------------------
+
+Iterations implement loops in Stratosphere programs. The iteration operators encapsulate a part of the program and execute it repeatedly, feeding back the result of one iteration (the partial solution) into the next iteration. There are two types of iterations in Stratosphere: **BulkIteration** and **DeltaIteration**.
+
+This section provides quick examples on how to use both operators. Check out the [Introduction to Iterations]({{site.baseurl}}/docs/0.5/programming_guides/iterations.html) page for a more detailed introduction.
+
+#### Bulk Iterations
+
+To create a BulkIteration call the `iterate(int)` method of the `DataSet` the iteration should start at. This will return an `IterativeDataSet`, which can be transformed with the regular operators. The single argument to the iterate call specifies the maximum number of iterations.
+
+To specify the end of an iteration call the `closeWith(DataSet)` method on the `IterativeDataSet` to specify which transformation should be fed back to the next iteration. You can optionally specify a termination criterion with `closeWith(DataSet, DataSet)`, which evaluates the second DataSet and terminates the iteration, if this DataSet is empty. If no termination criterion is specified, the iteration terminates after the given maximum number iterations.
+
+The following example iteratively estimates the number Pi. The goal is to count the number of random points, which fall into the unit circle. In each iteration, a random point is picked. If this point lies inside the unit circle, we increment the count. Pi is then estimated as the resulting count divided by the number of iterations multiplied by 4.
+
+{% highlight java %}
+final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+// Create initial IterativeDataSet
+IterativeDataSet<Integer> initial = env.fromElements(0).iterate(10000);
+
+DataSet<Integer> iteration = initial.map(new MapFunction<Integer, Integer>() {
+    @Override
+    public Integer map(Integer i) throws Exception {
+        double x = Math.random();
+        double y = Math.random();
+
+        return i + ((x * x + y * y < 1) ? 1 : 0);
+    }
+});
+
+// Iteratively transform the IterativeDataSet
+DataSet<Integer> count = initial.closeWith(iteration);
+
+count.map(new MapFunction<Integer, Double>() {
+    @Override
+    public Double map(Integer count) throws Exception {
+        return count / (double) 10000 * 4;
+    }
+}).print();
+
+env.execute("Iterative Pi Example");
+{% endhighlight %}
+
+You can also check out the [K-Means example](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/KMeans.java), which uses a BulkIteration to cluster a set of unlabeled points.
+
+#### Delta Iterations
+
+Delta iterations exploit the fact that certain algorithms do not change every data point of the solution in each iteration.
+
+In addition to the partial solution that is fed back (called workset) in every iteration, delta iterations maintain state across iterations (called solution set), which can be updated through deltas. The result of the iterative computation is the state after the last iteration. Please refer to the [Introduction to Iterations]({{site.baseurl}}/docs/0.5/programming_guides/iterations.html) for an overview of the basic principle of delta iterations.
+
+Defining a DeltaIteration is similar to defining a BulkIteration. For delta iterations, two data sets form the input to each iteration (workset and solution set), and two data sets are produced as the result (new workset, solution set delta) in each iteration.
+
+To create a DeltaIteration call the `iterateDelta(DataSet, int, int)` (or `iterateDelta(DataSet, int, int[])` respectively). This method is called on the initial solution set. The arguments are the initial delta set, the maximum number of iterations and the key positions. The returned `DeltaIteration` object gives you access to the DataSets representing the workset and solution set via the methods `iteration.getWorket()` and `iteration.getSolutionSet()`.
+
+Below is an example for the syntax of a delta iteration
+
+```java
+// read the initial data sets
+DataSet<Tuple2<Long, Double>> initialSolutionSet = // [...]
+
+DataSet<Tuple2<Long, Double>> initialDeltaSet = // [...]
+
+int maxIterations = 100;
+int keyPosition = 0;
+
+DeltaIteration<Tuple2<Long, Double>, Tuple2<Long, Double>> iteration = initialSolutionSet
+    .iterateDelta(initialDeltaSet, maxIterations, keyPosition);
+
+DataSet<Tuple2<Long, Double>> candidateUpdates = iteration.getWorkset()
+    .groupBy(1)
+    .reduceGroup(new ComputeCandidateChanges());
+
+DataSet<Tuple2<Long, Double>> deltas = candidateUpdates
+    .join(iteration.getSolutionSet())
+    .where(0)
+    .equalTo(0)
+    .with(new CompareChangesToCurrent());
+
+DataSet<Tuple2<Long, Double>> nextWorkset = deltas
+    .filter(new FilterByThreshold());
+
+iteration.closeWith(deltas, nextWorkset)
+	.writeAsCsv(outputPath);
+```
+
+<div class="back-to-top"><a href="#toc">Back to top</a></div>
+</section>
+
+<section id="annotations">
+Semantic Annotations
+-----------
+
+Semantic Annotations give hints about the behavior of a function by telling the system which fields in the input are accessed and which are constant between input and output data of a function (copied but not modified). Semantic annotations are a powerful means to speed up execution, because they allow the system to reason about reusing sort orders or partitions across multiple operations. Using semantic annotations may eventually save the program from unnecessary data shuffling or unnecessary sorts.
+
+Semantic annotations can be attached to functions through Java annotations, or passed as arguments when invoking a function on a DataSet. The following example illustrates that:
+
+```java
+@ConstantFields("1")
+public class DivideFirstbyTwo extends MapFunction<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> {
+  @Override
+  public Tuple2<Integer, Integer> map(Tuple2<Integer, Integer> value) {
+    value.f0 /= 2;
+    return value;
+  }
+}
+```
+
+The following annotations are currently available:
+
+* `@ConstantFields`: Declares constant fields (forwarded/copied) for functions with a single input data set (Map, Reduce, Filter, ...).
+
+* `@ConstantFieldsFirst`: Declares constant fields (forwarded/copied) for functions with a two input data sets (Join, CoGroup, ...), with respect to the first input data set.
+
+* `@ConstantFieldsSecond`: Declares constant fields (forwarded/copied) for functions with a two input data sets (Join, CoGroup, ...), with respect to the first second data set.
+
+* `@ConstantFieldsExcept`: Declares that all fields are constant, except for the specified fields. Applicable to functions with a single input data set.
+
+* `@ConstantFieldsFirstExcept`: Declares that all fields of the first input are constant, except for the specified fields. Applicable to functions with a two input data sets.
+
+* `@ConstantFieldsSecondExcept`: Declares that all fields of the second input are constant, except for the specified fields. Applicable to functions with a two input data sets.
+
+*(Note: The system currently evaluated annotations only on Tuple data types. This will be extended in the next versions)*
+
+**Note**: It is important to be conservative when providing annotations. Only annotate fields, when they are always constant for every call to the function. Otherwise the system has incorrect assumptions about the execution and the execution may produce wrong results. If the behavior of the operator is not clearly predictable, no annotation should be provided.
+
+<div class="back-to-top"><a href="#toc">Back to top</a></div>
+</section>
+
+<section id="broadcast_variables">
+Broadcast Variables
+-------------------
+
+Broadcast variables allow you to make a data set available to all parallel instances of an operation, in addition to the regular input of the operation. This is useful
+for auxiliary data sets, or data-dependent parameterization. The data set will then be accessible at the operator as an `Collection<T>`.
+
+- **Broadcast**: broadcast sets are registered by name via `withBroadcastSet(DataSet, String)`, and
+- **Access**: accessible via `getRuntimeContext().getBroadcastVariable(String)` at the target operator.
+
+```java
+// 1. The DataSet to be broadcasted
+DataSet<Integer> toBroadcast = env.fromElements(1, 2, 3);
+
+DataSet<String> data = env.fromElements("a", "b");
+
+data.map(new MapFunction<String, String>() {
+    @Override
+    public void open(Configuration parameters) throws Exception {
+      // 3. Access the broadcasted DataSet as a Collection
+      Collection<Integer> broadcastSet = getRuntimeContext().getBroadcastVariable("broadcastSetName");
+    }
+
+
+    @Override
+    public String map(String value) throws Exception {
+        ...
+    }
+}).withBroadcastSet(toBroadcast, "broadcastSetName"); // 2. Broadcast the DataSet
+```
+
+Make sure that the names (`broadcastSetName` in the previous example) match when registering and accessing broadcasted data sets. For a complete example program, have a look at [KMeans Algorithm](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/KMeans.java#L96).
+
+**Note**: As the content of broadcast variables is kept in-memory on each node, it should not become too large. For simpler things like scalar values you can simply make parameters part of the closure of a function, or use the `withParameters(...)` method to pass in a configuration.
+
+<div class="back-to-top"><a href="#toc">Back to top</a></div>
+</section>
+
+<section id="packaging">
+Program Packaging & Distributed Execution
+-----------------------------------------
+
+As described in the [program skeleton](#skeleton) section, Stratosphere programs can be executed on clusters by using the `RemoteEnvironment`. Alternatively, programs can be packaged into JAR Files (Java Archives) for execution. Packaging the program is a prerequisite to executing them through the [command line interface]({{ site.baseurl }}/docs/0.5/program_execution/cli_client.html) or the [web interface]({{ site.baseurl }}/docs/0.5/program_execution/web_interface.html).
+
+#### Packaging Programs
+
+To support execution from a packaged JAR file via the command line or web interface, a program must use the environment obtained by `ExecutionEnvironment.getExecutionEnvironment()`. This environment will act as the cluster's environment when the JAR is submitted to the command line or web interface. If the Stratosphere program is invoked differently than through these interfaces, the environment will act like a local environment.
+
+To package the program, simply export all involved classes as a JAR file. The JAR file's manifest must point to the class that contains the program's *entry point* (the class with the `public void main(String[])` method). The simplest way to do this is by putting the *main-class* entry into the manifest (such as `main-class: eu.stratosphere.example.MyProgram`). The *main-class* attribute is the same one that is used by the Java Virtual Machine to find the main method when executing a JAR files through the command `java -jar pathToTheJarFile`. Most IDEs offer to include that attribute automatically when exporting JAR files.
+
+
+#### Packaging Programs through Plans
+
+Additionally, the Java API supports packaging programs as *Plans*. This method resembles the way that the *Scala API* packages programs. Instead of defining a progam in the main method and calling `execute()` on the environment, plan packaging returns the *Program Plan*, which is a description of the program's data flow. To do that, the program must implement the `eu.stratosphere.api.common.Program` interface, defining the `getPlan(String...)` method. The strings passed to that method are the command line arguments. The program's plan can be created from the environment via the `ExecutionEnvironment#createProgramPlan()` method. When packaging the program's plan, the JAR manifest must point to the class implementing the `eu.stratosphere.api.common.Program` interface, instead of the class with the main method.
+
+
+#### Summary
+
+The overall procedure to invoke a packaged program is as follows:
+
+  1. The JAR's manifest is searched for a *main-class* or *program-class* attribute. If both attributes are found, the *program-class* attribute takes precedence over the *main-class* attribute. Both the command line and the web interface support a parameter to pass the entry point class name manually for cases where the JAR manifest contains neither attribute.
+  2. If the entry point class implements the `eu.stratosphere.api.common.Program`, then the system calls the `getPlan(String...)` method to obtain the program plan to execute. The `getPlan(String...)` method was the only possible way of defining a program in the *Record API* (see [0.4 docs]({{ site.baseurl }}/docs/0.4/)) and is also supported in the new Java API.
+  3. If the entry point class does not implement the `eu.stratosphere.api.common.Program` interface, the system will invoke the main method of the class.
+
+<div class="back-to-top"><a href="#toc">Back to top</a></div>
+</section>
+
+<section id="accumulators_counters">
+Accumulators &amp; Counters
+---------------------------
+
+Accumulators are simple constructs with an **add operation** and a **final accumulated result**, which is available after the job ended.
+
+The most straightforward accumulator is a **counter**: You can increment it using the ```Accumulator.add(V value)``` method. At the end of the job Stratosphere will sum up (merge) all partial results and send the result to the client. Since accumulators are very easy to use, they can be useful during debugging or if you quickly want to find out more about your data.
+
+Stratosphere currently has the following **built-in accumulators**. Each of them implements the [Accumulator](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-core/src/main/java/eu/stratosphere/api/common/accumulators/Accumulator.java) interface.
+
+- [__IntCounter__](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-core/src/main/java/eu/stratosphere/api/common/accumulators/IntCounter.java), [__LongCounter__](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-core/src/main/java/eu/stratosphere/api/common/accumulators/LongCounter.java) and [__DoubleCounter__](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-core/src/main/java/eu/stratosphere/api/common/accumulators/DoubleCounter.java): See below for an example using a counter.
+- [__Histogram__](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-core/src/main/java/eu/stratosphere/api/common/accumulators/Histogram.java): A histogram implementation for a discrete number of bins. Internally it is just a map from Integer to Integer. You can use this to compute distributions of values, e.g. the distribution of words-per-line for a word count program.
+
+__How to use accumulators:__
+
+First you have to create an accumulator object (here a counter) in the operator function where you want to use it. Operator function here refers to the (anonymous inner)
+class implementing the user defined code for an operator.
+
+    private IntCounter numLines = new IntCounter();
+
+Second you have to register the accumulator object, typically in the ```open()``` method of the operator function. Here you also define the name.
+
+    getRuntimeContext().addAccumulator("num-lines", this.numLines);
+
+You can now use the accumulator anywhere in the operator function, including in the ```open()``` and ```close()``` methods.
+
+    this.numLines.add(1);
+
+The overall result will be stored in the ```JobExecutionResult``` object which is returned when running a job using the Java API (currently this only works if the execution waits for the completion of the job).
+
+    myJobExecutionResult.getAccumulatorResult("num-lines")
+
+All accumulators share a single namespace per job. Thus you can use the same accumulator in different operator functions of your job. Stratosphere will internally merge all accumulators with the same name.
+
+A note on accumulators and iterations: Currently the result of accumulators is only available after the overall job ended. We plan to also make the result of the previous iteration available in the next iteration. You can use [Aggregators](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-java/src/main/java/eu/stratosphere/api/java/IterativeDataSet.java#L98) to compute per-iteration statistics and base the termination of iterations on such statistics.
+
+__Custom accumulators:__
+
+To implement your own accumulator you simply have to write your implementation of the Accumulator interface. Feel free to create a pull request if you think your custom accumulator should be shipped with Stratosphere.
+
+You have the choice to implement either [Accumulator](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-core/src/main/java/eu/stratosphere/api/common/accumulators/Accumulator.java) or [SimpleAccumulator](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-core/src/main/java/eu/stratosphere/api/common/accumulators/SimpleAccumulator.java). ```Accumulator<V,R>``` is most flexible: It defines a type ```V``` for the value to add, and a result type ```R``` for the final result. E.g. for a histogram, ```V``` is a number and ```R``` is a histogram. ```SimpleAccumulator``` is for the cases where both types are the same, e.g. for counters.
+
+<div class="back-to-top"><a href="#toc">Back to top</a></div>
+</section>
+
+<section id="execution_plan">
+Execution Plans
+---------------
+
+Depending on various parameters such as data size or number of machines in the cluster, Stratosphere's optimizer automatically chooses an execution strategy for your program. In many cases, it can be useful to know how exactly Stratosphere will execute your program.
+
+__Plan Visualization Tool__
+
+Stratosphere 0.5 comes packaged with a visualization tool for execution plans. The HTML document containing the visualizer is located under ```tools/planVisualizer.html```. It takes a JSON representation of the job execution plan and visualizes it as a graph with complete annotations of execution strategies.
+
+The following code shows how to print the execution plan JSON from your program:
+
+    final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+    ...
+
+    System.out.println(env.getExecutionPlan());
+
+
+To visualize the execution plan, do the following:
+
+1. **Open** ```planVisualizer.html``` with your web browser,
+2. **Paste** the JSON string into the text field, and
+3. **Press** the draw button.
+
+After these steps, a detailed execution plan will be visualized.
+
+![alt text](http://stratosphere.eu/img/blog/plan_visualizer2.png "A stratosphere job execution graph.")
+
+__Web Interface__
+
+Stratosphere offers a web interface for submitting and executing jobs. If you choose to use this interface to submit your packaged program, you have the option to also see the plan visualization.
+
+The script to start the webinterface is located under ```bin/start-webclient.sh```. After starting the webclient (per default on **port 8080**), your program can be uploaded and will be added to the list of available programs on the left side of the interface.
+
+You are able to specify program arguments in the textbox at the bottom of the page. Checking the plan visualization checkbox shows the execution plan before executing the actual program.
+
+<div class="back-to-top"><a href="#toc">Back to top</a></div>
+</section>


[26/53] [abbrv] [FLINK-962] Initial import of documentation from website into source code (closes #34)

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/map.svg
----------------------------------------------------------------------
diff --git a/docs/img/map.svg b/docs/img/map.svg
new file mode 100644
index 0000000..28fffed
--- /dev/null
+++ b/docs/img/map.svg
@@ -0,0 +1,295 @@
+<?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"
+   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+   width="800"
+   height="143.05"
+   id="svg35289"
+   version="1.1"
+   inkscape:version="0.48.3.1 r9886"
+   sodipodi:docname="New document 124">
+  <defs
+     id="defs35291">
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35250"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35252"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35254"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35256"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35258"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35260"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+  </defs>
+  <sodipodi:namedview
+     id="base"
+     pagecolor="#ffffff"
+     bordercolor="#666666"
+     borderopacity="1.0"
+     inkscape:pageopacity="0.0"
+     inkscape:pageshadow="2"
+     inkscape:zoom="0.49497475"
+     inkscape:cx="-815.60294"
+     inkscape:cy="64.274519"
+     inkscape:document-units="px"
+     inkscape:current-layer="layer1"
+     showgrid="false"
+     fit-margin-top="0"
+     fit-margin-left="0"
+     fit-margin-right="0"
+     fit-margin-bottom="0"
+     inkscape:window-width="1916"
+     inkscape:window-height="570"
+     inkscape:window-x="1400"
+     inkscape:window-y="609"
+     inkscape:window-maximized="0" />
+  <metadata
+     id="metadata35294">
+    <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>
+  <g
+     inkscape:label="Layer 1"
+     inkscape:groupmode="layer"
+     id="layer1"
+     transform="translate(-242.76955,-215.47765)">
+    <g
+       id="g35210"
+       transform="translate(2485.7696,-422.90351)">
+      <g
+         id="g33877">
+        <rect
+           style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.50000024;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.50000041, 4.50000041;stroke-dashoffset:0"
+           id="rect2994-3-8-0-9-2-8"
+           width="57.498154"
+           height="28.746929"
+           x="-1821.1605"
+           y="676.76147" />
+        <rect
+           style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.50000024;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.50000041, 4.50000041;stroke-dashoffset:0"
+           id="rect2994-3-8-0-9-2-6"
+           width="57.498154"
+           height="28.746929"
+           x="-1821.1605"
+           y="643.2868" />
+        <rect
+           style="opacity:0.95999995;fill:#f3f3f3;fill-opacity:1;stroke:#000000;stroke-width:1.50000024;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:25"
+           id="rect2994-5-8-9"
+           width="57.498154"
+           height="136.54524"
+           x="-1922.3376"
+           y="641.63354" />
+        <rect
+           style="opacity:0.95999995;fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-0-7-4"
+           width="20.371059"
+           height="20.622461"
+           x="-1913.9596"
+           y="647.34906" />
+        <rect
+           style="opacity:0.95999995;fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-5-6-6"
+           width="20.371059"
+           height="20.622461"
+           x="-1893.5885"
+           y="647.34906" />
+        <rect
+           style="opacity:0.95999995;fill:#f3de4a;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-1-46-7-1"
+           width="20.371059"
+           height="20.622461"
+           x="-1913.9596"
+           y="680.82367" />
+        <rect
+           style="opacity:0.95999995;fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-0-9-3-9"
+           width="20.371059"
+           height="20.622461"
+           x="-1893.5885"
+           y="680.82367" />
+        <rect
+           style="opacity:0.95999995;fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-4-2-5-4"
+           width="20.371059"
+           height="20.622461"
+           x="-1913.9596"
+           y="716.60101" />
+        <rect
+           style="opacity:0.95999995;fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-4-4-4-0"
+           width="20.371059"
+           height="20.622461"
+           x="-1893.5885"
+           y="716.60101" />
+        <rect
+           style="opacity:0.95999995;fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-1-4-75-5-3"
+           width="20.371059"
+           height="20.622461"
+           x="-1913.9596"
+           y="752.37817" />
+        <rect
+           style="opacity:0.95999995;fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-0-7-4-7-3"
+           width="20.371059"
+           height="20.622461"
+           x="-1893.5885"
+           y="752.37817" />
+        <rect
+           style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.50000024;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.50000041, 4.50000041;stroke-dashoffset:0"
+           id="rect2994-3-8-0-9-2"
+           width="57.498154"
+           height="28.746929"
+           x="-1821.1605"
+           y="712.60596" />
+        <rect
+           style="opacity:0.95999995;fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-1-7-2-6-4"
+           width="20.371059"
+           height="20.622461"
+           x="-1812.7825"
+           y="716.75659" />
+        <rect
+           style="opacity:0.95999995;fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-0-4-1-5-6"
+           width="20.371059"
+           height="20.622461"
+           x="-1792.4115"
+           y="716.75659" />
+        <rect
+           style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.50000024;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.50000043, 4.50000043;stroke-dashoffset:0"
+           id="rect2994-3-3-8-3-4"
+           width="57.498154"
+           height="29.100485"
+           x="-1821.1605"
+           y="747.47668" />
+        <rect
+           style="opacity:0.95999995;fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-4-8-1-8-3"
+           width="20.371059"
+           height="20.622461"
+           x="-1812.7825"
+           y="751.4505" />
+        <rect
+           style="opacity:0.95999995;fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-4-5-0-3-5"
+           width="20.371059"
+           height="20.622461"
+           x="-1792.4115"
+           y="751.4505" />
+        <rect
+           style="opacity:0.95999995;fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-7-8-9-5"
+           width="20.371059"
+           height="20.622461"
+           x="-1812.7825"
+           y="647.349" />
+        <rect
+           style="opacity:0.95999995;fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-8-9-9-6"
+           width="20.371059"
+           height="20.622461"
+           x="-1792.4115"
+           y="647.349" />
+        <rect
+           style="opacity:0.95999995;fill:#f3de4a;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-1-4-7-6-7-8"
+           width="20.371059"
+           height="20.622461"
+           x="-1812.7825"
+           y="680.82367" />
+        <rect
+           style="opacity:0.95999995;fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-0-7-5-4-1-6"
+           width="20.371059"
+           height="20.622461"
+           x="-1792.4115"
+           y="680.82367" />
+        <path
+           style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.50000024;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)"
+           d="m -1873.2175,657.6603 51.9673,0"
+           id="path4066-2-4-6"
+           sodipodi:nodetypes="cc"
+           inkscape:connector-curvature="0" />
+        <path
+           inkscape:connector-curvature="0"
+           sodipodi:nodetypes="cc"
+           id="path4066-2-4-6-3"
+           d="m -1873.2175,691.16137 51.9674,0"
+           style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.50000024;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+        <path
+           inkscape:connector-curvature="0"
+           sodipodi:nodetypes="cc"
+           id="path4066-2-4-6-4"
+           d="m -1873.2175,726.93859 51.9674,0"
+           style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.50000024;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+        <path
+           inkscape:connector-curvature="0"
+           sodipodi:nodetypes="cc"
+           id="path4066-2-4-6-6"
+           d="m -1873.2175,762.71582 51.9674,0"
+           style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.50000024;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      </g>
+    </g>
+  </g>
+</svg>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/operator.svg
----------------------------------------------------------------------
diff --git a/docs/img/operator.svg b/docs/img/operator.svg
new file mode 100644
index 0000000..88379ec
--- /dev/null
+++ b/docs/img/operator.svg
@@ -0,0 +1,241 @@
+<?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"
+   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+   width="800"
+   height="170"
+   id="svg6946"
+   version="1.1"
+   inkscape:version="0.48.3.1 r9886"
+   sodipodi:docname="operator.svg">
+  <defs
+     id="defs6948">
+    <marker
+       inkscape:stockid="Arrow1Mend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mend"
+       style="overflow:visible">
+      <path
+         id="path3864"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(-0.4,0,0,-0.4,-4,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+  </defs>
+  <sodipodi:namedview
+     id="base"
+     pagecolor="#ffffff"
+     bordercolor="#666666"
+     borderopacity="1.0"
+     inkscape:pageopacity="0.0"
+     inkscape:pageshadow="2"
+     inkscape:zoom="2.5452459"
+     inkscape:cx="388.74049"
+     inkscape:cy="101.23661"
+     inkscape:document-units="px"
+     inkscape:current-layer="layer1"
+     showgrid="false"
+     fit-margin-top="0"
+     fit-margin-left="0"
+     fit-margin-right="0"
+     fit-margin-bottom="0"
+     inkscape:window-width="1916"
+     inkscape:window-height="1161"
+     inkscape:window-x="1400"
+     inkscape:window-y="18"
+     inkscape:window-maximized="0" />
+  <metadata
+     id="metadata6951">
+    <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>
+  <g
+     inkscape:label="Layer 1"
+     inkscape:groupmode="layer"
+     id="layer1"
+     transform="translate(-127.12556,-140.3642)">
+    <g
+       id="g8234"
+       transform="translate(181.30051,-1.8278624)">
+      <g
+         id="g7569">
+        <rect
+           style="fill:#a7dbd8;fill-opacity:1;stroke:#333333;stroke-width:1.99999976;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           id="rect7007"
+           width="271.42856"
+           height="134.01833"
+           x="206.42857"
+           y="145.01993" />
+        <text
+           xml:space="preserve"
+           style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+           x="338.51025"
+           y="270.93359"
+           id="text7535"
+           sodipodi:linespacing="125%"><tspan
+             sodipodi:role="line"
+             x="338.51025"
+             y="270.93359"
+             id="tspan7539">Operator Function</tspan></text>
+      </g>
+      <g
+         transform="translate(2.7779195,-6.8185297)"
+         id="g7564">
+        <rect
+           style="fill:#4ecdc4;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           id="rect7007-0"
+           width="169.28571"
+           height="88.571434"
+           x="293.57141"
+           y="163.32396" />
+        <text
+           xml:space="preserve"
+           style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+           x="299.37442"
+           y="244.45761"
+           id="text7535-3"
+           sodipodi:linespacing="125%"><tspan
+             sodipodi:role="line"
+             x="299.37442"
+             y="244.45761"
+             id="tspan7539-2">User-defined Function</tspan></text>
+      </g>
+      <path
+         inkscape:connector-curvature="0"
+         id="rect7760"
+         d="m 158.27381,149.96659 0,25.28125 -26.71577,0 0,73.5625 26.71577,0 0,25.28125 36.08334,-62.0625 -36.08334,-62.0625 z"
+         style="fill:#b4a7d6;fill-opacity:1;stroke:#333333;stroke-width:1.99999988000000006;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0" />
+      <text
+         xml:space="preserve"
+         style="font-size:48px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+         x="162.55525"
+         y="287.18451"
+         id="text3845-0-1"
+         sodipodi:linespacing="125%"><tspan
+           sodipodi:role="line"
+           id="tspan3847-6-2"
+           x="162.55525"
+           y="287.18451"
+           style="font-size:16px;text-align:center;text-anchor:middle" /><tspan
+           sodipodi:role="line"
+           x="162.55525"
+           y="307.18451"
+           style="font-size:16px;text-align:center;text-anchor:middle"
+           id="tspan8170">Input Data</tspan></text>
+      <path
+         sodipodi:nodetypes="cccccccc"
+         inkscape:connector-curvature="0"
+         id="rect7760-2"
+         d="m 247.45907,159.58539 0,5.24456 -26.71577,0 0,6.85017 26.71577,0 0,4.88742 36.08334,-8.49108 z"
+         style="fill:#b4a7d6;fill-opacity:1;stroke:#333333;stroke-width:1.99999988000000006;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0" />
+      <path
+         sodipodi:nodetypes="cccccccc"
+         inkscape:connector-curvature="0"
+         id="rect7760-2-3"
+         d="m 247.45907,185.29972 0,5.24456 -26.71577,0 0,6.85017 26.71577,0 0,4.88742 36.08334,-8.49108 z"
+         style="fill:#b4a7d6;fill-opacity:1;stroke:#333333;stroke-width:1.99999988000000006;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0" />
+      <path
+         sodipodi:nodetypes="cccccccc"
+         inkscape:connector-curvature="0"
+         id="rect7760-2-2"
+         d="m 247.45907,211.01397 0,5.24456 -26.71577,0 0,6.85017 26.71577,0 0,4.88742 36.08334,-8.49108 z"
+         style="fill:#b4a7d6;fill-opacity:1;stroke:#333333;stroke-width:1.99999988000000006;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0" />
+      <g
+         transform="translate(8.8900712,-1.6668884)"
+         id="g8092">
+        <path
+           style="fill:#f3b33f;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           d="m 344.01749,180.66459 c -1.00984,0 -2.00307,0.0799 -2.97054,0.2303 l 0,2.95759 c -1.30309,0.241 -2.55224,0.63153 -3.72535,1.16364 l -1.4731,-2.53334 c -1.55054,0.73183 -2.98661,1.67422 -4.2732,2.77576 l 1.49745,2.56971 c -0.89425,0.81963 -1.6956,1.73909 -2.38618,2.7394 l -2.54444,-1.46667 c -0.94054,1.39727 -1.69549,2.93147 -2.24008,4.55759 l 2.60531,1.49092 c -0.30383,1.03528 -0.50752,2.11403 -0.59654,3.22425 l -2.93402,0 c -0.029,0.42867 -0.0487,0.86099 -0.0487,1.29698 0,1.29482 0.11742,2.55972 0.36523,3.78183 l 3.00706,0 c 0.23267,0.96251 0.55193,1.88931 0.9496,2.77576 l -2.54444,1.46667 c 0.72632,1.55023 1.66379,2.97813 2.76358,4.26668 l 2.56879,-1.47879 c 0.61288,0.67281 1.27837,1.29285 1.9966,1.85456 l -1.4731,2.53333 c 1.35328,1.01142 2.84542,1.85684 4.44364,2.48486 l 1.48527,-2.55758 c 0.82162,0.29731 1.67903,0.52861 2.55662,0.69091 l 0,2.95759 c 0.96746,0.1504 1.96069,0.2303 2.97054,0.2303 0.72232,0 1.42903,-0.0434 2.13051,-0.12122 l 0,-2.93333 c 0.92198,-0.12
 099 1.82433,-0.31291 2.69053,-0.58183 l 1.49746,2.56971 c 1.62727,-0.56836 3.1491,-1.35393 4.54103,-2.31516 l -1.46092,-2.53334 c 0.8221,-0.58602 1.59164,-1.23797 2.28878,-1.96364 l 2.58097,1.47879 c 1.13091,-1.26838 2.10046,-2.6809 2.86096,-4.21819 l -2.53226,-1.45456 c 0.47863,-0.99754 0.85459,-2.05339 1.12004,-3.15152 l 2.99488,0 c 0.24782,-1.22211 0.37742,-2.48701 0.37742,-3.78183 0,-0.43599 -0.0318,-0.86831 -0.0609,-1.29698 l -2.92185,0 c -0.10054,-1.25362 -0.34579,-2.46806 -0.71829,-3.62425 l 2.58096,-1.47879 c -0.58053,-1.61591 -1.37593,-3.12911 -2.34965,-4.50911 l -2.53227,1.45456 c -0.77844,-1.06674 -1.67797,-2.03832 -2.69053,-2.88487 l 1.4731,-2.54546 c -1.32833,-1.05198 -2.80059,-1.9333 -4.38277,-2.60607 l -1.4731,2.54546 c -1.22625,-0.48946 -2.53039,-0.82848 -3.88362,-1.00607 l 0,-2.93333 c -0.70148,-0.0778 -1.40819,-0.12122 -2.13051,-0.12122 z m 0,13.5758 c 3.01222,0 5.45411,2.43125 5.45411,5.43033 0,2.99908 -2.44189,5.43032 -5.45411,5.43032 -3.01223,0 -5.45411,-2.43124
  -5.45411,-5.43032 0,-2.99908 2.44188,-5.43033 5.45411,-5.43033 z"
+           id="path7903-09"
+           inkscape:connector-curvature="0" />
+        <path
+           style="fill:#f3b33f;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           d="m 369.14084,167.18601 c -0.63817,0 -1.26583,0.0505 -1.87721,0.14554 l 0,1.86903 c -0.82348,0.15229 -1.61288,0.39908 -2.35421,0.73534 l -0.9309,-1.60091 c -0.97987,0.46246 -1.88737,1.058 -2.70042,1.75411 l 0.94629,1.62391 c -0.56511,0.51796 -1.0715,1.099 -1.50792,1.73114 l -1.60794,-0.92685 c -0.59436,0.88299 -1.07144,1.8525 -1.4156,2.88013 l 1.64641,0.94217 c -0.192,0.65424 -0.32073,1.33593 -0.37697,2.03754 l -1.85414,0 c -0.0184,0.27088 -0.0308,0.54409 -0.0308,0.81961 0,0.81825 0.0742,1.61759 0.23081,2.38989 l 1.90028,0 c 0.14703,0.60825 0.34879,1.19393 0.60009,1.75412 l -1.60793,0.92685 c 0.45899,0.97966 1.05142,1.88199 1.74642,2.6963 l 1.62332,-0.93452 c 0.3873,0.42517 0.80786,0.81701 1.26174,1.17197 l -0.93092,1.60092 c 0.85519,0.63916 1.79814,1.17341 2.80812,1.57029 l 0.9386,-1.61625 c 0.51923,0.18789 1.06105,0.33405 1.61564,0.43662 l 0,1.86902 c 0.61137,0.0951 1.23904,0.14554 1.87721,0.14554 0.45645,0 0.90306,-0.0274 1.34635,-0.0765 l 0,-1.8537 c 0.58263,-0.0765 
 1.15287,-0.19773 1.70027,-0.36768 l 0.94629,1.62391 c 1.02834,-0.35918 1.99006,-0.85561 2.86967,-1.46305 l -0.92321,-1.60092 c 0.51952,-0.37033 1.00582,-0.78233 1.44637,-1.24091 l 1.63102,0.93451 c 0.71468,-0.80155 1.32738,-1.69418 1.80797,-2.66565 l -1.60025,-0.9192 c 0.30247,-0.63038 0.54006,-1.29762 0.7078,-1.99157 l 1.89258,0 c 0.15661,-0.7723 0.23852,-1.57165 0.23852,-2.3899 0,-0.27552 -0.0202,-0.54872 -0.0385,-0.81961 l -1.84643,0 c -0.0635,-0.79221 -0.21852,-1.55967 -0.45392,-2.29031 l 1.63102,-0.93452 c -0.36686,-1.02115 -0.86951,-1.97741 -1.48484,-2.84948 l -1.60025,0.91919 c -0.49193,-0.6741 -1.06037,-1.2881 -1.70026,-1.82307 l 0.93092,-1.60858 c -0.83943,-0.66479 -1.76981,-1.22172 -2.76966,-1.64689 l -0.93092,1.60859 c -0.77491,-0.30932 -1.59905,-0.52354 -2.45421,-0.63577 l 0,-1.8537 c -0.44329,-0.0491 -0.88989,-0.0765 -1.34635,-0.0765 z m 0,8.57911 c 1.90355,0 3.44667,1.5364 3.44667,3.43165 0,1.89525 -1.54312,3.43164 -3.44667,3.43164 -1.90355,0 -3.44669,-1.53639 -3.44669
 ,-3.43164 0,-1.89525 1.54314,-3.43165 3.44669,-3.43165 z"
+           id="path7903-0-4"
+           inkscape:connector-curvature="0" />
+        <path
+           style="fill:#f3b33f;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           d="m 399.41472,172.32967 c -1.00984,0 -2.00307,0.0799 -2.97054,0.2303 l 0,2.95759 c -1.30309,0.24099 -2.55225,0.63153 -3.72536,1.16364 l -1.47309,-2.53335 c -1.55054,0.73184 -2.98661,1.67423 -4.2732,2.77578 l 1.49745,2.5697 c -0.89425,0.81963 -1.69559,1.73908 -2.38618,2.73941 l -2.54444,-1.46668 c -0.94053,1.39727 -1.6955,2.93146 -2.24008,4.55759 l 2.60531,1.49091 c -0.30383,1.03529 -0.50751,2.11404 -0.59655,3.22426 l -2.93401,0 c -0.029,0.42866 -0.0487,0.86098 -0.0487,1.29697 0,1.29484 0.11742,2.55973 0.36523,3.78183 l 3.00707,0 c 0.23266,0.96252 0.55192,1.88932 0.94959,2.77577 l -2.54444,1.46666 c 0.72632,1.55025 1.66379,2.97814 2.76359,4.26669 l 2.56879,-1.47879 c 0.61288,0.6728 1.27836,1.29285 1.99659,1.85456 l -1.4731,2.53333 c 1.35327,1.01143 2.84542,1.85684 4.44364,2.48486 l 1.48526,-2.55758 c 0.82163,0.29731 1.67904,0.52861 2.55663,0.6909 l 0,2.95759 c 0.96745,0.15041 1.96069,0.2303 2.97054,0.2303 0.72232,0 1.42904,-0.0434 2.13051,-0.1212 l 0,-2.93335 c 0.92198,-0
 .12099 1.82433,-0.3129 2.69054,-0.58182 l 1.49744,2.56971 c 1.62728,-0.56836 3.14911,-1.35393 4.54104,-2.31516 l -1.46093,-2.53335 c 0.82211,-0.58602 1.59165,-1.23796 2.28879,-1.96364 l 2.58096,1.47879 c 1.13093,-1.26838 2.10047,-2.6809 2.86098,-4.21819 l -2.53227,-1.45456 c 0.47863,-0.99753 0.85459,-2.05338 1.12004,-3.15152 l 2.99489,0 c 0.2478,-1.2221 0.37741,-2.48699 0.37741,-3.78183 0,-0.43599 -0.0318,-0.86831 -0.0609,-1.29697 l -2.92185,0 c -0.10053,-1.25363 -0.34578,-2.46807 -0.71829,-3.62426 l 2.58097,-1.47879 c -0.58052,-1.61591 -1.37593,-3.12911 -2.34966,-4.5091 l -2.53226,1.45455 c -0.77844,-1.06673 -1.67798,-2.03832 -2.69053,-2.88486 l 1.4731,-2.54546 c -1.32833,-1.05199 -2.80059,-1.9333 -4.38277,-2.60608 l -1.4731,2.54547 c -1.22626,-0.48947 -2.53039,-0.82848 -3.88363,-1.00607 l 0,-2.93334 c -0.70147,-0.0778 -1.40818,-0.12121 -2.1305,-0.12121 z m 0,13.5758 c 3.01223,0 5.45412,2.43124 5.45412,5.43032 0,2.99909 -2.44189,5.43033 -5.45412,5.43033 -3.01222,0 -5.45411,-2.43124
  -5.45411,-5.43033 0,-2.99908 2.44189,-5.43032 5.45411,-5.43032 z"
+           id="path7903-6-1"
+           inkscape:connector-curvature="0" />
+      </g>
+      <g
+         transform="translate(367.1568,8.8134766e-7)"
+         id="g8115-1">
+        <path
+           style="fill:#b4a7d6;fill-opacity:1;stroke:#333333;stroke-width:1.99999988000000006;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           d="m 150.27381,149.96659 0,25.28125 -26.71577,0 0,73.5625 26.71577,0 0,25.28125 36.08334,-62.0625 -36.08334,-62.0625 z"
+           id="rect7760-0"
+           inkscape:connector-curvature="0" />
+      </g>
+      <text
+         xml:space="preserve"
+         style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+         x="251.40457"
+         y="247.67683"
+         id="text7535-1"
+         sodipodi:linespacing="125%"><tspan
+           style="font-style:italic;text-align:center;text-anchor:middle;-inkscape-font-specification:Arial Italic"
+           sodipodi:role="line"
+           x="251.40457"
+           y="247.67683"
+           id="tspan7539-3">UDF</tspan><tspan
+           style="font-style:italic;text-align:center;text-anchor:middle;-inkscape-font-specification:Arial Italic"
+           id="tspan8168"
+           sodipodi:role="line"
+           x="251.40457"
+           y="267.67682">Calls</tspan></text>
+      <text
+         xml:space="preserve"
+         style="font-size:48px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+         x="522.07141"
+         y="287.18451"
+         id="text3845-0-1-8"
+         sodipodi:linespacing="125%"><tspan
+           sodipodi:role="line"
+           id="tspan3847-6-2-4"
+           x="522.07141"
+           y="287.18451"
+           style="font-size:16px;text-align:center;text-anchor:middle" /><tspan
+           sodipodi:role="line"
+           x="522.07141"
+           y="307.18451"
+           style="font-size:16px;text-align:center;text-anchor:middle"
+           id="tspan8170-7">Output Data</tspan></text>
+      <text
+         xml:space="preserve"
+         style="font-size:48px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+         x="341.65457"
+         y="287.18451"
+         id="text3845-0-1-8-1"
+         sodipodi:linespacing="125%"><tspan
+           sodipodi:role="line"
+           id="tspan3847-6-2-4-5"
+           x="341.65457"
+           y="287.18451"
+           style="font-size:16px;text-align:center;text-anchor:middle" /><tspan
+           sodipodi:role="line"
+           x="341.65457"
+           y="307.18451"
+           style="font-size:16px;text-align:center;text-anchor:middle"
+           id="tspan8170-7-1">Operator</tspan></text>
+    </g>
+  </g>
+</svg>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/recorddm.svg
----------------------------------------------------------------------
diff --git a/docs/img/recorddm.svg b/docs/img/recorddm.svg
new file mode 100644
index 0000000..bcc20e0
--- /dev/null
+++ b/docs/img/recorddm.svg
@@ -0,0 +1,263 @@
+<?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"
+   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+   width="800"
+   height="125.61"
+   id="svg2"
+   version="1.1"
+   inkscape:version="0.48.3.1 r9886"
+   sodipodi:docname="recorddm.svg">
+  <defs
+     id="defs4">
+    <marker
+       inkscape:stockid="Arrow1Mend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mend"
+       style="overflow:visible">
+      <path
+         id="path3864"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(-0.4,0,0,-0.4,-4,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart"
+       style="overflow:visible">
+      <path
+         id="path3861"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Lstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Lstart"
+       style="overflow:visible">
+      <path
+         id="path3855"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.8,0,0,0.8,10,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+  </defs>
+  <sodipodi:namedview
+     id="base"
+     pagecolor="#ffffff"
+     bordercolor="#666666"
+     borderopacity="1.0"
+     inkscape:pageopacity="0.0"
+     inkscape:pageshadow="2"
+     inkscape:zoom="0.69753138"
+     inkscape:cx="-137.00168"
+     inkscape:cy="180.66416"
+     inkscape:document-units="px"
+     inkscape:current-layer="layer1"
+     showgrid="false"
+     fit-margin-top="0"
+     fit-margin-left="0"
+     fit-margin-right="0"
+     fit-margin-bottom="0"
+     inkscape:window-width="1916"
+     inkscape:window-height="986"
+     inkscape:window-x="1400"
+     inkscape:window-y="193"
+     inkscape:window-maximized="0" />
+  <metadata
+     id="metadata7">
+    <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 />
+      </cc:Work>
+    </rdf:RDF>
+  </metadata>
+  <g
+     inkscape:label="Layer 1"
+     inkscape:groupmode="layer"
+     id="layer1"
+     transform="translate(-330.7807,-543.08649)">
+    <flowRoot
+       xml:space="preserve"
+       id="flowRoot4781"
+       style="font-size:48px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+       transform="translate(92.933548,154.52058)"><flowRegion
+         id="flowRegion4783"><rect
+           id="rect4785"
+           width="326.42029"
+           height="111.51003"
+           x="256.47308"
+           y="449.82095" /></flowRegion><flowPara
+         id="flowPara4789" /></flowRoot>    <g
+       id="g6802"
+       transform="translate(269.2258,-3.4995098)">
+      <g
+         transform="matrix(0.35294118,0,0,0.35294118,328.10303,357.09655)"
+         id="g4847">
+        <rect
+           style="fill:#eeeeee;fill-opacity:1;stroke:#333333;stroke-width:5.66666651;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           id="rect2985-5-3-1"
+           width="482.55975"
+           height="82"
+           x="10.420063"
+           y="649.36505" />
+        <rect
+           style="fill:#4ecdc4;fill-opacity:1;stroke:#333333;stroke-width:5.66666651;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           id="rect2985-5"
+           width="77.441345"
+           height="78"
+           x="12.920331"
+           y="651.04889" />
+        <rect
+           style="fill:#eeeeee;fill-opacity:1;stroke:#333333;stroke-width:5.66666651;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           id="rect2985-5-3"
+           width="77.441345"
+           height="78"
+           x="93.032051"
+           y="651.04889" />
+        <rect
+           style="fill:#b4a7d6;fill-opacity:1;stroke:#333333;stroke-width:5.66666651;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           id="rect2985-5-7"
+           width="77.441345"
+           height="78"
+           x="173.1438"
+           y="651.04889" />
+        <rect
+           style="fill:#666666;fill-opacity:1;stroke:#333333;stroke-width:5.66666651;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           id="rect2985-5-8"
+           width="77.441345"
+           height="78"
+           x="253.25552"
+           y="651.04889" />
+        <rect
+           style="fill:#f3b33f;fill-opacity:1;stroke:#333333;stroke-width:5.66666651;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           id="rect2985-5-4"
+           width="77.441345"
+           height="78"
+           x="333.36725"
+           y="651.04889" />
+        <rect
+           style="fill:#e1f5c4;fill-opacity:1;stroke:#333333;stroke-width:5.66666651;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           id="rect2985-5-0"
+           width="77.441345"
+           height="78"
+           x="413.47897"
+           y="651.04889" />
+      </g>
+      <text
+         sodipodi:linespacing="125%"
+         id="text3845"
+         y="649.5304"
+         x="441.89203"
+         style="font-size:48px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+         xml:space="preserve"><tspan
+           style="font-size:16px"
+           y="649.5304"
+           x="441.89203"
+           id="tspan3847"
+           sodipodi:role="line">Field: Type</tspan></text>
+      <text
+         sodipodi:linespacing="125%"
+         id="text3845-0"
+         y="561.53864"
+         x="391.07892"
+         style="font-size:48px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+         xml:space="preserve"><tspan
+           style="font-size:16px"
+           y="561.53864"
+           x="391.07892"
+           id="tspan3847-6"
+           sodipodi:role="line">Record</tspan></text>
+      <text
+         sodipodi:linespacing="125%"
+         id="text4753"
+         y="639.59271"
+         x="553.21973"
+         style="font-size:48px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+         xml:space="preserve"><tspan
+           style="font-size:16px"
+           id="tspan4757"
+           y="639.59271"
+           x="553.21973"
+           sodipodi:role="line">Value</tspan></text>
+      <text
+         sodipodi:linespacing="125%"
+         id="text4799"
+         y="658.17999"
+         x="540.8418"
+         style="font-size:48px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+         xml:space="preserve"><tspan
+           style="font-size:12px"
+           y="658.17999"
+           x="540.8418"
+           id="tspan4801"
+           sodipodi:role="line">●</tspan></text>
+      <text
+         sodipodi:linespacing="125%"
+         id="text4799-8"
+         y="637.34369"
+         x="540.8418"
+         style="font-size:48px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+         xml:space="preserve"><tspan
+           style="font-size:12px"
+           y="637.34369"
+           x="540.8418"
+           id="tspan4801-8"
+           sodipodi:role="line">●</tspan></text>
+      <text
+         inkscape:transform-center-y="66.906019"
+         inkscape:transform-center-x="196.66315"
+         sodipodi:linespacing="125%"
+         id="text4827"
+         y="660.42902"
+         x="552.11816"
+         style="font-size:48px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+         xml:space="preserve"><tspan
+           style="font-size:16px"
+           y="660.42902"
+           x="552.11816"
+           id="tspan4829"
+           sodipodi:role="line">Key</tspan></text>
+      <path
+         sodipodi:nodetypes="ccccccc"
+         inkscape:connector-curvature="0"
+         id="path4878"
+         d="m 331.78518,578.49146 0,-3.70254 80.12761,0 5.47415,-4.53532 5.40619,4.53532 79.29829,0 0,3.62459"
+         style="fill:none;stroke:#000000;stroke-width:2;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+      <path
+         sodipodi:nodetypes="ccccccc"
+         inkscape:connector-curvature="0"
+         id="path4878-7"
+         d="m 446.54075,621.05807 0,3.77371 8.224,0 5.14272,4.62249 5.22951,-4.62249 8.86137,0 0,-3.69427"
+         style="fill:none;stroke:#000000;stroke-width:2;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+      <path
+         sodipodi:nodetypes="ccccccc"
+         inkscape:connector-curvature="0"
+         id="path4878-7-9"
+         d="m 537.48048,623.51239 -4.794,0 0,15.62417 -5.87227,6.39539 5.87227,6.50333 0,15.66121 4.69308,0"
+         style="fill:none;stroke:#000000;stroke-width:2;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+    </g>
+  </g>
+</svg>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/reduce.svg
----------------------------------------------------------------------
diff --git a/docs/img/reduce.svg b/docs/img/reduce.svg
new file mode 100644
index 0000000..e0e8df3
--- /dev/null
+++ b/docs/img/reduce.svg
@@ -0,0 +1,425 @@
+<?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"
+   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+   width="800"
+   height="185"
+   id="svg35289"
+   version="1.1"
+   inkscape:version="0.48.3.1 r9886"
+   sodipodi:docname="reduce.svg">
+  <defs
+     id="defs35291">
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35250"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35252"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35254"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35256"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35258"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35260"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend-0"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456-7"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35403"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35405"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35407"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35409"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35411"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35413"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend-6"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456-1"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+  </defs>
+  <sodipodi:namedview
+     id="base"
+     pagecolor="#ffffff"
+     bordercolor="#666666"
+     borderopacity="1.0"
+     inkscape:pageopacity="0.0"
+     inkscape:pageshadow="2"
+     inkscape:zoom="1.4"
+     inkscape:cx="233.75455"
+     inkscape:cy="86.830647"
+     inkscape:document-units="px"
+     inkscape:current-layer="layer1"
+     showgrid="false"
+     fit-margin-top="0"
+     fit-margin-left="0"
+     fit-margin-right="0"
+     fit-margin-bottom="0"
+     inkscape:window-width="1916"
+     inkscape:window-height="1161"
+     inkscape:window-x="1400"
+     inkscape:window-y="18"
+     inkscape:window-maximized="0" />
+  <metadata
+     id="metadata35294">
+    <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 />
+      </cc:Work>
+    </rdf:RDF>
+  </metadata>
+  <g
+     inkscape:label="Layer 1"
+     inkscape:groupmode="layer"
+     id="layer1"
+     transform="translate(-492.58981,-171.02527)">
+    <g
+       id="g44557"
+       transform="translate(284.86629,-3.7883928)">
+      <g
+         transform="translate(2158.7696,-421.90351)"
+         id="g35174">
+        <g
+           id="g33519"
+           transform="translate(-475,10)">
+          <g
+             transform="matrix(1,0,0,1.0123433,-1388.1056,2483.203)"
+             id="g7121-1"
+             style="opacity:0.95999995">
+            <rect
+               style="fill:#f3f3f3;fill-opacity:1;stroke:#000000;stroke-width:1.49082756;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:25"
+               id="rect2994-5-8-90"
+               width="57.498154"
+               height="134.88037"
+               x="267.76794"
+               y="-1829.9814" />
+            <g
+               id="g3792-4-7-6"
+               transform="matrix(0.4074212,0,0,0.40742032,190.00545,-1148.6406)"
+               style="stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+              <rect
+                 y="-1658.4716"
+                 x="211.42857"
+                 height="50"
+                 width="50"
+                 id="rect2992-0-7-5"
+                 style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+              <rect
+                 y="-1658.4716"
+                 x="261.42859"
+                 height="50"
+                 width="50"
+                 id="rect2992-8-5-6-7"
+                 style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+            </g>
+            <g
+               transform="matrix(0.4074212,0,0,0.40742032,190.00545,-1115.5741)"
+               id="g3792-8-9-8-25"
+               style="stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+              <rect
+                 y="-1658.4716"
+                 x="211.42857"
+                 height="50"
+                 width="50"
+                 id="rect2992-1-46-7-5"
+                 style="fill:#f3de4a;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+              <rect
+                 y="-1658.4716"
+                 x="261.42859"
+                 height="50"
+                 width="50"
+                 id="rect2992-8-0-9-3-0"
+                 style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+            </g>
+            <g
+               transform="matrix(0.4074212,0,0,0.40742032,190.00545,-1080.2331)"
+               id="g3792-0-2-2-3"
+               style="stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+              <rect
+                 y="-1658.4716"
+                 x="211.42857"
+                 height="50"
+                 width="50"
+                 id="rect2992-4-2-5-5"
+                 style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+              <rect
+                 y="-1658.4716"
+                 x="261.42859"
+                 height="50"
+                 width="50"
+                 id="rect2992-8-4-4-4-8"
+                 style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+            </g>
+            <g
+               transform="matrix(0.4074212,0,0,0.40742032,190.00545,-1044.8921)"
+               id="g3792-8-4-7-3-0"
+               style="stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+              <rect
+                 y="-1658.4716"
+                 x="211.42857"
+                 height="50"
+                 width="50"
+                 id="rect2992-1-4-75-5-7"
+                 style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+              <rect
+                 y="-1658.4716"
+                 x="261.42859"
+                 height="50"
+                 width="50"
+                 id="rect2992-8-0-7-4-7-36"
+                 style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+            </g>
+            <rect
+               style="fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49082756;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.47248249, 4.47248249;stroke-dashoffset:0"
+               id="rect2994-3-8-0-9-22"
+               width="57.498154"
+               height="28.256905"
+               x="368.94507"
+               y="-1759.9644" />
+            <g
+               transform="matrix(0.4074212,0,0,0.40742032,291.18262,-1080.0794)"
+               id="g3792-8-5-0-5-0"
+               style="stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+              <rect
+                 y="-1658.4716"
+                 x="211.42857"
+                 height="50"
+                 width="50"
+                 id="rect2992-1-7-2-6-3"
+                 style="fill:#f3de4a;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+              <rect
+                 y="-1658.4716"
+                 x="261.42859"
+                 height="50"
+                 width="50"
+                 id="rect2992-8-0-4-1-5-2"
+                 style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+            </g>
+            <rect
+               style="fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49082756;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.47248249, 4.47248249;stroke-dashoffset:0"
+               id="rect2994-3-3-8-3-7"
+               width="57.498154"
+               height="27.763"
+               x="368.94507"
+               y="-1725.4464" />
+            <g
+               transform="matrix(0.4074212,0,0,0.40742032,291.18262,-1045.8085)"
+               id="g3792-0-1-1-5-0"
+               style="stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+              <rect
+                 y="-1658.4716"
+                 x="211.42857"
+                 height="50"
+                 width="50"
+                 id="rect2992-4-8-1-8-8"
+                 style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+              <rect
+                 y="-1658.4716"
+                 x="261.42859"
+                 height="50"
+                 width="50"
+                 id="rect2992-8-4-5-0-3-7"
+                 style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+            </g>
+            <rect
+               style="fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49082756;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.47248248, 4.47248248;stroke-dashoffset:0"
+               id="rect2994-3-36-6-8"
+               width="57.498154"
+               height="60.043583"
+               x="368.94507"
+               y="-1828.2587" />
+            <g
+               id="g3792-1-2-1-9"
+               transform="matrix(0.4074212,0,0,0.40742032,291.0657,-1147.8529)"
+               style="stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+              <rect
+                 y="-1660.405"
+                 x="211.71547"
+                 height="50"
+                 width="50"
+                 id="rect2992-7-8-9-3"
+                 style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+              <rect
+                 y="-1660.405"
+                 x="261.71548"
+                 height="50"
+                 width="50"
+                 id="rect2992-8-8-9-9-4"
+                 style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+            </g>
+            <g
+               transform="matrix(0.4074212,0,0,0.40742032,291.18262,-1115.5741)"
+               id="g3792-8-4-9-0-1-4"
+               style="stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+              <rect
+                 y="-1658.4716"
+                 x="211.42857"
+                 height="50"
+                 width="50"
+                 id="rect2992-1-4-7-6-7-4"
+                 style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+              <rect
+                 y="-1658.4716"
+                 x="261.42859"
+                 height="50"
+                 width="50"
+                 id="rect2992-8-0-7-5-4-1-3"
+                 style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+            </g>
+            <path
+               style="fill:none;stroke:#000000;stroke-width:1.49082756;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)"
+               d="m 316.88805,-1814.1501 51.96735,0"
+               id="path4066-2-4-62"
+               sodipodi:nodetypes="cc"
+               inkscape:connector-curvature="0" />
+            <path
+               style="fill:none;stroke:#000000;stroke-width:1.49082756;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)"
+               d="m 316.88805,-1781.0836 31.9416,21.7791 c 3.17115,-4.7193 8.17328,-0.6496 5.3385,3.64 l 14.77695,10.0756"
+               id="path4066-8-5-0-1"
+               sodipodi:nodetypes="cccc"
+               inkscape:connector-curvature="0" />
+            <path
+               style="fill:none;stroke:#000000;stroke-width:1.49082756;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)"
+               d="m 316.88805,-1710.4016 51.94148,-70.682"
+               id="path4066-0-6-2-3"
+               sodipodi:nodetypes="cc"
+               inkscape:connector-curvature="0" />
+            <path
+               style="fill:none;stroke:#000000;stroke-width:1.49082756;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)"
+               d="m 316.88805,-1745.7426 14.775,9.772 c 3.00165,-5.0886 9.07942,-0.9487 5.48925,3.6306 l 31.78455,21.022"
+               id="path4066-8-5-2-5-4"
+               sodipodi:nodetypes="cccc"
+               inkscape:connector-curvature="0" />
+          </g>
+        </g>
+      </g>
+      <path
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.5;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)"
+         d="m 559.17478,185.71313 22.75265,-0.035 -0.12654,35.13393"
+         id="path4066-2-0-9-4-1-2"
+         sodipodi:nodetypes="ccc"
+         inkscape:connector-curvature="0" />
+      <text
+         sodipodi:linespacing="125%"
+         id="text39591"
+         y="190.05518"
+         x="491.41794"
+         style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+         xml:space="preserve"><tspan
+           y="190.05518"
+           x="491.41794"
+           id="tspan39593"
+           sodipodi:role="line">Key Field</tspan></text>
+    </g>
+  </g>
+</svg>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/spargel_example.png
----------------------------------------------------------------------
diff --git a/docs/img/spargel_example.png b/docs/img/spargel_example.png
new file mode 100644
index 0000000..21d20f8
Binary files /dev/null and b/docs/img/spargel_example.png differ

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/spargel_example_input.png
----------------------------------------------------------------------
diff --git a/docs/img/spargel_example_input.png b/docs/img/spargel_example_input.png
new file mode 100644
index 0000000..01ead46
Binary files /dev/null and b/docs/img/spargel_example_input.png differ

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/index.md
----------------------------------------------------------------------
diff --git a/docs/index.md b/docs/index.md
new file mode 100644
index 0000000..d26e081
--- /dev/null
+++ b/docs/index.md
@@ -0,0 +1,11 @@
+---
+layout: docs
+title: Apache Flink Documentation
+---
+
+Lorem ipsum dolor sit amet, consectetur adipisicing elit, sed do eiusmod
+tempor incididunt ut labore et dolore magna aliqua. Ut enim ad minim veniam,
+quis nostrud exercitation ullamco laboris nisi ut aliquip ex ea commodo
+consequat. Duis aute irure dolor in reprehenderit in voluptate velit esse
+cillum dolore eu fugiat nulla pariatur. Excepteur sint occaecat cupidatat non
+proident, sunt in culpa qui officia deserunt mollit anim id est laborum.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/iterations.md
----------------------------------------------------------------------
diff --git a/docs/iterations.md b/docs/iterations.md
new file mode 100644
index 0000000..063f121
--- /dev/null
+++ b/docs/iterations.md
@@ -0,0 +1,188 @@
+---
+title:  "Iterations"
+---
+
+Iterative algorithms occur in many domains of data analysis, such as *machine learning* or *graph analysis*. Such algorithms are crucial in order to realize the promise of Big Data to extract meaningful information out of your data. With increasing interest to run these kinds of algorithms on very large data sets, there is a need to execute iterations in a massively parallel fashion.
+
+Stratosphere programs implement iterative algorithms by defining a **step function** and embedding it into a special iteration operator. There are two  variants of this operator: **Iterate** and **Delta Iterate**. Both operators repeatedly invoke the step function on the current iteration state until a certain termination condition is reached.
+
+Here, we provide background on both operator variants and outline their usage. The [programming guides]({{ site.baseurl }}/docs/0.4/programming_guides/) explain how to implement the operators in both [Scala]({{ site.baseurl }}/docs/0.4/programming_guides/scala.html) and [Java]({{ site.baseurl }}/docs/0.4/programming_guides/java.html#iterations). We also provide a **vertex-centric graph processing API** called [Spargel]({{ site.baseurl }}/docs/0.4/programming_guides/spargel.html).
+
+The following table provides an overview of both operators:
+
+<table class="table table-striped table-hover table-bordered">
+	<thead>
+		<th></th>
+		<th class="text-center">Iterate</th>
+		<th class="text-center">Delta Iterate</th>
+	</thead>
+	<tr>
+		<td class="text-center" width="20%"><strong>Iteration Input</strong></td>
+		<td class="text-center" width="40%"><strong>Partial Solution</strong></td>
+		<td class="text-center" width="40%"><strong>Workset</strong> and <strong>Solution Set</strong></td>
+	</tr>
+	<tr>
+		<td class="text-center"><strong>Step Function</strong></td>
+		<td colspan="2" class="text-center">Arbitrary Data Flows</td>
+	</tr>
+	<tr>
+		<td class="text-center"><strong>State Update</strong></td>
+		<td class="text-center">Next <strong>partial solution</strong></td>
+		<td>
+			<ul>
+				<li>Next workset</li>
+				<li><strong>Changes to solution set</strong></li>
+			</ul>
+		</td>
+	</tr>
+	<tr>
+		<td class="text-center"><strong>Iteration Result</strong></td>
+		<td class="text-center">Last partial solution</td>
+		<td class="text-center">Solution set state after last iteration</td>
+	</tr>
+	<tr>
+		<td class="text-center"><strong>Termination</strong></td>
+		<td>
+			<ul>
+				<li><strong>Maximum number of iterations</strong> (default)</li>
+				<li>Custom aggregator convergence</li>
+			</ul>
+		</td>
+		<td>
+			<ul>
+				<li><strong>Maximum number of iterations or empty workset</strong> (default)</li>
+				<li>Custom aggregator convergence</li>
+			</ul>
+		</td>
+	</tr>
+</table>
+
+<section id="iterate">
+Iterate Operator
+----------------
+
+The **iterate operator** covers the *simple form of iterations*: in each iteration, the **step function** consumes the **entire input** (the *result of the previous iteration*, or the *initial data set*), and computes the **next version of the partial solution** (e.g. `map`, `reduce`, `join`, etc.).
+
+<p class="text-center">
+    <img alt="Iterate Operator" width="60%" src="{{ site.baseurl }}/docs/0.4/img/iterations_iterate_operator.png" />
+</p>
+
+  1. **Iteration Input**: Initial input for the *first iteration* from a *data source* or *previous operators*.
+  2. **Step Function**: The step function will be executed in each iteration. It is an arbitrary data flow consisting of operators like `map`, `reduce`, `join`, etc. (see [programming model]({{ site.baseurl }}/docs/0.4/programming_guides/pmodel.html) for details) and depends on your specific task at hand.
+  3. **Next Partial Solution**: In each iteration, the output of the step function will be fed back into the *next iteration*.
+  4. **Iteration Result**: Output of the *last iteration* is written to a *data sink* or used as input to the *following operators*.
+
+There are multiple options to specify **termination conditions** for an iteration:
+
+  - **Maximum number of iterations**: Without any further conditions, the iteration will be executed this many times.
+  - **Custom aggregator convergence**: Iterations allow to specify *custom aggregators* and *convergence criteria* like sum aggregate the number of emitted records (aggregator) and terminate if this number is zero (convergence criterion).
+
+You can also think about the iterate operator in pseudo-code:
+
+{% highlight java %}
+IterationState state = getInitialState();
+
+while (!terminationCriterion()) {
+	state = step(state);
+}
+
+setFinalState(state);
+{% endhighlight %}
+
+<div class="panel panel-default">
+	<div class="panel-body">
+	See the <strong><a href="{{ site.baseurl }}/docs/0.4/programming_guides/scala.html">Scala</a> and <a href="{{ site.baseurl }}/docs/0.4/programming_guides/java.html#iterations">Java</a> programming guides</strong> for details and code examples.</div>
+</div>
+
+### Example: Incrementing Numbers
+
+In the following example, we **iteratively incremenet a set numbers**:
+
+<p class="text-center">
+    <img alt="Iterate Operator Example" width="60%" src="{{ site.baseurl }}/docs/0.4/img/iterations_iterate_operator_example.png" />
+</p>
+
+  1. **Iteration Input**: The inital input is read from a data source and consists of five single-field records (integers `1` to `5`).
+  2. **Step function**: The step function is a single `map` operator, which increments the integer field from `i` to `i+1`. It will be applied to every record of the input.
+  3. **Next Partial Solution**: The output of the step function will be the output of the map operator, i.e. records with incremented integers.
+  4. **Iteration Result**: After ten iterations, the initial numbers will have been incremented ten times, resulting in integers `11` to `15`.
+
+```
+// 1st           2nd                       10th
+map(1) -> 2      map(2) -> 3      ...      map(10) -> 11
+map(2) -> 3      map(3) -> 4      ...      map(11) -> 12
+map(3) -> 4      map(4) -> 5      ...      map(12) -> 13
+map(4) -> 5      map(5) -> 6      ...      map(13) -> 14
+map(5) -> 6      map(6) -> 7      ...      map(14) -> 15
+```
+
+Note that **1**, **2**, and **4** can be arbitrary data flows.
+</section>
+
+<section id="delta_iterate">
+Delta Iterate Operator
+----------------------
+
+The **delta iterate operator** covers the case of **incremental iterations**. Incremental iterations **selectively modify elements** of their **solution solution** and evolve the solution rather than fully recompute it.
+
+Where applicable, this leads to **more efficient algorithms**, because not every element in the solution set changes in each iteration. This allows to **focus on the hot parts** of the solution and leave the **cold parts untouched**. Frequently, the majority of the solution cools down comparatively fast and the later iterations operate only on a small subset of the data.
+
+<p class="text-center">
+    <img alt="Delta Iterate Operator" width="60%" src="{{ site.baseurl }}/docs/0.4/img/iterations_delta_iterate_operator.png" />
+</p>
+
+  1. **Iteration Input**: The initial workset and solution set are read from *data sources* or *previous operators* as input to the first iteration.
+  2. **Step Function**: The step function will be executed in each iteration. It is an arbitrary data flow consisting of operators like `map`, `reduce`, `join`, etc. (see [programming model]({{ site.baseurl }}/docs/0.4/programming_guides/pmodel.html) for details) and depends on your specific task at hand.
+  3. **Next Workset/Update Solution Set**: The *next workset* drives the iterative computation and will be fed back into the *next iteration*. Furthermore, the solution set will be updated and implicitly forwarded (it is not required to be rebuild). Both data sets can be updated by different operators of the step function.
+  4. **Iteration Result**: After the *last iteration*, the *solution set* is written to a *data sink* or used as input to the *following operators*.
+
+The default **termination condition** for delta iterations is specified by the **empty workset convergence criterion** and a **maximum number of iterations**. The iteration will terminate when a produced *next workset* is empty or when the maximum number of iterations is reached. It is also possible to specify a **custom aggregator** and **convergence criterion**.
+
+You can also think about the iterate operator in pseudo-code:
+{% highlight java %}
+IterationState workset = getInitialState();
+IterationState solution = getInitialSolution();
+
+while (!terminationCriterion()) {
+	(delta, workset) = step(workset, solution);
+
+	solution.update(delta)
+}
+
+setFinalState(solution);
+{% endhighlight %}
+
+<div class="panel panel-default">
+	<div class="panel-body">
+	See the <strong><a href="{{ site.baseurl }}/docs/0.4/programming_guides/scala.html">Scala</a> and <a href="{{ site.baseurl }}/docs/0.4/programming_guides/java.html#iterations">Java</a> programming guides</strong> for details and code examples.</div>
+</div>
+
+### Example: Propagate Minimum in Graph
+
+In the following example, every vertex has an **ID** and a **coloring**. Each vertex will propagete its vertex ID to neighboring vertices. The **goal** is to *assign the minimum ID to every vertex in a subgraph*. If a received ID is smaller then the current one, it changes to the color of the vertex with the received ID. One application of this can be found in *community analysis* or *connected components* computation.
+
+<p class="text-center">
+    <img alt="Delta Iterate Operator Example" width="100%" src="{{ site.baseurl }}/docs/0.4/img/iterations_delta_iterate_operator_example.png" />
+</p>
+
+The **intial input** is set as **both workset and solution set.** In the above figure, the colors visualize the **evolution of the solution set**. With each iteration, the color of the minimum ID is spreading in the respective subgraph. At the same time, the amount of work (exchanged and compared vertex IDs) decreases with each iteration. This corresponds to the **decreasing size of the workset**, which goes from all seven vertices to zero after three iterations, at which time the iteration terminates. The **important observation** is that *the lower subgraph converges before the upper half* does and the delta iteration is able to capture this with the workset abstraction.
+
+In the upper subgraph **ID 1** (*orange*) is the **minimum ID**. In the **first iteration**, it will get propagated to vertex 2, which will subsequently change its color to orange. Vertices 3 and 4 will receive **ID 2** (in *yellow*) as their current minimum ID and change to yellow. Because the color of *vertex 1* didn't change in the first iteration, it can be skipped it in the next workset.
+
+In the lower subgraph **ID 5** (*cyan*) is the **minimum ID**. All vertices of the lower subgraph will receive it in the first iteration. Again, we can skip the unchanged vertices (*vertex 5*) for the next workset.
+
+In the **2nd iteration**, the workset size has already decreased from seven to five elements (vertices 2, 3, 4, 6, and 7). These are part of the iteration and further propagate their current minimum IDs. After this iteration, the lower subgraph has already converged (**cold part** of the graph), as it has no elements in the workset, whereas the upper half needs a further iteration (**hot part** of the graph) for the two remaining workset elements (vertices 3 and 4).
+
+The iteration **terminates**, when the workset is empty after the **3rd iteration**.
+</section>
+
+<section id="supersteps">
+Superstep Synchronization
+-------------------------
+
+We referred to each execution of the step function of an iteration operator as *a single iteration*. In parallel setups, **multiple instances of the step function are evaluated in parallel** on different partitions of the iteration state. In many settings, one evaluation of the step function on all parallel instances forms a so called **superstep**, which is also the granularity of synchronization. Therefore, *all* parallel tasks of an iteration need to complete the superstep, before a next superstep will be initialized. **Termination criteria** will also be evaluated at superstep barriers.
+
+<p class="text-center">
+    <img alt="Supersteps" width="50%" src="{{ site.baseurl }}/docs/0.4/img/iterations_supersteps.png" />
+</p>
+</section>
\ No newline at end of file


[53/53] [abbrv] git commit: enable build notifications to issues@flink.i.a.o from travis

Posted by rm...@apache.org.
enable build notifications to issues@flink.i.a.o from travis


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

Branch: refs/heads/travis_test
Commit: f9422539772bd007d174599afc5c705f972899ac
Parents: 69589fa
Author: Robert Metzger <rm...@apache.org>
Authored: Thu Jun 26 11:45:27 2014 +0200
Committer: Robert Metzger <rm...@apache.org>
Committed: Thu Jun 26 11:45:27 2014 +0200

----------------------------------------------------------------------
 .travis.yml | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f9422539/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index d36473a..db51eed 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -7,6 +7,11 @@ jdk:
   - openjdk6
 
 notifications:
+  email:
+    recipients:
+      - issues@flink.incubator.apache.org
+    on_success: always
+    on_failure: always
   webhooks:
     urls:
       - https://webhooks.gitter.im/e/d70a7e674cb9354c77b2


[15/53] [abbrv] Removed RuntimeEnvironment instantiation from execution graph construction. Removed legacy job vertex classes and input/output tasks.

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSinkTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSinkTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSinkTask.java
index 82359f5..cbe1766 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSinkTask.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSinkTask.java
@@ -351,64 +351,6 @@ public class DataSinkTask<IT> extends AbstractOutputTask {
 			throw new Exception("Illegal configuration: Number of input gates and group sizes are not consistent.");
 		}
 	}
-	
-	// ------------------------------------------------------------------------
-	//                     Degree of parallelism & checks
-	// ------------------------------------------------------------------------
-	
-
-	@Override
-	public int getMaximumNumberOfSubtasks() {
-		if (!(this.format instanceof FileOutputFormat<?>)) {
-			return -1;
-		}
-		
-		final FileOutputFormat<?> fileOutputFormat = (FileOutputFormat<?>) this.format;
-		
-		// ----------------- This code applies only to file inputs ------------------
-		
-		final Path path = fileOutputFormat.getOutputFilePath();
-		final WriteMode writeMode = fileOutputFormat.getWriteMode();
-		final OutputDirectoryMode outDirMode = fileOutputFormat.getOutputDirectoryMode();
-
-		// Prepare output path and determine max DOP		
-		try {
-			
-			int dop = getTaskConfiguration().getInteger(DEGREE_OF_PARALLELISM_KEY, -1);
-			final FileSystem fs = path.getFileSystem();
-			
-			if(dop == 1 && outDirMode == OutputDirectoryMode.PARONLY) {
-				// output is not written in parallel and should be written to a single file.
-				
-				if(fs.isDistributedFS()) {
-					// prepare distributed output path
-					if(!fs.initOutPathDistFS(path, writeMode, false)) {
-						// output preparation failed! Cancel task.
-						throw new IOException("Output path could not be initialized.");
-					}
-				}
-				
-				return 1;
-				
-			} else {
-				// output should be written to a directory
-				
-				if(fs.isDistributedFS()) {
-					// only distributed file systems can be initialized at start-up time.
-					if(!fs.initOutPathDistFS(path, writeMode, true)) {
-						throw new IOException("Output directory could not be created.");
-					}
-				}
-				
-				return -1;
-				
-			}
-		}
-		catch (IOException e) {
-			LOG.error("Could not access the file system to detemine the status of the output.", e);
-			throw new RuntimeException("I/O Error while accessing file", e);
-		}
-	}
 
 	// ------------------------------------------------------------------------
 	//                               Utilities

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSourceTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSourceTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSourceTask.java
index af176b9..f835ace 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSourceTask.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSourceTask.java
@@ -78,27 +78,12 @@ public class DataSourceTask<OT> extends AbstractInputTask<InputSplit> {
 	@Override
 	public void registerInputOutput()
 	{
-		if (LOG.isDebugEnabled()) {
+		initInputFormat();
+
+		if (LOG.isDebugEnabled())
 			LOG.debug(getLogString("Start registering input and output"));
 		}
 
-		if (this.userCodeClassLoader == null) {
-			try {
-				this.userCodeClassLoader = LibraryCacheManager.getClassLoader(getEnvironment().getJobID());
-			}
-			catch (IOException ioe) {
-				throw new RuntimeException("Usercode ClassLoader could not be obtained for job: " + 
-							getEnvironment().getJobID(), ioe);
-			}
-		}
-		
-		// obtain task configuration (including stub parameters)
-		Configuration taskConf = getTaskConfiguration();
-		taskConf.setClassLoader(this.userCodeClassLoader);
-		this.config = new TaskConfig(taskConf);
-		
-		initInputFormat(this.userCodeClassLoader);
-		
 		try {
 			initOutputs(this.userCodeClassLoader);
 		} catch (Exception ex) {
@@ -301,17 +286,42 @@ public class DataSourceTask<OT> extends AbstractInputTask<InputSplit> {
 
 	/**
 	 * Initializes the InputFormat implementation and configuration.
-	 * 
+l	 * 
 	 * @throws RuntimeException
 	 *         Throws if instance of InputFormat implementation can not be
 	 *         obtained.
 	 */
-	private void initInputFormat(ClassLoader cl) {
-		// instantiate the stub
-		@SuppressWarnings("unchecked")
-		Class<InputFormat<OT, InputSplit>> superClass = (Class<InputFormat<OT, InputSplit>>) (Class<?>) InputFormat.class;
-		this.format = RegularPactTask.instantiateUserCode(this.config, cl, superClass);
-		
+	private void initInputFormat() {
+		if (this.userCodeClassLoader == null) {
+			try {
+				this.userCodeClassLoader = LibraryCacheManager.getClassLoader(getEnvironment().getJobID());
+			}
+			catch (IOException ioe) {
+				throw new RuntimeException("Usercode ClassLoader could not be obtained for job: " +
+						getEnvironment().getJobID(), ioe);
+			}
+		}
+
+		// obtain task configuration (including stub parameters)
+		Configuration taskConf = getTaskConfiguration();
+		taskConf.setClassLoader(this.userCodeClassLoader);
+		this.config = new TaskConfig(taskConf);
+
+		try {
+			this.format = config.<InputFormat<OT, InputSplit>>getStubWrapper(this.userCodeClassLoader)
+					.getUserCodeObject(InputFormat.class, this.userCodeClassLoader);
+
+			// check if the class is a subclass, if the check is required
+			if (!InputFormat.class.isAssignableFrom(this.format.getClass())) {
+				throw new RuntimeException("The class '" + this.format.getClass().getName() + "' is not a subclass of '" +
+						InputFormat.class.getName() + "' as is required.");
+			}
+		}
+		catch (ClassCastException ccex) {
+			throw new RuntimeException("The stub class is not a proper subclass of " + InputFormat.class.getName(),
+					ccex);
+		}
+
 		// configure the stub. catch exceptions here extra, to report them as originating from the user code 
 		try {
 			this.format.configure(this.config.getStubParameters());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/TaskConfig.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/TaskConfig.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/TaskConfig.java
index a43f8cc..2eb003d 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/TaskConfig.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/TaskConfig.java
@@ -246,6 +246,10 @@ public class TaskConfig {
 	public String getTaskName() {
 		return this.config.getString(TASK_NAME, null);
 	}
+
+	public boolean hasStubWrapper() {
+		return this.config.containsKey(STUB_OBJECT);
+	}
 	
 	
 	public void setStubWrapper(UserCodeWrapper<?> wrapper) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/deployment/TaskDeploymentDescriptorTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/deployment/TaskDeploymentDescriptorTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/deployment/TaskDeploymentDescriptorTest.java
index 7000667..dc2e605 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/deployment/TaskDeploymentDescriptorTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/deployment/TaskDeploymentDescriptorTest.java
@@ -19,12 +19,12 @@ import static org.junit.Assert.fail;
 
 import java.io.IOException;
 
+import eu.stratosphere.pact.runtime.task.RegularPactTask;
 import org.junit.Test;
 
 import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
 import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
-import eu.stratosphere.nephele.util.FileLineReader;
 import eu.stratosphere.nephele.jobgraph.JobID;
 import eu.stratosphere.nephele.template.AbstractInvokable;
 import eu.stratosphere.nephele.util.SerializableArrayList;
@@ -50,7 +50,7 @@ public class TaskDeploymentDescriptorTest {
 		final int currentNumberOfSubtasks = 1;
 		final Configuration jobConfiguration = new Configuration();
 		final Configuration taskConfiguration = new Configuration();
-		final Class<? extends AbstractInvokable> invokableClass = FileLineReader.class;
+		final Class<? extends AbstractInvokable> invokableClass =  RegularPactTask.class;
 		final SerializableArrayList<GateDeploymentDescriptor> outputGates = new SerializableArrayList<GateDeploymentDescriptor>(
 			0);
 		final SerializableArrayList<GateDeploymentDescriptor> inputGates = new SerializableArrayList<GateDeploymentDescriptor>(
@@ -85,7 +85,7 @@ public class TaskDeploymentDescriptorTest {
 		final int currentNumberOfSubtasks = 1;
 		final Configuration jobConfiguration = new Configuration();
 		final Configuration taskConfiguration = new Configuration();
-		final Class<? extends AbstractInvokable> invokableClass = FileLineReader.class;
+		final Class<? extends AbstractInvokable> invokableClass = RegularPactTask.class;
 		final SerializableArrayList<GateDeploymentDescriptor> outputGates = new SerializableArrayList<GateDeploymentDescriptor>(
 			0);
 		final SerializableArrayList<GateDeploymentDescriptor> inputGates = new SerializableArrayList<GateDeploymentDescriptor>(
@@ -239,7 +239,7 @@ public class TaskDeploymentDescriptorTest {
 		final int currentNumberOfSubtasks = 1;
 		final Configuration jobConfiguration = new Configuration();
 		final Configuration taskConfiguration = new Configuration();
-		final Class<? extends AbstractInvokable> invokableClass = FileLineReader.class;
+		final Class<? extends AbstractInvokable> invokableClass = RegularPactTask.class;
 		final SerializableArrayList<GateDeploymentDescriptor> outputGates = new SerializableArrayList<GateDeploymentDescriptor>(
 			0);
 		final SerializableArrayList<GateDeploymentDescriptor> inputGates = new SerializableArrayList<GateDeploymentDescriptor>(

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/SelfCrossInputTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/SelfCrossInputTask.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/SelfCrossInputTask.java
deleted file mode 100644
index 1ce23e6..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/SelfCrossInputTask.java
+++ /dev/null
@@ -1,41 +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.executiongraph;
-
-import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.runtime.io.api.RecordWriter;
-import eu.stratosphere.nephele.template.AbstractFileInputTask;
-
-/**
- * This class represents the data source in the self cross unit test.
- * 
- */
-public class SelfCrossInputTask extends AbstractFileInputTask {
-
-
-	@Override
-	public void registerInputOutput() {
-
-		new RecordWriter<StringRecord>(this);
-		new RecordWriter<StringRecord>(this);
-	}
-
-
-	@Override
-	public void invoke() throws Exception {
-
-		// Nothing to do here
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/DoubleSourceTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/DoubleSourceTask.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/DoubleSourceTask.java
deleted file mode 100644
index 1e2be47..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/DoubleSourceTask.java
+++ /dev/null
@@ -1,81 +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.jobmanager;
-
-import java.util.Iterator;
-
-import eu.stratosphere.core.fs.FSDataInputStream;
-import eu.stratosphere.core.fs.FileInputSplit;
-import eu.stratosphere.core.fs.FileSystem;
-import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.runtime.io.api.RecordWriter;
-import eu.stratosphere.nephele.template.AbstractFileInputTask;
-import eu.stratosphere.runtime.fs.LineReader;
-
-public class DoubleSourceTask extends AbstractFileInputTask {
-
-	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 = getFileInputSplits();
-
-		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);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/DoubleTargetTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/DoubleTargetTask.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/DoubleTargetTask.java
index f0ca435..a1ce0b2 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/DoubleTargetTask.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/DoubleTargetTask.java
@@ -13,18 +13,18 @@
 
 package eu.stratosphere.nephele.jobmanager;
 
-import eu.stratosphere.core.io.StringRecord;
 import eu.stratosphere.runtime.io.api.RecordReader;
 import eu.stratosphere.runtime.io.api.RecordWriter;
 import eu.stratosphere.nephele.template.AbstractTask;
+import eu.stratosphere.types.Record;
 
 public class DoubleTargetTask extends AbstractTask {
 
-	private RecordReader<StringRecord> input1 = null;
+	private RecordReader<Record> input1 = null;
 
-	private RecordReader<StringRecord> input2 = null;
+	private RecordReader<Record> input2 = null;
 
-	private RecordWriter<StringRecord> output = null;
+	private RecordWriter<Record> output = null;
 
 	@Override
 	public void invoke() throws Exception {
@@ -33,13 +33,13 @@ public class DoubleTargetTask extends AbstractTask {
 
 		while (this.input1.hasNext()) {
 
-			StringRecord s = input1.next();
+			Record s = input1.next();
 			this.output.emit(s);
 		}
 
 		while (this.input2.hasNext()) {
 
-			StringRecord s = input2.next();
+			Record s = input2.next();
 			this.output.emit(s);
 		}
 
@@ -49,9 +49,9 @@ public class DoubleTargetTask extends AbstractTask {
 
 	@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);
+		this.input1 = new RecordReader<Record>(this, Record.class);
+		this.input2 = new RecordReader<Record>(this, Record.class);
+		this.output = new RecordWriter<Record>(this);
 	}
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ForwardTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ForwardTask.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ForwardTask.java
index 96be668..377e304 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ForwardTask.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ForwardTask.java
@@ -13,15 +13,15 @@
 
 package eu.stratosphere.nephele.jobmanager;
 
-import eu.stratosphere.core.io.StringRecord;
 import eu.stratosphere.runtime.io.api.RecordReader;
 import eu.stratosphere.runtime.io.api.RecordWriter;
 import eu.stratosphere.nephele.template.AbstractTask;
+import eu.stratosphere.types.Record;
 
 public class ForwardTask extends AbstractTask {
 
-	private RecordReader<StringRecord> input = null;
-	private RecordWriter<StringRecord> output = null;
+	private RecordReader<Record> input = null;
+	private RecordWriter<Record> output = null;
 
 	@Override
 	public void invoke() throws Exception {
@@ -30,7 +30,7 @@ public class ForwardTask extends AbstractTask {
 
 		while (this.input.hasNext()) {
 
-			StringRecord s = input.next();
+			Record s = input.next();
 			this.output.emit(s);
 		}
 
@@ -39,7 +39,7 @@ public class ForwardTask extends AbstractTask {
 
 	@Override
 	public void registerInputOutput() {
-		this.input = new RecordReader<StringRecord>(this, StringRecord.class);
-		this.output = new RecordWriter<StringRecord>(this);
+		this.input = new RecordReader<Record>(this, Record.class);
+		this.output = new RecordWriter<Record>(this);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/UnionTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/UnionTask.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/UnionTask.java
index 124a24d..209eff1 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/UnionTask.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/UnionTask.java
@@ -18,6 +18,7 @@ import eu.stratosphere.runtime.io.api.MutableRecordReader;
 import eu.stratosphere.runtime.io.api.RecordWriter;
 import eu.stratosphere.runtime.io.api.UnionRecordReader;
 import eu.stratosphere.nephele.template.AbstractTask;
+import eu.stratosphere.types.Record;
 
 /**
  * A simple implementation of a task using a {@link UnionRecordReader}.
@@ -27,21 +28,22 @@ public class UnionTask extends AbstractTask {
 	/**
 	 * The union record reader to be used during the tests.
 	 */
-	private UnionRecordReader<StringRecord> unionReader;
+	private UnionRecordReader<Record> unionReader;
 
-	private RecordWriter<StringRecord> writer;
+	private RecordWriter<Record> 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);
+		MutableRecordReader<Record>[] recordReaders = (MutableRecordReader<Record>[]) new
+				MutableRecordReader<?>[2];
+		recordReaders[0] = new MutableRecordReader<Record>(this);
+		recordReaders[1] = new MutableRecordReader<Record>(this);
+		this.unionReader = new UnionRecordReader<Record>(recordReaders, Record.class);
 		
-		this.writer = new RecordWriter<StringRecord>(this);
+		this.writer = new RecordWriter<Record>(this);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/DefaultSchedulerTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/DefaultSchedulerTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/DefaultSchedulerTest.java
index c8bcddc..6a41fe9 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/DefaultSchedulerTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/DefaultSchedulerTest.java
@@ -95,6 +95,47 @@ public class DefaultSchedulerTest {
 
 	}
 
+	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) throws IOException {
+
+		}
+
+		@Override
+		public void writeRecord(IntValue record) throws IOException {
+
+		}
+
+		@Override
+		public void close() throws IOException {
+
+		}
+
+		@Override
+		public void initialize(Configuration configuration) {
+
+		}
+	}
+
 	/**
 	 * Constructs a sample execution graph consisting of two vertices connected by a channel of the given type.
 	 * 
@@ -108,10 +149,12 @@ public class DefaultSchedulerTest {
 
 		final JobInputVertex inputVertex = new JobInputVertex("Input 1", jobGraph);
 		inputVertex.setInputClass(InputTask.class);
+		inputVertex.setInputFormat(new DummyInputFormat());
 		inputVertex.setNumberOfSubtasks(1);
 
 		final JobOutputVertex outputVertex = new JobOutputVertex("Output 1", jobGraph);
 		outputVertex.setOutputClass(OutputTask.class);
+		outputVertex.setOutputFormat(new DummyOutputFormat());
 		outputVertex.setNumberOfSubtasks(1);
 
 		try {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/FileLineReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/FileLineReader.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/FileLineReader.java
deleted file mode 100644
index fcb4fa1..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/FileLineReader.java
+++ /dev/null
@@ -1,80 +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.util;
-
-import java.util.Iterator;
-
-import eu.stratosphere.core.fs.FSDataInputStream;
-import eu.stratosphere.core.fs.FileInputSplit;
-import eu.stratosphere.core.fs.FileSystem;
-import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.runtime.io.api.RecordWriter;
-import eu.stratosphere.nephele.template.AbstractFileInputTask;
-import eu.stratosphere.runtime.fs.LineReader;
-
-/**
- * A file line reader reads the associated file input splits line by line and outputs the lines as string records.
- * 
- */
-public class FileLineReader extends AbstractFileInputTask {
-
-	private RecordWriter<StringRecord> output = null;
-
-	@Override
-	public void invoke() throws Exception {
-
-		output.initializeSerializers();
-
-		final Iterator<FileInputSplit> splitIterator = getFileInputSplits();
-
-		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);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/FileLineWriter.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/FileLineWriter.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/FileLineWriter.java
deleted file mode 100644
index bc738df..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/FileLineWriter.java
+++ /dev/null
@@ -1,75 +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.util;
-
-import eu.stratosphere.core.fs.FSDataOutputStream;
-import eu.stratosphere.core.fs.FileStatus;
-import eu.stratosphere.core.fs.FileSystem;
-import eu.stratosphere.core.fs.Path;
-import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.runtime.io.api.RecordReader;
-import eu.stratosphere.nephele.template.AbstractFileOutputTask;
-
-/**
- * A file line writer reads string records its input gate and writes them to the associated output file.
- * 
- */
-public class FileLineWriter extends AbstractFileOutputTask {
-
-	/**
-	 * The record reader through which incoming string records are received.
-	 */
-	private RecordReader<StringRecord> input = null;
-
-
-	@Override
-	public void invoke() throws Exception {
-
-		Path outputPath = getFileOutputPath();
-
-		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);
-	}
-
-
-	@Override
-	public int getMaximumNumberOfSubtasks() {
-		// The default implementation always returns -1
-		return -1;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/library/FileLineReadWriteTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/library/FileLineReadWriteTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/library/FileLineReadWriteTest.java
deleted file mode 100644
index 17c2f58..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/library/FileLineReadWriteTest.java
+++ /dev/null
@@ -1,136 +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.runtime.io.library;
-
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-import static org.powermock.api.mockito.PowerMockito.whenNew;
-
-import java.io.File;
-import java.io.IOException;
-
-import eu.stratosphere.runtime.io.api.RecordWriter;
-import eu.stratosphere.nephele.util.FileLineReader;
-import eu.stratosphere.nephele.util.FileLineWriter;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.powermock.modules.junit4.PowerMockRunner;
-import org.powermock.reflect.Whitebox;
-
-import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.core.fs.FileInputSplit;
-import eu.stratosphere.core.fs.Path;
-import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.execution.Environment;
-import eu.stratosphere.runtime.io.api.RecordReader;
-import eu.stratosphere.nephele.template.InputSplitProvider;
-
-/**
- * This class checks the functionality of the {@link eu.stratosphere.nephele.util.FileLineReader} and the {@link eu.stratosphere.nephele.util.FileLineWriter} class.
- * 
- */
-@RunWith(PowerMockRunner.class)
-@PrepareForTest(FileLineReader.class)
-public class FileLineReadWriteTest {
-
-	@Mock
-	private Environment environment;
-
-	@Mock
-	private Configuration conf;
-
-	@Mock
-	private RecordReader<StringRecord> recordReader;
-
-	@Mock
-	private RecordWriter<StringRecord> recordWriter;
-
-	@Mock
-	private InputSplitProvider inputSplitProvider;
-
-	private File file = new File("./tmp");
-
-	/**
-	 * Set up mocks
-	 * 
-	 * @throws IOException
-	 */
-	@Before
-	public void before() throws Exception {
-
-		MockitoAnnotations.initMocks(this);
-	}
-
-	/**
-	 * remove the temporary file
-	 */
-	@After
-	public void after() {
-		this.file.delete();
-	}
-
-	/**
-	 * Tests the read and write methods
-	 * 
-	 * @throws Exception
-	 */
-	@Test
-	public void testReadWrite() throws Exception {
-
-		this.file.createNewFile();
-		FileLineWriter writer = new FileLineWriter();
-		Whitebox.setInternalState(writer, "environment", this.environment);
-		Whitebox.setInternalState(writer, "input", this.recordReader);
-		when(this.environment.getTaskConfiguration()).thenReturn(this.conf);
-
-		when(this.conf.getString("outputPath", null)).thenReturn(this.file.toURI().toString());
-		when(this.recordReader.hasNext()).thenReturn(true, true, true, false);
-		StringRecord in = new StringRecord("abc");
-		try {
-			when(this.recordReader.next()).thenReturn(in);
-		} catch (IOException e) {
-			fail();
-			e.printStackTrace();
-		} catch (InterruptedException e) {
-			fail();
-			e.printStackTrace();
-		}
-		writer.invoke();
-
-		final FileInputSplit split = new FileInputSplit(0, new Path(this.file.toURI().toString()), 0,
-			this.file.length(), null);
-		when(this.environment.getInputSplitProvider()).thenReturn(this.inputSplitProvider);
-		when(this.inputSplitProvider.getNextInputSplit()).thenReturn(split, (FileInputSplit) null);
-
-		FileLineReader reader = new FileLineReader();
-		Whitebox.setInternalState(reader, "environment", this.environment);
-		Whitebox.setInternalState(reader, "output", this.recordWriter);
-		StringRecord record = mock(StringRecord.class);
-
-		whenNew(StringRecord.class).withNoArguments().thenReturn(record);
-
-		reader.invoke();
-
-		// verify the correct bytes have been written and read
-		verify(record, times(3)).set(in.getBytes());
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/DiscardingOutputFormat.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/DiscardingOutputFormat.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/DiscardingOutputFormat.java
index 57253d1..3de547e 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/DiscardingOutputFormat.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/DiscardingOutputFormat.java
@@ -44,4 +44,7 @@ public class DiscardingOutputFormat implements OutputFormat<Record> {
 	@Override
 	public void close() throws IOException
 	{}
+
+	@Override
+	public void initialize(Configuration configuration){}
 }


[07/53] [abbrv] Rework the Taskmanager to a slot based model and remove legacy cloud code

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java
index 6e25796..8a3cba4 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java
@@ -16,13 +16,14 @@ package eu.stratosphere.nephele.jobmanager;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.io.InputStream;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Map;
+import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -32,6 +33,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import eu.stratosphere.nephele.ExecutionMode;
 import eu.stratosphere.nephele.managementgraph.ManagementVertexID;
 import eu.stratosphere.nephele.taskmanager.TaskKillResult;
+
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.GnuParser;
@@ -68,14 +70,11 @@ import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
 import eu.stratosphere.nephele.executiongraph.GraphConversionException;
 import eu.stratosphere.nephele.executiongraph.InternalJobStatus;
 import eu.stratosphere.nephele.executiongraph.JobStatusListener;
-import eu.stratosphere.nephele.instance.AbstractInstance;
 import eu.stratosphere.nephele.instance.DummyInstance;
 import eu.stratosphere.nephele.instance.HardwareDescription;
+import eu.stratosphere.nephele.instance.Instance;
 import eu.stratosphere.nephele.instance.InstanceConnectionInfo;
 import eu.stratosphere.nephele.instance.InstanceManager;
-import eu.stratosphere.nephele.instance.InstanceType;
-import eu.stratosphere.nephele.instance.InstanceTypeDescription;
-import eu.stratosphere.nephele.instance.local.LocalInstanceManager;
 import eu.stratosphere.runtime.io.channels.ChannelID;
 import eu.stratosphere.nephele.ipc.RPC;
 import eu.stratosphere.nephele.ipc.Server;
@@ -85,7 +84,7 @@ import eu.stratosphere.nephele.jobgraph.JobID;
 import eu.stratosphere.nephele.jobmanager.accumulators.AccumulatorManager;
 import eu.stratosphere.nephele.jobmanager.archive.ArchiveListener;
 import eu.stratosphere.nephele.jobmanager.archive.MemoryArchivist;
-import eu.stratosphere.nephele.jobmanager.scheduler.AbstractScheduler;
+import eu.stratosphere.nephele.jobmanager.scheduler.DefaultScheduler;
 import eu.stratosphere.nephele.jobmanager.scheduler.SchedulingException;
 import eu.stratosphere.nephele.jobmanager.splitassigner.InputSplitManager;
 import eu.stratosphere.nephele.jobmanager.splitassigner.InputSplitWrapper;
@@ -106,6 +105,7 @@ import eu.stratosphere.nephele.taskmanager.TaskSubmissionResult;
 import eu.stratosphere.runtime.io.network.ConnectionInfoLookupResponse;
 import eu.stratosphere.runtime.io.network.RemoteReceiver;
 import eu.stratosphere.nephele.taskmanager.ExecutorThreadFactory;
+import eu.stratosphere.nephele.taskmanager.transferenvelope.RegisterTaskManagerResult;
 import eu.stratosphere.nephele.topology.NetworkTopology;
 import eu.stratosphere.nephele.types.IntegerRecord;
 import eu.stratosphere.nephele.util.SerializableArrayList;
@@ -135,7 +135,7 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 
 	private final InputSplitManager inputSplitManager;
 
-	private final AbstractScheduler scheduler;
+	private final DefaultScheduler scheduler;
 	
 	private AccumulatorManager accumulatorManager;
 
@@ -213,20 +213,11 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 		LOG.info("Starting job manager in " + executionMode + " mode");
 
 		// Try to load the instance manager for the given execution mode
-		// Try to load the scheduler for the given execution mode
-		if (executionMode == ExecutionMode.LOCAL) {
-			try {
-				this.instanceManager = new LocalInstanceManager();
-			} catch (Throwable t) {
-				throw new Exception("Cannot instantiate local instance manager: " + t.getMessage(), t);
-			}
-		} else {
-			final String instanceManagerClassName = JobManagerUtils.getInstanceManagerClassName(executionMode);
-			LOG.info("Trying to load " + instanceManagerClassName + " as instance manager");
-			this.instanceManager = JobManagerUtils.loadInstanceManager(instanceManagerClassName);
-			if (this.instanceManager == null) {
-				throw new Exception("Unable to load instance manager " + instanceManagerClassName);
-			}
+		final String instanceManagerClassName = JobManagerUtils.getInstanceManagerClassName(executionMode);
+		LOG.info("Trying to load " + instanceManagerClassName + " as instance manager");
+		this.instanceManager = JobManagerUtils.loadInstanceManager(instanceManagerClassName);
+		if (this.instanceManager == null) {
+			throw new Exception("Unable to load instance manager " + instanceManagerClassName);
 		}
 
 		// Try to load the scheduler for the given execution mode
@@ -479,7 +470,7 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 			ExecutionGraph eg;
 	
 			try {
-				eg = new ExecutionGraph(job, this.instanceManager);
+				eg = new ExecutionGraph(job, this.getAvailableSlots());
 			} catch (GraphConversionException e) {
 				if (e.getCause() == null) {
 					return new JobSubmissionResult(AbstractJobResult.ReturnCode.ERROR, StringUtils.stringifyException(e));
@@ -520,7 +511,7 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 			}
 	
 			try {
-				this.scheduler.schedulJob(eg);
+				this.scheduler.scheduleJob(eg);
 			} catch (SchedulingException e) {
 				unregisterJob(eg);
 				JobSubmissionResult result = new JobSubmissionResult(AbstractJobResult.ReturnCode.ERROR, StringUtils.stringifyException(e));
@@ -561,10 +552,6 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 			}
 		}
 
-		// Cancel all pending requests for instances
-		this.instanceManager.cancelPendingRequests(executionGraph.getJobID()); // getJobID is final member, no
-																				// synchronization necessary
-
 		// Remove job from input split manager
 		if (this.inputSplitManager != null) {
 			this.inputSplitManager.unregisterJob(executionGraph);
@@ -582,8 +569,7 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 
 
 	@Override
-	public void sendHeartbeat(final InstanceConnectionInfo instanceConnectionInfo,
-			final HardwareDescription hardwareDescription) {
+	public void sendHeartbeat(final InstanceConnectionInfo instanceConnectionInfo) {
 
 		// Delegate call to instance manager
 		if (this.instanceManager != null) {
@@ -592,7 +578,7 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 
 				@Override
 				public void run() {
-					instanceManager.reportHeartBeat(instanceConnectionInfo, hardwareDescription);
+					instanceManager.reportHeartBeat(instanceConnectionInfo);
 				}
 			};
 
@@ -600,6 +586,25 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 		}
 	}
 
+	@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());
+				}
+			};
+
+			this.executorService.execute(registerTaskManagerRunnable);
+			return new RegisterTaskManagerResult(RegisterTaskManagerResult.ReturnCode.SUCCESS);
+		}
+
+		return new RegisterTaskManagerResult(RegisterTaskManagerResult.ReturnCode.FAILURE);
+	}
+
 
 	@Override
 	public void updateTaskExecutionState(final TaskExecutionState executionState) throws IOException {
@@ -730,9 +735,10 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 
 		if (sourceChannelID.equals(edge.getInputChannelID())) {
 			// Request was sent from an input channel
+
 			final ExecutionVertex connectedVertex = edge.getOutputGate().getVertex();
 
-			final AbstractInstance assignedInstance = connectedVertex.getAllocatedResource().getInstance();
+			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");
@@ -758,6 +764,7 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 				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());
 
@@ -788,7 +795,7 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 			return ConnectionInfoLookupResponse.createReceiverNotReady();
 		}
 
-		final AbstractInstance assignedInstance = targetVertex.getAllocatedResource().getInstance();
+		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");
@@ -877,6 +884,7 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 		return eventList;
 	}
 
+
 	@Override
 	public void killTask(final JobID jobID, final ManagementVertexID id) throws IOException {
 
@@ -909,10 +917,11 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 		eg.executeCommand(runnable);
 	}
 
+
 	@Override
 	public void killInstance(final StringRecord instanceName) throws IOException {
 
-		final AbstractInstance instance = this.instanceManager.getInstanceByName(instanceName.toString());
+		final Instance instance = this.instanceManager.getInstanceByName(instanceName.toString());
 		if (instance == null) {
 			LOG.error("Cannot find instance with name " + instanceName + " to kill it");
 			return;
@@ -947,16 +956,6 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 	}
 
 
-	public Map<InstanceType, InstanceTypeDescription> getMapOfAvailableInstanceTypes() {
-
-		// Delegate call to the instance manager
-		if (this.instanceManager != null) {
-			return this.instanceManager.getMapOfAvailableInstanceTypes();
-		}
-
-		return null;
-	}
-
 
 	@Override
 	public void jobStatusHasChanged(final ExecutionGraph executionGraph, final InternalJobStatus newJobStatus,
@@ -987,7 +986,7 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 			return;
 		}
 
-		final Set<AbstractInstance> allocatedInstance = new HashSet<AbstractInstance>();
+		final Set<Instance> allocatedInstance = new HashSet<Instance>();
 
 		final Iterator<ExecutionVertex> it = new ExecutionGraphIterator(eg, true);
 		while (it.hasNext()) {
@@ -995,7 +994,7 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 			final ExecutionVertex vertex = it.next();
 			final ExecutionState state = vertex.getExecutionState();
 			if (state == ExecutionState.RUNNING || state == ExecutionState.FINISHING) {
-				final AbstractInstance instance = vertex.getAllocatedResource().getInstance();
+				final Instance instance = vertex.getAllocatedResource().getInstance();
 
 				if (instance instanceof DummyInstance) {
 					LOG.error("Found instance of type DummyInstance for vertex " + vertex.getName() + " (state "
@@ -1013,7 +1012,7 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 			@Override
 			public void run() {
 
-				final Iterator<AbstractInstance> it2 = allocatedInstance.iterator();
+				final Iterator<Instance> it2 = allocatedInstance.iterator();
 
 				try {
 					while (it2.hasNext()) {
@@ -1030,9 +1029,14 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 		this.executorService.execute(requestRunnable);
 	}
 
+	@Override
+	public int getAvailableSlots() {
+		return getInstanceManager().getNumberOfSlots();
+	}
+
 
 	@Override
-	public void deploy(final JobID jobID, final AbstractInstance instance,
+	public void deploy(final JobID jobID, final Instance instance,
 			final List<ExecutionVertex> verticesToBeDeployed) {
 
 		if (verticesToBeDeployed.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManagerUtils.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManagerUtils.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManagerUtils.java
index 5b0b30d..45506aa 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManagerUtils.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManagerUtils.java
@@ -20,12 +20,11 @@ import java.lang.reflect.InvocationTargetException;
 import java.util.Properties;
 
 import eu.stratosphere.nephele.ExecutionMode;
-
+import eu.stratosphere.nephele.instance.InstanceManager;
+import eu.stratosphere.nephele.jobmanager.scheduler.DefaultScheduler;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
-import eu.stratosphere.nephele.instance.InstanceManager;
-import eu.stratosphere.nephele.jobmanager.scheduler.AbstractScheduler;
 import eu.stratosphere.util.StringUtils;
 
 /**
@@ -47,7 +46,7 @@ public class JobManagerUtils {
 
 	/**
 	 * Tries to locate a class with given name and to
-	 * instantiate a {@link AbstractScheduler} object from it.
+	 * instantiate a {@link eu.stratosphere.nephele.jobmanager.scheduler.DefaultScheduler} object from it.
 	 * 
 	 * @param schedulerClassName
 	 *        the name of the class to instantiate the scheduler object from
@@ -55,21 +54,21 @@ public class JobManagerUtils {
 	 *        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 AbstractScheduler} object instantiated from the class with the provided name
+	 * @return the {@link eu.stratosphere.nephele.jobmanager.scheduler.DefaultScheduler} object instantiated from the class with the provided name
 	 */
 	@SuppressWarnings("unchecked")
-	static AbstractScheduler loadScheduler(final String schedulerClassName, final DeploymentManager deploymentManager,
+	static DefaultScheduler loadScheduler(final String schedulerClassName, final DeploymentManager deploymentManager,
 			final InstanceManager instanceManager) {
 
-		Class<? extends AbstractScheduler> schedulerClass;
+		Class<? extends DefaultScheduler> schedulerClass;
 		try {
-			schedulerClass = (Class<? extends AbstractScheduler>) Class.forName(schedulerClassName);
+			schedulerClass = (Class<? extends DefaultScheduler>) Class.forName(schedulerClassName);
 		} catch (ClassNotFoundException e) {
 			LOG.error("Cannot find class " + schedulerClassName + ": " + StringUtils.stringifyException(e));
 			return null;
 		}
 
-		Constructor<? extends AbstractScheduler> constructor;
+		Constructor<? extends DefaultScheduler> constructor;
 
 		try {
 
@@ -83,7 +82,7 @@ public class JobManagerUtils {
 			return null;
 		}
 
-		AbstractScheduler scheduler;
+		DefaultScheduler scheduler;
 
 		try {
 			scheduler = constructor.newInstance(deploymentManager, instanceManager);
@@ -110,7 +109,7 @@ public class JobManagerUtils {
 	 * 
 	 * @param instanceManagerClassName
 	 *        the name of the class to instantiate the instance manager object from
-	 * @return the {@link InstanceManager} object instantiated from the class with the provided name
+	 * @return the {@link eu.stratosphere.nephele.instance.InstanceManager} object instantiated from the class with the provided name
 	 */
 	@SuppressWarnings("unchecked")
 	static InstanceManager loadInstanceManager(final String instanceManagerClassName) {
@@ -139,53 +138,34 @@ public class JobManagerUtils {
 	}
 
 	/**
-	 * Tries to read the class name of the {@link AbstractScheduler} implementation from the global configuration which
+	 * Tries to read the class name of the {@link eu.stratosphere.nephele.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 AbstractScheduler} implementation to be used or <code>null</code> if no
+	 * @return the class name of the {@link eu.stratosphere.nephele.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) {
-		switch (executionMode) {
-		case LOCAL:
-			return "eu.stratosphere.nephele.jobmanager.scheduler.local.LocalScheduler";
-		case CLUSTER:
-			return "eu.stratosphere.nephele.jobmanager.scheduler.queue.QueueScheduler";
-		default:
-			throw new RuntimeException("Unrecognized Execution Mode.");
-		}
-//		String modeClass = getClassStringForMode(executionMode);
-//		String instanceManagerClassNameKey = "jobmanager.scheduler." + modeClass + ".classname";
-//		String schedulerClassName = GlobalConfiguration.getString(instanceManagerClassNameKey, null);
-//
-//		if (executionMode == ExecutionMode.LOCAL && schedulerClassName == null) {
-//			schedulerClassName = ConfigConstants.DEFAULT_LOCAL_MODE_SCHEDULER;
-//		}
-//		return schedulerClassName;
+		return "eu.stratosphere.nephele.jobmanager.scheduler.DefaultScheduler";
 	}
 
 	/**
-	 * Tries to read the class name of the {@link InstanceManager} implementation from the global configuration which is
+	 * Tries to read the class name of the {@link eu.stratosphere.nephele.instance.InstanceManager} 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 InstanceManager} implementation to be used or <code>null</code> if no
+	 * @return the class name of the {@link eu.stratosphere.nephele.instance.InstanceManager} implementation to be used or <code>null</code> if no
 	 *         implementation is configured for the given execution mode
 	 */
 	static String getInstanceManagerClassName(ExecutionMode executionMode) {
 		switch (executionMode) {
 		case LOCAL:
-			return "eu.stratosphere.nephele.instance.local.LocalInstanceManager";
+			return "eu.stratosphere.nephele.instance.LocalInstanceManager";
 		case CLUSTER:
-			return "eu.stratosphere.nephele.instance.cluster.ClusterManager";
+			return "eu.stratosphere.nephele.instance.DefaultInstanceManager";
 		default:
 			throw new RuntimeException("Unrecognized Execution Mode.");
 		}
-//		
-//		final String modeClass = getClassStringForMode(executionMode);
-//		final String instanceManagerClassNameKey = "jobmanager.instancemanager." + modeClass + ".classname";
-//		return GlobalConfiguration.getString(instanceManagerClassNameKey, null);
 	}
 	
 	/**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java
deleted file mode 100644
index 5b528c7..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java
+++ /dev/null
@@ -1,166 +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.jobmanager.scheduler;
-
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Set;
-
-import eu.stratosphere.nephele.execution.ExecutionListener;
-import eu.stratosphere.nephele.execution.ExecutionState;
-import eu.stratosphere.nephele.executiongraph.ExecutionGraph;
-import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertex;
-import eu.stratosphere.nephele.executiongraph.ExecutionPipeline;
-import eu.stratosphere.nephele.executiongraph.ExecutionVertex;
-import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
-import eu.stratosphere.nephele.executiongraph.InternalJobStatus;
-import eu.stratosphere.nephele.jobgraph.JobID;
-
-public abstract class AbstractExecutionListener implements ExecutionListener {
-
-	/**
-	 * The instance of the {@link LocalScheduler}.
-	 */
-	private final AbstractScheduler scheduler;
-
-	/**
-	 * The {@link ExecutionVertex} this wrapper object belongs to.
-	 */
-	private final ExecutionVertex executionVertex;
-
-	/**
-	 * Constructs a new wrapper object for the given {@link ExecutionVertex}.
-	 * 
-	 * @param AbstractScheduler
-	 *        the instance of the {@link AbstractScheduler}
-	 * @param executionVertex
-	 *        the {@link ExecutionVertex} the received notification refer to
-	 */
-	public AbstractExecutionListener(final AbstractScheduler 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());
-		}
-
-		// In case of an error, check if the vertex shall be recovered
-		if (newExecutionState == ExecutionState.FAILED) {
-			if (this.executionVertex.decrementRetriesLeftAndCheck()) {
-
-				final Set<ExecutionVertex> assignedVertices = new HashSet<ExecutionVertex>();
-
-				if (RecoveryLogic.recover(this.executionVertex, this.scheduler.getVerticesToBeRestarted(),
-					assignedVertices)) {
-
-					if (RecoveryLogic.hasInstanceAssigned(this.executionVertex)) {
-						// Run through the deployment procedure
-						this.scheduler.deployAssignedVertices(assignedVertices);
-					}
-
-				} else {
-
-					// Make sure the map with the vertices to be restarted is cleaned up properly
-					synchronized (eg) {
-
-						final Iterator<ExecutionVertex> it = this.scheduler.getVerticesToBeRestarted().values()
-							.iterator();
-
-						while (it.hasNext()) {
-							if (eg.equals(it.next().getExecutionGraph())) {
-								it.remove();
-							}
-						}
-					}
-
-					// Actual cancellation of job is performed by job manager
-				}
-			}
-		}
-
-	}
-
-
-	@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;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java
deleted file mode 100644
index 24e2970..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java
+++ /dev/null
@@ -1,662 +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.jobmanager.scheduler;
-
-import java.util.ArrayList;
-import java.util.Collection;
-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.concurrent.ConcurrentHashMap;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.nephele.execution.ExecutionState;
-import eu.stratosphere.nephele.executiongraph.ExecutionEdge;
-import eu.stratosphere.nephele.executiongraph.ExecutionGate;
-import eu.stratosphere.nephele.executiongraph.ExecutionGraph;
-import eu.stratosphere.nephele.executiongraph.ExecutionGraphIterator;
-import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertex;
-import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertexIterator;
-import eu.stratosphere.nephele.executiongraph.ExecutionPipeline;
-import eu.stratosphere.nephele.executiongraph.ExecutionStage;
-import eu.stratosphere.nephele.executiongraph.ExecutionVertex;
-import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
-import eu.stratosphere.nephele.executiongraph.InternalJobStatus;
-import eu.stratosphere.nephele.instance.AbstractInstance;
-import eu.stratosphere.nephele.instance.AllocatedResource;
-import eu.stratosphere.nephele.instance.AllocationID;
-import eu.stratosphere.nephele.instance.DummyInstance;
-import eu.stratosphere.nephele.instance.InstanceException;
-import eu.stratosphere.nephele.instance.InstanceListener;
-import eu.stratosphere.nephele.instance.InstanceManager;
-import eu.stratosphere.nephele.instance.InstanceRequestMap;
-import eu.stratosphere.nephele.instance.InstanceType;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.jobmanager.DeploymentManager;
-import eu.stratosphere.util.StringUtils;
-
-/**
- * This abstract scheduler must be extended by a scheduler implementations for Nephele. The abstract class defines the
- * fundamental methods for scheduling and removing jobs. While Nephele's
- * {@link eu.stratosphere.nephele.jobmanager.JobManager} is responsible for requesting the required instances for the
- * job at the {@link eu.stratosphere.nephele.instance.InstanceManager}, the scheduler is in charge of assigning the
- * individual tasks to the instances.
- * 
- */
-public abstract class AbstractScheduler implements InstanceListener {
-
-	/**
-	 * The LOG object to report events within the scheduler.
-	 */
-	protected static final Log LOG = LogFactory.getLog(AbstractScheduler.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>();
-
-	/**
-	 * 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
-	 */
-	protected AbstractScheduler(final DeploymentManager deploymentManager, final InstanceManager instanceManager) {
-
-		this.deploymentManager = deploymentManager;
-		this.instanceManager = instanceManager;
-		this.instanceManager.setInstanceListener(this);
-	}
-
-	/**
-	 * 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 abstract void schedulJob(ExecutionGraph executionGraph) throws SchedulingException;
-
-	/**
-	 * 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 abstract ExecutionGraph getExecutionGraphByID(JobID jobID);
-
-	/**
-	 * Returns the {@link InstanceManager} object which is used by the current scheduler.
-	 * 
-	 * @return the {@link InstanceManager} object which is used by the current scheduler
-	 */
-	public InstanceManager getInstanceManager() {
-		return this.instanceManager;
-	}
-
-	// void removeJob(JobID jobID);
-
-	/**
-	 * Shuts the scheduler down. After shut down no jobs can be added to the scheduler.
-	 */
-	public abstract void shutdown();
-
-	/**
-	 * 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();
-		final InstanceRequestMap instanceRequestMap = new InstanceRequestMap();
-
-		synchronized (executionStage) {
-
-			executionStage.collectRequiredInstanceTypes(instanceRequestMap, ExecutionState.CREATED);
-
-			final Iterator<Map.Entry<InstanceType, Integer>> it = instanceRequestMap.getMinimumIterator();
-			LOG.info("Requesting the following instances for job " + executionGraph.getJobID());
-			while (it.hasNext()) {
-				final Map.Entry<InstanceType, Integer> entry = it.next();
-				LOG.info(" " + entry.getKey() + " [" + entry.getValue().intValue() + ", "
-					+ instanceRequestMap.getMaximumNumberOfInstances(entry.getKey()) + "]");
-			}
-
-			if (instanceRequestMap.isEmpty()) {
-				return;
-			}
-
-			this.instanceManager.requestInstance(executionGraph.getJobID(), executionGraph.getJobConfiguration(),
-				instanceRequestMap, null);
-
-			// 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<AbstractInstance, List<ExecutionVertex>> verticesToBeDeployed,
-			final Set<ExecutionVertex> alreadyVisited) {
-
-		if (!alreadyVisited.add(vertex)) {
-			return;
-		}
-
-		if (vertex.compareAndUpdateExecutionState(ExecutionState.ASSIGNED, ExecutionState.READY)) {
-			final AbstractInstance 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 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<AbstractInstance, List<ExecutionVertex>> verticesToBeDeployed = new HashMap<AbstractInstance, List<ExecutionVertex>>();
-		final Set<ExecutionVertex> alreadyVisited = new HashSet<ExecutionVertex>();
-
-		findVerticesToBeDeployed(startVertex, verticesToBeDeployed, alreadyVisited);
-
-		if (!verticesToBeDeployed.isEmpty()) {
-
-			final Iterator<Map.Entry<AbstractInstance, List<ExecutionVertex>>> it2 = verticesToBeDeployed
-				.entrySet()
-				.iterator();
-
-			while (it2.hasNext()) {
-
-				final Map.Entry<AbstractInstance, 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 AllocatedResource} objects.
-	 * 
-	 * @param pipeline
-	 *        the execution pipeline to be deployed
-	 */
-	public void deployAssignedPipeline(final ExecutionPipeline pipeline) {
-
-		final JobID jobID = null;
-
-		final Map<AbstractInstance, List<ExecutionVertex>> verticesToBeDeployed = new HashMap<AbstractInstance, 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<AbstractInstance, List<ExecutionVertex>>> it2 = verticesToBeDeployed
-				.entrySet()
-				.iterator();
-
-			while (it2.hasNext()) {
-
-				final Map.Entry<AbstractInstance, 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 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<AbstractInstance, List<ExecutionVertex>> verticesToBeDeployed = new HashMap<AbstractInstance, 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<AbstractInstance, List<ExecutionVertex>>> it2 = verticesToBeDeployed
-				.entrySet()
-				.iterator();
-
-			while (it2.hasNext()) {
-
-				final Map.Entry<AbstractInstance, 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 AllocatedResource} objects.
-	 * 
-	 * @param executionGraph
-	 *        the execution graph to collect the vertices from
-	 */
-	public void deployAssignedInputVertices(final ExecutionGraph executionGraph) {
-
-		final Map<AbstractInstance, List<ExecutionVertex>> verticesToBeDeployed = new HashMap<AbstractInstance, 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<AbstractInstance, List<ExecutionVertex>>> it2 = verticesToBeDeployed
-				.entrySet()
-				.iterator();
-
-			while (it2.hasNext()) {
-
-				final Map.Entry<AbstractInstance, 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(jobID, null, 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) {
-									// In local mode, we do not consider any topology, only the instance type
-									if (vertex.getAllocatedResource().getInstanceType().equals(
-										allocatedResource.getInstanceType())) {
-										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(jobID, eg.getJobConfiguration(),
-									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(executionGraph.getJobID(), executionGraph
-					.getJobConfiguration(), allocatedResource);
-			} catch (InstanceException e) {
-				LOG.error(StringUtils.stringifyException(e));
-			}
-		}
-	}
-
-	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(allocatedResource
-							.getInstance()
-							.getType());
-						final AllocatedResource dummyResource = new AllocatedResource(dummyInstance,
-							allocatedResource.getInstanceType(), 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);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/DefaultExecutionListener.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/DefaultExecutionListener.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/DefaultExecutionListener.java
new file mode 100644
index 0000000..86b3c40
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/DefaultExecutionListener.java
@@ -0,0 +1,127 @@
+/***********************************************************************************************************************
+ * 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.jobmanager.scheduler;
+
+import eu.stratosphere.nephele.execution.ExecutionListener;
+import eu.stratosphere.nephele.execution.ExecutionState;
+import eu.stratosphere.nephele.executiongraph.ExecutionGraph;
+import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertex;
+import eu.stratosphere.nephele.executiongraph.ExecutionPipeline;
+import eu.stratosphere.nephele.executiongraph.ExecutionVertex;
+import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
+import eu.stratosphere.nephele.executiongraph.InternalJobStatus;
+import eu.stratosphere.nephele.jobgraph.JobID;
+
+public class DefaultExecutionListener implements ExecutionListener {
+
+	/**
+	 * The instance of the {@link eu.stratosphere.nephele.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;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/DefaultScheduler.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/DefaultScheduler.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/DefaultScheduler.java
new file mode 100644
index 0000000..745b199
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/DefaultScheduler.java
@@ -0,0 +1,762 @@
+/***********************************************************************************************************************
+ * 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.jobmanager.scheduler;
+
+import java.util.ArrayList;
+import java.util.Collection;
+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.concurrent.ConcurrentHashMap;
+import java.util.Deque;
+import java.util.ArrayDeque;
+
+import eu.stratosphere.nephele.executiongraph.ExecutionEdge;
+import eu.stratosphere.nephele.executiongraph.ExecutionGate;
+import eu.stratosphere.nephele.executiongraph.ExecutionGraph;
+import eu.stratosphere.nephele.executiongraph.ExecutionGraphIterator;
+import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertex;
+import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertexIterator;
+import eu.stratosphere.nephele.executiongraph.ExecutionPipeline;
+import eu.stratosphere.nephele.executiongraph.ExecutionStage;
+import eu.stratosphere.nephele.executiongraph.ExecutionStageListener;
+import eu.stratosphere.nephele.executiongraph.ExecutionVertex;
+import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
+import eu.stratosphere.nephele.executiongraph.InternalJobStatus;
+import eu.stratosphere.nephele.executiongraph.JobStatusListener;
+import eu.stratosphere.nephele.instance.AllocatedResource;
+import eu.stratosphere.nephele.instance.AllocationID;
+import eu.stratosphere.nephele.instance.DummyInstance;
+import eu.stratosphere.nephele.instance.InstanceException;
+import eu.stratosphere.nephele.instance.InstanceListener;
+import eu.stratosphere.nephele.instance.InstanceManager;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import eu.stratosphere.nephele.execution.ExecutionState;
+import eu.stratosphere.nephele.instance.Instance;
+import eu.stratosphere.nephele.jobgraph.JobID;
+import eu.stratosphere.nephele.jobmanager.DeploymentManager;
+import eu.stratosphere.util.StringUtils;
+
+/**
+ * The default scheduler for Nephele. While Nephele's
+ * {@link eu.stratosphere.nephele.jobmanager.JobManager} is responsible for requesting the required instances for the
+ * job at the {@link eu.stratosphere.nephele.instance.InstanceManager}, the scheduler is in charge of assigning the
+ * individual tasks to the instances.
+ * 
+ */
+public class DefaultScheduler implements InstanceListener, JobStatusListener, ExecutionStageListener {
+
+	/**
+	 * The LOG object to report events within the scheduler.
+	 */
+	protected static final Log LOG = LogFactory.getLog(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);
+	}
+
+	/**
+	 * 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("Not enough slots to schedule job " + executionGraph.getJobID());
+		}
+
+		// 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;
+	}
+
+	/**
+	 * Shuts the scheduler down. After shut down no jobs can be added to the scheduler.
+	 */
+	public void shutdown() {
+
+		synchronized (this.jobQueue) {
+			this.jobQueue.clear();
+		}
+
+	}
+
+	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 eu.stratosphere.nephele.instance.InstanceManager} object which is used by the current scheduler.
+	 * 
+	 * @return the {@link eu.stratosphere.nephele.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 + " 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 eu.stratosphere.nephele.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 eu.stratosphere.nephele.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 eu.stratosphere.nephele.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 eu.stratosphere.nephele.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;
+	}
+
+	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
+					 * }
+					 */
+				}
+			}
+
+			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);
+	}
+}


[31/53] [abbrv] git commit: Show version information including commit date in TaskMgr log

Posted by rm...@apache.org.
Show version information including commit date in TaskMgr log


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

Branch: refs/heads/travis_test
Commit: 7d0e89d2b0f8894ef5356346f7dd78c0a0361054
Parents: 9ff440f
Author: Robert Metzger <rm...@apache.org>
Authored: Tue Jun 24 11:01:06 2014 +0200
Committer: Robert Metzger <rm...@apache.org>
Committed: Tue Jun 24 11:01:06 2014 +0200

----------------------------------------------------------------------
 .../nephele/jobmanager/JobManager.java             |  7 ++++++-
 .../nephele/jobmanager/JobManagerUtils.java        | 14 ++++++++++++--
 .../jobmanager/web/JobmanagerInfoServlet.java      |  2 +-
 .../nephele/taskmanager/TaskManager.java           |  9 ++++++++-
 tools/merge_pull_request.sh.template               | 17 +++++++++++++++++
 5 files changed, 44 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/7d0e89d2/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java
index 40e2a0b..6401407 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java
@@ -79,6 +79,7 @@ import eu.stratosphere.nephele.ipc.Server;
 import eu.stratosphere.nephele.jobgraph.AbstractJobVertex;
 import eu.stratosphere.nephele.jobgraph.JobGraph;
 import eu.stratosphere.nephele.jobgraph.JobID;
+import eu.stratosphere.nephele.jobmanager.JobManagerUtils.RevisionInformation;
 import eu.stratosphere.nephele.jobmanager.accumulators.AccumulatorManager;
 import eu.stratosphere.nephele.jobmanager.archive.ArchiveListener;
 import eu.stratosphere.nephele.jobmanager.archive.MemoryArchivist;
@@ -291,7 +292,11 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 	 * Log Stratosphere version information.
 	 */
 	private static void logVersionInformation() {
-		LOG.info("Starting Stratosphere JobManager (Version: " + JobManagerUtils.getVersion() + ", Rev:" + JobManagerUtils.getRevision() + ")");
+		RevisionInformation rev = JobManagerUtils.getRevisionInformation();
+		LOG.info("Starting Stratosphere JobManager "
+				+ "(Version: " + JobManagerUtils.getVersion() + ", "
+					+ "Rev:" + rev.commitId + ", "
+					+ "Date:" + rev.commitDate + ")");
 	}
 	
 	/**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/7d0e89d2/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManagerUtils.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManagerUtils.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManagerUtils.java
index 45506aa..f2e1d33 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManagerUtils.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManagerUtils.java
@@ -182,18 +182,28 @@ public class JobManagerUtils {
 	 * Returns the revision of Stratosphere as String.
 	 * @return String
 	 */
-	public static String getRevision() {
+	public static RevisionInformation getRevisionInformation() {
+		RevisionInformation info = new RevisionInformation();
 		String revision = "<unknown>";
+		String commitDate = "<unknown>";
 		try {
 			Properties properties = new Properties();
 			InputStream propFile = JobManagerUtils.class.getClassLoader().getResourceAsStream(".version.properties");
 			if (propFile != null) {
 				properties.load(propFile);
 				revision = properties.getProperty("git.commit.id.abbrev");
+				commitDate = properties.getProperty("git.commit.time");
 			}
 		} catch (IOException e) {
 			LOG.info("Cannot determine code revision. Unable ro read version property file.");
 		}
-		return revision;
+		info.commitId = revision;
+		info.commitDate = commitDate;
+		return info;
+	}
+	
+	public static class RevisionInformation {
+		public String commitId;
+		public String commitDate;
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/7d0e89d2/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/web/JobmanagerInfoServlet.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/web/JobmanagerInfoServlet.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/web/JobmanagerInfoServlet.java
index bb35100..8ffbd57 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/web/JobmanagerInfoServlet.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/web/JobmanagerInfoServlet.java
@@ -517,7 +517,7 @@ public class JobmanagerInfoServlet extends HttpServlet {
 	private void writeJsonForVersion(PrintWriter wrt) {
 		wrt.write("{");
 		wrt.write("\"version\": \"" + JobManagerUtils.getVersion() + "\",");
-		wrt.write("\"revision\": \"" + JobManagerUtils.getRevision() + "\"");
+		wrt.write("\"revision\": \"" + JobManagerUtils.getRevisionInformation().commitId + "\"");
 		wrt.write("}");
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/7d0e89d2/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
index 3225ab7..9b623bd 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
@@ -80,6 +80,8 @@ import eu.stratosphere.nephele.instance.InstanceConnectionInfo;
 import eu.stratosphere.nephele.ipc.RPC;
 import eu.stratosphere.nephele.ipc.Server;
 import eu.stratosphere.nephele.jobgraph.JobID;
+import eu.stratosphere.nephele.jobmanager.JobManagerUtils;
+import eu.stratosphere.nephele.jobmanager.JobManagerUtils.RevisionInformation;
 import eu.stratosphere.nephele.net.NetUtils;
 import eu.stratosphere.nephele.profiling.ProfilingUtils;
 import eu.stratosphere.nephele.profiling.TaskManagerProfiler;
@@ -172,13 +174,18 @@ public class TaskManager implements TaskOperationProtocol {
 			throw new NullPointerException("Execution mode must not be null.");
 		}
 		
+		RevisionInformation rev = JobManagerUtils.getRevisionInformation();
+		LOG.info("Starting Stratosphere TaskManager "
+				+ "(Version: " + JobManagerUtils.getVersion() + ", "
+					+ "Rev:" + rev.commitId + ", "
+					+ "Date:" + rev.commitDate + ")");
+		
 		try {
 			LOG.info("TaskManager started as user " + UserGroupInformation.getCurrentUser().getShortUserName());
 		} catch (Throwable t) {
 			LOG.error("Cannot determine user group information.", t);
 		}
 			
-		LOG.info("User system property: " + System.getProperty("user.name"));
 		LOG.info("Execution mode: " + executionMode);
 
 		// IMPORTANT! At this point, the GlobalConfiguration must have been read!

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/7d0e89d2/tools/merge_pull_request.sh.template
----------------------------------------------------------------------
diff --git a/tools/merge_pull_request.sh.template b/tools/merge_pull_request.sh.template
index d506d9d..0333fec 100755
--- a/tools/merge_pull_request.sh.template
+++ b/tools/merge_pull_request.sh.template
@@ -1,5 +1,22 @@
 #!/bin/sh
 
+#
+# 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.
+#
+
 # the directory where you have your flink code
 export FLINK_HOME="/home/robert/projects/flink/incubator-flink"
 # Remote name which points to the Gihub site


[16/53] [abbrv] git commit: Removed RuntimeEnvironment instantiation from execution graph construction. Removed legacy job vertex classes and input/output tasks.

Posted by rm...@apache.org.
Removed RuntimeEnvironment instantiation from execution graph construction. Removed legacy job vertex classes and input/output 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/ea79186b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/ea79186b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/ea79186b

Branch: refs/heads/travis_test
Commit: ea79186b7ef787991fa1c4dbfa29f26c7aefd804
Parents: 429493d
Author: Till Rohrmann <ti...@mailbox.tu-berlin.de>
Authored: Wed Mar 26 02:58:15 2014 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Sun Jun 22 21:07:20 2014 +0200

----------------------------------------------------------------------
 .../plantranslate/NepheleJobGraphGenerator.java |  33 +--
 .../api/common/io/FileOutputFormat.java         |  38 ++++
 .../api/common/io/OutputFormat.java             |   7 +
 .../configuration/Configuration.java            |  12 ++
 .../api/java/io/PrintingOutputFormat.java       |   3 +
 .../nephele/execution/RuntimeEnvironment.java   |  56 ++----
 .../nephele/executiongraph/ExecutionGraph.java  |  86 ++------
 .../executiongraph/ExecutionGroupVertex.java    |  53 ++---
 .../jobgraph/AbstractJobInputVertex.java        |   7 +
 .../nephele/jobgraph/AbstractJobVertex.java     |  62 +-----
 .../nephele/jobgraph/JobFileInputVertex.java    | 195 ------------------
 .../nephele/jobgraph/JobFileOutputVertex.java   | 198 ------------------
 .../nephele/jobgraph/JobGenericInputVertex.java | 168 ----------------
 .../jobgraph/JobGenericOutputVertex.java        | 182 -----------------
 .../nephele/jobgraph/JobInputVertex.java        |  90 ++++++++-
 .../nephele/jobgraph/JobOutputVertex.java       |  56 ++++++
 .../nephele/jobgraph/JobTaskVertex.java         |  17 --
 .../splitassigner/InputSplitManager.java        |  13 +-
 .../LocatableInputSplitAssigner.java            |  14 +-
 .../file/FileInputSplitAssigner.java            |  14 +-
 .../nephele/template/AbstractFileInputTask.java | 201 -------------------
 .../template/AbstractFileOutputTask.java        |  46 -----
 .../template/AbstractGenericInputTask.java      |  39 ----
 .../nephele/template/AbstractInputTask.java     |  23 ---
 .../nephele/template/AbstractInvokable.java     |  33 ---
 .../nephele/template/GenericInputTask.java      |  39 ----
 .../pact/runtime/task/DataSinkTask.java         |  58 ------
 .../pact/runtime/task/DataSourceTask.java       |  60 +++---
 .../pact/runtime/task/util/TaskConfig.java      |   4 +
 .../TaskDeploymentDescriptorTest.java           |   8 +-
 .../executiongraph/SelfCrossInputTask.java      |  41 ----
 .../nephele/jobmanager/DoubleSourceTask.java    |  81 --------
 .../nephele/jobmanager/DoubleTargetTask.java    |  18 +-
 .../nephele/jobmanager/ForwardTask.java         |  12 +-
 .../nephele/jobmanager/UnionTask.java           |  16 +-
 .../scheduler/queue/DefaultSchedulerTest.java   |  43 ++++
 .../nephele/util/FileLineReader.java            |  80 --------
 .../nephele/util/FileLineWriter.java            |  75 -------
 .../io/library/FileLineReadWriteTest.java       | 136 -------------
 .../recordJobs/util/DiscardingOutputFormat.java |   3 +
 40 files changed, 412 insertions(+), 1908 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plantranslate/NepheleJobGraphGenerator.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plantranslate/NepheleJobGraphGenerator.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plantranslate/NepheleJobGraphGenerator.java
index b4c7560..200ef7c 100644
--- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plantranslate/NepheleJobGraphGenerator.java
+++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plantranslate/NepheleJobGraphGenerator.java
@@ -20,7 +20,14 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
+
+import eu.stratosphere.api.common.io.InputFormat;
+import eu.stratosphere.api.common.io.OutputFormat;
+import eu.stratosphere.api.common.operators.util.UserCodeWrapper;
+import eu.stratosphere.core.io.InputSplit;
+import eu.stratosphere.nephele.template.AbstractInputTask;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 
 import eu.stratosphere.api.common.aggregators.AggregatorRegistry;
 import eu.stratosphere.api.common.aggregators.AggregatorWithName;
@@ -805,31 +812,31 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 
 	private JobInputVertex createDataSourceVertex(SourcePlanNode node) throws CompilerException {
 		final JobInputVertex vertex = new JobInputVertex(node.getNodeName(), this.jobGraph);
-		final TaskConfig config = new TaskConfig(vertex.getConfiguration());
-		
+
 		// set task class
 		@SuppressWarnings("unchecked")
-		final Class<AbstractInputTask<?>> clazz = (Class<AbstractInputTask<?>>) (Class<?>) DataSourceTask.class;
+		final Class<AbstractInputTask<?>> clazz = (Class<AbstractInputTask<?>>) (Class<?>) DataSourceTask
+				.class;
 		vertex.setInputClass(clazz);
 
 		// set user code
-		config.setStubWrapper(node.getPactContract().getUserCodeWrapper());
-		config.setStubParameters(node.getPactContract().getParameters());
-		
-		config.setOutputSerializer(node.getSerializer());
+		vertex.setInputFormat((UserCodeWrapper<? extends InputFormat<?, InputSplit>>)node.getPactContract()
+				.getUserCodeWrapper());
+		vertex.setInputFormatParameters(node.getPactContract().getParameters());
+		vertex.setOutputSerializer(node.getSerializer());
 		return vertex;
 	}
 
 	private AbstractJobOutputVertex createDataSinkVertex(SinkPlanNode node) throws CompilerException {
 		final JobOutputVertex vertex = new JobOutputVertex(node.getNodeName(), this.jobGraph);
-		final TaskConfig config = new TaskConfig(vertex.getConfiguration());
-		
+
 		vertex.setOutputClass(DataSinkTask.class);
 		vertex.getConfiguration().setInteger(DataSinkTask.DEGREE_OF_PARALLELISM_KEY, node.getDegreeOfParallelism());
-		
+
 		// set user code
-		config.setStubWrapper(node.getPactContract().getUserCodeWrapper());
-		config.setStubParameters(node.getPactContract().getParameters());
+		vertex.setOutputFormat((UserCodeWrapper<? extends OutputFormat<?>>)node.getPactContract().getUserCodeWrapper
+				());
+		vertex.setOutputFormatParameters(node.getPactContract().getParameters());
 		
 		return vertex;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/FileOutputFormat.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/FileOutputFormat.java b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/FileOutputFormat.java
index b04ced9..7733c71 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/FileOutputFormat.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/FileOutputFormat.java
@@ -437,4 +437,42 @@ public abstract class FileOutputFormat<IT> implements OutputFormat<IT> {
 			super(targetConfig);
 		}
 	}
+
+	@Override
+	public void initialize(Configuration configuration){
+		final Path path = this.getOutputFilePath();
+		final WriteMode writeMode = this.getWriteMode();
+		final OutputDirectoryMode outDirMode = this.getOutputDirectoryMode();
+
+		// Prepare output path and determine max DOP
+		try {
+			final FileSystem fs = path.getFileSystem();
+
+			int dop = configuration.getInteger(DEGREE_OF_PARALLELISM_KEY, -1);
+			if(dop == 1 && outDirMode == OutputDirectoryMode.PARONLY) {
+				// output is not written in parallel and should be written to a single file.
+
+				if(fs.isDistributedFS()) {
+					// prepare distributed output path
+					if(!fs.initOutPathDistFS(path, writeMode, false)) {
+						// output preparation failed! Cancel task.
+						throw new IOException("Output path could not be initialized.");
+					}
+				}
+			} else {
+				// output should be written to a directory
+
+				if(fs.isDistributedFS()) {
+					// only distributed file systems can be initialized at start-up time.
+					if(!fs.initOutPathDistFS(path, writeMode, true)) {
+						throw new IOException("Output directory could not be created.");
+					}
+				}
+			}
+		}
+		catch (IOException e) {
+			LOG.error("Could not access the file system to detemine the status of the output.", e);
+			throw new RuntimeException("I/O Error while accessing file", e);
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/OutputFormat.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/OutputFormat.java b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/OutputFormat.java
index c32be78..3b66902 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/OutputFormat.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/OutputFormat.java
@@ -79,5 +79,12 @@ public interface OutputFormat<IT> extends Serializable {
 	 * @throws IOException Thrown, if the input could not be closed properly.
 	 */
 	void close() throws IOException;
+
+	/**
+	 * Method which is called on the master node prior to execution. It can be used to set up the output format.
+	 *
+	 * @param configuration The task configuration
+	 */
+	void initialize(Configuration configuration);
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-core/src/main/java/eu/stratosphere/configuration/Configuration.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/configuration/Configuration.java b/stratosphere-core/src/main/java/eu/stratosphere/configuration/Configuration.java
index 46cadc3..0271b59 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/configuration/Configuration.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/configuration/Configuration.java
@@ -405,6 +405,18 @@ public class Configuration implements IOReadableWritable {
 			}
 		}
 	}
+
+	/**
+	 * Checks whether there is an entry with key
+	 *
+	 * @param key key of entry
+	 * @return true if entry with key is stored in the configuration, otherwise false
+	 */
+	public boolean containsKey(String key){
+		synchronized (this.confData){
+			return this.confData.containsKey(key);
+		}
+	}
 	
 	// --------------------------------------------------------------------------------------------
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-java/src/main/java/eu/stratosphere/api/java/io/PrintingOutputFormat.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/io/PrintingOutputFormat.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/io/PrintingOutputFormat.java
index 5c09439..d1736d4 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/io/PrintingOutputFormat.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/io/PrintingOutputFormat.java
@@ -95,4 +95,7 @@ public class PrintingOutputFormat<T> implements OutputFormat<T> {
 	public String toString() {
 		return "Print to " + (target == STD_OUT ? "System.out" : "System.err");
 	}
+
+	@Override
+	public void initialize(Configuration configuration){}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java
index 4e07694..70718a9 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java
@@ -83,6 +83,12 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 	private final List<InputGate<? extends IOReadableWritable>> inputGates = new CopyOnWriteArrayList<InputGate<? extends IOReadableWritable>>();
 
 	/**
+	 * Queue of unbound output gate IDs which are required for deserializing an environment in the course of an RPC
+	 * call.
+	 */
+	private final Queue<GateID> unboundOutputGateIDs = new ArrayDeque<GateID>();
+
+	/**
 	 * Queue of unbound input gate IDs which are required for deserializing an environment in the course of an RPC
 	 * call.
 	 */
@@ -165,46 +171,18 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 	private volatile boolean canceled;
 
 	/**
-	 * Creates a new runtime environment object which contains the runtime information for the encapsulated Nephele
-	 * task.
-	 *
-	 * @param jobID             the ID of the original Nephele job
-	 * @param taskName          the name of task running in this environment
-	 * @param invokableClass    invokableClass the class that should be instantiated as a Nephele task
-	 * @param taskConfiguration the configuration object which was attached to the original JobVertex
-	 * @param jobConfiguration  the configuration object which was attached to the original JobGraph
-	 * @throws Exception thrown if an error occurs while instantiating the invokable class
-	 */
-	public RuntimeEnvironment(final JobID jobID, final String taskName,
-							final Class<? extends AbstractInvokable> invokableClass, final Configuration taskConfiguration,
-							final Configuration jobConfiguration)
-		throws Exception
-	{
-		this.jobID = jobID;
-		this.taskName = taskName;
-		this.invokableClass = invokableClass;
-		this.taskConfiguration = taskConfiguration;
-		this.jobConfiguration = jobConfiguration;
-		this.indexInSubtaskGroup = 0;
-		this.currentNumberOfSubtasks = 0;
-		this.memoryManager = null;
-		this.ioManager = null;
-		this.inputSplitProvider = null;
-		this.cacheCopyTasks = new HashMap<String, FutureTask<Path>>();
-		
-		this.invokable = this.invokableClass.newInstance();
-		this.invokable.setEnvironment(this);
-		this.invokable.registerInputOutput();
-	}
-
-	/**
 	 * 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
+	 * 
+	 * @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,

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java
index c5059f9..93e0a25 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java
@@ -399,17 +399,6 @@ public class ExecutionGraph implements ExecutionListener {
 			final ExecutionVertex sev = entry.getValue();
 			final ExecutionGroupVertex sgv = sev.getGroupVertex();
 
-			// First compare number of output gates
-			if (sjv.getNumberOfForwardConnections() != sgv.getEnvironment().getNumberOfOutputGates()) {
-				throw new GraphConversionException("Job and execution vertex " + sjv.getName()
-					+ " have different number of outputs");
-			}
-
-			if (sjv.getNumberOfBackwardConnections() != sgv.getEnvironment().getNumberOfInputGates()) {
-				throw new GraphConversionException("Job and execution vertex " + sjv.getName()
-					+ " have different number of inputs");
-			}
-
 			// First, build the group edges
 			for (int i = 0; i < sjv.getNumberOfForwardConnections(); ++i) {
 				final JobEdge edge = sjv.getForwardConnection(i);
@@ -488,16 +477,13 @@ public class ExecutionGraph implements ExecutionListener {
 
 			final InputSplit[] inputSplits;
 
-			// let the task code compute the input splits
-			if (groupVertex.getEnvironment().getInvokable() instanceof AbstractInputTask) {
-				try {
-					inputSplits = ((AbstractInputTask<?>) groupVertex.getEnvironment().getInvokable())
-						.computeInputSplits(jobVertex.getNumberOfSubtasks());
-				} catch (Exception e) {
-					throw new GraphConversionException("Cannot compute input splits for " + groupVertex.getName(), e);
-				}
-			} else {
-				throw new GraphConversionException("JobInputVertex contained a task class which was not an input task.");
+			final Class<? extends InputSplit> inputSplitType = jobInputVertex.getInputSplitType();
+
+			try{
+				inputSplits = jobInputVertex.getInputSplits(jobVertex.getNumberOfSubtasks());
+			}catch(Exception e) {
+				throw new GraphConversionException("Cannot compute input splits for " + groupVertex.getName() + ": "
+						+ StringUtils.stringifyException(e));
 			}
 
 			if (inputSplits == null) {
@@ -507,13 +493,19 @@ public class ExecutionGraph implements ExecutionListener {
 					+ " input splits");
 			}
 
-			// assign input splits
+			// assign input splits and type
 			groupVertex.setInputSplits(inputSplits);
+			groupVertex.setInputSplitType(inputSplitType);
 		}
-		// TODO: This is a quick workaround, problem can be solved in a more generic way
-		if (jobVertex instanceof JobFileOutputVertex) {
-			final JobFileOutputVertex jbov = (JobFileOutputVertex) jobVertex;
-			jobVertex.getConfiguration().setString("outputPath", jbov.getFilePath().toString());
+
+		if(jobVertex instanceof JobOutputVertex){
+			final JobOutputVertex jobOutputVertex = (JobOutputVertex) jobVertex;
+
+			final OutputFormat<?> outputFormat = jobOutputVertex.getOutputFormat();
+
+			if(outputFormat != null){
+				outputFormat.initialize(groupVertex.getConfiguration());
+			}
 		}
 
 		// Add group vertex to initial execution stage
@@ -796,48 +788,6 @@ public class ExecutionGraph implements ExecutionListener {
 	}
 
 	/**
-	 * Retrieves the maximum parallel degree of the job represented by this execution graph
-	 */
-	public int getMaxNumberSubtasks() {
-		int maxDegree = 0;
-		final Iterator<ExecutionStage> stageIterator = this.stages.iterator();
-
-		while(stageIterator.hasNext()){
-			final ExecutionStage stage = stageIterator.next();
-
-			int maxPerStageDegree = stage.getMaxNumberSubtasks();
-
-			if(maxPerStageDegree > maxDegree){
-				maxDegree = maxPerStageDegree;
-			}
-		}
-
-		return maxDegree;
-	}
-
-	/**
-	 * 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;
-	}
-
-	/**
 	 * Returns the stage which is currently executed.
 	 * 
 	 * @return the currently executed stage or <code>null</code> if the job execution is already completed

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java
index c865609..91e9e53 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java
@@ -114,6 +114,11 @@ public final class ExecutionGroupVertex {
 	private volatile InputSplit[] inputSplits = null;
 
 	/**
+	 * Input split type
+	 */
+	private volatile Class<? extends InputSplit> inputSplitType = null;
+
+	/**
 	 * The execution stage this vertex belongs to.
 	 */
 	private volatile ExecutionStage executionStage = null;
@@ -129,11 +134,6 @@ public final class ExecutionGroupVertex {
 	private final Class<? extends AbstractInvokable> invokableClass;
 
 	/**
-	 * The environment created to execute the vertex's task.
-	 */
-	private final RuntimeEnvironment environment;
-
-	/**
 	 * Constructs a new group vertex.
 	 * 
 	 * @param name
@@ -177,9 +177,6 @@ public final class ExecutionGroupVertex {
 		this.executionSignature = signature;
 
 		this.invokableClass = invokableClass;
-
-		this.environment = new RuntimeEnvironment(executionGraph.getJobID(), name, invokableClass, configuration,
-			executionGraph.getJobConfiguration());
 	}
 
 	/**
@@ -192,16 +189,6 @@ public final class ExecutionGroupVertex {
 	}
 
 	/**
-	 * Returns the environment of the instantiated {@link AbstractInvokable} object.
-	 * 
-	 * @return the environment of the instantiated {@link AbstractInvokable} object
-	 */
-	public RuntimeEnvironment getEnvironment() {
-
-		return this.environment;
-	}
-
-	/**
 	 * Sets the execution stage this group vertex is associated with.
 	 * 
 	 * @param executionStage
@@ -407,20 +394,6 @@ public final class ExecutionGroupVertex {
 			}
 		}
 
-		// Make sure the value of newNumber is valid
-		// TODO: Move these checks to some other place
-		/*
-		 * if (this.getMinimumNumberOfGroupMember() < 1) {
-		 * throw new GraphConversionException("The minimum number of members is below 1 for group vertex "
-		 * + this.getName());
-		 * }
-		 * if ((this.getMaximumNumberOfGroupMembers() != -1)
-		 * && (this.getMaximumNumberOfGroupMembers() < this.getMinimumNumberOfGroupMember())) {
-		 * throw new GraphConversionException(
-		 * "The maximum number of members is smaller than the minimum for group vertex " + this.getName());
-		 * }
-		 */
-
 		final ExecutionVertex originalVertex = this.getGroupMember(0);
 		int currentNumberOfExecutionVertices = this.getCurrentNumberOfGroupMembers();
 
@@ -453,6 +426,14 @@ public final class ExecutionGroupVertex {
 	}
 
 	/**
+	 * Sets the input split type class
+	 *
+	 * @param inputSplitType Input split type class
+	 */
+	public void setInputSplitType(final Class<? extends InputSplit> inputSplitType) { this.inputSplitType =
+			inputSplitType; }
+
+	/**
 	 * Returns the input splits assigned to this group vertex.
 	 * 
 	 * @return the input splits, possibly <code>null</code> if the group vertex does not represent an input vertex
@@ -462,6 +443,14 @@ public final class ExecutionGroupVertex {
 		return this.inputSplits;
 	}
 
+	/**
+	 * Returns the input split type class
+	 *
+	 * @return the input split type class, possibly <code>null</code> if the group vertex does not represent an input
+	 * vertex
+	 */
+	public Class<? extends InputSplit> getInputSplitType() { return this.inputSplitType; }
+
 	public ExecutionGroupEdge getForwardEdge(int index) {
 
 		if (index < 0) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobInputVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobInputVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobInputVertex.java
index 958ed9d..22b4d7c 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobInputVertex.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobInputVertex.java
@@ -13,6 +13,10 @@
 
 package eu.stratosphere.nephele.jobgraph;
 
+import eu.stratosphere.core.io.InputSplit;
+
+import java.io.IOException;
+
 /**
  * An abstract base class for input vertices in Nephele.
  * 
@@ -34,4 +38,7 @@ public abstract class AbstractJobInputVertex extends AbstractJobVertex {
 
 		jobGraph.addVertex(this);
 	}
+
+	public abstract Class<? extends InputSplit> getInputSplitType();
+	public abstract InputSplit[] getInputSplits(int minNumSplits) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java
index d64c622..7cec46a 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java
@@ -573,65 +573,15 @@ public abstract class AbstractJobVertex implements IOReadableWritable {
 	}
 
 	/**
-	 * Performs task specific checks if the
-	 * respective task has been configured properly.
-	 * 
-	 * @param invokable
-	 *        an instance of the task this vertex represents
+	 * Performs check whether the vertex has been properly configured
+	 *
+	 * @param configuration
+	 *        configuration of this vertex
 	 * @throws IllegalConfigurationException
 	 *         thrown if the respective tasks is not configured properly
 	 */
-	public void checkConfiguration(final AbstractInvokable invokable) throws IllegalConfigurationException {
-
-		if (invokable == null) {
-			throw new IllegalArgumentException("Argument invokable is null");
-		}
-
-		// see if the task itself has a valid configuration
-		// because this is user code running on the master, we embed it in a catch-all block
-		try {
-			invokable.checkConfiguration();
-		} catch (IllegalConfigurationException icex) {
-			throw icex; // simply forward
-		} catch (Throwable t) {
-			throw new IllegalConfigurationException("Checking the invokable's configuration caused an error: "
-				+ StringUtils.stringifyException(t));
-		}
-	}
-
-	/**
-	 * Returns the minimum number of subtasks the respective task
-	 * must be split into at runtime.
-	 * 
-	 * @param invokable
-	 *        an instance of the task this vertex represents
-	 * @return the minimum number of subtasks the respective task must be split into at runtime
-	 */
-	public int getMinimumNumberOfSubtasks(final AbstractInvokable invokable) {
-
-		if (invokable == null) {
-			throw new IllegalArgumentException("Argument invokable is null");
-		}
-
-		return invokable.getMinimumNumberOfSubtasks();
-	}
-
-	/**
-	 * Returns the maximum number of subtasks the respective task
-	 * can be split into at runtime.
-	 * 
-	 * @param invokable
-	 *        an instance of the task this vertex represents
-	 * @return the maximum number of subtasks the respective task can be split into at runtime, <code>-1</code> for
-	 *         infinity
-	 */
-	public int getMaximumNumberOfSubtasks(final AbstractInvokable invokable) {
-
-		if (invokable == null) {
-			throw new IllegalArgumentException("Argument invokable is null");
-		}
-
-		return invokable.getMaximumNumberOfSubtasks();
+	public void checkConfiguration(final Configuration configuration) throws IllegalConfigurationException {
+		//default configuration check
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobFileInputVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobFileInputVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobFileInputVertex.java
deleted file mode 100644
index 65685ee..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobFileInputVertex.java
+++ /dev/null
@@ -1,195 +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.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import eu.stratosphere.configuration.IllegalConfigurationException;
-import eu.stratosphere.core.fs.FileStatus;
-import eu.stratosphere.core.fs.FileSystem;
-import eu.stratosphere.core.fs.Path;
-import eu.stratosphere.nephele.template.AbstractFileInputTask;
-import eu.stratosphere.nephele.template.AbstractInvokable;
-import eu.stratosphere.util.StringUtils;
-
-/**
- * A JobFileInputVertex is a specific subtype of a {@link AbstractJobInputVertex} and is designed
- * for Nephele tasks which read data from a local or distributed file system. As every job input vertex
- * A JobFileInputVertex must not have any further input.
- * 
- */
-public final class JobFileInputVertex extends AbstractJobInputVertex {
-
-	/**
-	 * The path pointing to the input file/directory.
-	 */
-	private Path path = null;
-
-	/**
-	 * Creates a new job file input vertex with the specified name.
-	 * 
-	 * @param name
-	 *        the name of the new job file input vertex
-	 * @param id
-	 *        the ID of this vertex
-	 * @param jobGraph
-	 *        the job graph this vertex belongs to
-	 */
-	public JobFileInputVertex(final String name, final JobVertexID id, final 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(final String name, final JobGraph jobGraph) {
-		super(name, null, jobGraph);
-	}
-
-	/**
-	 * Creates a new job file input vertex.
-	 * 
-	 * @param jobGraph
-	 *        the job graph this vertex belongs to
-	 */
-	public JobFileInputVertex(final JobGraph jobGraph) {
-		super(null, 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;
-	}
-
-	/**
-	 * Sets the class of the vertex's input task.
-	 * 
-	 * @param inputClass
-	 *        the class of the vertex's input task.
-	 */
-	public void setFileInputClass(final Class<? extends AbstractFileInputTask> inputClass) {
-		this.invokableClass = inputClass;
-	}
-
-	/**
-	 * Returns the class of the vertex's input task.
-	 * 
-	 * @return the class of the vertex's input task or <code>null</code> if no task has yet been set
-	 */
-	@SuppressWarnings("unchecked")
-	public Class<? extends AbstractFileInputTask> getFileInputClass() {
-		return (Class<? extends AbstractFileInputTask>) this.invokableClass;
-	}
-
-
-	@Override
-	public void read(final DataInput 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 DataOutput 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 void checkConfiguration(final AbstractInvokable invokable) throws IllegalConfigurationException {
-
-		// Check if the user has specified a path
-		if (this.path == null) {
-			throw new IllegalConfigurationException(this.getName() + " does not specify an input path");
-		}
-
-		// Check if the path is valid
-		try {
-			final FileSystem fs = this.path.getFileSystem();
-			final FileStatus f = fs.getFileStatus(this.path);
-			if (f == null) {
-				throw new IOException(this.path.toString() + " led to a null object");
-			}
-		} catch (IOException e) {
-			throw new IllegalConfigurationException("Cannot access file or directory: "
-				+ StringUtils.stringifyException(e));
-		}
-
-		// register the path in the configuration
-		invokable.getTaskConfiguration()
-			.setString(AbstractFileInputTask.INPUT_PATH_CONFIG_KEY, this.path.toString());
-
-		// Finally, see if the task itself has a valid configuration
-		super.checkConfiguration(invokable);
-	}
-
-
-	@Override
-	public int getMaximumNumberOfSubtasks(final AbstractInvokable invokable) {
-
-		int numberOfBlocks = -1;
-
-		if (this.path == null) {
-			return -1;
-		}
-
-		try {
-			final FileSystem fs = this.path.getFileSystem();
-			final FileStatus f = fs.getFileStatus(this.path);
-			numberOfBlocks = fs.getNumberOfBlocks(f);
-
-		} catch (IOException e) {
-			return -1;
-		}
-
-		return (int) Math.min(numberOfBlocks, invokable.getMaximumNumberOfSubtasks());
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobFileOutputVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobFileOutputVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobFileOutputVertex.java
deleted file mode 100644
index 645041a..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobFileOutputVertex.java
+++ /dev/null
@@ -1,198 +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.io.DataInput;
-import java.io.DataOutput;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-
-import eu.stratosphere.configuration.IllegalConfigurationException;
-import eu.stratosphere.core.fs.FileStatus;
-import eu.stratosphere.core.fs.FileSystem;
-import eu.stratosphere.core.fs.Path;
-import eu.stratosphere.nephele.template.AbstractFileOutputTask;
-import eu.stratosphere.nephele.template.AbstractInvokable;
-
-/**
- * A JobFileOutputVertex is a specific subtype of a {@link AbstractJobOutputVertex} and is designed
- * for Nephele tasks which write data to a local or distributed file system. As every job output vertex
- * A JobFileOutputVertex must not have any further output.
- * 
- */
-public class JobFileOutputVertex extends AbstractJobOutputVertex {
-
-	/**
-	 * The path pointing to the output file/directory.
-	 */
-	private Path path = null;
-
-	/**
-	 * Creates a new job file output vertex with the specified name.
-	 * 
-	 * @param name
-	 *        the name of the new job file output vertex
-	 * @param id
-	 *        the ID of this vertex
-	 * @param jobGraph
-	 *        the job graph this vertex belongs to
-	 */
-	public JobFileOutputVertex(final String name, final JobVertexID id, final 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(final String name, final JobGraph jobGraph) {
-		super(name, null, jobGraph);
-	}
-
-	/**
-	 * Creates a new job file input vertex.
-	 * 
-	 * @param jobGraph
-	 *        the job graph this vertex belongs to
-	 */
-	public JobFileOutputVertex(final JobGraph jobGraph) {
-		super(null, 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(final Path path) {
-		this.path = path;
-	}
-
-	/**
-	 * 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;
-	}
-
-	/**
-	 * Sets the class of the vertex's output task.
-	 * 
-	 * @param outputClass
-	 *        the class of the vertex's output task.
-	 */
-	public void setFileOutputClass(final Class<? extends AbstractFileOutputTask> outputClass) {
-		this.invokableClass = outputClass;
-	}
-
-	/**
-	 * Returns the class of the vertex's output task.
-	 * 
-	 * @return the class of the vertex's output task or <code>null</code> if no task has yet been set
-	 */
-	@SuppressWarnings("unchecked")
-	public Class<? extends AbstractFileOutputTask> getFileOutputClass() {
-		return (Class<? extends AbstractFileOutputTask>) this.invokableClass;
-	}
-
-
-	@Override
-	public void read(final DataInput 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 DataOutput 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 void checkConfiguration(final AbstractInvokable invokable) throws IllegalConfigurationException {
-
-		// Check if the user has specified a path
-		if (this.path == null) {
-			throw new IllegalConfigurationException(this.getName() + " does not specify an output path");
-		}
-
-		super.checkConfiguration(invokable);
-	}
-
-
-	@Override
-	public int getMaximumNumberOfSubtasks(final AbstractInvokable invokable) {
-
-		if (this.path == null) {
-			return 0;
-		}
-
-		// Check if the path is valid
-		try {
-			final FileSystem fs = path.getFileSystem();
-
-			try {
-				final FileStatus f = fs.getFileStatus(path);
-
-				if (f == null) {
-					return 1;
-				}
-
-				// If the path points to a directory we allow an infinity number of subtasks
-				if (f.isDir()) {
-					return -1;
-				}
-			} catch (FileNotFoundException fnfex) {
-				// The exception is thrown if the requested file/directory does not exist.
-				// if the degree of parallelism is > 1, we create a directory for this path
-				if (getNumberOfSubtasks() > 1) {
-					fs.mkdirs(path);
-					return -1;
-				} else {
-					// a none existing file and a degree of parallelism that is one
-					return 1;
-				}
-			}
-		} catch (IOException e) {
-			// any other kind of I/O exception: we assume only a degree of one here
-			return 1;
-		}
-
-		return 1;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobGenericInputVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobGenericInputVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobGenericInputVertex.java
deleted file mode 100644
index 658ea0d..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobGenericInputVertex.java
+++ /dev/null
@@ -1,168 +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.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import eu.stratosphere.configuration.IllegalConfigurationException;
-import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
-import eu.stratosphere.nephele.template.AbstractInputTask;
-import eu.stratosphere.nephele.template.AbstractInvokable;
-import eu.stratosphere.util.StringUtils;
-
-public class JobGenericInputVertex extends JobInputVertex
-{
-	/**
-	 * Class of input task.
-	 */
-	protected Class<? extends AbstractInputTask<?>> inputClass = null;
-
-	/**
-	 * Creates a new job input vertex with the specified name.
-	 * 
-	 * @param name The name of the new job file input vertex.
-	 * @param id The ID of this vertex.
-	 * @param jobGraph The job graph this vertex belongs to.
-	 */
-	public JobGenericInputVertex(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 JobGenericInputVertex(String name, JobGraph jobGraph) {
-		super(name, null, jobGraph);
-	}
-
-	/**
-	 * Creates a new job file input vertex.
-	 * 
-	 * @param jobGraph The job graph this vertex belongs to.
-	 */
-	public JobGenericInputVertex(JobGraph jobGraph) {
-		super(null, null, jobGraph);
-	}
-
-	/**
-	 * Sets the class of the vertex's input task.
-	 * 
-	 * @param inputClass The class of the vertex's input task.
-	 */
-	public void setInputClass(Class<? extends AbstractInputTask<?>> inputClass) {
-		this.inputClass = inputClass;
-	}
-
-	/**
-	 * Returns the class of the vertex's input task.
-	 * 
-	 * @return the class of the vertex's input task or <code>null</code> if no task has yet been set
-	 */
-	public Class<? extends AbstractInputTask<?>> getInputClass() {
-		return this.inputClass;
-	}
-
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public void read(DataInput in) throws IOException
-	{
-		super.read(in);
-
-		// Read class
-		boolean isNotNull = in.readBoolean();
-		if (isNotNull) {
-			// Read the name of the class and try to instantiate the class object
-			final ClassLoader cl = LibraryCacheManager.getClassLoader(this.getJobGraph().getJobID());
-			if (cl == null) {
-				throw new IOException("Cannot find class loader for vertex " + getID());
-			}
-
-			// Read the name of the expected class
-			final String className = StringRecord.readString(in);
-
-			try {
-				this.inputClass = (Class<? extends AbstractInputTask<?>>) Class.forName(className, true, cl).asSubclass(AbstractInputTask.class);
-			}
-			catch (ClassNotFoundException cnfe) {
-				throw new IOException("Class " + className + " not found in one of the supplied jar files: "
-					+ StringUtils.stringifyException(cnfe));
-			}
-			catch (ClassCastException ccex) {
-				throw new IOException("Class " + className + " is not a subclass of "
-					+ AbstractInputTask.class.getName() + ": " + StringUtils.stringifyException(ccex));
-			}
-		}
-	}
-
-
-	@Override
-	public void write(DataOutput out) throws IOException
-	{
-		super.write(out);
-
-		// Write out the name of the class
-		if (this.inputClass == null) {
-			out.writeBoolean(false);
-		} else {
-			out.writeBoolean(true);
-			StringRecord.writeString(out, this.inputClass.getName());
-		}
-	}
-
-
-	@Override
-	public void checkConfiguration(AbstractInvokable invokable) throws IllegalConfigurationException
-	{
-		// see if the task itself has a valid configuration
-		// because this is user code running on the master, we embed it in a catch-all block
-		try {
-			invokable.checkConfiguration();
-		}
-		catch (IllegalConfigurationException icex) {
-			throw icex; // simply forward
-		}
-		catch (Throwable t) {
-			throw new IllegalConfigurationException("Checking the invokable's configuration caused an error: " 
-				+ StringUtils.stringifyException(t));
-		}
-	}
-
-
-	@Override
-	public Class<? extends AbstractInvokable> getInvokableClass() {
-
-		return this.inputClass;
-	}
-
-
-	@Override
-	public int getMaximumNumberOfSubtasks(AbstractInvokable invokable)
-	{
-		return invokable.getMaximumNumberOfSubtasks();
-	}
-
-
-	@Override
-	public int getMinimumNumberOfSubtasks(AbstractInvokable invokable) {
-
-		return invokable.getMinimumNumberOfSubtasks();
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobGenericOutputVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobGenericOutputVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobGenericOutputVertex.java
deleted file mode 100644
index a5b0665..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobGenericOutputVertex.java
+++ /dev/null
@@ -1,182 +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.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import eu.stratosphere.configuration.IllegalConfigurationException;
-import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
-import eu.stratosphere.nephele.template.AbstractInvokable;
-import eu.stratosphere.nephele.template.AbstractOutputTask;
-import eu.stratosphere.util.StringUtils;
-
-/**
- * A JobGenericOutputVertex is a specific subtype of a {@link JobOutputVertex} and is designed
- * for Nephele tasks which sink data in a not further specified way. As every job output vertex,
- * a JobGenericOutputVertex must not have any further output.
- * 
- */
-public class JobGenericOutputVertex extends JobOutputVertex {
-
-	/**
-	 * The class of the output task.
-	 */
-	protected Class<? extends AbstractOutputTask> outputClass = null;
-
-
-	/**
-	 * Creates a new job file output vertex with the specified name.
-	 * 
-	 * @param name
-	 *        the name of the new job file output vertex
-	 * @param id
-	 *        the ID of this vertex
-	 * @param jobGraph
-	 *        the job graph this vertex belongs to
-	 */
-	public JobGenericOutputVertex(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 JobGenericOutputVertex(String name, JobGraph jobGraph) {
-		super(name, null, jobGraph);
-	}
-
-	/**
-	 * Creates a new job file input vertex.
-	 * 
-	 * @param jobGraph
-	 *        the job graph this vertex belongs to
-	 */
-	public JobGenericOutputVertex(JobGraph jobGraph) {
-		super(null, null, jobGraph);
-	}
-
-	/**
-	 * Sets the class of the vertex's output task.
-	 * 
-	 * @param outputClass The class of the vertex's output task.
-	 */
-	public void setOutputClass(Class<? extends AbstractOutputTask> outputClass) {
-		this.outputClass = outputClass;
-	}
-
-	/**
-	 * Returns the class of the vertex's output task.
-	 * 
-	 * @return The class of the vertex's output task or <code>null</code> if no task has yet been set.
-	 */
-	public Class<? extends AbstractOutputTask> getOutputClass() {
-		return this.outputClass;
-	}
-
-
-	@Override
-	public void read(DataInput in) throws IOException {
-		super.read(in);
-
-		// Read class
-		boolean isNotNull = in.readBoolean();
-		if (isNotNull) {
-
-			// Read the name of the class and try to instantiate the class object
-			final ClassLoader cl = LibraryCacheManager.getClassLoader(this.getJobGraph().getJobID());
-			if (cl == null) {
-				throw new IOException("Cannot find class loader for vertex " + getID());
-			}
-
-			// Read the name of the expected class
-			final String className = StringRecord.readString(in);
-
-			try {
-				this.outputClass = Class.forName(className, true, cl).asSubclass(AbstractOutputTask.class);
-			}
-			catch (ClassNotFoundException cnfe) {
-				throw new IOException("Class " + className + " not found in one of the supplied jar files: "
-					+ StringUtils.stringifyException(cnfe));
-			}
-			catch (ClassCastException ccex) {
-				throw new IOException("Class " + className + " is not a subclass of "
-					+ AbstractOutputTask.class.getName() + ": " + StringUtils.stringifyException(ccex));
-			}
-		}
-	}
-
-
-	@Override
-	public void write(DataOutput out) throws IOException {
-		super.write(out);
-
-		// Write out the name of the class
-		if (this.outputClass == null) {
-			out.writeBoolean(false);
-		}
-		else {
-			out.writeBoolean(true);
-			StringRecord.writeString(out, this.outputClass.getName());
-		}
-	}
-
-
-	@Override
-	public void checkConfiguration(AbstractInvokable invokable) throws IllegalConfigurationException
-	{
-		// see if the task itself has a valid configuration
-		// because this is user code running on the master, we embed it in a catch-all block
-		try {
-			invokable.checkConfiguration();
-		}
-		catch (IllegalConfigurationException icex) {
-			throw icex; // simply forward
-		}
-		catch (Throwable t) {
-			throw new IllegalConfigurationException("Checking the invokable's configuration caused an error: " 
-				+ StringUtils.stringifyException(t));
-		}
-	}
-
-
-	@Override
-	public Class<? extends AbstractInvokable> getInvokableClass() {
-
-		return this.outputClass;
-	}
-
-
-	@Override
-	public int getMaximumNumberOfSubtasks(AbstractInvokable invokable)
-	{
-		// Delegate call to invokable
-		return invokable.getMaximumNumberOfSubtasks();
-	}
-
-
-	@Override
-	public int getMinimumNumberOfSubtasks(AbstractInvokable invokable)
-	{
-		// Delegate call to invokable
-		return invokable.getMinimumNumberOfSubtasks();
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobInputVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobInputVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobInputVertex.java
index a22d7ca..9e5f6c7 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobInputVertex.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobInputVertex.java
@@ -13,9 +13,21 @@
 
 package eu.stratosphere.nephele.jobgraph;
 
+import eu.stratosphere.api.common.io.InputFormat;
+import eu.stratosphere.api.common.operators.util.UserCodeObjectWrapper;
+import eu.stratosphere.api.common.operators.util.UserCodeWrapper;
+import eu.stratosphere.api.common.typeutils.TypeSerializerFactory;
+import eu.stratosphere.configuration.Configuration;
+import eu.stratosphere.core.io.InputSplit;
+import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
 import eu.stratosphere.nephele.template.AbstractInputTask;
+import eu.stratosphere.pact.runtime.task.util.TaskConfig;
+
+import java.io.DataInput;
+import java.io.IOException;
 
 public class JobInputVertex extends AbstractJobInputVertex {
+	private volatile InputFormat<?, ? extends InputSplit> inputFormat = null;
 
 	/**
 	 * Creates a new job input vertex with the specified name.
@@ -55,7 +67,7 @@ public class JobInputVertex extends AbstractJobInputVertex {
 
 	/**
 	 * Sets the class of the vertex's input task.
-	 * 
+	 *
 	 * @param inputClass
 	 *        The class of the vertex's input task.
 	 */
@@ -72,4 +84,80 @@ public class JobInputVertex extends AbstractJobInputVertex {
 	public Class<? extends AbstractInputTask<?>> getInputClass() {
 		return (Class<? extends AbstractInputTask<?>>) this.invokableClass;
 	}
+
+	public void setInputFormat(UserCodeWrapper<? extends InputFormat<?, ? extends InputSplit>> inputFormatWrapper) {
+		TaskConfig config = new TaskConfig(this.getConfiguration());
+		config.setStubWrapper(inputFormatWrapper);
+
+		inputFormat = inputFormatWrapper.getUserCodeObject();
+	}
+
+	public void setInputFormat(InputFormat<?, ? extends InputSplit> inputFormat) {
+		this.inputFormat = inputFormat;
+
+		UserCodeWrapper<? extends InputFormat<?, ? extends InputSplit>> wrapper = new
+				UserCodeObjectWrapper<InputFormat<?, ? extends InputSplit>>(inputFormat);
+		TaskConfig config = new TaskConfig(this.getConfiguration());
+		config.setStubWrapper(wrapper);
+	}
+
+	public void setInputFormatParameters(Configuration inputFormatParameters){
+		TaskConfig config = new TaskConfig(this.getConfiguration());
+		config.setStubParameters(inputFormatParameters);
+
+		if(inputFormat == null){
+			throw new RuntimeException("There is no input format set in job vertex: " + this.getID());
+		}
+
+		inputFormat.configure(inputFormatParameters);
+	}
+
+	public void setOutputSerializer(TypeSerializerFactory<?> factory){
+		TaskConfig config = new TaskConfig(this.getConfiguration());
+		config.setOutputSerializer(factory);
+	}
+
+
+	@Override
+	public void read(final DataInput input) throws IOException{
+		super.read(input);
+
+		// load input format wrapper from the config
+		ClassLoader cl = null;
+
+		try{
+			cl = LibraryCacheManager.getClassLoader(this.getJobGraph().getJobID());
+		}
+		catch (IOException ioe) {
+			throw new RuntimeException("Usercode ClassLoader could not be obtained for job: " +
+					this.getJobGraph().getJobID(), ioe);
+		}
+
+		final Configuration config = this.getConfiguration();
+		config.setClassLoader(cl);
+		final TaskConfig taskConfig = new TaskConfig(config);
+
+		inputFormat = taskConfig.<InputFormat<?, InputSplit>>getStubWrapper(cl).getUserCodeObject(InputFormat.class,
+				cl);
+
+		inputFormat.configure(taskConfig.getStubParameters());
+	}
+
+	@Override
+	public Class<? extends InputSplit> getInputSplitType() {
+		if(inputFormat == null){
+			throw new RuntimeException("No input format has been set for job vertex: "+ this.getID());
+		}
+
+		return inputFormat.getInputSplitType();
+	}
+
+	@Override
+	public InputSplit[] getInputSplits(int minNumSplits) throws IOException {
+		if(inputFormat == null){
+			throw new RuntimeException("No input format has been set for job vertex: "+ this.getID());
+		}
+
+		return inputFormat.createInputSplits(minNumSplits);
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobOutputVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobOutputVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobOutputVertex.java
index 31452c3..154e639 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobOutputVertex.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobOutputVertex.java
@@ -13,7 +13,16 @@
 
 package eu.stratosphere.nephele.jobgraph;
 
+import eu.stratosphere.api.common.io.OutputFormat;
+import eu.stratosphere.api.common.operators.util.UserCodeObjectWrapper;
+import eu.stratosphere.api.common.operators.util.UserCodeWrapper;
+import eu.stratosphere.configuration.Configuration;
+import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
 import eu.stratosphere.nephele.template.AbstractOutputTask;
+import eu.stratosphere.pact.runtime.task.util.TaskConfig;
+
+import java.io.DataInput;
+import java.io.IOException;
 
 /**
  * A JobOutputVertex is a specific subtype of a {@link AbstractJobOutputVertex} and is designed
@@ -22,6 +31,7 @@ import eu.stratosphere.nephele.template.AbstractOutputTask;
  * 
  */
 public class JobOutputVertex extends AbstractJobOutputVertex {
+	private volatile OutputFormat<?> outputFormat = null;
 
 	/**
 	 * Creates a new job file output vertex with the specified name.
@@ -78,4 +88,50 @@ public class JobOutputVertex extends AbstractJobOutputVertex {
 	public Class<? extends AbstractOutputTask> getOutputClass() {
 		return (Class<? extends AbstractOutputTask>) this.invokableClass;
 	}
+
+	public void setOutputFormat(UserCodeWrapper<? extends OutputFormat<?>> outputFormatWrapper){
+		TaskConfig config = new TaskConfig(this.getConfiguration());
+		config.setStubWrapper(outputFormatWrapper);
+		outputFormat = outputFormatWrapper.getUserCodeObject();
+	}
+
+	public void setOutputFormat(OutputFormat<?> outputFormat){
+		this.outputFormat = outputFormat;
+		UserCodeWrapper<? extends OutputFormat<?>> wrapper = new UserCodeObjectWrapper<OutputFormat<?>>
+				(outputFormat);
+		TaskConfig config = new TaskConfig(this.getConfiguration());
+		config.setStubWrapper(wrapper);
+	}
+
+	public void setOutputFormatParameters(Configuration parameters){
+		TaskConfig config = new TaskConfig(this.getConfiguration());
+		config.setStubParameters(parameters);
+
+		outputFormat.configure(parameters);
+	}
+
+	@Override
+	public void read(final DataInput input) throws IOException{
+		super.read(input);
+
+		ClassLoader cl = null;
+		try{
+			cl = LibraryCacheManager.getClassLoader(this.getJobGraph().getJobID());
+		}
+		catch (IOException ioe) {
+			throw new RuntimeException("Usercode ClassLoader could not be obtained for job: " +
+					this.getJobGraph().getJobID(), ioe);
+		}
+
+		final Configuration config = this.getConfiguration();
+		config.setClassLoader(cl);
+		final TaskConfig taskConfig = new TaskConfig(config);
+
+		if(taskConfig.hasStubWrapper()){
+			outputFormat = taskConfig.<OutputFormat<?> >getStubWrapper(cl).getUserCodeObject(OutputFormat.class,cl);
+			outputFormat.configure(taskConfig.getStubParameters());
+		}
+	}
+
+	public OutputFormat<?> getOutputFormat() { return outputFormat; }
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobTaskVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobTaskVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobTaskVertex.java
index 61eb66c..8672aeb 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobTaskVertex.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobTaskVertex.java
@@ -13,7 +13,6 @@
 
 package eu.stratosphere.nephele.jobgraph;
 
-import eu.stratosphere.nephele.template.AbstractInvokable;
 import eu.stratosphere.nephele.template.AbstractTask;
 
 /**
@@ -84,20 +83,4 @@ public class JobTaskVertex extends AbstractJobVertex {
 	public Class<? extends AbstractTask> getTaskClass() {
 		return (Class<? extends AbstractTask>) this.invokableClass;
 	}
-
-
-	@Override
-	public int getMaximumNumberOfSubtasks(final AbstractInvokable invokable) {
-
-		// Delegate call to invokable
-		return invokable.getMaximumNumberOfSubtasks();
-	}
-
-
-	@Override
-	public int getMinimumNumberOfSubtasks(final AbstractInvokable invokable) {
-
-		// Delegate call to invokable
-		return invokable.getMinimumNumberOfSubtasks();
-	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitManager.java
index bbef991..790aca9 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitManager.java
@@ -102,18 +102,7 @@ public final class InputSplitManager {
 				continue;
 			}
 
-			final AbstractInvokable invokable = groupVertex.getEnvironment().getInvokable();
-			if (!(invokable instanceof AbstractInputTask)) {
-				LOG.error(groupVertex.getName() + " has " + inputSplits.length
-					+ " input splits, but is not of typt AbstractInputTask, ignoring...");
-				continue;
-			}
-
-			@SuppressWarnings("unchecked")
-			final AbstractInputTask<? extends InputSplit> inputTask = (AbstractInputTask<? extends InputSplit>) invokable;
-			final Class<? extends InputSplit> splitType = inputTask.getInputSplitType();
-
-			final InputSplitAssigner assigner = getAssignerByType(splitType, true);
+			final InputSplitAssigner assigner = getAssignerByType(groupVertex.getInputSplitType(), true);
 			// Add entry to cache for fast retrieval during the job execution
 			this.assignerCache.put(groupVertex, assigner);
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/LocatableInputSplitAssigner.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/LocatableInputSplitAssigner.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/LocatableInputSplitAssigner.java
index 3717fbf..1e6929d 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/LocatableInputSplitAssigner.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/LocatableInputSplitAssigner.java
@@ -49,18 +49,8 @@ public final class LocatableInputSplitAssigner implements InputSplitAssigner {
 	@Override
 	public void registerGroupVertex(final ExecutionGroupVertex groupVertex) {
 
-		// Do some sanity checks first
-		final AbstractInvokable invokable = groupVertex.getEnvironment().getInvokable();
-
-		// if (!(invokable instanceof AbstractFileInputTask)) {
-		// LOG.error(groupVertex.getName() + " is not an input vertex, ignoring vertex...");
-		// return;
-		// }
-
-		@SuppressWarnings("unchecked")
-		final AbstractInputTask<? extends InputSplit> inputTask = (AbstractInputTask<? extends InputSplit>) invokable;
-		if (!LocatableInputSplit.class.isAssignableFrom(inputTask.getInputSplitType())) {
-			LOG.error(groupVertex.getName() + " produces input splits of type " + inputTask.getInputSplitType()
+		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;
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/file/FileInputSplitAssigner.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/file/FileInputSplitAssigner.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/file/FileInputSplitAssigner.java
index 7894334..048562c 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/file/FileInputSplitAssigner.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/file/FileInputSplitAssigner.java
@@ -50,18 +50,8 @@ public final class FileInputSplitAssigner implements InputSplitAssigner {
 	@Override
 	public void registerGroupVertex(final ExecutionGroupVertex groupVertex) {
 
-		// Do some sanity checks first
-		final AbstractInvokable invokable = groupVertex.getEnvironment().getInvokable();
-
-		// if (!(invokable instanceof AbstractFileInputTask)) {
-		// LOG.error(groupVertex.getName() + " is not an input vertex, ignoring vertex...");
-		// return;
-		// }
-
-		@SuppressWarnings("unchecked")
-		final AbstractInputTask<? extends InputSplit> inputTask = (AbstractInputTask<? extends InputSplit>) invokable;
-		if (!FileInputSplit.class.equals(inputTask.getInputSplitType())) {
-			LOG.error(groupVertex.getName() + " produces input splits of type " + inputTask.getInputSplitType()
+		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;
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractFileInputTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractFileInputTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractFileInputTask.java
deleted file mode 100644
index d16e757..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractFileInputTask.java
+++ /dev/null
@@ -1,201 +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.template;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Iterator;
-import java.util.List;
-
-import eu.stratosphere.core.fs.BlockLocation;
-import eu.stratosphere.core.fs.FileInputSplit;
-import eu.stratosphere.core.fs.FileStatus;
-import eu.stratosphere.core.fs.FileSystem;
-import eu.stratosphere.core.fs.Path;
-
-/**
- * Specialized subtype of {@link AbstractInputTask} for tasks which are supposed to generate input from
- * a file. In addition to {@link AbstractInputTask} this class includes a method to query file splits
- * which should be read during the task's execution.
- * 
- */
-public abstract class AbstractFileInputTask extends AbstractInputTask<FileInputSplit> {
-
-	public static final String INPUT_PATH_CONFIG_KEY = "input.path";
-
-	/**
-	 * The fraction that the last split may be larger than the others.
-	 */
-	private static final float MAX_SPLIT_SIZE_DISCREPANCY = 1.1f;
-
-	// --------------------------------------------------------------------------------------------
-
-	/**
-	 * Returns an iterator to a (possible empty) list of file input splits which is expected to be consumed by this
-	 * instance of the {@link AbstractFileInputTask}.
-	 * 
-	 * @return an iterator to a (possible empty) list of file input splits.
-	 */
-	public Iterator<FileInputSplit> getFileInputSplits() {
-
-		return new InputSplitIterator<FileInputSplit>(getEnvironment().getInputSplitProvider());
-	}
-
-
-	@Override
-	public FileInputSplit[] computeInputSplits(final int minNumSplits) throws IOException {
-
-		final String pathURI = getTaskConfiguration().getString(INPUT_PATH_CONFIG_KEY, null);
-		if (pathURI == null) {
-			throw new IOException("The path to the file was not found in the runtime configuration.");
-		}
-
-		final Path path;
-		try {
-			path = new Path(pathURI);
-		} catch (Exception iaex) {
-			throw new IOException("Invalid file path specifier: ", iaex);
-		}
-
-		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;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractFileOutputTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractFileOutputTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractFileOutputTask.java
deleted file mode 100644
index 5f231c1..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractFileOutputTask.java
+++ /dev/null
@@ -1,46 +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.template;
-
-import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.core.fs.Path;
-
-/**
- * Specialized subtype of {@link AbstractOutputTask} for tasks which are supposed to write output to
- * a file.
- * 
- */
-public abstract class AbstractFileOutputTask extends AbstractOutputTask {
-
-	/**
-	 * Returns the output path which has been assigned to the original {@link JobFileOutputVertex}.
-	 * 
-	 * @return the output path which has been assigned to the original {@link JobFileOutputVertex} or <code>null</code>
-	 *         if the path cannot be retrieved
-	 */
-	public Path getFileOutputPath() {
-
-		// TODO: This is a quick workaround, problem can be solved in a more generic way
-		final Configuration conf = getEnvironment().getTaskConfiguration();
-
-		final String outputPath = conf.getString("outputPath", null);
-
-		if (outputPath != null) {
-			return new Path(outputPath);
-		}
-
-		return null;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractGenericInputTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractGenericInputTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractGenericInputTask.java
deleted file mode 100644
index cf6d916..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractGenericInputTask.java
+++ /dev/null
@@ -1,39 +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.template;
-
-import eu.stratosphere.core.io.GenericInputSplit;
-
-/**
- * An input task that processes generic input splits (partitions).
- */
-public abstract class AbstractGenericInputTask extends AbstractInputTask<GenericInputSplit> {
-
-
-	@Override
-	public GenericInputSplit[] computeInputSplits(final int requestedMinNumber) throws Exception {
-		GenericInputSplit[] splits = new GenericInputSplit[requestedMinNumber];
-		for (int i = 0; i < requestedMinNumber; i++) {
-			splits[i] = new GenericInputSplit(i,requestedMinNumber);
-		}
-		return splits;
-	}
-
-
-	@Override
-	public Class<GenericInputSplit> getInputSplitType() {
-
-		return GenericInputSplit.class;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractInputTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractInputTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractInputTask.java
index 76c9377..88e4fcb 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractInputTask.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractInputTask.java
@@ -27,29 +27,6 @@ import eu.stratosphere.core.io.InputSplit;
 public abstract class AbstractInputTask<T extends InputSplit> extends AbstractInvokable {
 
 	/**
-	 * This method computes the different splits of the input that can be processed in parallel. It needs
-	 * to be implemented by classes that describe input tasks.
-	 * <p>
-	 * Note that this method does not return the input splits for the task instance only, but it computes all splits for
-	 * all parallel instances. Those computed splits are then assigned to the individual task instances by the Job
-	 * Manager. To obtain the input splits for the current task instance, use the {@link #getTaskInputSplits()} method.
-	 * 
-	 * @param requestedMinNumber
-	 *        The minimum number of splits to create. This is a hint by the system how many splits
-	 *        should be generated at least (typically because there are that many parallel task
-	 *        instances), but it is no hard constraint
-	 * @return The input splits for the input to be processed by all instances of this input task
-	 */
-	public abstract T[] computeInputSplits(int requestedMinNumber) throws Exception;
-
-	/**
-	 * Returns the type of input splits that is generated by this input task.
-	 * 
-	 * @return the type of input splits that is generated by this input task
-	 */
-	public abstract Class<T> getInputSplitType();
-
-	/**
 	 * Returns an iterator to a (possible empty) list of input splits which is expected to be consumed by this
 	 * instance of the {@link AbstractInputTask}.
 	 * 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractInvokable.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractInvokable.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractInvokable.java
index a37f592..79390f8 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractInvokable.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractInvokable.java
@@ -62,39 +62,6 @@ public abstract class AbstractInvokable {
 		return this.environment;
 	}
 
-	/**
-	 * Overwrite this method to implement task specific checks if the
-	 * respective task has been configured properly.
-	 * 
-	 * @throws IllegalConfigurationException
-	 *         thrown if the respective tasks is not configured properly
-	 */
-	public void checkConfiguration() throws IllegalConfigurationException {
-		// The default implementation does nothing
-	}
-
-	/**
-	 * Overwrite this method to provide the minimum number of subtasks the respective task
-	 * must be split into at runtime.
-	 * 
-	 * @return the minimum number of subtasks the respective task must be split into at runtime
-	 */
-	public int getMinimumNumberOfSubtasks() {
-		// The default implementation always returns 1
-		return 1;
-	}
-
-	/**
-	 * Overwrite this method to provide the maximum number of subtasks the respective task
-	 * can be split into at runtime.
-	 * 
-	 * @return the maximum number of subtasks the respective task can be split into at runtime, <code>-1</code> for
-	 *         infinity
-	 */
-	public int getMaximumNumberOfSubtasks() {
-		// The default implementation always returns -1
-		return -1;
-	}
 
 	/**
 	 * Returns the current number of subtasks the respective task is split into.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ea79186b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/GenericInputTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/GenericInputTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/GenericInputTask.java
deleted file mode 100644
index c2cbbc1..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/GenericInputTask.java
+++ /dev/null
@@ -1,39 +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.template;
-
-import eu.stratosphere.core.io.GenericInputSplit;
-
-/**
- * An input task that processes generic input splits (partitions).
- */
-public abstract class GenericInputTask extends AbstractInputTask<GenericInputSplit> {
-
-
-	@Override
-	public GenericInputSplit[] computeInputSplits(final int requestedMinNumber) throws Exception {
-		GenericInputSplit[] splits = new GenericInputSplit[requestedMinNumber];
-		for (int i = 0; i < requestedMinNumber; i++) {
-			splits[i] = new GenericInputSplit(i, requestedMinNumber);
-		}
-		return splits;
-	}
-
-
-	@Override
-	public Class<GenericInputSplit> getInputSplitType() {
-
-		return GenericInputSplit.class;
-	}
-}


[10/53] [abbrv] Rework the Taskmanager to a slot based model and remove legacy cloud code

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/WorksetIterationNode.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/WorksetIterationNode.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/WorksetIterationNode.java
index 2c70794..f425695 100644
--- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/WorksetIterationNode.java
+++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/WorksetIterationNode.java
@@ -155,8 +155,7 @@ public class WorksetIterationNode extends TwoInputNode implements IterationNode
 		UnaryOperatorNode solutionSetDeltaUpdateAux = new UnaryOperatorNode("Solution-Set Delta", getSolutionSetKeyFields(),
 				new SolutionSetDeltaOperator(getSolutionSetKeyFields()));
 		solutionSetDeltaUpdateAux.setDegreeOfParallelism(getDegreeOfParallelism());
-		solutionSetDeltaUpdateAux.setSubtasksPerInstance(getSubtasksPerInstance());
-		
+
 		PactConnection conn = new PactConnection(solutionSetDelta, solutionSetDeltaUpdateAux);
 		solutionSetDeltaUpdateAux.setIncomingConnection(conn);
 		solutionSetDelta.addOutgoingConnection(conn);
@@ -218,11 +217,6 @@ public class WorksetIterationNode extends TwoInputNode implements IterationNode
 	// --------------------------------------------------------------------------------------------
 	
 	@Override
-	public boolean isMemoryConsumer() {
-		return true;
-	}
-	
-	@Override
 	protected List<OperatorDescriptorDual> getPossibleProperties() {
 		return new ArrayList<OperatorDescriptorDual>(1);
 	}
@@ -331,13 +325,12 @@ public class WorksetIterationNode extends TwoInputNode implements IterationNode
 				else if (report == FeedbackPropertiesMeetRequirementsReport.NOT_MET) {
 					// attach a no-op node through which we create the properties of the original input
 					Channel toNoOp = new Channel(candidate);
-					globPropsReqWorkset.parameterizeChannel(toNoOp, false, false);
+					globPropsReqWorkset.parameterizeChannel(toNoOp, false);
 					locPropsReqWorkset.parameterizeChannel(toNoOp);
 					
 					UnaryOperatorNode rebuildWorksetPropertiesNode = new UnaryOperatorNode("Rebuild Workset Properties", FieldList.EMPTY_LIST);
 					
 					rebuildWorksetPropertiesNode.setDegreeOfParallelism(candidate.getDegreeOfParallelism());
-					rebuildWorksetPropertiesNode.setSubtasksPerInstance(candidate.getSubtasksPerInstance());
 					
 					SingleInputPlanNode rebuildWorksetPropertiesPlanNode = new SingleInputPlanNode(rebuildWorksetPropertiesNode, "Rebuild Workset Properties", toNoOp, DriverStrategy.UNARY_NO_OP);
 					rebuildWorksetPropertiesPlanNode.initProperties(toNoOp.getGlobalProperties(), toNoOp.getLocalProperties());
@@ -518,7 +511,6 @@ public class WorksetIterationNode extends TwoInputNode implements IterationNode
 			super(new NoOpBinaryUdfOp<Nothing>(new NothingTypeInfo()));
 			
 			setDegreeOfParallelism(1);
-			setSubtasksPerInstance(1);
 		}
 		
 		public void setInputs(PactConnection input1, PactConnection input2) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dataproperties/RequestedGlobalProperties.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dataproperties/RequestedGlobalProperties.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dataproperties/RequestedGlobalProperties.java
index 574922a..e769508 100644
--- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dataproperties/RequestedGlobalProperties.java
+++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dataproperties/RequestedGlobalProperties.java
@@ -53,8 +53,7 @@ public final class RequestedGlobalProperties implements Cloneable {
 	/**
 	 * Sets the partitioning property for the global properties.
 	 * 
-	 * @param partitioning The new partitioning to set.
-	 * @param partitionedFields 
+	 * @param partitionedFields
 	 */
 	public void setHashPartitioned(FieldSet partitionedFields) {
 		if (partitionedFields == null) {
@@ -218,7 +217,7 @@ public final class RequestedGlobalProperties implements Cloneable {
 	 * @param globalDopChange
 	 * @param localDopChange
 	 */
-	public void parameterizeChannel(Channel channel, boolean globalDopChange, boolean localDopChange) {
+	public void parameterizeChannel(Channel channel, boolean globalDopChange) {
 		// if we request nothing, then we need no special strategy. forward, if the number of instances remains
 		// the same, randomly repartition otherwise
 		if (isTrivial()) {
@@ -228,8 +227,7 @@ public final class RequestedGlobalProperties implements Cloneable {
 		
 		final GlobalProperties inGlobals = channel.getSource().getGlobalProperties();
 		// if we have no global parallelism change, check if we have already compatible global properties
-		if (!globalDopChange && !localDopChange && isMetBy(inGlobals)) {
-			// we meet already everything, so go forward
+		if (!globalDopChange && isMetBy(inGlobals)) {
 			channel.setShipStrategy(ShipStrategyType.FORWARD);
 			return;
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/AllGroupWithPartialPreGroupProperties.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/AllGroupWithPartialPreGroupProperties.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/AllGroupWithPartialPreGroupProperties.java
index b389855..0ef277e 100644
--- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/AllGroupWithPartialPreGroupProperties.java
+++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/AllGroupWithPartialPreGroupProperties.java
@@ -48,8 +48,7 @@ public final class AllGroupWithPartialPreGroupProperties extends OperatorDescrip
 			// create an input node for combine with same DOP as input node
 			GroupReduceNode combinerNode = ((GroupReduceNode) node).getCombinerUtilityNode();
 			combinerNode.setDegreeOfParallelism(in.getSource().getDegreeOfParallelism());
-			combinerNode.setSubtasksPerInstance(in.getSource().getSubtasksPerInstance());
-			
+
 			SingleInputPlanNode combiner = new SingleInputPlanNode(combinerNode, "Combine ("+node.getPactContract().getName()+")", toCombiner, DriverStrategy.ALL_GROUP_COMBINE);
 			combiner.setCosts(new Costs(0, 0));
 			combiner.initProperties(toCombiner.getGlobalProperties(), toCombiner.getLocalProperties());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/AllReduceProperties.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/AllReduceProperties.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/AllReduceProperties.java
index be3ed74..867b9d9 100644
--- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/AllReduceProperties.java
+++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/AllReduceProperties.java
@@ -48,8 +48,7 @@ public final class AllReduceProperties extends OperatorDescriptorSingle
 			// create an input node for combine with same DOP as input node
 			ReduceNode combinerNode = ((ReduceNode) node).getCombinerUtilityNode();
 			combinerNode.setDegreeOfParallelism(in.getSource().getDegreeOfParallelism());
-			combinerNode.setSubtasksPerInstance(in.getSource().getSubtasksPerInstance());
-			
+
 			SingleInputPlanNode combiner = new SingleInputPlanNode(combinerNode, "Combine ("+node.getPactContract().getName()+")", toCombiner, DriverStrategy.ALL_REDUCE);
 			combiner.setCosts(new Costs(0, 0));
 			combiner.initProperties(toCombiner.getGlobalProperties(), toCombiner.getLocalProperties());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/GroupReduceWithCombineProperties.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/GroupReduceWithCombineProperties.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/GroupReduceWithCombineProperties.java
index 980cf6d..ec45a53 100644
--- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/GroupReduceWithCombineProperties.java
+++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/GroupReduceWithCombineProperties.java
@@ -85,9 +85,9 @@ public final class GroupReduceWithCombineProperties extends OperatorDescriptorSi
 			// create an input node for combine with same DOP as input node
 			GroupReduceNode combinerNode = ((GroupReduceNode) node).getCombinerUtilityNode();
 			combinerNode.setDegreeOfParallelism(in.getSource().getDegreeOfParallelism());
-			combinerNode.setSubtasksPerInstance(in.getSource().getSubtasksPerInstance());
-			
-			SingleInputPlanNode combiner = new SingleInputPlanNode(combinerNode, "Combine ("+node.getPactContract().getName()+")", toCombiner, DriverStrategy.SORTED_GROUP_COMBINE, this.keyList);
+
+			SingleInputPlanNode combiner = new SingleInputPlanNode(combinerNode, "Combine("+node.getPactContract()
+					.getName()+")", toCombiner, DriverStrategy.SORTED_GROUP_COMBINE, this.keyList);
 			combiner.setCosts(new Costs(0, 0));
 			combiner.initProperties(toCombiner.getGlobalProperties(), toCombiner.getLocalProperties());
 			

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/PartialGroupProperties.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/PartialGroupProperties.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/PartialGroupProperties.java
index a28feeb..9fb97b5 100644
--- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/PartialGroupProperties.java
+++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/PartialGroupProperties.java
@@ -44,9 +44,9 @@ public final class PartialGroupProperties extends OperatorDescriptorSingle {
 		// create in input node for combine with same DOP as input node
 		GroupReduceNode combinerNode = new GroupReduceNode((GroupReduceOperatorBase<?, ?, ?>) node.getPactContract());
 		combinerNode.setDegreeOfParallelism(in.getSource().getDegreeOfParallelism());
-		combinerNode.setSubtasksPerInstance(in.getSource().getSubtasksPerInstance());
-		
-		return new SingleInputPlanNode(combinerNode, "Combine("+node.getPactContract().getName()+")", in, DriverStrategy.SORTED_GROUP_COMBINE, this.keyList);
+
+		return new SingleInputPlanNode(combinerNode, "Combine("+node.getPactContract().getName()+")", in,
+				DriverStrategy.SORTED_GROUP_COMBINE, this.keyList);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/ReduceProperties.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/ReduceProperties.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/ReduceProperties.java
index 4539da5..0db3fa5 100644
--- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/ReduceProperties.java
+++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/ReduceProperties.java
@@ -56,8 +56,7 @@ public final class ReduceProperties extends OperatorDescriptorSingle {
 			// create an input node for combine with same DOP as input node
 			ReduceNode combinerNode = ((ReduceNode) node).getCombinerUtilityNode();
 			combinerNode.setDegreeOfParallelism(in.getSource().getDegreeOfParallelism());
-			combinerNode.setSubtasksPerInstance(in.getSource().getSubtasksPerInstance());
-			
+
 			SingleInputPlanNode combiner = new SingleInputPlanNode(combinerNode, "Combine ("+node.getPactContract().getName()+")", toCombiner, DriverStrategy.SORTED_PARTIAL_REDUCE, this.keyList);
 			combiner.setCosts(new Costs(0, 0));
 			combiner.initProperties(toCombiner.getGlobalProperties(), toCombiner.getLocalProperties());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plan/Channel.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plan/Channel.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plan/Channel.java
index 8fe95c9..6f9418f 100644
--- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plan/Channel.java
+++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plan/Channel.java
@@ -68,11 +68,11 @@ public class Channel implements EstimateProvider, Cloneable, DumpableConnection<
 	
 	private TempMode tempMode;
 	
-	private long tempMemory;
+	private double relativeTempMemory;
 	
-	private long memoryGlobalStrategy;
+	private double relativeMemoryGlobalStrategy;
 	
-	private long memoryLocalStrategy;
+	private double relativeMemoryLocalStrategy;
 	
 	private int replicationFactor = 1;
 	
@@ -200,17 +200,17 @@ public class Channel implements EstimateProvider, Cloneable, DumpableConnection<
 	 *
 	 * @return The temp memory.
 	 */
-	public long getTempMemory() {
-		return this.tempMemory;
+	public double getRelativeTempMemory() {
+		return this.relativeTempMemory;
 	}
 	
 	/**
 	 * Sets the memory for materializing the channel's result from this Channel.
 	 *
-	 * @param tempMemory The memory for materialization.
+	 * @param relativeTempMemory The memory for materialization.
 	 */
-	public void setTempMemory(long tempMemory) {
-		this.tempMemory = tempMemory;
+	public void setRelativeTempMemory(double relativeTempMemory) {
+		this.relativeTempMemory = relativeTempMemory;
 	}
 	
 	/**
@@ -286,20 +286,20 @@ public class Channel implements EstimateProvider, Cloneable, DumpableConnection<
 		this.localStrategyComparator = localStrategyComparator;
 	}
 	
-	public long getMemoryGlobalStrategy() {
-		return memoryGlobalStrategy;
+	public double getRelativeMemoryGlobalStrategy() {
+		return relativeMemoryGlobalStrategy;
 	}
 	
-	public void setMemoryGlobalStrategy(long memoryGlobalStrategy) {
-		this.memoryGlobalStrategy = memoryGlobalStrategy;
+	public void setRelativeMemoryGlobalStrategy(double relativeMemoryGlobalStrategy) {
+		this.relativeMemoryGlobalStrategy = relativeMemoryGlobalStrategy;
 	}
 	
-	public long getMemoryLocalStrategy() {
-		return memoryLocalStrategy;
+	public double getRelativeMemoryLocalStrategy() {
+		return relativeMemoryLocalStrategy;
 	}
 	
-	public void setMemoryLocalStrategy(long memoryLocalStrategy) {
-		this.memoryLocalStrategy = memoryLocalStrategy;
+	public void setRelativeMemoryLocalStrategy(double relativeMemoryLocalStrategy) {
+		this.relativeMemoryLocalStrategy = relativeMemoryLocalStrategy;
 	}
 	
 	public boolean isOnDynamicPath() {
@@ -437,33 +437,6 @@ public class Channel implements EstimateProvider, Cloneable, DumpableConnection<
 		}
 		throw new CompilerException("Unrecognized Ship Strategy Type: " + this.shipStrategy);
 	}
-	
-	public void adjustGlobalPropertiesForLocalParallelismChange() {
-		if (this.shipStrategy == null || this.shipStrategy == ShipStrategyType.NONE) {
-			throw new IllegalStateException("Cannot adjust channel for degree of parallelism " +
-					"change before the ship strategy is set.");
-		}
-		
-		// make sure the properties are acquired
-		if (this.globalProps == null) {
-			getGlobalProperties();
-		}
-		
-		// some strategies globally reestablish properties
-		switch (this.shipStrategy) {
-		case FORWARD:
-			this.globalProps.reset();
-			return;
-		case NONE: // excluded by sanity check. just here to silence compiler warnings check completion
-		case BROADCAST:
-		case PARTITION_HASH:
-		case PARTITION_RANGE:
-		case PARTITION_RANDOM:
-			return;
-		}
-		
-		throw new CompilerException("Unrecognized Ship Strategy Type: " + this.shipStrategy);
-	}
 
 	// --------------------------------------------------------------------------------------------
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plan/PlanNode.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plan/PlanNode.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plan/PlanNode.java
index 69263bc..539006c 100644
--- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plan/PlanNode.java
+++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plan/PlanNode.java
@@ -65,12 +65,10 @@ public abstract class PlanNode implements Visitable<PlanNode>, DumpableNode<Plan
 
 	protected Costs cumulativeCosts;					// the cumulative costs of all operators in the sub-tree
 	
-	private long memoryPerSubTask;					// the amount of memory dedicated to each task, in bytes
+	private double relativeMemoryPerSubTask;					// the amount of memory dedicated to each task, in bytes
 	
 	private int degreeOfParallelism;
 	
-	private int subtasksPerInstance;
-	
 	private boolean pFlag;							// flag for the internal pruning algorithm
 	
 	// --------------------------------------------------------------------------------------------
@@ -83,8 +81,7 @@ public abstract class PlanNode implements Visitable<PlanNode>, DumpableNode<Plan
 		this.driverStrategy = strategy;
 		
 		this.degreeOfParallelism = template.getDegreeOfParallelism();
-		this.subtasksPerInstance = template.getSubtasksPerInstance();
-		
+
 		// check, if there is branch at this node. if yes, this candidate must be associated with
 		// the branching template node.
 		if (template.isBranching()) {
@@ -166,17 +163,17 @@ public abstract class PlanNode implements Visitable<PlanNode>, DumpableNode<Plan
 	 * 
 	 * @return The memory per task, in bytes.
 	 */
-	public long getMemoryPerSubTask() {
-		return this.memoryPerSubTask;
+	public double getRelativeMemoryPerSubTask() {
+		return this.relativeMemoryPerSubTask;
 	}
 
 	/**
 	 * Sets the memory dedicated to each task for this node.
 	 * 
-	 * @param memoryPerTask The memory per sub-task, in bytes.
+	 * @param relativeMemoryPerSubtask The relative memory per sub-task
 	 */
-	public void setMemoryPerSubTask(long memoryPerTask) {
-		this.memoryPerSubTask = memoryPerTask;
+	public void setRelativeMemoryPerSubtask(double relativeMemoryPerSubtask) {
+		this.relativeMemoryPerSubTask = relativeMemoryPerSubtask;
 	}
 	
 	/**
@@ -303,18 +300,10 @@ public abstract class PlanNode implements Visitable<PlanNode>, DumpableNode<Plan
 		this.degreeOfParallelism = parallelism;
 	}
 	
-	public void setSubtasksPerInstance(int subTasksPerInstance) {
-		this.subtasksPerInstance = subTasksPerInstance;
-	}
-	
 	public int getDegreeOfParallelism() {
 		return this.degreeOfParallelism;
 	}
 	
-	public int getSubtasksPerInstance() {
-		return this.subtasksPerInstance;
-	}
-	
 	public long getGuaranteedAvailableMemory() {
 		return this.template.getMinimalMemoryAcrossAllSubTasks();
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plandump/PlanJSONDumpGenerator.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plandump/PlanJSONDumpGenerator.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plandump/PlanJSONDumpGenerator.java
index 82d757c..a1baff1 100644
--- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plandump/PlanJSONDumpGenerator.java
+++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plandump/PlanJSONDumpGenerator.java
@@ -252,9 +252,6 @@ public class PlanJSONDumpGenerator {
 		writer.print(",\n\t\t\"parallelism\": \""
 			+ (n.getDegreeOfParallelism() >= 1 ? n.getDegreeOfParallelism() : "default") + "\"");
 		
-		writer.print(",\n\t\t\"subtasks_per_instance\": \""
-				+ (n.getSubtasksPerInstance() >= 1 ? n.getSubtasksPerInstance() : "default") + "\"");
-
 		// output node predecessors
 		Iterator<? extends DumpableConnection<?>> inConns = node.getDumpableInputs().iterator();
 		String child1name = "", child2name = "";

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plantranslate/NepheleJobGraphGenerator.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plantranslate/NepheleJobGraphGenerator.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plantranslate/NepheleJobGraphGenerator.java
index 5a30fb6..b4c7560 100644
--- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plantranslate/NepheleJobGraphGenerator.java
+++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plantranslate/NepheleJobGraphGenerator.java
@@ -22,9 +22,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
 import eu.stratosphere.api.common.aggregators.AggregatorRegistry;
 import eu.stratosphere.api.common.aggregators.AggregatorWithName;
 import eu.stratosphere.api.common.aggregators.ConvergenceCriterion;
@@ -101,7 +98,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 	
 	private static final boolean mergeIterationAuxTasks = GlobalConfiguration.getBoolean(MERGE_ITERATION_AUX_TASKS_KEY, true);
 	
-	private static final Log LOG = LogFactory.getLog(NepheleJobGraphGenerator.class);
+//	private static final Log LOG = LogFactory.getLog(NepheleJobGraphGenerator.class);
 	
 	private static final TaskInChain ALREADY_VISITED_PLACEHOLDER = new TaskInChain(null, null, null);
 	
@@ -186,13 +183,6 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 			t.addChainedTask(tic.getChainedTask(), tic.getTaskConfig(), tic.getTaskName());
 		}
 
-		// now that all have been created, make sure that all share their instances with the one
-		// with the highest degree of parallelism
-		if (program.getInstanceTypeName() != null) {
-			this.maxDegreeVertex.setInstanceType(program.getInstanceTypeName());
-		} else {
-			LOG.warn("No instance type assigned to JobVertex.");
-		}
 		for (AbstractJobVertex vertex : this.vertices.values()) {
 			if (vertex != this.maxDegreeVertex) {
 				vertex.setVertexToShareInstancesWith(this.maxDegreeVertex);
@@ -231,7 +221,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 	 * @param node
 	 *        The node that is currently processed.
 	 * @return True, if the visitor should descend to the node's children, false if not.
-	 * @see eu.stratosphere.util.Visitor#preVisit(eu.stratosphere.pact.common.plan.Visitable)
+	 * @see eu.stratosphere.util.Visitor#preVisit(eu.stratosphere.util.Visitable)
 	 */
 	@Override
 	public boolean preVisit(PlanNode node) {
@@ -260,8 +250,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 				// operator with the tail, if they have the same DOP. not merging is currently not
 				// implemented
 				PlanNode root = iterationNode.getRootOfStepFunction();
-				if (root.getDegreeOfParallelism() != node.getDegreeOfParallelism() || 
-						root.getSubtasksPerInstance() != node.getSubtasksPerInstance()) 
+				if (root.getDegreeOfParallelism() != node.getDegreeOfParallelism())
 				{
 					throw new CompilerException("Error: The final operator of the step " +
 							"function has a different degree of parallelism than the iteration operator itself.");
@@ -278,14 +267,12 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 				PlanNode nextWorkSet = iterationNode.getNextWorkSetPlanNode();
 				PlanNode solutionSetDelta  = iterationNode.getSolutionSetDeltaPlanNode();
 				
-				if (nextWorkSet.getDegreeOfParallelism() != node.getDegreeOfParallelism() || 
-					nextWorkSet.getSubtasksPerInstance() != node.getSubtasksPerInstance())
+				if (nextWorkSet.getDegreeOfParallelism() != node.getDegreeOfParallelism())
 				{
 					throw new CompilerException("It is currently not supported that the final operator of the step " +
 							"function has a different degree of parallelism than the iteration operator itself.");
 				}
-				if (solutionSetDelta.getDegreeOfParallelism() != node.getDegreeOfParallelism() || 
-					solutionSetDelta.getSubtasksPerInstance() != node.getSubtasksPerInstance())
+				if (solutionSetDelta.getDegreeOfParallelism() != node.getDegreeOfParallelism())
 				{
 					throw new CompilerException("It is currently not supported that the final operator of the step " +
 							"function has a different degree of parallelism than the iteration operator itself.");
@@ -364,11 +351,6 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 			if (this.maxDegreeVertex == null || this.maxDegreeVertex.getNumberOfSubtasks() < pd) {
 				this.maxDegreeVertex = vertex;
 			}
-	
-			// set the number of tasks per instance
-			if (node.getSubtasksPerInstance() >= 1) {
-				vertex.setNumberOfSubtasksPerInstance(node.getSubtasksPerInstance());
-			}
 			
 			// check whether this vertex is part of an iteration step function
 			if (this.currentIteration != null) {
@@ -377,10 +359,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 				if (iterationNode.getDegreeOfParallelism() < pd) {
 					throw new CompilerException("Error: All functions that are part of an iteration must have the same, or a lower, degree-of-parallelism than the iteration operator.");
 				}
-				if (iterationNode.getSubtasksPerInstance() < node.getSubtasksPerInstance()) {
-					throw new CompilerException("Error: All functions that are part of an iteration must have the same, or a lower, number of subtasks-per-node than the iteration operator.");
-				}
-				
+
 				// 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());
@@ -401,7 +380,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 	 * 
 	 * @param node
 	 *        The node currently processed during the post-visit.
-	 * @see eu.stratosphere.util.Visitor#postVisit(eu.stratosphere.pact.common.plan.Visitable)
+	 * @see eu.stratosphere.util.Visitor#postVisit(eu.stratosphere.util.Visitable) t
 	 */
 	@Override
 	public void postVisit(PlanNode node) {
@@ -739,7 +718,6 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 					inConn.getLocalStrategy() == LocalStrategy.NONE &&
 					pred.getOutgoingChannels().size() == 1 &&
 					node.getDegreeOfParallelism() == pred.getDegreeOfParallelism() && 
-					node.getSubtasksPerInstance() == pred.getSubtasksPerInstance() &&
 					node.getBroadcastInputs().isEmpty();
 			
 			// cannot chain the nodes that produce the next workset or the next solution set, if they are not the
@@ -879,7 +857,6 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 					c.getLocalStrategy() == LocalStrategy.NONE &&
 					c.getTempMode() == TempMode.NONE &&
 					successor.getDegreeOfParallelism() == pspn.getDegreeOfParallelism() &&
-					successor.getSubtasksPerInstance() == pspn.getSubtasksPerInstance() &&
 					!(successor instanceof NAryUnionPlanNode) &&
 					successor != iteration.getRootOfStepFunction() &&
 					iteration.getInput().getLocalStrategy() == LocalStrategy.NONE;
@@ -948,7 +925,6 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 					c.getLocalStrategy() == LocalStrategy.NONE &&
 					c.getTempMode() == TempMode.NONE &&
 					successor.getDegreeOfParallelism() == wspn.getDegreeOfParallelism() &&
-					successor.getSubtasksPerInstance() == wspn.getSubtasksPerInstance() &&
 					!(successor instanceof NAryUnionPlanNode) &&
 					successor != iteration.getNextWorkSetPlanNode() &&
 					iteration.getInitialWorksetInput().getLocalStrategy() == LocalStrategy.NONE;
@@ -995,17 +971,17 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 	}
 	
 	private void assignDriverResources(PlanNode node, TaskConfig config) {
-		final long mem = node.getMemoryPerSubTask();
-		if (mem > 0) {
-			config.setMemoryDriver(mem);
+		final double relativeMem = node.getRelativeMemoryPerSubTask();
+		if (relativeMem > 0) {
+			config.setRelativeMemoryDriver(relativeMem);
 			config.setFilehandlesDriver(this.defaultMaxFan);
 			config.setSpillingThresholdDriver(this.defaultSortSpillingThreshold);
 		}
 	}
 	
 	private void assignLocalStrategyResources(Channel c, TaskConfig config, int inputNum) {
-		if (c.getMemoryLocalStrategy() > 0) {
-			config.setMemoryInput(inputNum, c.getMemoryLocalStrategy());
+		if (c.getRelativeMemoryLocalStrategy() > 0) {
+			config.setRelativeMemoryInput(inputNum, c.getRelativeMemoryLocalStrategy());
 			config.setFilehandlesInput(inputNum, this.defaultMaxFan);
 			config.setSpillingThresholdInput(inputNum, this.defaultSortSpillingThreshold);
 		}
@@ -1020,13 +996,13 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 	 * channel is then the channel into the union node, the local strategy channel the one from the union to the
 	 * actual target operator.
 	 *
-	 * @param channelForGlobalStrategy
-	 * @param channelForLocalStrategy
+	 * @param channel
 	 * @param inputNumber
 	 * @param sourceVertex
 	 * @param sourceConfig
 	 * @param targetVertex
 	 * @param targetConfig
+	 * @param isBroadcast
 	 * @throws JobGraphDefinitionException
 	 * @throws CompilerException
 	 */
@@ -1133,10 +1109,10 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 			
 			if (needsMemory) {
 				// sanity check
-				if (tm == null || tm == TempMode.NONE || channel.getTempMemory() < 1) {
+				if (tm == null || tm == TempMode.NONE || channel.getRelativeTempMemory() <= 0) {
 					throw new CompilerException("Bug in compiler: Inconsistent description of input materialization.");
 				}
-				config.setInputMaterializationMemory(inputNum, channel.getTempMemory());
+				config.setRelativeInputMaterializationMemory(inputNum, channel.getRelativeTempMemory());
 			}
 		}
 	}
@@ -1153,11 +1129,11 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 		final int numFinalOuts = headFinalOutputConfig.getNumOutputs();
 		headConfig.setIterationHeadFinalOutputConfig(headFinalOutputConfig);
 		headConfig.setIterationHeadIndexOfSyncOutput(numStepFunctionOuts + numFinalOuts);
-		final long memForBackChannel = bulkNode.getMemoryPerSubTask();
-		if (memForBackChannel <= 0) {
+		final double relativeMemForBackChannel = bulkNode.getRelativeMemoryPerSubTask();
+		if (relativeMemForBackChannel <= 0) {
 			throw new CompilerException("Bug: No memory has been assigned to the iteration back channel.");
 		}
-		headConfig.setBackChannelMemory(memForBackChannel);
+		headConfig.setRelativeBackChannelMemory(relativeMemForBackChannel);
 		
 		// --------------------------- create the sync task ---------------------------
 		final JobOutputVertex sync = new JobOutputVertex("Sync(" +
@@ -1219,7 +1195,6 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 			JobOutputVertex fakeTail = new JobOutputVertex("Fake Tail", this.jobGraph);
 			fakeTail.setOutputClass(FakeOutputTask.class);
 			fakeTail.setNumberOfSubtasks(headVertex.getNumberOfSubtasks());
-			fakeTail.setNumberOfSubtasksPerInstance(headVertex.getNumberOfSubtasksPerInstance());
 			this.auxVertices.add(fakeTail);
 			
 			// connect the fake tail
@@ -1262,7 +1237,6 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 			JobOutputVertex fakeTailTerminationCriterion = new JobOutputVertex("Fake Tail for Termination Criterion", this.jobGraph);
 			fakeTailTerminationCriterion.setOutputClass(FakeOutputTask.class);
 			fakeTailTerminationCriterion.setNumberOfSubtasks(headVertex.getNumberOfSubtasks());
-			fakeTailTerminationCriterion.setNumberOfSubtasksPerInstance(headVertex.getNumberOfSubtasksPerInstance());
 			this.auxVertices.add(fakeTailTerminationCriterion);
 		
 			// connect the fake tail
@@ -1310,14 +1284,14 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 			final int numFinalOuts = headFinalOutputConfig.getNumOutputs();
 			headConfig.setIterationHeadFinalOutputConfig(headFinalOutputConfig);
 			headConfig.setIterationHeadIndexOfSyncOutput(numStepFunctionOuts + numFinalOuts);
-			final long mem = iterNode.getMemoryPerSubTask();
-			if (mem <= 0) {
+			final double relativeMemory = iterNode.getRelativeMemoryPerSubTask();
+			if (relativeMemory <= 0) {
 				throw new CompilerException("Bug: No memory has been assigned to the workset iteration.");
 			}
 			
 			headConfig.setIsWorksetIteration();
-			headConfig.setBackChannelMemory(mem / 2);
-			headConfig.setSolutionSetMemory(mem / 2);
+			headConfig.setRelativeBackChannelMemory(relativeMemory / 2);
+			headConfig.setRelativeSolutionSetMemory(relativeMemory / 2);
 			
 			// set the solution set serializer and comparator
 			headConfig.setSolutionSetSerializer(iterNode.getSolutionSetSerializer());
@@ -1396,7 +1370,6 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 					JobOutputVertex fakeTail = new JobOutputVertex("Fake Tail", this.jobGraph);
 					fakeTail.setOutputClass(FakeOutputTask.class);
 					fakeTail.setNumberOfSubtasks(headVertex.getNumberOfSubtasks());
-					fakeTail.setNumberOfSubtasksPerInstance(headVertex.getNumberOfSubtasksPerInstance());
 					this.auxVertices.add(fakeTail);
 					
 					// connect the fake tail
@@ -1435,7 +1408,6 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 					JobOutputVertex fakeTail = new JobOutputVertex("Fake Tail", this.jobGraph);
 					fakeTail.setOutputClass(FakeOutputTask.class);
 					fakeTail.setNumberOfSubtasks(headVertex.getNumberOfSubtasks());
-					fakeTail.setNumberOfSubtasksPerInstance(headVertex.getNumberOfSubtasksPerInstance());
 					this.auxVertices.add(fakeTail);
 					
 					// connect the fake tail
@@ -1502,9 +1474,9 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 		
 		private AbstractJobVertex containingVertex;
 
-		@SuppressWarnings("unchecked")
-		TaskInChain(@SuppressWarnings("rawtypes") Class<? extends ChainedDriver> chainedTask, TaskConfig taskConfig, String taskName) {
-			this.chainedTask = (Class<? extends ChainedDriver<?, ?>>) chainedTask;
+		TaskInChain(Class<? extends ChainedDriver<?, ?>> chainedTask, TaskConfig taskConfig,
+					String taskName) {
+			this.chainedTask = chainedTask;
 			this.taskConfig = taskConfig;
 			this.taskName = taskName;
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/CompilerTestBase.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/CompilerTestBase.java b/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/CompilerTestBase.java
index e0da85d..f534ad9 100644
--- a/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/CompilerTestBase.java
+++ b/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/CompilerTestBase.java
@@ -12,7 +12,6 @@
  **********************************************************************************************************************/
 package eu.stratosphere.pact.compiler;
 
-import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -37,12 +36,6 @@ import eu.stratosphere.compiler.costs.DefaultCostEstimator;
 import eu.stratosphere.compiler.plan.OptimizedPlan;
 import eu.stratosphere.compiler.plan.PlanNode;
 import eu.stratosphere.compiler.plan.SingleInputPlanNode;
-import eu.stratosphere.nephele.instance.HardwareDescription;
-import eu.stratosphere.nephele.instance.HardwareDescriptionFactory;
-import eu.stratosphere.nephele.instance.InstanceType;
-import eu.stratosphere.nephele.instance.InstanceTypeDescription;
-import eu.stratosphere.nephele.instance.InstanceTypeDescriptionFactory;
-import eu.stratosphere.nephele.instance.InstanceTypeFactory;
 import eu.stratosphere.util.LogUtils;
 import eu.stratosphere.util.OperatingSystem;
 import eu.stratosphere.util.Visitor;
@@ -72,8 +65,6 @@ public abstract class CompilerTestBase implements java.io.Serializable {
 	
 	protected transient PactCompiler noStatsCompiler;
 	
-	protected transient InstanceTypeDescription instanceType;
-	
 	private transient int statCounter;
 	
 	// ------------------------------------------------------------------------	
@@ -85,29 +76,22 @@ public abstract class CompilerTestBase implements java.io.Serializable {
 	
 	@Before
 	public void setup() {
-		InetSocketAddress dummyAddr = new InetSocketAddress("localhost", 12345);
-		
 		this.dataStats = new DataStatistics();
-		this.withStatsCompiler = new PactCompiler(this.dataStats, new DefaultCostEstimator(), dummyAddr);
+		this.withStatsCompiler = new PactCompiler(this.dataStats, new DefaultCostEstimator());
 		this.withStatsCompiler.setDefaultDegreeOfParallelism(DEFAULT_PARALLELISM);
 		
-		this.noStatsCompiler = new PactCompiler(null, new DefaultCostEstimator(), dummyAddr);
+		this.noStatsCompiler = new PactCompiler(null, new DefaultCostEstimator());
 		this.noStatsCompiler.setDefaultDegreeOfParallelism(DEFAULT_PARALLELISM);
-		
-		// create the instance type description
-		InstanceType iType = InstanceTypeFactory.construct("standard", 6, 2, 4096, 100, 0);
-		HardwareDescription hDesc = HardwareDescriptionFactory.construct(2, 4096 * 1024 * 1024, 2000 * 1024 * 1024);
-		this.instanceType = InstanceTypeDescriptionFactory.construct(iType, hDesc, DEFAULT_PARALLELISM * 2);
 	}
 	
 	// ------------------------------------------------------------------------
 	
 	public OptimizedPlan compileWithStats(Plan p) {
-		return this.withStatsCompiler.compile(p, this.instanceType);
+		return this.withStatsCompiler.compile(p);
 	}
 	
 	public OptimizedPlan compileNoStats(Plan p) {
-		return this.noStatsCompiler.compile(p, this.instanceType);
+		return this.noStatsCompiler.compile(p);
 	}
 	
 	public void setSourceStatistics(GenericDataSourceBase<?, ?> source, long size, float recordWidth) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-core/src/main/java/eu/stratosphere/configuration/ConfigConstants.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/configuration/ConfigConstants.java b/stratosphere-core/src/main/java/eu/stratosphere/configuration/ConfigConstants.java
index 0e4177e..eff48cc 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/configuration/ConfigConstants.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/configuration/ConfigConstants.java
@@ -99,6 +99,11 @@ public final class ConfigConstants {
 	public static final String TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY = "taskmanager.network.bufferSizeInBytes";
 
 	/**
+	 * The config parameter defining the number of task slots of a task manager.
+	 */
+	public static final String TASK_MANAGER_NUM_TASK_SLOTS = "taskmanager.numberOfTaskSlots";
+
+	/**
 	 * The number of incoming network IO threads (e.g. incoming connection threads used in NettyConnectionManager
 	 * for the ServerBootstrap.)
 	 */
@@ -290,12 +295,7 @@ public final class ConfigConstants {
 	/**
 	 * The default degree of parallelism for operations.
 	 */
-	public static final int DEFAULT_PARALLELIZATION_DEGREE = -1;
-
-	/**
-	 * The default intra-node parallelism.
-	 */
-	public static final int DEFAULT_MAX_INTRA_NODE_PARALLELIZATION_DEGREE = -1;
+	public static final int DEFAULT_PARALLELIZATION_DEGREE = 1;
 	
 	// ------------------------------ Runtime ---------------------------------
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-core/src/main/java/eu/stratosphere/util/ClassUtils.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/util/ClassUtils.java b/stratosphere-core/src/main/java/eu/stratosphere/util/ClassUtils.java
index 96be666..f79dd2a 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/util/ClassUtils.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/util/ClassUtils.java
@@ -40,6 +40,7 @@ public final class ClassUtils {
 			throws ClassNotFoundException {
 
 		if (!className.contains("Protocol")) {
+			System.out.println(className);
 			throw new ClassNotFoundException("Only use this method for protocols!");
 		}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/job/VertexAssignmentEvent.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/job/VertexAssignmentEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/job/VertexAssignmentEvent.java
index f01e62d..c86c12b 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/job/VertexAssignmentEvent.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/job/VertexAssignmentEvent.java
@@ -38,11 +38,6 @@ public final class VertexAssignmentEvent extends AbstractEvent implements Manage
 	private String instanceName;
 
 	/**
-	 * The type of the instance the vertex is now assigned to.
-	 */
-	private String instanceType;
-
-	/**
 	 * Constructs a new event.
 	 * 
 	 * @param timestamp
@@ -51,16 +46,13 @@ public final class VertexAssignmentEvent extends AbstractEvent implements Manage
 	 *        identifies the vertex this event refers to
 	 * @param instanceName
 	 *        the name of the instance the vertex is now assigned to
-	 * @param instanceType
-	 *        the type of the instance the vertex is now assigned to
 	 */
 	public VertexAssignmentEvent(final long timestamp, final ManagementVertexID managementVertexID,
-			final String instanceName, final String instanceType) {
+			final String instanceName) {
 		super(timestamp);
 
 		this.managementVertexID = managementVertexID;
 		this.instanceName = instanceName;
-		this.instanceType = instanceType;
 	}
 
 	/**
@@ -90,16 +82,6 @@ public final class VertexAssignmentEvent extends AbstractEvent implements Manage
 		return this.instanceName;
 	}
 
-	/**
-	 * Returns the type of the instance the vertex is now assigned to.
-	 * 
-	 * @return the type of the instance the vertex is now assigned to
-	 */
-	public String getInstanceType() {
-		return this.instanceType;
-	}
-
-
 	@Override
 	public void read(final DataInput in) throws IOException {
 
@@ -107,7 +89,6 @@ public final class VertexAssignmentEvent extends AbstractEvent implements Manage
 
 		this.managementVertexID.read(in);
 		this.instanceName = StringRecord.readString(in);
-		this.instanceType = StringRecord.readString(in);
 	}
 
 
@@ -118,7 +99,6 @@ public final class VertexAssignmentEvent extends AbstractEvent implements Manage
 
 		this.managementVertexID.write(out);
 		StringRecord.writeString(out, this.instanceName);
-		StringRecord.writeString(out, this.instanceType);
 	}
 
 
@@ -149,16 +129,6 @@ public final class VertexAssignmentEvent extends AbstractEvent implements Manage
 			}
 		}
 
-		if (this.instanceType == null) {
-			if (vae.getInstanceType() != null) {
-				return false;
-			}
-		} else {
-			if (!this.instanceType.equals(vae.getInstanceType())) {
-				return false;
-			}
-		}
-
 		return true;
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionEdge.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionEdge.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionEdge.java
index 0106361..920c47e 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionEdge.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionEdge.java
@@ -19,7 +19,6 @@ import eu.stratosphere.runtime.io.channels.ChannelType;
 /**
  * Objects of this class represent a pair of {@link eu.stratosphere.runtime.io.serialization.io.channels.InputChannel} and {@link AbstractOutputChannel} objects
  * within an {@link ExecutionGraph}, Nephele's internal scheduling representation for jobs.
- * 
  */
 public final class ExecutionEdge {
 
@@ -51,42 +50,34 @@ public final class ExecutionEdge {
 	}
 
 	public ExecutionGate getInputGate() {
-
 		return this.inputGate;
 	}
 
 	public ExecutionGate getOutputGate() {
-
 		return this.outputGate;
 	}
 
 	public ChannelID getOutputChannelID() {
-
 		return this.outputChannelID;
 	}
 
 	public ChannelID getInputChannelID() {
-
 		return this.inputChannelID;
 	}
 
 	public int getOutputGateIndex() {
-
 		return this.outputGateIndex;
 	}
 
 	public int getInputGateIndex() {
-
 		return this.inputGateIndex;
 	}
 	
 	public ChannelType getChannelType() {
-		
 		return this.groupEdge.getChannelType();
 	}
 	
 	public int getConnectionID() {
-		
 		return this.groupEdge.getConnectionID();
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java
index ca7eddb..c5059f9 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java
@@ -38,8 +38,6 @@ import eu.stratosphere.nephele.execution.ExecutionListener;
 import eu.stratosphere.nephele.execution.ExecutionState;
 import eu.stratosphere.nephele.instance.AllocatedResource;
 import eu.stratosphere.nephele.instance.DummyInstance;
-import eu.stratosphere.nephele.instance.InstanceManager;
-import eu.stratosphere.nephele.instance.InstanceType;
 import eu.stratosphere.nephele.jobgraph.DistributionPattern;
 import eu.stratosphere.runtime.io.gates.GateID;
 import eu.stratosphere.runtime.io.channels.ChannelID;
@@ -160,18 +158,18 @@ public class ExecutionGraph implements ExecutionListener {
 	 * 
 	 * @param job
 	 *        the user's job graph
-	 * @param instanceManager
-	 *        the instance manager
+	 * @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(final JobGraph job, final InstanceManager instanceManager)
+	public ExecutionGraph(final JobGraph job, final int defaultParallelism)
 																					throws GraphConversionException {
 		this(job.getJobID(), job.getName(), job.getJobConfiguration());
 
 		// Start constructing the new execution graph from given job graph
 		try {
-			constructExecutionGraph(job, instanceManager);
+			constructExecutionGraph(job, defaultParallelism);
 		} catch (GraphConversionException e) {
 			throw e; // forward graph conversion exceptions
 		} catch (Exception e) {
@@ -217,7 +215,6 @@ public class ExecutionGraph implements ExecutionListener {
 			final ExecutionGroupVertex groupVertex = it2.next();
 			if (groupVertex.isNumberOfMembersUserDefined()) {
 				groupVertex.createInitialExecutionVertices(groupVertex.getUserDefinedNumberOfMembers());
-				groupVertex.repairSubtasksPerInstance();
 			}
 		}
 
@@ -253,12 +250,12 @@ public class ExecutionGraph implements ExecutionListener {
 	 * 
 	 * @param jobGraph
 	 *        the job graph to create the execution graph from
-	 * @param instanceManager
-	 *        the instance manager
+	 * @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 InstanceManager instanceManager)
+	private void constructExecutionGraph(final JobGraph jobGraph, final int defaultParallelism)
 			throws GraphConversionException {
 
 		// Clean up temporary data structures
@@ -272,8 +269,11 @@ public class ExecutionGraph implements ExecutionListener {
 		// Convert job vertices to execution vertices and initialize them
 		final AbstractJobVertex[] all = jobGraph.getAllJobVertices();
 		for (int i = 0; i < all.length; i++) {
-			final ExecutionVertex createdVertex = createVertex(all[i], instanceManager, initialExecutionStage,
-				jobGraph.getJobConfiguration());
+			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());
 		}
@@ -444,37 +444,15 @@ public class ExecutionGraph implements ExecutionListener {
 	 * 
 	 * @param jobVertex
 	 *        the job vertex to create the execution vertex from
-	 * @param instanceManager
-	 *        the instanceManager
 	 * @param initialExecutionStage
 	 *        the initial execution stage all group vertices are added to
-	 * @param jobConfiguration
-	 *        the configuration object originally attached to the {@link JobGraph}
 	 * @return the new execution vertex
 	 * @throws GraphConversionException
 	 *         thrown if the job vertex is of an unknown subclass
 	 */
-	private ExecutionVertex createVertex(final AbstractJobVertex jobVertex, final InstanceManager instanceManager,
-			final ExecutionStage initialExecutionStage, final Configuration jobConfiguration)
+	private ExecutionVertex createVertex(final AbstractJobVertex jobVertex, final ExecutionStage initialExecutionStage)
 			throws GraphConversionException {
 
-		// If the user has requested instance type, check if the type is known by the current instance manager
-		InstanceType instanceType = null;
-		boolean userDefinedInstanceType = false;
-		if (jobVertex.getInstanceType() != null) {
-
-			userDefinedInstanceType = true;
-			instanceType = instanceManager.getInstanceTypeByName(jobVertex.getInstanceType());
-			if (instanceType == null) {
-				throw new GraphConversionException("Requested instance type " + jobVertex.getInstanceType()
-					+ " is not known to the instance manager");
-			}
-		}
-
-		if (instanceType == null) {
-			instanceType = instanceManager.getDefaultInstanceType();
-		}
-
 		// Create an initial execution vertex for the job vertex
 		final Class<? extends AbstractInvokable> invokableClass = jobVertex.getInvokableClass();
 		if (invokableClass == null) {
@@ -491,8 +469,7 @@ public class ExecutionGraph implements ExecutionListener {
 		ExecutionGroupVertex groupVertex = null;
 		try {
 			groupVertex = new ExecutionGroupVertex(jobVertex.getName(), jobVertex.getID(), this,
-				jobVertex.getNumberOfSubtasks(), instanceType, userDefinedInstanceType,
-				jobVertex.getNumberOfSubtasksPerInstance(), jobVertex.getVertexToShareInstancesWith() != null ? true
+				jobVertex.getNumberOfSubtasks(), jobVertex.getVertexToShareInstancesWith() != null ? true
 					: false, jobVertex.getNumberOfExecutionRetries(), jobVertex.getConfiguration(), signature,
 				invokableClass);
 		} catch (Throwable t) {
@@ -506,39 +483,6 @@ public class ExecutionGraph implements ExecutionListener {
 			throw new GraphConversionException(StringUtils.stringifyException(e));
 		}
 
-		// Check if the user's specifications for the number of subtasks are valid
-		final int minimumNumberOfSubtasks = jobVertex.getMinimumNumberOfSubtasks(groupVertex.getEnvironment()
-			.getInvokable());
-		final int maximumNumberOfSubtasks = jobVertex.getMaximumNumberOfSubtasks(groupVertex.getEnvironment()
-			.getInvokable());
-		if (jobVertex.getNumberOfSubtasks() != -1) {
-			if (jobVertex.getNumberOfSubtasks() < 1) {
-				throw new GraphConversionException("Cannot split task " + jobVertex.getName() + " into "
-					+ jobVertex.getNumberOfSubtasks() + " subtasks");
-			}
-
-			if (jobVertex.getNumberOfSubtasks() < minimumNumberOfSubtasks) {
-				throw new GraphConversionException("Number of subtasks must be at least " + minimumNumberOfSubtasks);
-			}
-
-			if (maximumNumberOfSubtasks != -1) {
-				if (jobVertex.getNumberOfSubtasks() > maximumNumberOfSubtasks) {
-					throw new GraphConversionException("Number of subtasks for vertex " + jobVertex.getName()
-						+ " can be at most " + maximumNumberOfSubtasks);
-				}
-			}
-		}
-
-		// Check number of subtasks per instance
-		if (jobVertex.getNumberOfSubtasksPerInstance() != -1 && jobVertex.getNumberOfSubtasksPerInstance() < 1) {
-			throw new GraphConversionException("Cannot set number of subtasks per instance to "
-				+ jobVertex.getNumberOfSubtasksPerInstance() + " for vertex " + jobVertex.getName());
-		}
-
-		// Assign min/max to the group vertex (settings are actually applied in applyUserDefinedSettings)
-		groupVertex.setMinMemberSize(minimumNumberOfSubtasks);
-		groupVertex.setMaxMemberSize(maximumNumberOfSubtasks);
-
 		// Register input and output vertices separately
 		if (jobVertex instanceof AbstractJobInputVertex) {
 
@@ -579,8 +523,7 @@ public class ExecutionGraph implements ExecutionListener {
 			jobVertex.getNumberOfBackwardConnections());
 
 		// Assign initial instance to vertex (may be overwritten later on when user settings are applied)
-		ev.setAllocatedResource(new AllocatedResource(DummyInstance.createDummyInstance(instanceType), instanceType,
-			null));
+		ev.setAllocatedResource(new AllocatedResource(DummyInstance.createDummyInstance(), null));
 
 		return ev;
 	}
@@ -853,6 +796,48 @@ public class ExecutionGraph implements ExecutionListener {
 	}
 
 	/**
+	 * Retrieves the maximum parallel degree of the job represented by this execution graph
+	 */
+	public int getMaxNumberSubtasks() {
+		int maxDegree = 0;
+		final Iterator<ExecutionStage> stageIterator = this.stages.iterator();
+
+		while(stageIterator.hasNext()){
+			final ExecutionStage stage = stageIterator.next();
+
+			int maxPerStageDegree = stage.getMaxNumberSubtasks();
+
+			if(maxPerStageDegree > maxDegree){
+				maxDegree = maxPerStageDegree;
+			}
+		}
+
+		return maxDegree;
+	}
+
+	/**
+	 * 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;
+	}
+
+	/**
 	 * Returns the stage which is currently executed.
 	 * 
 	 * @return the currently executed stage or <code>null</code> if the job execution is already completed
@@ -1318,25 +1303,16 @@ public class ExecutionGraph implements ExecutionListener {
 		return this.jobName;
 	}
 
-
 	@Override
-	public void userThreadStarted(final JobID jobID, final ExecutionVertexID vertexID, final Thread userThread) {
-		// TODO Auto-generated method stub
-
-	}
-
+	public void userThreadStarted(JobID jobID, ExecutionVertexID vertexID, Thread userThread) {}
 
 	@Override
-	public void userThreadFinished(final JobID jobID, final ExecutionVertexID vertexID, final Thread userThread) {
-		// TODO Auto-generated method stub
-
-	}
+	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()) {
 
@@ -1345,39 +1321,17 @@ public class ExecutionGraph implements ExecutionListener {
 	}
 
 	/**
-	 * Calculates the connection IDs of the graph to avoid deadlocks in the data flow at runtime.
-	 */
-	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);
-			}
-		}
-	}
-
-	/**
 	 * 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();
 	}
 
 
 	@Override
 	public int getPriority() {
-
 		return 1;
 	}
 
@@ -1388,7 +1342,22 @@ public class ExecutionGraph implements ExecutionListener {
 	 *        the update command to be asynchronously executed on this graph
 	 */
 	public void executeCommand(final Runnable command) {
-
 		this.executorService.execute(command);
 	}
+	
+	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);
+			}
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java
index 89b4b6d..c865609 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java
@@ -18,7 +18,6 @@ import eu.stratosphere.core.io.InputSplit;
 import eu.stratosphere.nephele.execution.RuntimeEnvironment;
 import eu.stratosphere.nephele.instance.AllocatedResource;
 import eu.stratosphere.nephele.instance.DummyInstance;
-import eu.stratosphere.nephele.instance.InstanceType;
 import eu.stratosphere.nephele.jobgraph.JobVertexID;
 import eu.stratosphere.nephele.template.AbstractInvokable;
 import eu.stratosphere.runtime.io.channels.ChannelType;
@@ -69,41 +68,11 @@ public final class ExecutionGroupVertex {
 	private final CopyOnWriteArrayList<ExecutionVertex> groupMembers = new CopyOnWriteArrayList<ExecutionVertex>();
 
 	/**
-	 * Maximum number of execution vertices this group vertex can manage.
-	 */
-	private volatile int maxMemberSize = 1;
-
-	/**
-	 * Minimum number of execution vertices this group vertex can manage.
-	 */
-	private volatile int minMemberSize = 1;
-
-	/**
 	 * The user defined number of execution vertices, -1 if the user has not specified it.
 	 */
 	private final int userDefinedNumberOfMembers;
 
 	/**
-	 * The instance type to be used for execution vertices this group vertex manages.
-	 */
-	private volatile InstanceType instanceType = null;
-
-	/**
-	 * Stores whether the instance type is user defined.
-	 */
-	private final boolean userDefinedInstanceType;
-
-	/**
-	 * Stores the number of subtasks per instance.
-	 */
-	private volatile int numberOfSubtasksPerInstance = -1;
-
-	/**
-	 * Stores whether the number of subtasks per instance is user defined.
-	 */
-	private final boolean userDefinedNumberOfSubtasksPerInstance;
-
-	/**
 	 * Number of retries in case of an error before the task represented by this vertex is considered as failed.
 	 */
 	private final int numberOfExecutionRetries;
@@ -175,12 +144,6 @@ public final class ExecutionGroupVertex {
 	 *        the execution graph is group vertex belongs to
 	 * @param userDefinedNumberOfMembers
 	 *        the user defined number of subtasks, -1 if the user did not specify the number
-	 * @param instanceType
-	 *        the instance type to be used for execution vertices this group vertex manages.
-	 * @param userDefinedInstanceType
-	 *        <code>true</code> if the instance type is user defined, <code>false</code> otherwise
-	 * @param numberOfSubtasksPerInstance
-	 *        the user defined number of subtasks per instance, -1 if the user did not specify the number
 	 * @param userDefinedVertexToShareInstanceWith
 	 *        <code>true</code> if the user specified another vertex to share instances with, <code>false</code>
 	 *        otherwise
@@ -197,24 +160,13 @@ public final class ExecutionGroupVertex {
 	 *         throws if an error occurs while instantiating the {@link AbstractInvokable}
 	 */
 	public ExecutionGroupVertex(final String name, final JobVertexID jobVertexID, final ExecutionGraph executionGraph,
-			final int userDefinedNumberOfMembers, final InstanceType instanceType,
-			final boolean userDefinedInstanceType, final int numberOfSubtasksPerInstance,
-			final boolean userDefinedVertexToShareInstanceWith, final int numberOfExecutionRetries,
-			final Configuration configuration, final ExecutionSignature signature,
+			final int userDefinedNumberOfMembers, final boolean userDefinedVertexToShareInstanceWith,
+			final int numberOfExecutionRetries, final Configuration configuration, final ExecutionSignature signature,
 			final Class<? extends AbstractInvokable> invokableClass) throws Exception {
 
 		this.name = (name != null) ? name : "";
 		this.jobVertexID = jobVertexID;
 		this.userDefinedNumberOfMembers = userDefinedNumberOfMembers;
-		this.instanceType = instanceType;
-		this.userDefinedInstanceType = userDefinedInstanceType;
-		if (numberOfSubtasksPerInstance != -1) {
-			this.numberOfSubtasksPerInstance = numberOfSubtasksPerInstance;
-			this.userDefinedNumberOfSubtasksPerInstance = true;
-		} else {
-			this.numberOfSubtasksPerInstance = 1;
-			this.userDefinedNumberOfSubtasksPerInstance = false;
-		}
 		if (numberOfExecutionRetries >= 0) {
 			this.numberOfExecutionRetries = numberOfExecutionRetries;
 		} else {
@@ -309,32 +261,6 @@ public final class ExecutionGroupVertex {
 	}
 
 	/**
-	 * Sets the maximum number of members this group vertex can have.
-	 * 
-	 * @param maxSize
-	 *        the maximum number of members this group vertex can have
-	 */
-	void setMaxMemberSize(final int maxSize) {
-
-		// TODO: Add checks here
-
-		this.maxMemberSize = maxSize;
-	}
-
-	/**
-	 * Sets the minimum number of members this group vertex must have.
-	 * 
-	 * @param minSize
-	 *        the minimum number of members this group vertex must have
-	 */
-	void setMinMemberSize(final int minSize) {
-
-		// TODO: Add checks here
-
-		this.minMemberSize = minSize;
-	}
-
-	/**
 	 * Returns the current number of members this group vertex has.
 	 * 
 	 * @return the current number of members this group vertex has
@@ -345,24 +271,6 @@ public final class ExecutionGroupVertex {
 	}
 
 	/**
-	 * Returns the maximum number of members this group vertex can have.
-	 * 
-	 * @return the maximum number of members this group vertex can have
-	 */
-	public int getMaximumNumberOfGroupMembers() {
-		return this.maxMemberSize;
-	}
-
-	/**
-	 * Returns the minimum number of members this group vertex must have.
-	 * 
-	 * @return the minimum number of members this group vertex must have
-	 */
-	public int getMinimumNumberOfGroupMember() {
-		return this.minMemberSize;
-	}
-
-	/**
 	 * Wires this group vertex to the specified group vertex and creates
 	 * a back link.
 	 * 
@@ -376,10 +284,6 @@ public final class ExecutionGroupVertex {
 	 *        the channel type to be used for this edge
 	 * @param userDefinedChannelType
 	 *        <code>true</code> if the channel type is user defined, <code>false</code> otherwise
-	 * @param compressionLevel
-	 *        the compression level to be used for this edge
-	 * @param userDefinedCompressionLevel
-	 *        <code>true</code> if the compression level is user defined, <code>false</code> otherwise
 	 * @param distributionPattern
 	 *        the distribution pattern to create the wiring between the group members
 	 * @param isBroadcast
@@ -480,10 +384,10 @@ public final class ExecutionGroupVertex {
 	 * @throws GraphConversionException
 	 *         thrown if the number of execution vertices for this group vertex cannot be set to the desired value
 	 */
-	void createInitialExecutionVertices(final int initalNumberOfVertices) throws GraphConversionException {
+	void createInitialExecutionVertices(final int initialNumberOfVertices) throws GraphConversionException {
 
 		// If the requested number of group vertices does not change, do nothing
-		if (initalNumberOfVertices == this.getCurrentNumberOfGroupMembers()) {
+		if (initialNumberOfVertices == this.getCurrentNumberOfGroupMembers()) {
 			return;
 		}
 
@@ -517,25 +421,14 @@ public final class ExecutionGroupVertex {
 		 * }
 		 */
 
-		if (initalNumberOfVertices < this.getMinimumNumberOfGroupMember()) {
-			throw new GraphConversionException("Number of members must be at least "
-				+ this.getMinimumNumberOfGroupMember());
-		}
-
-		if ((this.getMaximumNumberOfGroupMembers() != -1)
-			&& (initalNumberOfVertices > this.getMaximumNumberOfGroupMembers())) {
-			throw new GraphConversionException("Number of members cannot exceed "
-				+ this.getMaximumNumberOfGroupMembers());
-		}
-
 		final ExecutionVertex originalVertex = this.getGroupMember(0);
 		int currentNumberOfExecutionVertices = this.getCurrentNumberOfGroupMembers();
 
-		while (currentNumberOfExecutionVertices++ < initalNumberOfVertices) {
+		while (currentNumberOfExecutionVertices++ < initialNumberOfVertices) {
 
 			final ExecutionVertex vertex = originalVertex.splitVertex();
 			vertex.setAllocatedResource(new AllocatedResource(DummyInstance
-				.createDummyInstance(this.instanceType), this.instanceType, null));
+				.createDummyInstance(), null));
 			this.groupMembers.add(vertex);
 		}
 
@@ -645,53 +538,6 @@ public final class ExecutionGroupVertex {
 		return this.userDefinedNumberOfMembers;
 	}
 
-	boolean isInstanceTypeUserDefined() {
-
-		return this.userDefinedInstanceType;
-	}
-
-	void setInstanceType(final InstanceType instanceType) throws GraphConversionException {
-
-		if (instanceType == null) {
-			throw new IllegalArgumentException("Argument instanceType must not be null");
-		}
-
-		if (this.userDefinedInstanceType) {
-			throw new GraphConversionException("Cannot overwrite user defined instance type "
-				+ instanceType.getIdentifier());
-		}
-
-		this.instanceType = instanceType;
-
-		// Reset instance allocation of all members and let reassignInstances do the work
-		for (int i = 0; i < this.groupMembers.size(); i++) {
-			final ExecutionVertex vertex = this.groupMembers.get(i);
-			vertex.setAllocatedResource(null);
-		}
-	}
-
-	InstanceType getInstanceType() {
-		return this.instanceType;
-	}
-
-	boolean isNumberOfSubtasksPerInstanceUserDefined() {
-
-		return this.userDefinedNumberOfSubtasksPerInstance;
-	}
-
-	void setNumberOfSubtasksPerInstance(final int numberOfSubtasksPerInstance) throws GraphConversionException {
-
-		if (this.userDefinedNumberOfSubtasksPerInstance
-			&& (numberOfSubtasksPerInstance != this.numberOfSubtasksPerInstance)) {
-			throw new GraphConversionException("Cannot overwrite user defined number of subtasks per instance");
-		}
-
-		this.numberOfSubtasksPerInstance = numberOfSubtasksPerInstance;
-	}
-
-	int getNumberOfSubtasksPerInstance() {
-		return this.numberOfSubtasksPerInstance;
-	}
 
 	/**
 	 * Returns the number of retries in case of an error before the task represented by this vertex is considered as
@@ -766,27 +612,13 @@ public final class ExecutionGroupVertex {
 
 	}
 
-	void repairSubtasksPerInstance() {
-
-		final Iterator<ExecutionVertex> it = this.groupMembers.iterator();
-		int count = 0;
-		while (it.hasNext()) {
-
-			final ExecutionVertex v = it.next();
-			v.setAllocatedResource(this.groupMembers.get(
-				(count++ / this.numberOfSubtasksPerInstance) * this.numberOfSubtasksPerInstance)
-				.getAllocatedResource());
-		}
-	}
-
 	void repairInstanceSharing(final Set<AllocatedResource> availableResources) {
 
 		// Number of required resources by this group vertex
-		final int numberOfRequiredInstances = (this.groupMembers.size() / this.numberOfSubtasksPerInstance)
-			+ (((this.groupMembers.size() % this.numberOfSubtasksPerInstance) != 0) ? 1 : 0);
+		final int numberOfRequiredSlots = this.groupMembers.size();
 
 		// Number of resources to be replaced
-		final int resourcesToBeReplaced = Math.min(availableResources.size(), numberOfRequiredInstances);
+		final int resourcesToBeReplaced = Math.min(availableResources.size(), numberOfRequiredSlots);
 
 		// Build the replacement map if necessary
 		final Map<AllocatedResource, AllocatedResource> replacementMap = new HashMap<AllocatedResource, AllocatedResource>();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionStage.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionStage.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionStage.java
index eab2375..df29aef 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionStage.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionStage.java
@@ -15,18 +15,10 @@ package eu.stratosphere.nephele.executiongraph;
 
 import java.util.HashSet;
 import java.util.Iterator;
-import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.CopyOnWriteArrayList;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.nephele.execution.ExecutionState;
-import eu.stratosphere.nephele.instance.AbstractInstance;
-import eu.stratosphere.nephele.instance.DummyInstance;
-import eu.stratosphere.nephele.instance.InstanceRequestMap;
-import eu.stratosphere.nephele.instance.InstanceType;
+import eu.stratosphere.nephele.instance.Instance;
 import eu.stratosphere.runtime.io.channels.ChannelType;
 
 /**
@@ -35,16 +27,10 @@ import eu.stratosphere.runtime.io.channels.ChannelType;
  * 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 log object used for debugging.
-	 */
-	private static final Log LOG = LogFactory.getLog(ExecutionStage.class);
-
-	/**
 	 * The execution graph that this stage belongs to.
 	 */
 	private final ExecutionGraph executionGraph;
@@ -242,69 +228,6 @@ public final class ExecutionStage {
 	}
 
 	/**
-	 * Checks which instance types and how many instances of these types are required to execute this stage
-	 * of the job graph. The required instance types and the number of instances are collected in the given map. Note
-	 * that this method does not clear the map before collecting the instances.
-	 * 
-	 * @param instanceRequestMap
-	 *        the map containing the instances types and the required number of instances of the respective type
-	 * @param executionState
-	 *        the execution state the considered vertices must be in
-	 */
-	public void collectRequiredInstanceTypes(final InstanceRequestMap instanceRequestMap,
-			final ExecutionState executionState) {
-
-		final Set<AbstractInstance> collectedInstances = new HashSet<AbstractInstance>();
-		final ExecutionGroupVertexIterator groupIt = new ExecutionGroupVertexIterator(this.getExecutionGraph(), true,
-			this.stageNum);
-
-		while (groupIt.hasNext()) {
-
-			final ExecutionGroupVertex groupVertex = groupIt.next();
-			final Iterator<ExecutionVertex> vertexIt = groupVertex.iterator();
-			while (vertexIt.hasNext()) {
-
-				// Get the instance type from the execution vertex if it
-				final ExecutionVertex vertex = vertexIt.next();
-				if (vertex.getExecutionState() == executionState) {
-					final AbstractInstance instance = vertex.getAllocatedResource().getInstance();
-
-					if (collectedInstances.contains(instance)) {
-						continue;
-					} else {
-						collectedInstances.add(instance);
-					}
-
-					if (instance instanceof DummyInstance) {
-
-						final InstanceType instanceType = instance.getType();
-						int num = instanceRequestMap.getMaximumNumberOfInstances(instanceType);
-						++num;
-						instanceRequestMap.setMaximumNumberOfInstances(instanceType, num);
-						if (groupVertex.isInputVertex()) {
-							num = instanceRequestMap.getMinimumNumberOfInstances(instanceType);
-							++num;
-							instanceRequestMap.setMinimumNumberOfInstances(instanceType, num);
-						}
-					} else {
-						LOG.debug("Execution Vertex " + vertex.getName() + " (" + vertex.getID()
-							+ ") is already assigned to non-dummy instance, skipping...");
-					}
-				}
-			}
-		}
-
-		final Iterator<Map.Entry<InstanceType, Integer>> it = instanceRequestMap.getMaximumIterator();
-		while (it.hasNext()) {
-
-			final Map.Entry<InstanceType, Integer> entry = it.next();
-			if (instanceRequestMap.getMinimumNumberOfInstances(entry.getKey()) == 0) {
-				instanceRequestMap.setMinimumNumberOfInstances(entry.getKey(), entry.getValue());
-			}
-		}
-	}
-
-	/**
 	 * Returns the execution graph that this stage belongs to.
 	 * 
 	 * @return the execution graph that this stage belongs to
@@ -446,4 +369,37 @@ public final class ExecutionStage {
 			}
 		}
 	}
+
+	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/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java
index 8e9395a..1e8d538 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java
@@ -855,7 +855,6 @@ public final class ExecutionVertex {
 	 *         <code>false/<code> otherwise
 	 */
 	public boolean decrementRetriesLeftAndCheck() {
-
 		return (this.retriesLeft.decrementAndGet() > 0);
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/InternalJobStatus.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/InternalJobStatus.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/InternalJobStatus.java
index 3a41aa2..8565495 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/InternalJobStatus.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/InternalJobStatus.java
@@ -74,6 +74,7 @@ public enum InternalJobStatus {
 	 *        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) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java
index 72e3651..04c68b1 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java
@@ -17,8 +17,8 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 
-import eu.stratosphere.nephele.instance.AbstractInstance;
 import eu.stratosphere.runtime.io.channels.ChannelType;
+import eu.stratosphere.nephele.instance.Instance;
 import eu.stratosphere.nephele.managementgraph.ManagementEdge;
 import eu.stratosphere.nephele.managementgraph.ManagementEdgeID;
 import eu.stratosphere.nephele.managementgraph.ManagementGate;
@@ -120,12 +120,11 @@ public class ManagementGraphFactory {
 			final ExecutionVertex ev = iterator.next();
 			final ManagementGroupVertex parent = groupMap.get(ev.getGroupVertex());
 
-			final AbstractInstance instance = ev.getAllocatedResource().getInstance();
+			final Instance instance = ev.getAllocatedResource().getInstance();
 			final ManagementVertex managementVertex = new ManagementVertex(
 						parent, 
 						ev.getID().toManagementVertexID(),
-						(instance.getInstanceConnectionInfo() != null) ? instance.getInstanceConnectionInfo().toString() : instance.toString(), 
-						instance.getType().toString(), 
+						(instance.getInstanceConnectionInfo() != null) ? instance.getInstanceConnectionInfo().toString() : instance.toString(),
 						ev.getIndexInVertexGroup()
 					);
 			managementVertex.setExecutionState(ev.getExecutionState());


[09/53] [abbrv] Rework the Taskmanager to a slot based model and remove legacy cloud code

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java
deleted file mode 100644
index 56b4eae..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java
+++ /dev/null
@@ -1,297 +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.instance;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.List;
-import java.util.Set;
-
-import eu.stratosphere.nephele.deployment.TaskDeploymentDescriptor;
-import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
-import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileRequest;
-import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileResponse;
-import eu.stratosphere.nephele.execution.librarycache.LibraryCacheUpdate;
-import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
-import eu.stratosphere.nephele.taskmanager.TaskKillResult;
-import eu.stratosphere.runtime.io.channels.ChannelID;
-import eu.stratosphere.nephele.ipc.RPC;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.net.NetUtils;
-import eu.stratosphere.nephele.protocols.TaskOperationProtocol;
-import eu.stratosphere.nephele.taskmanager.TaskCancelResult;
-import eu.stratosphere.nephele.taskmanager.TaskSubmissionResult;
-import eu.stratosphere.nephele.topology.NetworkNode;
-import eu.stratosphere.nephele.topology.NetworkTopology;
-
-/**
- * An abstract instance represents a resource a {@link eu.stratosphere.nephele.taskmanager.TaskManager} runs on.
- * 
- */
-public abstract class AbstractInstance extends NetworkNode {
-
-	/**
-	 * The type of the instance.
-	 */
-	private final InstanceType instanceType;
-
-	/**
-	 * The connection info identifying the instance.
-	 */
-	private final InstanceConnectionInfo instanceConnectionInfo;
-
-	/**
-	 * The hardware description as reported by the instance itself.
-	 */
-	private final HardwareDescription hardwareDescription;
-
-	/**
-	 * Stores the RPC stub object for the instance's task manager.
-	 */
-	private TaskOperationProtocol taskManager = null;
-
-	/**
-	 * Constructs an abstract instance object.
-	 * 
-	 * @param instanceType
-	 *        the type of the instance
-	 * @param instanceConnectionInfo
-	 *        the connection info identifying the instance
-	 * @param parentNode
-	 *        the parent node in the network topology
-	 * @param networkTopology
-	 *        the network topology this node is a part of
-	 * @param hardwareDescription
-	 *        the hardware description provided by the instance itself
-	 */
-	public AbstractInstance(final InstanceType instanceType, final InstanceConnectionInfo instanceConnectionInfo,
-			final NetworkNode parentNode, final NetworkTopology networkTopology,
-			final HardwareDescription hardwareDescription) {
-		super((instanceConnectionInfo == null) ? null : instanceConnectionInfo.toString(), parentNode, networkTopology);
-		this.instanceType = instanceType;
-		this.instanceConnectionInfo = instanceConnectionInfo;
-		this.hardwareDescription = hardwareDescription;
-	}
-
-	/**
-	 * 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());
-		}
-
-		return this.taskManager;
-	}
-
-	/**
-	 * 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;
-		}
-	}
-
-	/**
-	 * Returns the type of the instance.
-	 * 
-	 * @return the type of the instance
-	 */
-	public final InstanceType getType() {
-		return this.instanceType;
-	}
-
-	/**
-	 * Returns the instance's connection information object.
-	 * 
-	 * @return the instance's connection information object
-	 */
-	public final InstanceConnectionInfo getInstanceConnectionInfo() {
-		return this.instanceConnectionInfo;
-	}
-
-	/**
-	 * Returns the instance's hardware description as reported by the instance itself.
-	 * 
-	 * @return the instance's hardware description
-	 */
-	public HardwareDescription getHardwareDescription() {
-		return this.hardwareDescription;
-	}
-
-	/**
-	 * 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);
-		}
-
-		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);
-			}
-		}
-	}
-
-	/**
-	 * Submits a list of tasks to the instance's {@link eu.stratosphere.nephele.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);
-	}
-
-	/**
-	 * Cancels the task identified by the given ID at the instance's
-	 * {@link eu.stratosphere.nephele.taskmanager.TaskManager}.
-	 * 
-	 * @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
-	 */
-	public synchronized TaskCancelResult cancelTask(final ExecutionVertexID id) throws IOException {
-
-		return getTaskManagerProxy().cancelTask(id);
-	}
-
-	/**
-	 * Kills the task identified by the given ID at the instance's
-	 * {@link eu.stratosphere.nephele.taskmanager.TaskManager}.
-	 *
-	 * @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
-	 */
-	public synchronized TaskKillResult killTask(final ExecutionVertexID id) throws IOException {
-
-		return getTaskManagerProxy().killTask(id);
-	}
-
-	@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 AbstractInstance)) {
-			return false;
-		}
-
-		final AbstractInstance abstractInstance = (AbstractInstance) 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.
-	 */
-	public synchronized void destroyProxies() {
-
-		destroyTaskManagerProxy();
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AllocatedResource.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AllocatedResource.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AllocatedResource.java
index eb0a835..7f2ad04 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AllocatedResource.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AllocatedResource.java
@@ -23,7 +23,7 @@ import eu.stratosphere.nephele.executiongraph.ExecutionVertex;
 /**
  * An allocated resource object unambiguously defines the
  * hardware resources which have been assigned to an {@link eu.stratosphere.nephele.executiongraph.ExecutionVertex} for
- * executing a task. The allocated resource is comprised of an {@link eu.stratosphere.nephele.instance.AbstractInstance}
+ * 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 eu.stratosphere.nephele.instance.AllocationID} which determines the resources the task is scheduled to
  * allocate within the node.
@@ -36,12 +36,7 @@ public final class AllocatedResource {
 	/**
 	 * The instance a task is scheduled to run on.
 	 */
-	private final AbstractInstance instance;
-
-	/**
-	 * The instance type this allocated resource represents.
-	 */
-	private final InstanceType instanceType;
+	private final Instance instance;
 
 	/**
 	 * The allocation ID identifying the resources within the instance
@@ -60,24 +55,20 @@ public final class AllocatedResource {
 	 * 
 	 * @param instance
 	 *        the instance a task is scheduled to run on.
-	 * @param instanceType
-	 *        the instance type this allocated resource represents
 	 * @param allocationID
 	 *        the allocation ID identifying the allocated resources within the instance
 	 */
-	public AllocatedResource(final AbstractInstance instance, final InstanceType instanceType,
-			final AllocationID allocationID) {
+	public AllocatedResource(final Instance instance, final AllocationID allocationID) {
 		this.instance = instance;
-		this.instanceType = instanceType;
 		this.allocationID = allocationID;
 	}
 
 	/**
 	 * Returns the instance a task is scheduled to run on.
-	 * 
+	 *
 	 * @return the instance a task is scheduled to run on
 	 */
-	public AbstractInstance getInstance() {
+	public Instance getInstance() {
 		return this.instance;
 	}
 
@@ -90,15 +81,6 @@ public final class AllocatedResource {
 		return this.allocationID;
 	}
 
-	/**
-	 * Returns the instance type this allocated resource represents.
-	 * 
-	 * @return the instance type this allocated resource represents
-	 */
-	public InstanceType getInstanceType() {
-		return this.instanceType;
-	}
-
 
 	@Override
 	public boolean equals(final Object obj) {
@@ -120,16 +102,6 @@ public final class AllocatedResource {
 				}
 			}
 
-			if (this.instanceType == null) {
-				if (allocatedResource.instance != null) {
-					return false;
-				}
-			} else {
-				if (!this.instanceType.equals(allocatedResource.getInstanceType())) {
-					return false;
-				}
-			}
-
 			return true;
 		}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AllocatedSlot.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AllocatedSlot.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AllocatedSlot.java
new file mode 100644
index 0000000..0641944
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AllocatedSlot.java
@@ -0,0 +1,65 @@
+/***********************************************************************************************************************
+ * 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.instance;
+
+import eu.stratosphere.nephele.jobgraph.JobID;
+
+/**
+ * An allocated slot is a part of an instance which is assigned to a job.
+ * <p>
+ * This class is thread-safe.
+ * 
+ */
+public class AllocatedSlot {
+
+	/**
+	 * The allocation ID which identifies the resources occupied by this slot.
+	 */
+	private final AllocationID allocationID;
+
+	/**
+	 * The ID of the job this slice belongs to.
+	 */
+	private final JobID jobID;
+
+	/**
+	 * 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();
+		this.jobID = jobID;
+	}
+
+	/**
+	 * 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.
+	 * 
+	 * @return the ID of the job this allocated slice belongs to
+	 */
+	public JobID getJobID() {
+		return this.jobID;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AllocationID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AllocationID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AllocationID.java
index 3c83b80..3ed5013 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AllocationID.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AllocationID.java
@@ -17,8 +17,8 @@ import eu.stratosphere.nephele.AbstractID;
 
 /**
  * An allocation ID unambiguously identifies the allocated resources
- * within an {@link AbstractInstance}. The ID is necessary if an {@link InstanceManager} decides to partition
- * {@link AbstractInstance}s
+ * 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.
  * 
  */

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/DefaultInstanceManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/DefaultInstanceManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/DefaultInstanceManager.java
new file mode 100644
index 0000000..7d5f31b
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/DefaultInstanceManager.java
@@ -0,0 +1,393 @@
+/***********************************************************************************************************************
+ * 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.instance;
+
+import eu.stratosphere.configuration.Configuration;
+import eu.stratosphere.configuration.GlobalConfiguration;
+import eu.stratosphere.nephele.jobgraph.JobID;
+import eu.stratosphere.nephele.topology.NetworkNode;
+import eu.stratosphere.nephele.topology.NetworkTopology;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.util.Map;
+import java.util.HashMap;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.Collection;
+import java.util.TimerTask;
+import java.util.Timer;
+
+/**
+ * In Nephele an instance manager maintains the set of available compute resources. It is responsible for allocating new
+ * compute resources,
+ * provisioning available compute resources to the JobManager and keeping track of the availability of the utilized
+ * compute resources in order
+ * to report unexpected resource outages.
+ * 
+ */
+public class DefaultInstanceManager implements InstanceManager {
+
+	// ------------------------------------------------------------------------
+	// Internal Constants
+	// ------------------------------------------------------------------------
+
+	/**
+	 * The log object used to report debugging and error information.
+	 */
+	private static final Log LOG = LogFactory.getLog(DefaultInstanceManager.class);
+
+	/**
+	 * Default duration after which a host is purged in case it did not send
+	 * a heart-beat message.
+	 */
+	private static final int DEFAULT_CLEANUP_INTERVAL = 2 * 60; // 2 min.
+
+	/**
+	 * The key to retrieve the clean up interval from the configuration.
+	 */
+	private static final String CLEANUP_INTERVAL_KEY = "instancemanager.cluster.cleanupinterval";
+
+	// ------------------------------------------------------------------------
+	// Fields
+	// ------------------------------------------------------------------------
+
+	private final Object lock = new Object();
+
+	/**
+	 * Duration after which a host is purged in case it did not send a
+	 * heart-beat message.
+	 */
+	private final long cleanUpInterval;
+
+	/**
+	 * Set of hosts known to run a task manager that are thus able to execute
+	 * tasks.
+	 */
+	private final Map<InstanceConnectionInfo, Instance> registeredHosts;
+
+	/**
+	 * The network topology of the cluster.
+	 */
+	private final NetworkTopology networkTopology;
+
+	/**
+	 * Object that is notified if instances become available or vanish.
+	 */
+	private InstanceListener instanceListener;
+
+
+	private boolean shutdown;
+
+	/**
+	 * 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() {
+
+			synchronized (DefaultInstanceManager.this.lock) {
+
+				final List<Map.Entry<InstanceConnectionInfo, Instance>> hostsToRemove =
+						new ArrayList<Map.Entry<InstanceConnectionInfo, Instance>>();
+
+				final Map<JobID, List<AllocatedResource>> staleResources = new HashMap<JobID, List<AllocatedResource>>();
+
+				// check all hosts whether they did not send heart-beat messages.
+				for (Map.Entry<InstanceConnectionInfo, Instance> entry : registeredHosts.entrySet()) {
+
+					final Instance host = entry.getValue();
+					if (!host.isStillAlive(cleanUpInterval)) {
+
+						// this host has not sent the heart-beat messages
+						// -> we terminate all instances running on this host and notify the jobs
+						final Collection<AllocatedSlot> slots = host.removeAllocatedSlots();
+						for (AllocatedSlot slot : slots) {
+
+							final JobID jobID = slot.getJobID();
+
+							List<AllocatedResource> staleResourcesOfJob = staleResources.get(jobID);
+							if (staleResourcesOfJob == null) {
+								staleResourcesOfJob = new ArrayList<AllocatedResource>();
+								staleResources.put(jobID, staleResourcesOfJob);
+							}
+
+							staleResourcesOfJob.add(new AllocatedResource(host,	slot.getAllocationID()));
+						}
+
+						hostsToRemove.add(entry);
+					}
+				}
+
+				registeredHosts.entrySet().removeAll(hostsToRemove);
+
+				final Iterator<Map.Entry<JobID, List<AllocatedResource>>> it = staleResources.entrySet().iterator();
+				while (it.hasNext()) {
+					final Map.Entry<JobID, List<AllocatedResource>> entry = it.next();
+					if (instanceListener != null) {
+						instanceListener.allocatedResourcesDied(entry.getKey(), entry.getValue());
+					}
+				}
+			}
+		}
+	};
+
+	// ------------------------------------------------------------------------
+	// Constructor and set-up
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Constructor.
+	 */
+	public DefaultInstanceManager() {
+
+		this.registeredHosts = new HashMap<InstanceConnectionInfo, Instance>();
+
+		long tmpCleanUpInterval = (long) GlobalConfiguration.getInteger(CLEANUP_INTERVAL_KEY, DEFAULT_CLEANUP_INTERVAL) * 1000;
+
+		if (tmpCleanUpInterval < 10) { // Clean up interval must be at least ten seconds
+			LOG.warn("Invalid clean up interval. Reverting to default cleanup interval of " + DEFAULT_CLEANUP_INTERVAL
+					+ " secs.");
+			tmpCleanUpInterval = DEFAULT_CLEANUP_INTERVAL;
+		}
+
+		this.cleanUpInterval = tmpCleanUpInterval;
+
+		this.networkTopology = NetworkTopology.createEmptyTopology();
+
+		// look every BASEINTERVAL milliseconds for crashed hosts
+		final boolean runTimerAsDaemon = true;
+		new Timer(runTimerAsDaemon).schedule(cleanupStaleMachines, 1000, 1000);
+	}
+
+	@Override
+	public void shutdown() {
+		synchronized (this.lock) {
+			if (this.shutdown) {
+				return;
+			}
+
+			this.cleanupStaleMachines.cancel();
+
+			Iterator<Instance> it = this.registeredHosts.values().iterator();
+			while (it.hasNext()) {
+				it.next().destroyProxies();
+			}
+			this.registeredHosts.clear();
+
+			this.shutdown = true;
+		}
+	}
+
+	@Override
+	public void releaseAllocatedResource(AllocatedResource allocatedResource) throws InstanceException
+	{
+		synchronized (this.lock) {
+			// release the instance from the host
+			final Instance clusterInstance = allocatedResource.getInstance();
+			clusterInstance.releaseSlot(allocatedResource.getAllocationID());
+		}
+	}
+
+	/**
+	 * Creates a new {@link Instance} object to manage instances that can
+	 * be executed on that host.
+	 *
+	 * @param instanceConnectionInfo
+	 *        the connection information for the instance
+	 * @param hardwareDescription
+	 *        the hardware description provided by the new instance
+	 * @param numberOfSlots
+	 * 		  number of slots available on the instance
+	 * @return a new {@link Instance} object or <code>null</code> if the cluster instance could not be created
+	 */
+	private Instance createNewHost(final InstanceConnectionInfo instanceConnectionInfo,
+							final HardwareDescription hardwareDescription, int numberOfSlots) {
+
+		// Try to match new host with a stub host from the existing topology
+		String instanceName = instanceConnectionInfo.hostname();
+		NetworkNode parentNode = this.networkTopology.getRootNode();
+		NetworkNode currentStubNode = null;
+
+		// Try to match new host using the host name
+		while (true) {
+
+			currentStubNode = this.networkTopology.getNodeByName(instanceName);
+			if (currentStubNode != null) {
+				break;
+			}
+
+			final int pos = instanceName.lastIndexOf('.');
+			if (pos == -1) {
+				break;
+			}
+
+			/*
+			 * If host name is reported as FQDN, iterative remove parts
+			 * of the domain name until a match occurs or no more dots
+			 * can be found in the host name.
+			 */
+			instanceName = instanceName.substring(0, pos);
+		}
+
+		// Try to match the new host using the IP address
+		if (currentStubNode == null) {
+			instanceName = instanceConnectionInfo.address().toString();
+			instanceName = instanceName.replaceAll("/", ""); // Remove any / characters
+			currentStubNode = this.networkTopology.getNodeByName(instanceName);
+		}
+
+		if (currentStubNode != null) {
+			/*
+			 * The instance name will be the same as the one of the stub node. That way
+			 * the stub now will be removed from the network topology and replaced be
+			 * the new node.
+			 */
+			if (currentStubNode.getParentNode() != null) {
+				parentNode = currentStubNode.getParentNode();
+			}
+			// Remove the stub node from the tree
+			currentStubNode.remove();
+		}
+
+		LOG.info("Creating instance for " + instanceConnectionInfo + ", parent is "
+				+ parentNode.getName());
+		final Instance host = new Instance(instanceConnectionInfo, parentNode,
+				this.networkTopology, hardwareDescription, numberOfSlots);
+
+		return host;
+	}
+
+	@Override
+	public void reportHeartBeat(InstanceConnectionInfo instanceConnectionInfo) {
+
+		synchronized (this.lock) {
+			Instance host = registeredHosts.get(instanceConnectionInfo);
+
+			if(host == null){
+				LOG.error("Task manager with connection info " + instanceConnectionInfo + " has not been registered.");
+				return;
+			}
+
+			host.reportHeartBeat();
+		}
+	}
+
+	@Override
+	public void registerTaskManager(InstanceConnectionInfo instanceConnectionInfo,
+									HardwareDescription hardwareDescription, int numberOfSlots){
+		synchronized(this.lock){
+			if(registeredHosts.containsKey(instanceConnectionInfo)){
+				LOG.error("Task manager with connection info " + instanceConnectionInfo + " has already been " +
+						"registered.");
+				return;
+			}
+
+			Instance host = createNewHost(instanceConnectionInfo, hardwareDescription, numberOfSlots);
+
+			if(host == null){
+				LOG.error("Could not create a new host object for register task manager for connection info " +
+						instanceConnectionInfo);
+				return;
+			}
+
+			this.registeredHosts.put(instanceConnectionInfo, host);
+			LOG.info("New number of registered hosts is " + this.registeredHosts.size());
+
+			host.reportHeartBeat();
+		}
+	}
+
+	@Override
+	public void requestInstance(JobID jobID, Configuration conf,  int requiredSlots)
+			throws InstanceException
+	{
+
+		synchronized(this.lock) {
+			Iterator<Instance> clusterIterator = this.registeredHosts.values().iterator();
+			Instance instance = null;
+			List<AllocatedResource> allocatedResources = new ArrayList<AllocatedResource>();
+			int allocatedSlots = 0;
+
+			while(clusterIterator.hasNext()) {
+				instance = clusterIterator.next();
+				while(instance.getNumberOfAvailableSlots() >0  && allocatedSlots < requiredSlots){
+					AllocatedResource resource = instance.allocateSlot(jobID);
+					allocatedResources.add(resource);
+					allocatedSlots++;
+				}
+			}
+
+			if(allocatedSlots < requiredSlots){
+				throw new InstanceException("Cannot allocate the required number of slots: " + requiredSlots + ".");
+			}
+
+			if (this.instanceListener != null) {
+				final InstanceNotifier instanceNotifier = new InstanceNotifier(
+						this.instanceListener, jobID, allocatedResources);
+				instanceNotifier.start();
+			}
+		}
+	}
+
+	@Override
+	public NetworkTopology getNetworkTopology(JobID jobID) {
+		return this.networkTopology;
+	}
+
+	@Override
+	public void setInstanceListener(InstanceListener instanceListener) {
+		synchronized (this.lock) {
+			this.instanceListener = instanceListener;
+		}
+	}
+
+	@Override
+	public Instance getInstanceByName(String name) {
+		if (name == null) {
+			throw new IllegalArgumentException("Argument name must not be null");
+		}
+
+		synchronized (this.lock) {
+			final Iterator<Instance> it = this.registeredHosts.values().iterator();
+			while (it.hasNext()) {
+				final Instance instance = it.next();
+				if (name.equals(instance.getName())) {
+					return instance;
+				}
+			}
+		}
+
+		return null;
+	}
+
+	@Override
+	public int getNumberOfTaskTrackers() {
+		return this.registeredHosts.size();
+	}
+
+	@Override
+	public int getNumberOfSlots() {
+		int slots = 0;
+
+		for(Instance instance: registeredHosts.values()){
+			slots += instance.getNumberOfSlots();
+		}
+
+		return slots;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/DummyInstance.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/DummyInstance.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/DummyInstance.java
index 4e0f004..56f44c6 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/DummyInstance.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/DummyInstance.java
@@ -14,32 +14,30 @@
 package eu.stratosphere.nephele.instance;
 
 /**
- * A DummyInstance is a stub implementation of the {@link AbstractInstance} interface.
+ * 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 AbstractInstance {
+public class DummyInstance extends Instance {
 
 	private static int nextID = 0;
 
 	private final String name;
 
-	public static synchronized DummyInstance createDummyInstance(InstanceType type) {
+	public static synchronized DummyInstance createDummyInstance() {
 
-		return new DummyInstance(type, nextID++);
+		return new DummyInstance(nextID++);
 	}
 
 	/**
 	 * Constructs a new dummy instance of the given instance type.
 	 * 
-	 * @param type
-	 *        the type of the new dummy instance
 	 * @param id
 	 *        the ID of the dummy instance
 	 */
-	private DummyInstance(InstanceType type, int id) {
-		super(type, null, null, null, null);
+	private DummyInstance(int id) {
+		super(null, null, null, null, 0);
 
 		this.name = "DummyInstance_" + Integer.toString(id);
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/Hardware.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/Hardware.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/Hardware.java
new file mode 100644
index 0000000..398a2a8
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/Hardware.java
@@ -0,0 +1,24 @@
+/***********************************************************************************************************************
+ * 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.instance;
+
+/**
+ * Convenience class to extract hardware specifics of the computer executing this class
+ */
+public class Hardware {
+
+	public static int getNumberCPUCores() {
+		return Runtime.getRuntime().availableProcessors();
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/Instance.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/Instance.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/Instance.java
new file mode 100644
index 0000000..fa17745
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/Instance.java
@@ -0,0 +1,362 @@
+/***********************************************************************************************************************
+ * 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.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.Set;
+import java.util.Collection;
+
+import eu.stratosphere.nephele.deployment.TaskDeploymentDescriptor;
+import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
+import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileRequest;
+import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileResponse;
+import eu.stratosphere.nephele.execution.librarycache.LibraryCacheUpdate;
+import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
+import eu.stratosphere.nephele.ipc.RPC;
+import eu.stratosphere.nephele.jobgraph.JobID;
+import eu.stratosphere.nephele.net.NetUtils;
+import eu.stratosphere.nephele.protocols.TaskOperationProtocol;
+import eu.stratosphere.nephele.taskmanager.TaskCancelResult;
+import eu.stratosphere.nephele.taskmanager.TaskKillResult;
+import eu.stratosphere.nephele.taskmanager.TaskSubmissionResult;
+import eu.stratosphere.nephele.topology.NetworkNode;
+import eu.stratosphere.nephele.topology.NetworkTopology;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+
+/**
+ * An instance represents a resource a {@link eu.stratosphere.nephele.taskmanager.TaskManager} runs on.
+ * 
+ */
+public class Instance extends NetworkNode {
+	/**
+	 * The connection info identifying the instance.
+	 */
+	private final InstanceConnectionInfo instanceConnectionInfo;
+
+	/**
+	 * The hardware description as reported by the instance itself.
+	 */
+	private final HardwareDescription hardwareDescription;
+
+	/**
+	 * Number of slots available on the node
+	 */
+	private final int numberOfSlots;
+
+	/**
+	 * Allocated slots on this instance
+	 */
+	private final Map<AllocationID, AllocatedSlot> allocatedSlots = new HashMap<AllocationID, AllocatedSlot>();
+
+	/**
+	 * Stores the RPC stub object for the instance's task manager.
+	 */
+	private TaskOperationProtocol taskManager = null;
+
+	/**
+	 * Time when last heat beat has been received from the task manager running on this instance.
+	 */
+	private long lastReceivedHeartBeat = System.currentTimeMillis();
+
+	/**
+	 * Constructs an abstract instance object.
+	 * 
+	 * @param instanceConnectionInfo
+	 *        the connection info identifying the instance
+	 * @param parentNode
+	 *        the parent node in the network topology
+	 * @param networkTopology
+	 *        the network topology this node is a part of
+	 * @param hardwareDescription
+	 *        the hardware description provided by the instance itself
+	 */
+	public Instance(final InstanceConnectionInfo instanceConnectionInfo,
+					final NetworkNode parentNode, final NetworkTopology networkTopology,
+					final HardwareDescription hardwareDescription, int numberOfSlots) {
+		super((instanceConnectionInfo == null) ? null : instanceConnectionInfo.toString(), parentNode, networkTopology);
+		this.instanceConnectionInfo = instanceConnectionInfo;
+		this.hardwareDescription = hardwareDescription;
+		this.numberOfSlots = numberOfSlots;
+	}
+
+	/**
+	 * 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());
+		}
+
+		return this.taskManager;
+	}
+
+	/**
+	 * 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;
+		}
+	}
+
+	/**
+	 * Returns the instance's connection information object.
+	 * 
+	 * @return the instance's connection information object
+	 */
+	public final InstanceConnectionInfo getInstanceConnectionInfo() {
+		return this.instanceConnectionInfo;
+	}
+
+	/**
+	 * Returns the instance's hardware description as reported by the instance itself.
+	 * 
+	 * @return the instance's hardware description
+	 */
+	public HardwareDescription getHardwareDescription() {
+		return this.hardwareDescription;
+	}
+
+	/**
+	 * 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);
+		}
+
+		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);
+			}
+		}
+	}
+
+	/**
+	 * Submits a list of tasks to the instance's {@link eu.stratosphere.nephele.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);
+	}
+
+	/**
+	 * Cancels the task identified by the given ID at the instance's
+	 * {@link eu.stratosphere.nephele.taskmanager.TaskManager}.
+	 * 
+	 * @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
+	 */
+	public synchronized TaskCancelResult cancelTask(final ExecutionVertexID id) throws IOException {
+
+		return getTaskManagerProxy().cancelTask(id);
+	}
+
+	/**
+	 * Kills the task identified by the given ID at the instance's
+	 * {@link eu.stratosphere.nephele.taskmanager.TaskManager}.
+	 * 
+	 * @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
+	 */
+	public synchronized TaskKillResult killTask(final ExecutionVertexID id) throws IOException {
+
+		return getTaskManagerProxy().killTask(id);
+	}
+
+	/**
+	 * Updates the time of last received heart beat to the current system time.
+	 */
+	public synchronized void reportHeartBeat() {
+		this.lastReceivedHeartBeat = System.currentTimeMillis();
+	}
+
+	/**
+	 * Returns whether the host is still alive.
+	 *
+	 * @param cleanUpInterval
+	 *        duration (in milliseconds) after which a host is
+	 *        considered dead if it has no received heat-beats.
+	 * @return <code>true</code> if the host has received a heat-beat before the <code>cleanUpInterval</code> duration
+	 *         has expired, <code>false</code> otherwise
+	 */
+	public synchronized boolean isStillAlive(final long cleanUpInterval) {
+
+		if (this.lastReceivedHeartBeat + cleanUpInterval < System.currentTimeMillis()) {
+			return false;
+		}
+		return true;
+	}
+
+
+	@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.
+	 */
+	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;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceManager.java
index a1015b5..00795f4 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceManager.java
@@ -13,157 +13,32 @@
 
 package eu.stratosphere.nephele.instance;
 
-import java.util.List;
-import java.util.Map;
 
 import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.nephele.jobgraph.JobID;
 import eu.stratosphere.nephele.topology.NetworkTopology;
 
-/**
- * In Nephele an instance manager maintains the set of available compute resources. It is responsible for allocating new
- * compute resources,
- * provisioning available compute resources to the JobManager and keeping track of the availability of the utilized
- * compute resources in order
- * to report unexpected resource outages.
- * 
- */
 public interface InstanceManager {
 
-	/**
-	 * Requests an instance of the provided instance type from the instance manager.
-	 * 
-	 * @param jobID
-	 *        the ID of the job this instance is requested for
-	 * @param conf
-	 *        a configuration object including additional request information (e.g. credentials)
-	 * @param instanceRequestMap
-	 *        a map specifying the instances requested by this call
-	 * @param count
-	 *        the number of instances
-	 * @throws InstanceException
-	 *         thrown if an error occurs during the instance request
-	 */
-	void requestInstance(JobID jobID, Configuration conf, InstanceRequestMap instanceRequestMap,
-			List<String> splitAffinityList) throws InstanceException;
 
-	/**
-	 * Releases an allocated resource from a job.
-	 * 
-	 * @param jobID
-	 *        the ID of the job the instance has been used for
-	 * @param conf
-	 *        a configuration object including additional release information (e.g. credentials)
-	 * @param allocatedResource
-	 *        the allocated resource to be released
-	 * @throws InstanceException
-	 *         thrown if an error occurs during the release process
-	 */
-	void releaseAllocatedResource(JobID jobID, Configuration conf, AllocatedResource allocatedResource)
-			throws InstanceException;
-
-	/**
-	 * Suggests a suitable instance type according to the provided hardware characteristics.
-	 * 
-	 * @param minNumComputeUnits
-	 *        the minimum number of compute units
-	 * @param minNumCPUCores
-	 *        the minimum number of CPU cores
-	 * @param minMemorySize
-	 *        the minimum number of main memory (in MB)
-	 * @param minDiskCapacity
-	 *        the minimum hard disk capacity (in GB)
-	 * @param maxPricePerHour
-	 *        the maximum price per hour for the instance
-	 * @return the instance type matching the requested hardware profile best or <code>null</code> if no such instance
-	 *         type is available
-	 */
-	InstanceType getSuitableInstanceType(int minNumComputeUnits, int minNumCPUCores, int minMemorySize,
-			int minDiskCapacity, int maxPricePerHour);
+	void shutdown();
 
-	/**
-	 * Reports a heart beat message of an instance.
-	 * 
-	 * @param instanceConnectionInfo
-	 *        the {@link InstanceConnectionInfo} object attached to the heart beat message
-	 * @param hardwareDescription
-	 *        a hardware description with details on the instance's compute resources.
-	 */
-	void reportHeartBeat(InstanceConnectionInfo instanceConnectionInfo, HardwareDescription hardwareDescription);
+	void releaseAllocatedResource(AllocatedResource allocatedResource) throws InstanceException;
 
-	/**
-	 * Translates the name of an instance type to the corresponding instance type object.
-	 * 
-	 * @param instanceTypeName
-	 *        the name of the instance type
-	 * @return the instance type object matching the name or <code>null</code> if no such instance type exists
-	 */
-	InstanceType getInstanceTypeByName(String instanceTypeName);
+	void reportHeartBeat(InstanceConnectionInfo instanceConnectionInfo);
 
-	/**
-	 * Returns the default instance type used by the instance manager.
-	 * 
-	 * @return the default instance type
-	 */
-	InstanceType getDefaultInstanceType();
+	void registerTaskManager(InstanceConnectionInfo instanceConnectionInfo,
+									HardwareDescription hardwareDescription, int numberOfSlots);
+	void requestInstance(JobID jobID, Configuration conf,  int requiredSlots)
+			throws InstanceException;
 
-	/**
-	 * Returns the network topology for the job with the given ID. The network topology
-	 * for the job might only be an excerpt of the overall network topology. It only
-	 * includes those instances as leaf nodes which are really allocated for the
-	 * execution of the job.
-	 * 
-	 * @param jobID
-	 *        the ID of the job to get the topology for
-	 * @return the network topology for the job
-	 */
 	NetworkTopology getNetworkTopology(JobID jobID);
 
-	/**
-	 * Sets the {@link InstanceListener} object which is supposed to be
-	 * notified about instance availability and deaths.
-	 * 
-	 * @param instanceListener
-	 *        the instance listener to set for this instance manager
-	 */
 	void setInstanceListener(InstanceListener instanceListener);
 
-	/**
-	 * Returns a map of all instance types which are currently available to Nephele. The map contains a description of
-	 * the hardware characteristics for each instance type as provided in the configuration file. Moreover, it contains
-	 * the actual hardware description as reported by task managers running on the individual instances. If available,
-	 * the map also contains the maximum number instances Nephele can allocate of each instance type (i.e. if no other
-	 * job occupies instances).
-	 * 
-	 * @return a list of all instance types available to Nephele
-	 */
-	Map<InstanceType, InstanceTypeDescription> getMapOfAvailableInstanceTypes();
-
-	/**
-	 * Returns the {@link AbstractInstance} with the given name.
-	 * 
-	 * @param name
-	 *        the name of the instance
-	 * @return the instance with the given name or <code>null</code> if no such instance could be found
-	 */
-	AbstractInstance getInstanceByName(String name);
+	Instance getInstanceByName(String name);
 
-	/**
-	 * Cancels all pending instance requests that might still exist for the job with the given ID.
-	 * 
-	 * @param jobID
-	 *        the ID of the job to cancel the pending instance requests for
-	 */
-	void cancelPendingRequests(JobID jobID);
-
-	/**
-	 * Shuts the instance manager down and stops all its internal processes.
-	 */
-	void shutdown();
-
-	/**
-	 * 
-	 * @return the number of available (registered) TaskTrackers
-	 */
 	int getNumberOfTaskTrackers();
+
+	int getNumberOfSlots();
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceNotifier.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceNotifier.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceNotifier.java
new file mode 100644
index 0000000..2df3d3d
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceNotifier.java
@@ -0,0 +1,71 @@
+/***********************************************************************************************************************
+ * 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.instance;
+
+import java.util.List;
+
+import eu.stratosphere.nephele.instance.AllocatedResource;
+import eu.stratosphere.nephele.instance.InstanceListener;
+import eu.stratosphere.nephele.jobgraph.JobID;
+
+/**
+ * This class is an auxiliary class to send the notification
+ * about the availability of an {@link eu.stratosphere.nephele.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/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceRequestMap.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceRequestMap.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceRequestMap.java
deleted file mode 100644
index 4167f67..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceRequestMap.java
+++ /dev/null
@@ -1,184 +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.instance;
-
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-
-/**
- * An instance request map specifies the required types of instances to run a specific job and the respective number
- * thereof. For each instance type it is possible to specify the minimum number of instances required to run the job. If
- * the {@link InstanceManager} cannot manage to provide at least this minimum numbers of instances for the given type,
- * the job will be rejected.
- * <p>
- * In addition, is it also possible to specify the optimal number of instances for a particular instance type. The
- * {@link InstanceManager} will try to provide this optimal number of instances, but will also start the job with less
- * instances.
- * <p>
- * This class is not thread-safe.
- * 
- */
-public final class InstanceRequestMap {
-
-	/**
-	 * The map holding the minimum number of instances to be requested for each instance type.
-	 */
-	private final Map<InstanceType, Integer> minimumMap = new HashMap<InstanceType, Integer>();
-
-	/**
-	 * The map holding the maximum number of instances to be requested for each instance type.
-	 */
-	private final Map<InstanceType, Integer> maximumMap = new HashMap<InstanceType, Integer>();
-
-	/**
-	 * Sets the minimum number of instances to be requested from the given instance type.
-	 * 
-	 * @param instanceType
-	 *        the type of instance to request
-	 * @param number
-	 *        the minimum number of instances to request
-	 */
-	public void setMinimumNumberOfInstances(final InstanceType instanceType, final int number) {
-
-		this.minimumMap.put(instanceType, Integer.valueOf(number));
-	}
-
-	/**
-	 * Sets the maximum number of instances to be requested from the given instance type.
-	 * 
-	 * @param instanceType
-	 *        the type of instance to request
-	 * @param number
-	 *        the maximum number of instances to request
-	 */
-	public void setMaximumNumberOfInstances(final InstanceType instanceType, final int number) {
-
-		this.maximumMap.put(instanceType, Integer.valueOf(number));
-	}
-
-	/**
-	 * Sets both the minimum and the maximum number of instances to be requested from the given instance type.
-	 * 
-	 * @param instanceType
-	 *        the type of instance to request
-	 * @param number
-	 *        the minimum and the maximum number of instances to request
-	 */
-	public void setNumberOfInstances(final InstanceType instanceType, final int number) {
-
-		setMinimumNumberOfInstances(instanceType, number);
-		setMaximumNumberOfInstances(instanceType, number);
-	}
-
-	/**
-	 * Returns the minimum number of instances to be requested from the given instance type.
-	 * 
-	 * @param instanceType
-	 *        the type of instance to request
-	 * @return the minimum number of instances to be requested from the given instance type
-	 */
-	public int getMinimumNumberOfInstances(final InstanceType instanceType) {
-
-		final Integer val = this.minimumMap.get(instanceType);
-		if (val != null) {
-			return val.intValue();
-		}
-
-		return 0;
-	}
-
-	/**
-	 * Returns the maximum number of instances to be requested from the given instance type.
-	 * 
-	 * @param instanceType
-	 *        the type of instance to request
-	 * @return the maximum number of instances to be requested from the given instance type
-	 */
-	public int getMaximumNumberOfInstances(final InstanceType instanceType) {
-
-		final Integer val = this.maximumMap.get(instanceType);
-		if (val != null) {
-			return val.intValue();
-		}
-
-		return 0;
-	}
-
-	/**
-	 * Checks if this instance request map is empty, i.e. neither contains an entry for the minimum or maximum number of
-	 * instances to be requested for any instance type.
-	 * 
-	 * @return <code>true</code> if the map is empty, <code>false</code> otherwise
-	 */
-	public boolean isEmpty() {
-
-		if (!this.maximumMap.isEmpty()) {
-			return false;
-		}
-
-		if (!this.minimumMap.isEmpty()) {
-			return false;
-		}
-
-		return true;
-	}
-
-	/**
-	 * Returns an {@link Iterator} object which allows to traverse the minimum number of instances to be requested for
-	 * each instance type.
-	 * 
-	 * @return an iterator to traverse the minimum number of instances to be requested for each instance type
-	 */
-	public Iterator<Map.Entry<InstanceType, Integer>> getMaximumIterator() {
-
-		return this.maximumMap.entrySet().iterator();
-	}
-
-	/**
-	 * Returns an {@link Iterator} object which allows to traverse the maximum number of instances to be requested for
-	 * each instance type.
-	 * 
-	 * @return an iterator to traverse the maximum number of instances to be requested for each instance type
-	 */
-	public Iterator<Map.Entry<InstanceType, Integer>> getMinimumIterator() {
-
-		return this.minimumMap.entrySet().iterator();
-	}
-
-	/**
-	 * Returns the number of different instance types stored in this request map.
-	 * 
-	 * @return the number of different instance types stored in this request map
-	 */
-	public int size() {
-
-		final int s = this.maximumMap.size();
-
-		if (s != this.minimumMap.size()) {
-			throw new IllegalStateException("InstanceRequestMap is in an inconsistent state");
-		}
-
-		return s;
-	}
-
-	/**
-	 * Clears the instance request map.
-	 */
-	public void clear() {
-
-		this.maximumMap.clear();
-		this.minimumMap.clear();
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceType.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceType.java
deleted file mode 100644
index f2bb4e5..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceType.java
+++ /dev/null
@@ -1,199 +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.instance;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.core.io.StringRecord;
-
-/**
- * An instance type describes the hardware resources a task manager runs on. According
- * to its type an instance has a specific number of CPU cores, computation units, a certain
- * amount of main memory and disk space. In addition, it has a specific price per hour.
- * 
- */
-public final class InstanceType implements IOReadableWritable {
-
-	/**
-	 * The identifier for this instance type.
-	 */
-	private String identifier;
-
-	/**
-	 * The number of computational units of this instance type.
-	 * A computational unit is a virtual compute capacity. A host with a
-	 * single-core 2 GHz CPU may possess 20 compute units (1*20), while a
-	 * dual-core 2.5 GHz CPU may possess 50 compute units (2*25). The
-	 * specified number of compute units expresses the fraction of the
-	 * CPU capacity promised to a user.
-	 */
-	private int numberOfComputeUnits = 0;
-
-	/**
-	 * The number of CPU cores of this instance type.
-	 */
-	private int numberOfCores = 0;
-
-	/**
-	 * The amount of main memory of this instance type (in MB).
-	 */
-	private int memorySize = 0;
-
-	/**
-	 * The disk capacity of this instance type (in GB).
-	 */
-	private int diskCapacity = 0;
-
-	/**
-	 * The price per hour that is charged for running instances of this type.
-	 */
-	private int pricePerHour = 0;
-
-	/**
-	 * Public constructor required for the serialization process.
-	 */
-	public InstanceType() {
-	}
-
-	/**
-	 * Creates a new instance type.
-	 * 
-	 * @param identifier
-	 *        identifier for this instance type
-	 * @param numberOfComputeUnits
-	 *        number of computational units of this instance type
-	 * @param numberOfCores
-	 *        number of CPU cores of this instance type
-	 * @param memorySize
-	 *        amount of main memory of this instance type (in MB)
-	 * @param diskCapacity
-	 *        disk capacity of this instance type (in GB)
-	 * @param pricePerHour
-	 *        price per hour that is charged for running instances of this type
-	 */
-	InstanceType(final String identifier, final int numberOfComputeUnits, final int numberOfCores,
-			final int memorySize,
-			final int diskCapacity, final int pricePerHour) {
-
-		this.identifier = identifier;
-		this.numberOfComputeUnits = numberOfComputeUnits;
-		this.numberOfCores = numberOfCores;
-		this.memorySize = memorySize;
-		this.diskCapacity = diskCapacity;
-		this.pricePerHour = pricePerHour;
-	}
-
-	/**
-	 * Returns the instance type's number of computational units.
-	 * 
-	 * @return the instance type's number of computational units
-	 */
-	public int getNumberOfComputeUnits() {
-		return this.numberOfComputeUnits;
-	}
-
-	/**
-	 * Returns the instance type's number of CPU cores.
-	 * 
-	 * @return the instance type's number of CPU cores
-	 */
-	public int getNumberOfCores() {
-		return this.numberOfCores;
-	}
-
-	/**
-	 * Returns the instance type's amount of main memory.
-	 * 
-	 * @return the instance type's amount of main memory
-	 */
-	public int getMemorySize() {
-		return this.memorySize;
-	}
-
-	/**
-	 * Returns the instance type's disk capacity.
-	 * 
-	 * @return the instance type's disk capacity
-	 */
-	public int getDiskCapacity() {
-		return this.diskCapacity;
-	}
-
-	/**
-	 * Returns the instance type's price per hour.
-	 * 
-	 * @return the instance type's price per hour
-	 */
-	public int getPricePerHour() {
-		return this.pricePerHour;
-	}
-
-	/**
-	 * Returns the instance type's identifier.
-	 * 
-	 * @return the instance type's identifier
-	 */
-	public String getIdentifier() {
-		return this.identifier;
-	}
-
-
-	@Override
-	public String toString() {
-
-		final StringBuilder bld = new StringBuilder(32);
-		bld.append(this.identifier);
-		bld.append(' ');
-		bld.append('(');
-		bld.append(this.numberOfComputeUnits);
-		bld.append(',');
-		bld.append(this.numberOfCores);
-		bld.append(',');
-		bld.append(this.memorySize);
-		bld.append(',');
-		bld.append(this.diskCapacity);
-		bld.append(',');
-		bld.append(this.pricePerHour);
-		bld.append(')');
-
-		return bld.toString();
-	}
-
-
-	@Override
-	public void write(final DataOutput out) throws IOException {
-
-		StringRecord.writeString(out, this.identifier);
-		out.writeInt(this.numberOfComputeUnits);
-		out.writeInt(this.numberOfCores);
-		out.writeInt(this.memorySize);
-		out.writeInt(this.diskCapacity);
-		out.writeInt(this.pricePerHour);
-	}
-
-
-	@Override
-	public void read(final DataInput in) throws IOException {
-
-		this.identifier = StringRecord.readString(in);
-		this.numberOfComputeUnits = in.readInt();
-		this.numberOfCores = in.readInt();
-		this.memorySize = in.readInt();
-		this.diskCapacity = in.readInt();
-		this.pricePerHour = in.readInt();
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceTypeDescription.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceTypeDescription.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceTypeDescription.java
deleted file mode 100644
index ce0a694..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceTypeDescription.java
+++ /dev/null
@@ -1,137 +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.instance;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-
-/**
- * An instance type description provides details of instance type. Is can comprise both the hardware description from
- * the instance type description (as provided by the operator/administrator of the instance) as well as the actual
- * hardware description which has been determined on the compute instance itself.
- * 
- */
-public final class InstanceTypeDescription implements IOReadableWritable {
-
-	/**
-	 * The instance type.
-	 */
-	private InstanceType instanceType = null;
-
-	/**
-	 * The hardware description as created by the {@link InstanceManager}.
-	 */
-	private HardwareDescription hardwareDescription = null;
-
-	/**
-	 * The maximum number of available instances of this type.
-	 */
-	private int maximumNumberOfAvailableInstances = 0;
-
-	/**
-	 * Public default constructor required for serialization process.
-	 */
-	public InstanceTypeDescription() {
-	}
-
-	/**
-	 * Constructs a new instance type description.
-	 * 
-	 * @param instanceType
-	 *        the instance type
-	 * @param hardwareDescription
-	 *        the hardware description as created by the {@link InstanceManager}
-	 * @param maximumNumberOfAvailableInstances
-	 *        the maximum number of available instances of this type
-	 */
-	InstanceTypeDescription(final InstanceType instanceType, final HardwareDescription hardwareDescription,
-			final int maximumNumberOfAvailableInstances) {
-
-		this.instanceType = instanceType;
-		this.hardwareDescription = hardwareDescription;
-		this.maximumNumberOfAvailableInstances = maximumNumberOfAvailableInstances;
-	}
-
-
-	@Override
-	public void write(final DataOutput out) throws IOException {
-
-		if (this.instanceType == null) {
-			out.writeBoolean(false);
-		} else {
-			out.writeBoolean(true);
-			this.instanceType.write(out);
-		}
-
-		if (this.hardwareDescription == null) {
-			out.writeBoolean(false);
-		} else {
-			out.writeBoolean(true);
-			this.hardwareDescription.write(out);
-		}
-
-		out.writeInt(this.maximumNumberOfAvailableInstances);
-	}
-
-
-	@Override
-	public void read(final DataInput in) throws IOException {
-
-		if (in.readBoolean()) {
-			this.instanceType = new InstanceType();
-			this.instanceType.read(in);
-		} else {
-			this.instanceType = null;
-		}
-
-		if (in.readBoolean()) {
-			this.hardwareDescription = new HardwareDescription();
-			this.hardwareDescription.read(in);
-		}
-
-		this.maximumNumberOfAvailableInstances = in.readInt();
-	}
-
-	/**
-	 * Returns the hardware description as created by the {@link InstanceManager}.
-	 * 
-	 * @return the instance's hardware description or <code>null</code> if no description is available
-	 */
-	public HardwareDescription getHardwareDescription() {
-		return this.hardwareDescription;
-	}
-
-	/**
-	 * Returns the instance type as determined by the {@link InstanceManager}.
-	 * 
-	 * @return the instance type
-	 */
-	public InstanceType getInstanceType() {
-		return this.instanceType;
-	}
-
-	/**
-	 * Returns the maximum number of instances the {@link InstanceManager} can at most allocate of this instance type
-	 * (i.e. when no other jobs are occupying any resources).
-	 * 
-	 * @return the maximum number of instances of this type or <code>-1</code> if the number is unknown to the
-	 *         {@link InstanceManager}
-	 */
-	public int getMaximumNumberOfAvailableInstances() {
-		return this.maximumNumberOfAvailableInstances;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceTypeDescriptionFactory.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceTypeDescriptionFactory.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceTypeDescriptionFactory.java
deleted file mode 100644
index 2b3e7db..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceTypeDescriptionFactory.java
+++ /dev/null
@@ -1,46 +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.instance;
-
-/**
- * This factory produces {@link InstanceTypeDescription} objects.
- * <p>
- * This class is thread-safe.
- * 
- */
-public class InstanceTypeDescriptionFactory {
-
-	/**
-	 * Private constructor, so class cannot be instantiated.
-	 */
-	private InstanceTypeDescriptionFactory() {
-	}
-
-	/**
-	 * Constructs a new {@link InstaceTypeDescription} object.
-	 * 
-	 * @param instanceType
-	 *        the instance type
-	 * @param hardwareDescription
-	 *        the hardware description as created by the {@link InstanceManager}
-	 * @param numberOfAvailableInstances
-	 *        the number of available instances of this type
-	 * @return the instance type description
-	 */
-	public static InstanceTypeDescription construct(InstanceType instanceType, HardwareDescription hardwareDescription,
-			int numberOfAvailableInstances) {
-
-		return new InstanceTypeDescription(instanceType, hardwareDescription, numberOfAvailableInstances);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceTypeFactory.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceTypeFactory.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceTypeFactory.java
deleted file mode 100644
index ff501c4..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceTypeFactory.java
+++ /dev/null
@@ -1,91 +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.instance;
-
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-/**
- * This factory constructs {@link InstanceType} objects.
- * 
- */
-public class InstanceTypeFactory {
-
-	/**
-	 * The logger used to report errors.
-	 */
-	private static final Log LOG = LogFactory.getLog(InstanceTypeFactory.class);
-
-	/**
-	 * The pattern used to parse the hardware descriptions of instance types.
-	 */
-	private static Pattern INSTANCE_TYPE_PATTERN = Pattern.compile("^([^,]+),(\\d+),(\\d+),(\\d+),(\\d+),(\\d+)$");
-
-	/**
-	 * Private constructor, so class cannot be instantiated.
-	 */
-	private InstanceTypeFactory() {
-	}
-
-	/**
-	 * Constructs an {@link InstanceType} object by parsing a hardware description string.
-	 * 
-	 * @param description
-	 *        the hardware description reflected by this instance type
-	 * @return an instance type reflecting the given hardware description or <code>null</code> if the description cannot
-	 *         be parsed
-	 */
-	public static InstanceType constructFromDescription(String description) {
-
-		final Matcher m = INSTANCE_TYPE_PATTERN.matcher(description);
-		if (!m.matches()) {
-			LOG.error("Cannot extract instance type from string " + description);
-			return null;
-		}
-
-		final String identifier = m.group(1);
-		final int numComputeUnits = Integer.parseInt(m.group(2));
-		final int numCores = Integer.parseInt(m.group(3));
-		final int memorySize = Integer.parseInt(m.group(4));
-		final int diskCapacity = Integer.parseInt(m.group(5));
-		final int pricePerHour = Integer.parseInt(m.group(6));
-
-		return new InstanceType(identifier, numComputeUnits, numCores, memorySize, diskCapacity, pricePerHour);
-	}
-
-	/**
-	 * Constructs an {@link InstanceType} from the given parameters.
-	 * 
-	 * @param identifier
-	 *        identifier for this instance type
-	 * @param numberOfComputeUnits
-	 *        number of computational units of this instance type
-	 * @param numberOfCores
-	 *        number of CPU cores of this instance type
-	 * @param memorySize
-	 *        amount of main memory of this instance type (in MB)
-	 * @param diskCapacity
-	 *        disk capacity of this instance type (in GB)
-	 * @param pricePerHour
-	 *        price per hour that is charged for running instances of this type
-	 */
-	public static InstanceType construct(String identifier, int numberOfComputeUnits, int numberOfCores,
-			int memorySize, int diskCapacity, int pricePerHour) {
-
-		return new InstanceType(identifier, numberOfComputeUnits, numberOfCores, memorySize, diskCapacity, pricePerHour);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/LocalInstanceManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/LocalInstanceManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/LocalInstanceManager.java
new file mode 100644
index 0000000..1576649
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/LocalInstanceManager.java
@@ -0,0 +1,60 @@
+/***********************************************************************************************************************
+ * 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.instance;
+
+
+import eu.stratosphere.configuration.ConfigConstants;
+import eu.stratosphere.configuration.Configuration;
+import eu.stratosphere.configuration.GlobalConfiguration;
+import eu.stratosphere.nephele.ExecutionMode;
+import eu.stratosphere.nephele.taskmanager.TaskManager;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class LocalInstanceManager extends DefaultInstanceManager {
+	
+	private List<TaskManager> taskManagers = new ArrayList<TaskManager>();
+
+	public LocalInstanceManager() throws Exception{
+		int numTaskManager = GlobalConfiguration.getInteger(ConfigConstants
+				.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, 1);
+
+		ExecutionMode execMode = numTaskManager == 1 ? ExecutionMode.LOCAL : ExecutionMode.CLUSTER;
+		
+		for (int i=0; i < numTaskManager; i++){
+			Configuration tm = new Configuration();
+			int ipcPort = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY,
+					ConfigConstants.DEFAULT_TASK_MANAGER_IPC_PORT);
+			int dataPort = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
+					ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT);
+
+			tm.setInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, ipcPort + i);
+			tm.setInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, dataPort + i);
+
+			GlobalConfiguration.includeConfiguration(tm);
+
+			taskManagers.add(new TaskManager(execMode));
+		}
+	}
+
+	@Override
+	public void shutdown(){
+		for(TaskManager taskManager: taskManagers){
+			taskManager.shutdown();
+		}
+
+		super.shutdown();
+	}
+}


[46/53] [abbrv] git commit: Change string construction to work around a known JVM performance bug still present in some older JVM versions.

Posted by rm...@apache.org.
Change string construction to work around a known JVM performance bug still present in some older JVM versions.


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

Branch: refs/heads/travis_test
Commit: ef623e9b883dea3f57d0fb33c3af3a804dc041e6
Parents: 8431395
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Jun 25 16:12:33 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Jun 25 16:14:18 2014 +0200

----------------------------------------------------------------------
 .../api/common/io/DelimitedInputFormat.java     |  2 +-
 .../types/parser/AsciiStringParser.java         |  9 ++----
 .../api/java/io/TextInputFormat.java            | 29 +++++++-------------
 3 files changed, 13 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ef623e9b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/DelimitedInputFormat.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/DelimitedInputFormat.java b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/DelimitedInputFormat.java
index 88eee94..185f7f8 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/DelimitedInputFormat.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/DelimitedInputFormat.java
@@ -258,7 +258,7 @@ public abstract class DelimitedInputFormat<OT> extends FileInputFormat<OT> {
 	 * 
 	 * @return returns whether the record was successfully deserialized or not.
 	 */
-	public abstract OT readRecord(OT reuse, byte[] bytes, int offset, int numBytes);
+	public abstract OT readRecord(OT reuse, byte[] bytes, int offset, int numBytes) throws IOException;
 	
 	// --------------------------------------------------------------------------------------------
 	//  Pre-flight: Configuration, Splits, Sampling

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ef623e9b/stratosphere-core/src/main/java/eu/stratosphere/types/parser/AsciiStringParser.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/types/parser/AsciiStringParser.java b/stratosphere-core/src/main/java/eu/stratosphere/types/parser/AsciiStringParser.java
index 381d968..c3e6784 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/types/parser/AsciiStringParser.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/types/parser/AsciiStringParser.java
@@ -13,17 +13,12 @@
 
 package eu.stratosphere.types.parser;
 
-import java.nio.charset.Charset;
-
 /**
  * Converts a variable length field of a byte array into a {@link String}. The byte contents between
  * delimiters is interpreted as an ASCII string. The string may be quoted in double quotes. For quoted
  * strings, whitespaces (space and tab) leading and trailing before and after the quotes are removed.
  */
 public class AsciiStringParser extends FieldParser<String> {
-
-	// the default (ascii style) charset. should be available really everywhere.
-	private static final Charset CHARSET = Charset.forName("ISO-8859-1");
 	
 	private static final byte WHITESPACE_SPACE = (byte) ' ';
 	private static final byte WHITESPACE_TAB = (byte) '\t';
@@ -58,7 +53,7 @@ public class AsciiStringParser extends FieldParser<String> {
 			
 			if (i < limit) {
 				// end of the string
-				this.result = new String(bytes, quoteStart, i-quoteStart, CHARSET);
+				this.result = new String(bytes, quoteStart, i-quoteStart);
 				
 				i++; // the quote
 				
@@ -87,7 +82,7 @@ public class AsciiStringParser extends FieldParser<String> {
 			}
 			
 			// set from the beginning. unquoted strings include the leading whitespaces
-			this.result = new String(bytes, startPos, i-startPos, CHARSET);
+			this.result = new String(bytes, startPos, i-startPos);
 			return (i == limit ? limit : i+1);
 		}
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ef623e9b/stratosphere-java/src/main/java/eu/stratosphere/api/java/io/TextInputFormat.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/io/TextInputFormat.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/io/TextInputFormat.java
index 6febb74..759c9e9 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/io/TextInputFormat.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/io/TextInputFormat.java
@@ -14,6 +14,7 @@
  **********************************************************************************************************************/
 package eu.stratosphere.api.java.io;
 
+import java.io.IOException;
 import java.nio.charset.Charset;
 
 import eu.stratosphere.api.common.io.DelimitedInputFormat;
@@ -25,12 +26,6 @@ public class TextInputFormat extends DelimitedInputFormat<String> {
 	
 	private static final long serialVersionUID = 1L;
 	
-	private String charsetName = "UTF-8";
-	
-//	private boolean skipInvalidLines;
-	
-	private transient Charset charset;
-
 	/**
 	 * Code of \r, used to remove \r from a line when the line ends with \r\n
 	 */
@@ -40,7 +35,12 @@ public class TextInputFormat extends DelimitedInputFormat<String> {
 	 * Code of \n, used to identify if \n is used as delimiter
 	 */
 	private static final byte NEW_LINE = (byte) '\n';
-
+	
+	
+	/**
+	 * The name of the charset to use for decoding.
+	 */
+	private String charsetName = "UTF-8";
 	
 	// --------------------------------------------------------------------------------------------
 	
@@ -62,14 +62,6 @@ public class TextInputFormat extends DelimitedInputFormat<String> {
 		this.charsetName = charsetName;
 	}
 	
-//	public boolean isSkipInvalidLines() {
-//		return skipInvalidLines;
-//	}
-//	
-//	public void setSkipInvalidLines(boolean skipInvalidLines) {
-//		this.skipInvalidLines = skipInvalidLines;
-//	}
-	
 	// --------------------------------------------------------------------------------------------
 
 	@Override
@@ -79,13 +71,12 @@ public class TextInputFormat extends DelimitedInputFormat<String> {
 		if (charsetName == null || !Charset.isSupported(charsetName)) {
 			throw new RuntimeException("Unsupported charset: " + charsetName);
 		}
-		this.charset = Charset.forName(charsetName);
 	}
 
 	// --------------------------------------------------------------------------------------------
 
 	@Override
-	public String readRecord(String reusable, byte[] bytes, int offset, int numBytes) {
+	public String readRecord(String reusable, byte[] bytes, int offset, int numBytes) throws IOException {
 		//Check if \n is used as delimiter and the end of this line is a \r, then remove \r from the line
 		if (this.getDelimiter() != null && this.getDelimiter().length == 1 
 				&& this.getDelimiter()[0] == NEW_LINE && offset+numBytes >= 1 
@@ -93,13 +84,13 @@ public class TextInputFormat extends DelimitedInputFormat<String> {
 			numBytes -= 1;
 		}
 		
-		return new String(bytes, offset, numBytes, this.charset);
+		return new String(bytes, offset, numBytes, this.charsetName);
 	}
 	
 	// --------------------------------------------------------------------------------------------
 	
 	@Override
 	public String toString() {
-		return "TextInputFormat (" + getFilePath() + ") - " + this.charsetName; // + (this.skipInvalidLines ? "(skipping invalid lines)" : "");
+		return "TextInputFormat (" + getFilePath() + ") - " + this.charsetName;
 	}
 }


[27/53] [abbrv] [FLINK-962] Initial import of documentation from website into source code (closes #34)

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/dataflow.svg
----------------------------------------------------------------------
diff --git a/docs/img/dataflow.svg b/docs/img/dataflow.svg
new file mode 100644
index 0000000..07d6e31
--- /dev/null
+++ b/docs/img/dataflow.svg
@@ -0,0 +1,979 @@
+<?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"
+   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+   width="800"
+   height="248"
+   id="svg10359"
+   version="1.1"
+   inkscape:version="0.48.3.1 r9886"
+   sodipodi:docname="dataflow.svg">
+  <defs
+     id="defs10361">
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart"
+       style="overflow:visible">
+      <path
+         id="path11612"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Lend"
+       style="overflow:visible">
+      <path
+         id="path11609"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(-0.8,0,0,-0.8,-10,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-7"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-4"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-5"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-47"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-3"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-8"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-32"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-0"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-4"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-1"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-8"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-7"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-0"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-3"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-37"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-82"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-05"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-5"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-03"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-00"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-6"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-84"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-1"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-6"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-42"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-07"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-74"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-32"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-89"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-80"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-77"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-69"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-79"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-03"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-41"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-02"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-45"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-01"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+  </defs>
+  <sodipodi:namedview
+     id="base"
+     pagecolor="#ffffff"
+     bordercolor="#666666"
+     borderopacity="1.0"
+     inkscape:pageopacity="0.0"
+     inkscape:pageshadow="2"
+     inkscape:zoom="1.4"
+     inkscape:cx="250.86321"
+     inkscape:cy="254.0586"
+     inkscape:document-units="px"
+     inkscape:current-layer="layer1"
+     showgrid="false"
+     fit-margin-top="0"
+     fit-margin-left="0"
+     fit-margin-right="0"
+     fit-margin-bottom="0"
+     inkscape:window-width="1916"
+     inkscape:window-height="1161"
+     inkscape:window-x="1400"
+     inkscape:window-y="18"
+     inkscape:window-maximized="0" />
+  <metadata
+     id="metadata10364">
+    <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>
+  <g
+     inkscape:label="Layer 1"
+     inkscape:groupmode="layer"
+     id="layer1"
+     transform="translate(-101.14286,-83.672699)">
+    <g
+       id="g12838"
+       transform="translate(94.357143,-2.3120884)">
+      <g
+         transform="translate(-10.618665,-105.0985)"
+         id="g11342">
+        <path
+           sodipodi:type="arc"
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           id="path10539"
+           sodipodi:cx="130.71428"
+           sodipodi:cy="229.1479"
+           sodipodi:rx="34.285713"
+           sodipodi:ry="33.214287"
+           d="m 164.99999,229.1479 c 0,18.34375 -15.35023,33.21429 -34.28571,33.21429 -18.93548,0 -34.285714,-14.87054 -34.285714,-33.21429 0,-18.34374 15.350234,-33.21428 34.285714,-33.21428 18.93548,0 34.28571,14.87054 34.28571,33.21428 z"
+           transform="translate(16.33296,-1.5382389)" />
+        <text
+           xml:space="preserve"
+           style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+           x="147.0238"
+           y="223.23857"
+           id="text10541"
+           sodipodi:linespacing="125%"><tspan
+             sodipodi:role="line"
+             id="tspan10543"
+             x="147.0238"
+             y="223.23857"
+             style="text-align:center;text-anchor:middle">Data</tspan><tspan
+             sodipodi:role="line"
+             x="147.0238"
+             y="243.23857"
+             id="tspan10545"
+             style="text-align:center;text-anchor:middle">Source</tspan></text>
+      </g>
+      <g
+         transform="translate(-126.14284,-10.618054)"
+         id="g11348">
+        <path
+           sodipodi:type="arc"
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           id="path10539-2"
+           sodipodi:cx="130.71428"
+           sodipodi:cy="229.1479"
+           sodipodi:rx="34.285713"
+           sodipodi:ry="33.214287"
+           d="m 164.99999,229.1479 c 0,18.34375 -15.35023,33.21429 -34.28571,33.21429 -18.93548,0 -34.285714,-14.87054 -34.285714,-33.21429 0,-18.34374 15.350234,-33.21428 34.285714,-33.21428 18.93548,0 34.28571,14.87054 34.28571,33.21428 z"
+           transform="translate(240.00001,78.928564)" />
+        <text
+           xml:space="preserve"
+           style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+           x="370.69086"
+           y="303.70538"
+           id="text10541-6"
+           sodipodi:linespacing="125%"><tspan
+             sodipodi:role="line"
+             id="tspan10543-6"
+             x="370.69086"
+             y="303.70538"
+             style="text-align:center;text-anchor:middle">Data</tspan><tspan
+             sodipodi:role="line"
+             x="370.69086"
+             y="323.70538"
+             id="tspan10545-9"
+             style="text-align:center;text-anchor:middle">Source</tspan></text>
+      </g>
+      <g
+         transform="translate(-69.857101,-112.71429)"
+         id="g11503">
+        <g
+           id="g11348-9"
+           transform="translate(160,98.571427)">
+          <path
+             transform="translate(240.00001,78.928564)"
+             d="m 164.99999,229.1479 c 0,18.34375 -15.35023,33.21429 -34.28571,33.21429 -18.93548,0 -34.285714,-14.87054 -34.285714,-33.21429 0,-18.34374 15.350234,-33.21428 34.285714,-33.21428 18.93548,0 34.28571,14.87054 34.28571,33.21428 z"
+             sodipodi:ry="33.214287"
+             sodipodi:rx="34.285713"
+             sodipodi:cy="229.1479"
+             sodipodi:cx="130.71428"
+             id="path10539-2-2"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+             sodipodi:type="arc" />
+          <text
+             sodipodi:linespacing="125%"
+             id="text10541-6-8"
+             y="303.70538"
+             x="370.69086"
+             style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+             xml:space="preserve"><tspan
+               style="text-align:center;text-anchor:middle"
+               y="303.70538"
+               x="370.69086"
+               id="tspan10543-6-9"
+               sodipodi:role="line">Data</tspan><tspan
+               style="text-align:center;text-anchor:middle"
+               id="tspan10545-9-8"
+               y="323.70538"
+               x="370.69086"
+               sodipodi:role="line">Source</tspan></text>
+        </g>
+      </g>
+      <g
+         transform="translate(-84.847654,-64.285714)"
+         id="g11488">
+        <path
+           transform="translate(198.70482,-42.351028)"
+           d="m 164.99999,229.1479 c 0,18.34375 -15.35023,33.21429 -34.28571,33.21429 -18.93548,0 -34.285714,-14.87054 -34.285714,-33.21429 0,-18.34374 15.350234,-33.21428 34.285714,-33.21428 18.93548,0 34.28571,14.87054 34.28571,33.21428 z"
+           sodipodi:ry="33.214287"
+           sodipodi:rx="34.285713"
+           sodipodi:cy="229.1479"
+           sodipodi:cx="130.71428"
+           id="path10539-4"
+           style="fill:#4ecdc4;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           sodipodi:type="arc" />
+        <text
+           xml:space="preserve"
+           style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+           x="313.57144"
+           y="190.93359"
+           id="text11423"
+           sodipodi:linespacing="125%"><tspan
+             sodipodi:role="line"
+             id="tspan11425"
+             x="313.57144"
+             y="190.93359">Map</tspan></text>
+      </g>
+      <g
+         transform="translate(22.684718,-61.12656)"
+         id="g11483">
+        <path
+           transform="translate(199.31529,86.152908)"
+           d="m 164.99999,229.1479 c 0,18.34375 -15.35023,33.21429 -34.28571,33.21429 -18.93548,0 -34.285714,-14.87054 -34.285714,-33.21429 0,-18.34374 15.350234,-33.21428 34.285714,-33.21428 18.93548,0 34.28571,14.87054 34.28571,33.21428 z"
+           sodipodi:ry="33.214287"
+           sodipodi:rx="34.285713"
+           sodipodi:cy="229.1479"
+           sodipodi:cx="130.71428"
+           id="path10539-08"
+           style="fill:#f3b33f;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           sodipodi:type="arc" />
+        <text
+           xml:space="preserve"
+           style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+           x="302.14285"
+           y="320.93362"
+           id="text11427"
+           sodipodi:linespacing="125%"><tspan
+             sodipodi:role="line"
+             id="tspan11429"
+             x="302.14285"
+             y="320.93362">Reduce</tspan></text>
+      </g>
+      <g
+         transform="translate(5.4420293,-37.982104)"
+         id="g11493">
+        <path
+           transform="translate(324.70088,19.724304)"
+           d="m 164.99999,229.1479 c 0,18.34375 -15.35023,33.21429 -34.28571,33.21429 -18.93548,0 -34.285714,-14.87054 -34.285714,-33.21429 0,-18.34374 15.350234,-33.21428 34.285714,-33.21428 18.93548,0 34.28571,14.87054 34.28571,33.21428 z"
+           sodipodi:ry="33.214287"
+           sodipodi:rx="34.285713"
+           sodipodi:cy="229.1479"
+           sodipodi:cx="130.71428"
+           id="path10539-7"
+           style="fill:#b4a7d6;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           sodipodi:type="arc" />
+        <text
+           xml:space="preserve"
+           style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+           x="433.57141"
+           y="254.50502"
+           id="text11431"
+           sodipodi:linespacing="125%"><tspan
+             sodipodi:role="line"
+             id="tspan11433"
+             x="433.57141"
+             y="254.50502">Match</tspan></text>
+      </g>
+      <g
+         transform="translate(-4.272862,3.5697637)"
+         id="g11498">
+        <path
+           transform="translate(442.55859,19.694183)"
+           d="m 164.99999,229.1479 c 0,18.34375 -15.35023,33.21429 -34.28571,33.21429 -18.93548,0 -34.285714,-14.87054 -34.285714,-33.21429 0,-18.34374 15.350234,-33.21428 34.285714,-33.21428 18.93548,0 34.28571,14.87054 34.28571,33.21428 z"
+           sodipodi:ry="33.214287"
+           sodipodi:rx="34.285713"
+           sodipodi:cy="229.1479"
+           sodipodi:cx="130.71428"
+           id="path10539-0"
+           style="fill:#87d69b;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           sodipodi:type="arc" />
+        <text
+           xml:space="preserve"
+           style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+           x="540.71429"
+           y="253.07646"
+           id="text11450"
+           sodipodi:linespacing="125%"><tspan
+             sodipodi:role="line"
+             id="tspan11452"
+             x="540.71429"
+             y="253.07646">CoGroup</tspan></text>
+      </g>
+      <g
+         transform="translate(24.571445,-118.61493)"
+         id="g11478">
+        <path
+           transform="translate(89.285721,100.35713)"
+           d="m 164.99999,229.1479 c 0,18.34375 -15.35023,33.21429 -34.28571,33.21429 -18.93548,0 -34.285714,-14.87054 -34.285714,-33.21429 0,-18.34374 15.350234,-33.21428 34.285714,-33.21428 18.93548,0 34.28571,14.87054 34.28571,33.21428 z"
+           sodipodi:ry="33.214287"
+           sodipodi:rx="34.285713"
+           sodipodi:cy="229.1479"
+           sodipodi:cx="130.71428"
+           id="path10539-4-2"
+           style="fill:#4ecdc4;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           sodipodi:type="arc" />
+        <text
+           xml:space="preserve"
+           style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+           x="204.15234"
+           y="333.64175"
+           id="text11423-8"
+           sodipodi:linespacing="125%"><tspan
+             sodipodi:role="line"
+             id="tspan11425-8"
+             x="204.15234"
+             y="333.64175">Map</tspan></text>
+      </g>
+      <g
+         transform="translate(146.42857,-154.23604)"
+         id="g11503-1">
+        <g
+           id="g11348-9-6"
+           transform="translate(160,98.571427)">
+          <path
+             transform="translate(240.00001,78.928564)"
+             d="m 164.99999,229.1479 c 0,18.34375 -15.35023,33.21429 -34.28571,33.21429 -18.93548,0 -34.285714,-14.87054 -34.285714,-33.21429 0,-18.34374 15.350234,-33.21428 34.285714,-33.21428 18.93548,0 34.28571,14.87054 34.28571,33.21428 z"
+             sodipodi:ry="33.214287"
+             sodipodi:rx="34.285713"
+             sodipodi:cy="229.1479"
+             sodipodi:cx="130.71428"
+             id="path10539-2-2-4"
+             style="fill:#e1f5c4;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+             sodipodi:type="arc" />
+          <text
+             sodipodi:linespacing="125%"
+             id="text10541-6-8-2"
+             y="303.70538"
+             x="370.69086"
+             style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+             xml:space="preserve"><tspan
+               style="text-align:center;text-anchor:middle"
+               y="303.70538"
+               x="370.69086"
+               id="tspan10543-6-9-6"
+               sodipodi:role="line">Data</tspan><tspan
+               style="text-align:center;text-anchor:middle"
+               id="tspan10545-9-8-8"
+               y="323.70538"
+               x="370.69086"
+               sodipodi:role="line">Sink</tspan></text>
+        </g>
+      </g>
+      <g
+         transform="translate(-10.618665,-16.719561)"
+         id="g11342-0">
+        <path
+           sodipodi:type="arc"
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           id="path10539-20"
+           sodipodi:cx="130.71428"
+           sodipodi:cy="229.1479"
+           sodipodi:rx="34.285713"
+           sodipodi:ry="33.214287"
+           d="m 164.99999,229.1479 c 0,18.34375 -15.35023,33.21429 -34.28571,33.21429 -18.93548,0 -34.285714,-14.87054 -34.285714,-33.21429 0,-18.34374 15.350234,-33.21428 34.285714,-33.21428 18.93548,0 34.28571,14.87054 34.28571,33.21428 z"
+           transform="translate(16.33296,-1.5382389)" />
+        <text
+           xml:space="preserve"
+           style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+           x="147.0238"
+           y="223.23857"
+           id="text10541-9"
+           sodipodi:linespacing="125%"><tspan
+             sodipodi:role="line"
+             id="tspan10543-0"
+             x="147.0238"
+             y="223.23857"
+             style="text-align:center;text-anchor:middle">Data</tspan><tspan
+             sodipodi:role="line"
+             x="147.0238"
+             y="243.23857"
+             id="tspan10545-2"
+             style="text-align:center;text-anchor:middle">Source</tspan></text>
+      </g>
+      <g
+         transform="translate(26.003898,-327.17553)"
+         id="g11595">
+        <path
+           sodipodi:type="arc"
+           style="fill:#93cbf8;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           id="path10539-7-3"
+           sodipodi:cx="130.71428"
+           sodipodi:cy="229.1479"
+           sodipodi:rx="34.285713"
+           sodipodi:ry="33.214287"
+           d="m 164.99999,229.1479 c 0,18.34375 -15.35023,33.21429 -34.28571,33.21429 -18.93548,0 -34.285714,-14.87054 -34.285714,-33.21429 0,-18.34374 15.350234,-33.21428 34.285714,-33.21428 18.93548,0 34.28571,14.87054 34.28571,33.21428 z"
+           transform="translate(195.99611,264.72826)" />
+        <text
+           xml:space="preserve"
+           style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+           x="305.71429"
+           y="499.50507"
+           id="text11591"
+           sodipodi:linespacing="125%"><tspan
+             sodipodi:role="line"
+             id="tspan11593"
+             x="305.71429"
+             y="499.50507">Union</tspan></text>
+      </g>
+      <path
+         inkscape:connector-curvature="0"
+         id="path11600"
+         d="m 207.29525,122.36218 -36.6524,0"
+         style="fill:none;stroke:#333333;stroke-width:1.99999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:url(#Arrow1Mstart)" />
+      <path
+         inkscape:connector-curvature="0"
+         id="path11600-3"
+         d="m 207.61192,210.8901 -36.6524,0"
+         style="fill:none;stroke:#333333;stroke-width:1.99999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:url(#Arrow1Mstart)" />
+      <path
+         sodipodi:nodetypes="cc"
+         inkscape:connector-curvature="0"
+         id="path11600-5"
+         d="M 316.50477,153.07647 276.10269,135.86026"
+         style="fill:none;stroke:#333333;stroke-width:1.99999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:url(#Arrow1Mstart)" />
+      <path
+         sodipodi:nodetypes="cc"
+         inkscape:connector-curvature="0"
+         id="path11600-30"
+         d="m 318.50402,182.481 -41.52749,16.69655"
+         style="fill:none;stroke:#333333;stroke-width:1.99999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:url(#Arrow1Mstart)" />
+      <path
+         inkscape:connector-curvature="0"
+         id="path11600-3-1"
+         d="m 640.16549,252.41185 -36.65241,0"
+         style="fill:none;stroke:#333333;stroke-width:2;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:url(#Arrow1Mstart)" />
+      <path
+         sodipodi:nodetypes="cc"
+         inkscape:connector-curvature="0"
+         id="path11600-5-4"
+         d="M 425.80414,196.37869 384.84328,179.32635"
+         style="fill:none;stroke:#333333;stroke-width:1.99999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:url(#Arrow1Mstart)" />
+      <path
+         sodipodi:nodetypes="cc"
+         inkscape:connector-curvature="0"
+         id="path11600-5-4-6"
+         d="M 535.14863,238.16024 493.24884,221.04335"
+         style="fill:none;stroke:#333333;stroke-width:1.99999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:url(#Arrow1Mstart)" />
+      <path
+         sodipodi:nodetypes="cc"
+         inkscape:connector-curvature="0"
+         id="path11600-30-7"
+         d="m 425.9661,225.06044 -41.52749,16.69655"
+         style="fill:none;stroke:#333333;stroke-width:1.99999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:url(#Arrow1Mstart)" />
+      <path
+         sodipodi:nodetypes="cc"
+         inkscape:connector-curvature="0"
+         id="path11600-30-7-0"
+         d="m 317.69053,267.33329 -41.52749,16.69655"
+         style="fill:none;stroke:#333333;stroke-width:1.99999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:url(#Arrow1Mstart)" />
+      <path
+         sodipodi:nodetypes="cc"
+         inkscape:connector-curvature="0"
+         id="path11600-30-7-7"
+         d="m 537.22878,272.70837 -41.52749,16.69655"
+         style="fill:none;stroke:#333333;stroke-width:1.99999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:url(#Arrow1Mstart)" />
+    </g>
+    <g
+       id="g12953"
+       style="fill:#cccccc;stroke-width:5.2870717;stroke-miterlimit:4;stroke-dasharray:none"
+       transform="matrix(0.41814798,0,0,0.40644285,558.62945,282.71148)">
+      <path
+         id="path12920-9"
+         d="m -278.8125,-54.03125 0,53.625 c -0.0646,0.44696162 -0.0937,0.92003723 -0.0937,1.375 0,10.49341 17.46089,19 39,19 21.49586,0 38.93008,-8.473146 39,-18.9375 l 0.0937,0 0,-55.0625 -78,0 z"
+         style="fill:#cccccc;fill-opacity:1;stroke:#333333;stroke-width:4.85137939;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+         inkscape:connector-curvature="0"
+         transform="translate(101.14286,83.672699)" />
+      <path
+         transform="matrix(1.0434603,0,0,1.3435029,44.746165,-114.27442)"
+         d="m -138.3909,105.56849 c 0,7.81048 -16.73364,14.14213 -37.37564,14.14213 -20.642,0 -37.37565,-6.33165 -37.37565,-14.14213 0,-7.810487 16.73365,-14.142137 37.37565,-14.142137 20.642,0 37.37564,6.33165 37.37564,14.142137 z"
+         sodipodi:ry="14.142136"
+         sodipodi:rx="37.375645"
+         sodipodi:cy="105.56849"
+         sodipodi:cx="-175.76654"
+         id="path12920"
+         style="fill:#cccccc;fill-opacity:1;stroke:#333333;stroke-width:4.09739685;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+         sodipodi:type="arc" />
+    </g>
+    <g
+       id="g13048"
+       transform="matrix(0.53768785,0,0,0.6237725,211.71852,24.496616)">
+      <g
+         id="g12998">
+        <path
+           style="fill:#cccccc;fill-opacity:1;stroke:#333333;stroke-width:3.45343852;stroke-miterlimit:4;stroke-opacity:1;stroke-dashoffset:0"
+           d="m -83.252987,150.6069 46.538697,0 0,46.65177 0,9.41403 -10.803621,0 -35.735076,0 z"
+           id="rect12981"
+           inkscape:connector-curvature="0"
+           sodipodi:nodetypes="ccccccc" />
+        <path
+           style="fill:none;stroke:#333333;stroke-width:3.45343876;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           d="m -150.90663,122.80518 6.55098,-10.69463 6.44621,4.69223"
+           id="path12984"
+           inkscape:connector-curvature="0"
+           transform="translate(101.14286,83.672699)"
+           sodipodi:nodetypes="ccc" />
+      </g>
+      <g
+         id="g12998-8"
+         transform="translate(-5.7638991,-4.842059)">
+        <path
+           style="fill:#cccccc;fill-opacity:1;stroke:#333333;stroke-width:3.45343852;stroke-miterlimit:4;stroke-opacity:1;stroke-dashoffset:0"
+           d="m -83.252987,150.6069 46.538697,0 0,46.65177 0,9.41403 -10.803621,0 -35.735076,0 z"
+           id="rect12981-0"
+           inkscape:connector-curvature="0"
+           sodipodi:nodetypes="ccccccc" />
+        <path
+           style="fill:none;stroke:#333333;stroke-width:3.45343876;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           d="m -150.90663,122.80518 6.55098,-10.69463 6.44621,4.69223"
+           id="path12984-5"
+           inkscape:connector-curvature="0"
+           transform="translate(101.14286,83.672699)"
+           sodipodi:nodetypes="ccc" />
+      </g>
+      <g
+         id="g12998-82"
+         transform="translate(-11.5278,-9.6840945)">
+        <path
+           style="fill:#cccccc;fill-opacity:1;stroke:#333333;stroke-width:3.45343852;stroke-miterlimit:4;stroke-opacity:1;stroke-dashoffset:0"
+           d="m -83.252987,150.6069 46.538697,0 0,46.65177 0,9.41403 -10.803621,0 -35.735076,0 z"
+           id="rect12981-4"
+           inkscape:connector-curvature="0"
+           sodipodi:nodetypes="ccccccc" />
+        <path
+           style="fill:none;stroke:#333333;stroke-width:3.45343876;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           d="m -150.90663,122.80518 6.55098,-10.69463 6.44621,4.69223"
+           id="path12984-0"
+           inkscape:connector-curvature="0"
+           transform="translate(101.14286,83.672699)"
+           sodipodi:nodetypes="ccc" />
+      </g>
+      <g
+         id="g12998-1"
+         transform="translate(-17.291695,-14.526155)">
+        <path
+           style="fill:#cccccc;fill-opacity:1;stroke:#333333;stroke-width:3.45343852;stroke-miterlimit:4;stroke-opacity:1;stroke-dashoffset:0"
+           d="m -83.252987,150.6069 46.538697,0 0,46.65177 0,9.41403 -10.803621,0 -35.735076,0 z"
+           id="rect12981-1"
+           inkscape:connector-curvature="0"
+           sodipodi:nodetypes="ccccccc" />
+        <path
+           style="fill:none;stroke:#333333;stroke-width:3.45343876;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           d="m -150.90663,122.80518 6.55098,-10.69463 6.44621,4.69223"
+           id="path12984-6"
+           inkscape:connector-curvature="0"
+           transform="translate(101.14286,83.672699)"
+           sodipodi:nodetypes="ccc" />
+      </g>
+    </g>
+    <g
+       id="g13048-1"
+       transform="matrix(0.53768785,0,0,0.6237725,211.80554,112.87556)">
+      <g
+         id="g12998-3">
+        <path
+           style="fill:#cccccc;fill-opacity:1;stroke:#333333;stroke-width:3.45343852;stroke-miterlimit:4;stroke-opacity:1;stroke-dashoffset:0"
+           d="m -83.252987,150.6069 46.538697,0 0,46.65177 0,9.41403 -10.803621,0 -35.735076,0 z"
+           id="rect12981-3"
+           inkscape:connector-curvature="0"
+           sodipodi:nodetypes="ccccccc" />
+        <path
+           style="fill:none;stroke:#333333;stroke-width:3.45343876;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           d="m -150.90663,122.80518 6.55098,-10.69463 6.44621,4.69223"
+           id="path12984-04"
+           inkscape:connector-curvature="0"
+           transform="translate(101.14286,83.672699)"
+           sodipodi:nodetypes="ccc" />
+      </g>
+      <g
+         id="g12998-8-8"
+         transform="translate(-5.7638991,-4.842059)">
+        <path
+           style="fill:#cccccc;fill-opacity:1;stroke:#333333;stroke-width:3.45343852;stroke-miterlimit:4;stroke-opacity:1;stroke-dashoffset:0"
+           d="m -83.252987,150.6069 46.538697,0 0,46.65177 0,9.41403 -10.803621,0 -35.735076,0 z"
+           id="rect12981-0-0"
+           inkscape:connector-curvature="0"
+           sodipodi:nodetypes="ccccccc" />
+        <path
+           style="fill:none;stroke:#333333;stroke-width:3.45343876;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           d="m -150.90663,122.80518 6.55098,-10.69463 6.44621,4.69223"
+           id="path12984-5-0"
+           inkscape:connector-curvature="0"
+           transform="translate(101.14286,83.672699)"
+           sodipodi:nodetypes="ccc" />
+      </g>
+      <g
+         id="g12998-82-0"
+         transform="translate(-11.5278,-9.6840945)">
+        <path
+           style="fill:#cccccc;fill-opacity:1;stroke:#333333;stroke-width:3.45343852;stroke-miterlimit:4;stroke-opacity:1;stroke-dashoffset:0"
+           d="m -83.252987,150.6069 46.538697,0 0,46.65177 0,9.41403 -10.803621,0 -35.735076,0 z"
+           id="rect12981-4-0"
+           inkscape:connector-curvature="0"
+           sodipodi:nodetypes="ccccccc" />
+        <path
+           style="fill:none;stroke:#333333;stroke-width:3.45343876;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           d="m -150.90663,122.80518 6.55098,-10.69463 6.44621,4.69223"
+           id="path12984-0-3"
+           inkscape:connector-curvature="0"
+           transform="translate(101.14286,83.672699)"
+           sodipodi:nodetypes="ccc" />
+      </g>
+      <g
+         id="g12998-1-0"
+         transform="translate(-17.291695,-14.526155)">
+        <path
+           style="fill:#cccccc;fill-opacity:1;stroke:#333333;stroke-width:3.45343852;stroke-miterlimit:4;stroke-opacity:1;stroke-dashoffset:0"
+           d="m -83.252987,150.6069 46.538697,0 0,46.65177 0,9.41403 -10.803621,0 -35.735076,0 z"
+           id="rect12981-1-7"
+           inkscape:connector-curvature="0"
+           sodipodi:nodetypes="ccccccc" />
+        <path
+           style="fill:none;stroke:#333333;stroke-width:3.45343876;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           d="m -150.90663,122.80518 6.55098,-10.69463 6.44621,4.69223"
+           id="path12984-6-0"
+           inkscape:connector-curvature="0"
+           transform="translate(101.14286,83.672699)"
+           sodipodi:nodetypes="ccc" />
+      </g>
+    </g>
+    <g
+       id="g13048-3"
+       transform="matrix(0.53768785,0,0,0.6237725,319.87156,199.44387)">
+      <g
+         id="g12998-87">
+        <path
+           style="fill:#cccccc;fill-opacity:1;stroke:#333333;stroke-width:3.45343852;stroke-miterlimit:4;stroke-opacity:1;stroke-dashoffset:0"
+           d="m -83.252987,150.6069 46.538697,0 0,46.65177 0,9.41403 -10.803621,0 -35.735076,0 z"
+           id="rect12981-48"
+           inkscape:connector-curvature="0"
+           sodipodi:nodetypes="ccccccc" />
+        <path
+           style="fill:none;stroke:#333333;stroke-width:3.45343876;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           d="m -150.90663,122.80518 6.55098,-10.69463 6.44621,4.69223"
+           id="path12984-2"
+           inkscape:connector-curvature="0"
+           transform="translate(101.14286,83.672699)"
+           sodipodi:nodetypes="ccc" />
+      </g>
+      <g
+         id="g12998-8-86"
+         transform="translate(-5.7638991,-4.842059)">
+        <path
+           style="fill:#cccccc;fill-opacity:1;stroke:#333333;stroke-width:3.45343852;stroke-miterlimit:4;stroke-opacity:1;stroke-dashoffset:0"
+           d="m -83.252987,150.6069 46.538697,0 0,46.65177 0,9.41403 -10.803621,0 -35.735076,0 z"
+           id="rect12981-0-2"
+           inkscape:connector-curvature="0"
+           sodipodi:nodetypes="ccccccc" />
+        <path
+           style="fill:none;stroke:#333333;stroke-width:3.45343876;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           d="m -150.90663,122.80518 6.55098,-10.69463 6.44621,4.69223"
+           id="path12984-5-3"
+           inkscape:connector-curvature="0"
+           transform="translate(101.14286,83.672699)"
+           sodipodi:nodetypes="ccc" />
+      </g>
+      <g
+         id="g12998-82-6"
+         transform="translate(-11.5278,-9.6840945)">
+        <path
+           style="fill:#cccccc;fill-opacity:1;stroke:#333333;stroke-width:3.45343852;stroke-miterlimit:4;stroke-opacity:1;stroke-dashoffset:0"
+           d="m -83.252987,150.6069 46.538697,0 0,46.65177 0,9.41403 -10.803621,0 -35.735076,0 z"
+           id="rect12981-4-7"
+           inkscape:connector-curvature="0"
+           sodipodi:nodetypes="ccccccc" />
+        <path
+           style="fill:none;stroke:#333333;stroke-width:3.45343876;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           d="m -150.90663,122.80518 6.55098,-10.69463 6.44621,4.69223"
+           id="path12984-0-7"
+           inkscape:connector-curvature="0"
+           transform="translate(101.14286,83.672699)"
+           sodipodi:nodetypes="ccc" />
+      </g>
+      <g
+         id="g12998-1-6"
+         transform="translate(-17.291695,-14.526155)">
+        <path
+           style="fill:#cccccc;fill-opacity:1;stroke:#333333;stroke-width:3.45343852;stroke-miterlimit:4;stroke-opacity:1;stroke-dashoffset:0"
+           d="m -83.252987,150.6069 46.538697,0 0,46.65177 0,9.41403 -10.803621,0 -35.735076,0 z"
+           id="rect12981-1-8"
+           inkscape:connector-curvature="0"
+           sodipodi:nodetypes="ccccccc" />
+        <path
+           style="fill:none;stroke:#333333;stroke-width:3.45343876;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           d="m -150.90663,122.80518 6.55098,-10.69463 6.44621,4.69223"
+           id="path12984-6-1"
+           inkscape:connector-curvature="0"
+           transform="translate(101.14286,83.672699)"
+           sodipodi:nodetypes="ccc" />
+      </g>
+    </g>
+    <g
+       id="g13190"
+       transform="translate(447.14285,-100.13051)">
+      <path
+         sodipodi:nodetypes="ccccccc"
+         inkscape:connector-curvature="0"
+         id="rect12981-1-8-1"
+         d="m 363.80979,348.47226 25.02329,0 0,29.10009 0,5.87221 -5.80897,0 -19.21432,0 z"
+         style="fill:#cccccc;fill-opacity:1;stroke:#333333;stroke-width:1.99999976;stroke-miterlimit:4;stroke-opacity:1;stroke-dashoffset:0" />
+      <path
+         sodipodi:nodetypes="ccc"
+         inkscape:connector-curvature="0"
+         id="path12984-6-1-4"
+         d="m 381.81653,383.32304 3.52238,-6.67102 3.46605,2.92689"
+         style="fill:none;stroke:#333333;stroke-width:1.99999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+    </g>
+  </g>
+</svg>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/datatypes.svg
----------------------------------------------------------------------
diff --git a/docs/img/datatypes.svg b/docs/img/datatypes.svg
new file mode 100644
index 0000000..a949677
--- /dev/null
+++ b/docs/img/datatypes.svg
@@ -0,0 +1,143 @@
+<?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"
+   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+   width="800"
+   height="85.809998"
+   id="svg5014"
+   version="1.1"
+   inkscape:version="0.48.3.1 r9886"
+   sodipodi:docname="datatypes.svg">
+  <defs
+     id="defs5016" />
+  <sodipodi:namedview
+     id="base"
+     pagecolor="#ffffff"
+     bordercolor="#666666"
+     borderopacity="1.0"
+     inkscape:pageopacity="0.0"
+     inkscape:pageshadow="2"
+     inkscape:zoom="0.7"
+     inkscape:cx="-61.192684"
+     inkscape:cy="64.192611"
+     inkscape:document-units="px"
+     inkscape:current-layer="layer1"
+     showgrid="false"
+     inkscape:snap-global="false"
+     fit-margin-top="0"
+     fit-margin-left="0"
+     fit-margin-right="0"
+     fit-margin-bottom="0"
+     showguides="true"
+     inkscape:guide-bbox="true"
+     inkscape:window-width="1916"
+     inkscape:window-height="747"
+     inkscape:window-x="1400"
+     inkscape:window-y="18"
+     inkscape:window-maximized="0" />
+  <metadata
+     id="metadata5019">
+    <rdf:RDF>
+      <cc:Work
+         rdf:about="">
+        <dc:format>image/svg+xml</dc:format>
+        <dc:type
+           rdf:resource="http://purl.org/dc/dcmitype/StillImage" />
+      </cc:Work>
+    </rdf:RDF>
+  </metadata>
+  <g
+     inkscape:label="Layer 1"
+     inkscape:groupmode="layer"
+     id="layer1"
+     transform="translate(-110.75,-300.24867)">
+    <g
+       id="g5646"
+       transform="translate(223.38524,-2.9974298)">
+      <rect
+         y="307.24353"
+         x="113.73705"
+         height="28.124744"
+         width="28.797562"
+         id="rect2985-5-8"
+         style="fill:#666666;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0" />
+      <rect
+         y="345.68823"
+         x="113.73705"
+         height="28.124744"
+         width="28.797562"
+         id="rect2985-5-4"
+         style="fill:#f3b33f;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0" />
+      <text
+         transform="scale(1.0155339,0.98470371)"
+         sodipodi:linespacing="125%"
+         id="text4753"
+         y="331.45355"
+         x="147.99387"
+         style="font-size:17.57638931px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+         xml:space="preserve"><tspan
+           style="font-size:14.64699078px"
+           id="tspan4757"
+           y="331.45355"
+           x="147.99387"
+           sodipodi:role="line"><tspan
+   id="tspan5642"
+   style="font-size:16px">Value</tspan>:</tspan></text>
+      <text
+         transform="scale(1.0155339,0.98470371)"
+         inkscape:transform-center-y="24.124547"
+         inkscape:transform-center-x="73.131734"
+         sodipodi:linespacing="125%"
+         id="text4827"
+         y="369.04004"
+         x="147.99387"
+         style="font-size:17.57638931px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+         xml:space="preserve"><tspan
+           style="font-size:14.64699078px"
+           y="369.04004"
+           x="147.99387"
+           id="tspan4829"
+           sodipodi:role="line"><tspan
+   id="tspan5644"
+   style="font-size:16px">Key</tspan>:</tspan></text>
+      <text
+         transform="scale(1.0155339,0.98470371)"
+         sodipodi:linespacing="125%"
+         id="text5588"
+         y="331.45355"
+         x="195.89445"
+         style="font-size:17.57638931px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Courier New;-inkscape-font-specification:Courier New"
+         xml:space="preserve"><tspan
+           style="font-size:16px"
+           y="331.45355"
+           x="195.89445"
+           id="tspan5590"
+           sodipodi:role="line">Serialize(), Deserialize()</tspan></text>
+      <text
+         transform="scale(1.0155339,0.98470371)"
+         sodipodi:linespacing="125%"
+         id="text5588-5"
+         y="369.04004"
+         x="195.89445"
+         style="font-size:17.57638931px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Courier New;-inkscape-font-specification:Courier New"
+         xml:space="preserve"><tspan
+           style="font-size:16px"
+           y="369.04004"
+           x="195.89445"
+           id="tspan5590-2"
+           sodipodi:role="line">Serialize(), Deserialize(), </tspan><tspan
+           id="tspan5613"
+           style="font-size:16px"
+           y="389.04004"
+           x="195.89445"
+           sodipodi:role="line">Hash(),      Compare(Key)</tspan></text>
+    </g>
+  </g>
+</svg>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/iterations_delta_iterate_operator.png
----------------------------------------------------------------------
diff --git a/docs/img/iterations_delta_iterate_operator.png b/docs/img/iterations_delta_iterate_operator.png
new file mode 100644
index 0000000..470485a
Binary files /dev/null and b/docs/img/iterations_delta_iterate_operator.png differ

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/iterations_delta_iterate_operator_example.png
----------------------------------------------------------------------
diff --git a/docs/img/iterations_delta_iterate_operator_example.png b/docs/img/iterations_delta_iterate_operator_example.png
new file mode 100644
index 0000000..15f2b54
Binary files /dev/null and b/docs/img/iterations_delta_iterate_operator_example.png differ

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/iterations_iterate_operator.png
----------------------------------------------------------------------
diff --git a/docs/img/iterations_iterate_operator.png b/docs/img/iterations_iterate_operator.png
new file mode 100644
index 0000000..aaf4158
Binary files /dev/null and b/docs/img/iterations_iterate_operator.png differ

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/iterations_iterate_operator_example.png
----------------------------------------------------------------------
diff --git a/docs/img/iterations_iterate_operator_example.png b/docs/img/iterations_iterate_operator_example.png
new file mode 100644
index 0000000..be4841c
Binary files /dev/null and b/docs/img/iterations_iterate_operator_example.png differ

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/iterations_supersteps.png
----------------------------------------------------------------------
diff --git a/docs/img/iterations_supersteps.png b/docs/img/iterations_supersteps.png
new file mode 100644
index 0000000..331dbc7
Binary files /dev/null and b/docs/img/iterations_supersteps.png differ

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/japi_example_overview.png
----------------------------------------------------------------------
diff --git a/docs/img/japi_example_overview.png b/docs/img/japi_example_overview.png
new file mode 100644
index 0000000..c035e3b
Binary files /dev/null and b/docs/img/japi_example_overview.png differ

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/join.svg
----------------------------------------------------------------------
diff --git a/docs/img/join.svg b/docs/img/join.svg
new file mode 100644
index 0000000..d5ef018
--- /dev/null
+++ b/docs/img/join.svg
@@ -0,0 +1,615 @@
+<?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"
+   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+   width="800"
+   height="195"
+   id="svg35289"
+   version="1.1"
+   inkscape:version="0.48.3.1 r9886"
+   sodipodi:docname="match.svg">
+  <defs
+     id="defs35291">
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35250"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35252"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35254"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35256"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35258"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35260"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend-0"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456-7"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35403"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35405"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35407"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35409"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35411"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35413"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend-9"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456-75"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35572"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35574"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35576"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35578"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35580"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35582"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35584"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35586"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35588"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35590"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend-5"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456-2"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+  </defs>
+  <sodipodi:namedview
+     id="base"
+     pagecolor="#ffffff"
+     bordercolor="#666666"
+     borderopacity="1.0"
+     inkscape:pageopacity="0.0"
+     inkscape:pageshadow="2"
+     inkscape:zoom="3.959798"
+     inkscape:cx="236.8846"
+     inkscape:cy="139.10312"
+     inkscape:document-units="px"
+     inkscape:current-layer="layer1"
+     showgrid="false"
+     fit-margin-top="0"
+     fit-margin-left="0"
+     fit-margin-right="0"
+     fit-margin-bottom="0"
+     inkscape:window-width="1916"
+     inkscape:window-height="374"
+     inkscape:window-x="1400"
+     inkscape:window-y="412"
+     inkscape:window-maximized="0" />
+  <metadata
+     id="metadata35294">
+    <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 />
+      </cc:Work>
+    </rdf:RDF>
+  </metadata>
+  <g
+     inkscape:label="Layer 1"
+     inkscape:groupmode="layer"
+     id="layer1"
+     transform="translate(-484.80217,-248.14373)">
+    <g
+       id="g35027"
+       transform="translate(1951.0537,-629.64646)">
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cc"
+         id="path4066-2-0-9-5-0-7"
+         d="m -938.8172,909.44427 0,23.67486"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.5;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <rect
+         y="932.67249"
+         x="-1223.4713"
+         height="136.06552"
+         width="57.281906"
+         id="rect2994-5-9-0-0"
+         style="opacity:0.95999995;fill:#f3f3f3;fill-opacity:1;stroke:#000000;stroke-width:1.5;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:25" />
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cc"
+         id="path4066-2-0-3-1-6"
+         d="m -1174.1847,1054.5519 32.1253,0"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.59638965;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <g
+         style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+         transform="matrix(0.40774172,0,0,0.41277371,-1301.4257,1623.5379)"
+         id="g3792-4-1-7-5">
+        <rect
+           style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65630722;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-0-0-3-0"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65630722;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-5-7-1-7"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <g
+         style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+         id="g3792-8-9-5-1-2"
+         transform="matrix(0.40774188,0,0,0.41277387,-1301.4258,1658.2654)">
+        <rect
+           style="fill:#f3de4a;fill-opacity:1;stroke:#000000;stroke-width:3.65630603;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-1-46-8-9-8"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65630603;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-0-9-7-0-0"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <g
+         style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+         id="g3792-0-2-0-5-2"
+         transform="matrix(0.40454759,0,0,0.40954016,-1300.5907,1722.3573)">
+        <rect
+           style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.6851759;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-4-2-4-6-3"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.6851759;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-4-4-8-7-5"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <g
+         style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+         id="g3792-8-4-7-0-7-4"
+         transform="matrix(0.40560715,0,0,0.41061279,-1300.8677,1689.4088)">
+        <rect
+           style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.67554951;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-1-4-75-4-4-3"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.67554951;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-0-7-4-2-0-0"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <rect
+         y="881.41754"
+         x="-1141.5946"
+         height="32.721252"
+         width="232.56281"
+         id="rect2994-5-9-4-2-2"
+         style="opacity:0.95999995;fill:#f3f3f3;fill-opacity:1;stroke:#000000;stroke-width:1.5;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:25" />
+      <g
+         style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+         transform="matrix(0.40774188,0,0,0.41277387,-1045.3506,1572.0715)"
+         id="g3792-4-1-1-6-7">
+        <rect
+           style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65630603;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-0-0-9-47-1"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65630603;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-5-7-6-4-8"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <g
+         style="opacity:0.95999995;stroke-width:1.53030467;stroke-miterlimit:4;stroke-dasharray:none"
+         id="g3792-0-2-0-2-8-8"
+         transform="matrix(0.39966779,0,0,0.40460014,-1161.2482,1558.5157)">
+        <rect
+           style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.73017049;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-4-2-4-5-5-7"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.73017049;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-4-4-8-4-8-4"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cccc"
+         id="path4066-2-0-78-4"
+         d="m -1174.4433,949.28372 112.1612,-0.58473 c 0,-7.97392 10.4003,-7.71183 10.4003,0.2622 l 28.6711,0.32253"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.5;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cccc"
+         id="path4066-2-0-7-4-2"
+         d="m -1174.5501,1018.6844 112.2161,-0.597 c 0,-8.0094 10.5429,-7.991 10.5429,0 l 28.5804,0.5971"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.5;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cc"
+         id="path4066-2-0-9-9-8"
+         d="m -1056.6686,907.98234 -0.3925,129.59686"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.51504707;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cc"
+         id="path4066-2-0-9-5-0-8-8"
+         d="m -938.7653,907.97323 0.1239,94.69557"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.51167059;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <g
+         transform="translate(426.5,-0.5)"
+         id="g34412">
+        <rect
+           style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49999988;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.49999962, 4.49999962;stroke-dashoffset:0"
+           id="rect2994-3-0-0-6-8"
+           width="103.5"
+           height="28.5"
+           x="-1562.7062"
+           y="1041.1628" />
+        <g
+           transform="translate(-391.67928,287.33529)"
+           id="g34070-6">
+          <g
+             style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+             id="g3792-0-1-17-4-6-9"
+             transform="matrix(0.40475762,0,0,0.40915735,-1251.7436,1436.4245)">
+            <rect
+               style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-4-8-4-1-8-9"
+               width="50"
+               height="50"
+               x="211.42857"
+               y="-1658.4716" />
+            <rect
+               style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-8-4-5-3-2-1-8"
+               width="50"
+               height="50"
+               x="261.42859"
+               y="-1658.4716" />
+          </g>
+          <g
+             style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+             id="g3792-0-2-0-2-3-7-5-9"
+             transform="matrix(0.40475762,0,0,0.40915735,-1198.4405,1436.4245)">
+            <rect
+               style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-4-2-4-5-3-9-5-9"
+               width="50"
+               height="50"
+               x="211.42857"
+               y="-1658.4716" />
+            <rect
+               style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-8-4-4-8-4-7-60-6-3"
+               width="50"
+               height="50"
+               x="261.42859"
+               y="-1658.4716" />
+          </g>
+        </g>
+      </g>
+      <g
+         transform="translate(428.43934,-1.3232233)"
+         id="g34422">
+        <rect
+           style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49999988;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.49999962, 4.49999962;stroke-dashoffset:0"
+           id="rect2994-3-0-0-5-0"
+           width="103.5"
+           height="28.5"
+           x="-1446.7938"
+           y="937.06146" />
+        <g
+           transform="translate(-391.67928,287.33529)"
+           id="g34078-7">
+          <g
+             style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+             transform="matrix(0.40795244,0,0,0.41238689,-1138.1239,1338.3958)"
+             id="g3792-1-9-8-4-0">
+            <rect
+               style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-7-3-3-6-1"
+               width="50"
+               height="50"
+               x="211.71547"
+               y="-1660.405" />
+            <rect
+               style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-8-8-6-83-2-4"
+               width="50"
+               height="50"
+               x="261.71548"
+               y="-1660.405" />
+          </g>
+          <g
+             style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+             transform="matrix(0.40795244,0,0,0.41238689,-1082.0231,1337.5985)"
+             id="g3792-4-1-1-4-3-1">
+            <rect
+               style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-0-0-9-1-0-7"
+               width="50"
+               height="50"
+               x="211.42857"
+               y="-1658.4716" />
+            <rect
+               style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-8-5-7-6-0-2-4"
+               width="50"
+               height="50"
+               x="261.42859"
+               y="-1658.4716" />
+          </g>
+        </g>
+      </g>
+      <g
+         transform="translate(428.60562,67.816701)"
+         id="g34422-3">
+        <rect
+           style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49999988;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.49999962, 4.49999962;stroke-dashoffset:0"
+           id="rect2994-3-0-0-5-0-0"
+           width="103.5"
+           height="28.5"
+           x="-1446.7938"
+           y="937.06146" />
+        <g
+           transform="translate(-391.67928,287.33529)"
+           id="g34078-7-9">
+          <g
+             style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+             transform="matrix(0.40795244,0,0,0.41238689,-1138.1239,1338.3958)"
+             id="g3792-1-9-8-4-0-9">
+            <rect
+               style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-7-3-3-6-1-1"
+               width="50"
+               height="50"
+               x="211.71547"
+               y="-1660.405" />
+            <rect
+               style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-8-8-6-83-2-4-4"
+               width="50"
+               height="50"
+               x="261.71548"
+               y="-1660.405" />
+          </g>
+          <g
+             style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+             transform="matrix(0.40795244,0,0,0.41238689,-1082.0231,1337.5985)"
+             id="g3792-4-1-1-4-3-1-8">
+            <rect
+               style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-0-0-9-1-0-7-9"
+               width="50"
+               height="50"
+               x="211.42857"
+               y="-1658.4716" />
+            <rect
+               style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-8-5-7-6-0-2-4-9"
+               width="50"
+               height="50"
+               x="261.42859"
+               y="-1658.4716" />
+          </g>
+        </g>
+      </g>
+    </g>
+    <path
+       inkscape:connector-curvature="0"
+       sodipodi:nodetypes="ccc"
+       id="path4066-2-0-9-4-1-2"
+       d="m 723.38563,271.07503 22.75265,-0.035 -0.12654,35.13393"
+       style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.5;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+    <text
+       xml:space="preserve"
+       style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+       x="655.62878"
+       y="275.41705"
+       id="text39591"
+       sodipodi:linespacing="125%"><tspan
+         sodipodi:role="line"
+         id="tspan39593"
+         x="655.62878"
+         y="275.41705">Key Field</tspan></text>
+  </g>
+</svg>


[33/53] [abbrv] git commit: Revert unintentional change in 1357ea3a40f747e4e51b5b52612323fe55747da7

Posted by rm...@apache.org.
Revert unintentional change in 1357ea3a40f747e4e51b5b52612323fe55747da7


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

Branch: refs/heads/travis_test
Commit: f3c54284b0dd20e360941d072b5cbd736acfec76
Parents: 1357ea3
Author: uce <u....@fu-berlin.de>
Authored: Tue Jun 24 14:41:11 2014 +0200
Committer: uce <u....@fu-berlin.de>
Committed: Tue Jun 24 14:41:11 2014 +0200

----------------------------------------------------------------------
 .../stratosphere/test/exampleJavaPrograms/WordCountITCase.java  | 5 -----
 1 file changed, 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f3c54284/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/WordCountITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/WordCountITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/WordCountITCase.java
index ec67390..c3ec112 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/WordCountITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/WordCountITCase.java
@@ -17,9 +17,6 @@ package eu.stratosphere.test.exampleJavaPrograms;
 import eu.stratosphere.example.java.wordcount.WordCount;
 import eu.stratosphere.test.testdata.WordCountData;
 import eu.stratosphere.test.util.JavaProgramTestBase;
-import eu.stratosphere.util.LogUtils;
-import org.apache.log4j.Level;
-
 
 public class WordCountITCase extends JavaProgramTestBase {
 
@@ -27,8 +24,6 @@ public class WordCountITCase extends JavaProgramTestBase {
 	protected String resultPath;
 
 	public WordCountITCase(){
-		LogUtils.initializeDefaultConsoleLogger(Level.DEBUG);
-
 		setDegreeOfParallelism(4);
 		setNumTaskTracker(2);
 		setTaskManagerNumSlots(2);


[40/53] [abbrv] git commit: [FLINK-926] Add shallow copy, deep equality, and hashCode to Tuple classes

Posted by rm...@apache.org.
[FLINK-926] Add shallow copy, deep equality, and hashCode to Tuple classes

This closes #17.


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

Branch: refs/heads/travis_test
Commit: 96e76a58417a820267dae1b134f28e988e903fcf
Parents: 28863ee
Author: Kostas Tzoumas <Ko...@gmail.com>
Authored: Fri Jun 13 17:40:24 2014 +0200
Committer: uce <u....@fu-berlin.de>
Committed: Wed Jun 25 11:55:13 2014 +0200

----------------------------------------------------------------------
 .../eu/stratosphere/api/java/tuple/Tuple1.java  |  28 ++++++
 .../eu/stratosphere/api/java/tuple/Tuple10.java |  55 ++++++++++
 .../eu/stratosphere/api/java/tuple/Tuple11.java |  58 +++++++++++
 .../eu/stratosphere/api/java/tuple/Tuple12.java |  61 +++++++++++
 .../eu/stratosphere/api/java/tuple/Tuple13.java |  64 ++++++++++++
 .../eu/stratosphere/api/java/tuple/Tuple14.java |  67 +++++++++++++
 .../eu/stratosphere/api/java/tuple/Tuple15.java |  70 +++++++++++++
 .../eu/stratosphere/api/java/tuple/Tuple16.java |  73 ++++++++++++++
 .../eu/stratosphere/api/java/tuple/Tuple17.java |  76 ++++++++++++++
 .../eu/stratosphere/api/java/tuple/Tuple18.java |  79 +++++++++++++++
 .../eu/stratosphere/api/java/tuple/Tuple19.java |  82 +++++++++++++++
 .../eu/stratosphere/api/java/tuple/Tuple2.java  |  31 ++++++
 .../eu/stratosphere/api/java/tuple/Tuple20.java |  85 ++++++++++++++++
 .../eu/stratosphere/api/java/tuple/Tuple21.java |  88 ++++++++++++++++
 .../eu/stratosphere/api/java/tuple/Tuple22.java |  91 +++++++++++++++++
 .../eu/stratosphere/api/java/tuple/Tuple23.java |  94 +++++++++++++++++
 .../eu/stratosphere/api/java/tuple/Tuple24.java |  97 ++++++++++++++++++
 .../eu/stratosphere/api/java/tuple/Tuple25.java | 100 +++++++++++++++++++
 .../eu/stratosphere/api/java/tuple/Tuple3.java  |  34 +++++++
 .../eu/stratosphere/api/java/tuple/Tuple4.java  |  37 +++++++
 .../eu/stratosphere/api/java/tuple/Tuple5.java  |  40 ++++++++
 .../eu/stratosphere/api/java/tuple/Tuple6.java  |  43 ++++++++
 .../eu/stratosphere/api/java/tuple/Tuple7.java  |  46 +++++++++
 .../eu/stratosphere/api/java/tuple/Tuple8.java  |  49 +++++++++
 .../eu/stratosphere/api/java/tuple/Tuple9.java  |  52 ++++++++++
 .../api/java/tuple/TupleGenerator.java          |  72 ++++++++++++-
 26 files changed, 1670 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/96e76a58/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple1.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple1.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple1.java
index 8752dca..66be617 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple1.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple1.java
@@ -104,4 +104,32 @@ public class Tuple1<T0> extends Tuple {
 		return "(" + StringUtils.arrayAwareToString(this.f0)
 			+ ")";
 	}
+
+	/**
+	 * Deep equality for tuples by calling equals() on the tuple members
+	 * @param o the object checked for equality
+	 * @return true if this is equal to o.
+	 */
+	@Override
+	public boolean equals(Object o) {
+		if(this == o) { return true; }
+		if (!(o instanceof Tuple1)) { return false; }
+		Tuple1 tuple = (Tuple1) o;
+		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
+		return true;
+	}
+
+	@Override
+	public int hashCode() {
+		int result = f0 != null ? f0.hashCode() : 0;
+		return result;
+	}
+	/**
+	* Shallow tuple copy.
+	* @returns A new Tuple with the same fields as this.
+	 */
+	public Tuple1<T0> copy(){ 
+		return new Tuple1<T0>(this.f0);
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/96e76a58/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple10.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple10.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple10.java
index 246350d..3a8ffd9 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple10.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple10.java
@@ -212,4 +212,59 @@ public class Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9> extends Tuple {
 			+ ", " + StringUtils.arrayAwareToString(this.f9)
 			+ ")";
 	}
+
+	/**
+	 * Deep equality for tuples by calling equals() on the tuple members
+	 * @param o the object checked for equality
+	 * @return true if this is equal to o.
+	 */
+	@Override
+	public boolean equals(Object o) {
+		if(this == o) { return true; }
+		if (!(o instanceof Tuple10)) { return false; }
+		Tuple10 tuple = (Tuple10) o;
+		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
+		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }
+		if (f2 != null ? !f2.equals(tuple.f2) : tuple.f2 != null) { return false; }
+		if (f3 != null ? !f3.equals(tuple.f3) : tuple.f3 != null) { return false; }
+		if (f4 != null ? !f4.equals(tuple.f4) : tuple.f4 != null) { return false; }
+		if (f5 != null ? !f5.equals(tuple.f5) : tuple.f5 != null) { return false; }
+		if (f6 != null ? !f6.equals(tuple.f6) : tuple.f6 != null) { return false; }
+		if (f7 != null ? !f7.equals(tuple.f7) : tuple.f7 != null) { return false; }
+		if (f8 != null ? !f8.equals(tuple.f8) : tuple.f8 != null) { return false; }
+		if (f9 != null ? !f9.equals(tuple.f9) : tuple.f9 != null) { return false; }
+		return true;
+	}
+
+	@Override
+	public int hashCode() {
+		int result = f0 != null ? f0.hashCode() : 0;
+		result = 31 * result + (f1 != null ? f1.hashCode() : 0);
+		result = 31 * result + (f2 != null ? f2.hashCode() : 0);
+		result = 31 * result + (f3 != null ? f3.hashCode() : 0);
+		result = 31 * result + (f4 != null ? f4.hashCode() : 0);
+		result = 31 * result + (f5 != null ? f5.hashCode() : 0);
+		result = 31 * result + (f6 != null ? f6.hashCode() : 0);
+		result = 31 * result + (f7 != null ? f7.hashCode() : 0);
+		result = 31 * result + (f8 != null ? f8.hashCode() : 0);
+		result = 31 * result + (f9 != null ? f9.hashCode() : 0);
+		return result;
+	}
+	/**
+	* Shallow tuple copy.
+	* @returns A new Tuple with the same fields as this.
+	 */
+	public Tuple10<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9> copy(){ 
+		return new Tuple10<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9>(this.f0,
+			this.f1,
+			this.f2,
+			this.f3,
+			this.f4,
+			this.f5,
+			this.f6,
+			this.f7,
+			this.f8,
+			this.f9);
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/96e76a58/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple11.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple11.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple11.java
index 8bfb6a8..9cafa32 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple11.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple11.java
@@ -224,4 +224,62 @@ public class Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10> extends Tuple
 			+ ", " + StringUtils.arrayAwareToString(this.f10)
 			+ ")";
 	}
+
+	/**
+	 * Deep equality for tuples by calling equals() on the tuple members
+	 * @param o the object checked for equality
+	 * @return true if this is equal to o.
+	 */
+	@Override
+	public boolean equals(Object o) {
+		if(this == o) { return true; }
+		if (!(o instanceof Tuple11)) { return false; }
+		Tuple11 tuple = (Tuple11) o;
+		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
+		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }
+		if (f2 != null ? !f2.equals(tuple.f2) : tuple.f2 != null) { return false; }
+		if (f3 != null ? !f3.equals(tuple.f3) : tuple.f3 != null) { return false; }
+		if (f4 != null ? !f4.equals(tuple.f4) : tuple.f4 != null) { return false; }
+		if (f5 != null ? !f5.equals(tuple.f5) : tuple.f5 != null) { return false; }
+		if (f6 != null ? !f6.equals(tuple.f6) : tuple.f6 != null) { return false; }
+		if (f7 != null ? !f7.equals(tuple.f7) : tuple.f7 != null) { return false; }
+		if (f8 != null ? !f8.equals(tuple.f8) : tuple.f8 != null) { return false; }
+		if (f9 != null ? !f9.equals(tuple.f9) : tuple.f9 != null) { return false; }
+		if (f10 != null ? !f10.equals(tuple.f10) : tuple.f10 != null) { return false; }
+		return true;
+	}
+
+	@Override
+	public int hashCode() {
+		int result = f0 != null ? f0.hashCode() : 0;
+		result = 31 * result + (f1 != null ? f1.hashCode() : 0);
+		result = 31 * result + (f2 != null ? f2.hashCode() : 0);
+		result = 31 * result + (f3 != null ? f3.hashCode() : 0);
+		result = 31 * result + (f4 != null ? f4.hashCode() : 0);
+		result = 31 * result + (f5 != null ? f5.hashCode() : 0);
+		result = 31 * result + (f6 != null ? f6.hashCode() : 0);
+		result = 31 * result + (f7 != null ? f7.hashCode() : 0);
+		result = 31 * result + (f8 != null ? f8.hashCode() : 0);
+		result = 31 * result + (f9 != null ? f9.hashCode() : 0);
+		result = 31 * result + (f10 != null ? f10.hashCode() : 0);
+		return result;
+	}
+	/**
+	* Shallow tuple copy.
+	* @returns A new Tuple with the same fields as this.
+	 */
+	public Tuple11<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9,T10> copy(){ 
+		return new Tuple11<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9,T10>(this.f0,
+			this.f1,
+			this.f2,
+			this.f3,
+			this.f4,
+			this.f5,
+			this.f6,
+			this.f7,
+			this.f8,
+			this.f9,
+			this.f10);
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/96e76a58/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple12.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple12.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple12.java
index 810af81..81204a9 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple12.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple12.java
@@ -236,4 +236,65 @@ public class Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11> extends T
 			+ ", " + StringUtils.arrayAwareToString(this.f11)
 			+ ")";
 	}
+
+	/**
+	 * Deep equality for tuples by calling equals() on the tuple members
+	 * @param o the object checked for equality
+	 * @return true if this is equal to o.
+	 */
+	@Override
+	public boolean equals(Object o) {
+		if(this == o) { return true; }
+		if (!(o instanceof Tuple12)) { return false; }
+		Tuple12 tuple = (Tuple12) o;
+		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
+		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }
+		if (f2 != null ? !f2.equals(tuple.f2) : tuple.f2 != null) { return false; }
+		if (f3 != null ? !f3.equals(tuple.f3) : tuple.f3 != null) { return false; }
+		if (f4 != null ? !f4.equals(tuple.f4) : tuple.f4 != null) { return false; }
+		if (f5 != null ? !f5.equals(tuple.f5) : tuple.f5 != null) { return false; }
+		if (f6 != null ? !f6.equals(tuple.f6) : tuple.f6 != null) { return false; }
+		if (f7 != null ? !f7.equals(tuple.f7) : tuple.f7 != null) { return false; }
+		if (f8 != null ? !f8.equals(tuple.f8) : tuple.f8 != null) { return false; }
+		if (f9 != null ? !f9.equals(tuple.f9) : tuple.f9 != null) { return false; }
+		if (f10 != null ? !f10.equals(tuple.f10) : tuple.f10 != null) { return false; }
+		if (f11 != null ? !f11.equals(tuple.f11) : tuple.f11 != null) { return false; }
+		return true;
+	}
+
+	@Override
+	public int hashCode() {
+		int result = f0 != null ? f0.hashCode() : 0;
+		result = 31 * result + (f1 != null ? f1.hashCode() : 0);
+		result = 31 * result + (f2 != null ? f2.hashCode() : 0);
+		result = 31 * result + (f3 != null ? f3.hashCode() : 0);
+		result = 31 * result + (f4 != null ? f4.hashCode() : 0);
+		result = 31 * result + (f5 != null ? f5.hashCode() : 0);
+		result = 31 * result + (f6 != null ? f6.hashCode() : 0);
+		result = 31 * result + (f7 != null ? f7.hashCode() : 0);
+		result = 31 * result + (f8 != null ? f8.hashCode() : 0);
+		result = 31 * result + (f9 != null ? f9.hashCode() : 0);
+		result = 31 * result + (f10 != null ? f10.hashCode() : 0);
+		result = 31 * result + (f11 != null ? f11.hashCode() : 0);
+		return result;
+	}
+	/**
+	* Shallow tuple copy.
+	* @returns A new Tuple with the same fields as this.
+	 */
+	public Tuple12<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9,T10,T11> copy(){ 
+		return new Tuple12<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9,T10,T11>(this.f0,
+			this.f1,
+			this.f2,
+			this.f3,
+			this.f4,
+			this.f5,
+			this.f6,
+			this.f7,
+			this.f8,
+			this.f9,
+			this.f10,
+			this.f11);
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/96e76a58/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple13.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple13.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple13.java
index c4f4655..5a4b02a 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple13.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple13.java
@@ -248,4 +248,68 @@ public class Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12> exte
 			+ ", " + StringUtils.arrayAwareToString(this.f12)
 			+ ")";
 	}
+
+	/**
+	 * Deep equality for tuples by calling equals() on the tuple members
+	 * @param o the object checked for equality
+	 * @return true if this is equal to o.
+	 */
+	@Override
+	public boolean equals(Object o) {
+		if(this == o) { return true; }
+		if (!(o instanceof Tuple13)) { return false; }
+		Tuple13 tuple = (Tuple13) o;
+		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
+		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }
+		if (f2 != null ? !f2.equals(tuple.f2) : tuple.f2 != null) { return false; }
+		if (f3 != null ? !f3.equals(tuple.f3) : tuple.f3 != null) { return false; }
+		if (f4 != null ? !f4.equals(tuple.f4) : tuple.f4 != null) { return false; }
+		if (f5 != null ? !f5.equals(tuple.f5) : tuple.f5 != null) { return false; }
+		if (f6 != null ? !f6.equals(tuple.f6) : tuple.f6 != null) { return false; }
+		if (f7 != null ? !f7.equals(tuple.f7) : tuple.f7 != null) { return false; }
+		if (f8 != null ? !f8.equals(tuple.f8) : tuple.f8 != null) { return false; }
+		if (f9 != null ? !f9.equals(tuple.f9) : tuple.f9 != null) { return false; }
+		if (f10 != null ? !f10.equals(tuple.f10) : tuple.f10 != null) { return false; }
+		if (f11 != null ? !f11.equals(tuple.f11) : tuple.f11 != null) { return false; }
+		if (f12 != null ? !f12.equals(tuple.f12) : tuple.f12 != null) { return false; }
+		return true;
+	}
+
+	@Override
+	public int hashCode() {
+		int result = f0 != null ? f0.hashCode() : 0;
+		result = 31 * result + (f1 != null ? f1.hashCode() : 0);
+		result = 31 * result + (f2 != null ? f2.hashCode() : 0);
+		result = 31 * result + (f3 != null ? f3.hashCode() : 0);
+		result = 31 * result + (f4 != null ? f4.hashCode() : 0);
+		result = 31 * result + (f5 != null ? f5.hashCode() : 0);
+		result = 31 * result + (f6 != null ? f6.hashCode() : 0);
+		result = 31 * result + (f7 != null ? f7.hashCode() : 0);
+		result = 31 * result + (f8 != null ? f8.hashCode() : 0);
+		result = 31 * result + (f9 != null ? f9.hashCode() : 0);
+		result = 31 * result + (f10 != null ? f10.hashCode() : 0);
+		result = 31 * result + (f11 != null ? f11.hashCode() : 0);
+		result = 31 * result + (f12 != null ? f12.hashCode() : 0);
+		return result;
+	}
+	/**
+	* Shallow tuple copy.
+	* @returns A new Tuple with the same fields as this.
+	 */
+	public Tuple13<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9,T10,T11,T12> copy(){ 
+		return new Tuple13<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9,T10,T11,T12>(this.f0,
+			this.f1,
+			this.f2,
+			this.f3,
+			this.f4,
+			this.f5,
+			this.f6,
+			this.f7,
+			this.f8,
+			this.f9,
+			this.f10,
+			this.f11,
+			this.f12);
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/96e76a58/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple14.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple14.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple14.java
index 737aba6..205e0b4 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple14.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple14.java
@@ -260,4 +260,71 @@ public class Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>
 			+ ", " + StringUtils.arrayAwareToString(this.f13)
 			+ ")";
 	}
+
+	/**
+	 * Deep equality for tuples by calling equals() on the tuple members
+	 * @param o the object checked for equality
+	 * @return true if this is equal to o.
+	 */
+	@Override
+	public boolean equals(Object o) {
+		if(this == o) { return true; }
+		if (!(o instanceof Tuple14)) { return false; }
+		Tuple14 tuple = (Tuple14) o;
+		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
+		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }
+		if (f2 != null ? !f2.equals(tuple.f2) : tuple.f2 != null) { return false; }
+		if (f3 != null ? !f3.equals(tuple.f3) : tuple.f3 != null) { return false; }
+		if (f4 != null ? !f4.equals(tuple.f4) : tuple.f4 != null) { return false; }
+		if (f5 != null ? !f5.equals(tuple.f5) : tuple.f5 != null) { return false; }
+		if (f6 != null ? !f6.equals(tuple.f6) : tuple.f6 != null) { return false; }
+		if (f7 != null ? !f7.equals(tuple.f7) : tuple.f7 != null) { return false; }
+		if (f8 != null ? !f8.equals(tuple.f8) : tuple.f8 != null) { return false; }
+		if (f9 != null ? !f9.equals(tuple.f9) : tuple.f9 != null) { return false; }
+		if (f10 != null ? !f10.equals(tuple.f10) : tuple.f10 != null) { return false; }
+		if (f11 != null ? !f11.equals(tuple.f11) : tuple.f11 != null) { return false; }
+		if (f12 != null ? !f12.equals(tuple.f12) : tuple.f12 != null) { return false; }
+		if (f13 != null ? !f13.equals(tuple.f13) : tuple.f13 != null) { return false; }
+		return true;
+	}
+
+	@Override
+	public int hashCode() {
+		int result = f0 != null ? f0.hashCode() : 0;
+		result = 31 * result + (f1 != null ? f1.hashCode() : 0);
+		result = 31 * result + (f2 != null ? f2.hashCode() : 0);
+		result = 31 * result + (f3 != null ? f3.hashCode() : 0);
+		result = 31 * result + (f4 != null ? f4.hashCode() : 0);
+		result = 31 * result + (f5 != null ? f5.hashCode() : 0);
+		result = 31 * result + (f6 != null ? f6.hashCode() : 0);
+		result = 31 * result + (f7 != null ? f7.hashCode() : 0);
+		result = 31 * result + (f8 != null ? f8.hashCode() : 0);
+		result = 31 * result + (f9 != null ? f9.hashCode() : 0);
+		result = 31 * result + (f10 != null ? f10.hashCode() : 0);
+		result = 31 * result + (f11 != null ? f11.hashCode() : 0);
+		result = 31 * result + (f12 != null ? f12.hashCode() : 0);
+		result = 31 * result + (f13 != null ? f13.hashCode() : 0);
+		return result;
+	}
+	/**
+	* Shallow tuple copy.
+	* @returns A new Tuple with the same fields as this.
+	 */
+	public Tuple14<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9,T10,T11,T12,T13> copy(){ 
+		return new Tuple14<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9,T10,T11,T12,T13>(this.f0,
+			this.f1,
+			this.f2,
+			this.f3,
+			this.f4,
+			this.f5,
+			this.f6,
+			this.f7,
+			this.f8,
+			this.f9,
+			this.f10,
+			this.f11,
+			this.f12,
+			this.f13);
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/96e76a58/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple15.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple15.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple15.java
index 9cecba7..6ffaa4e 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple15.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple15.java
@@ -272,4 +272,74 @@ public class Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13,
 			+ ", " + StringUtils.arrayAwareToString(this.f14)
 			+ ")";
 	}
+
+	/**
+	 * Deep equality for tuples by calling equals() on the tuple members
+	 * @param o the object checked for equality
+	 * @return true if this is equal to o.
+	 */
+	@Override
+	public boolean equals(Object o) {
+		if(this == o) { return true; }
+		if (!(o instanceof Tuple15)) { return false; }
+		Tuple15 tuple = (Tuple15) o;
+		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
+		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }
+		if (f2 != null ? !f2.equals(tuple.f2) : tuple.f2 != null) { return false; }
+		if (f3 != null ? !f3.equals(tuple.f3) : tuple.f3 != null) { return false; }
+		if (f4 != null ? !f4.equals(tuple.f4) : tuple.f4 != null) { return false; }
+		if (f5 != null ? !f5.equals(tuple.f5) : tuple.f5 != null) { return false; }
+		if (f6 != null ? !f6.equals(tuple.f6) : tuple.f6 != null) { return false; }
+		if (f7 != null ? !f7.equals(tuple.f7) : tuple.f7 != null) { return false; }
+		if (f8 != null ? !f8.equals(tuple.f8) : tuple.f8 != null) { return false; }
+		if (f9 != null ? !f9.equals(tuple.f9) : tuple.f9 != null) { return false; }
+		if (f10 != null ? !f10.equals(tuple.f10) : tuple.f10 != null) { return false; }
+		if (f11 != null ? !f11.equals(tuple.f11) : tuple.f11 != null) { return false; }
+		if (f12 != null ? !f12.equals(tuple.f12) : tuple.f12 != null) { return false; }
+		if (f13 != null ? !f13.equals(tuple.f13) : tuple.f13 != null) { return false; }
+		if (f14 != null ? !f14.equals(tuple.f14) : tuple.f14 != null) { return false; }
+		return true;
+	}
+
+	@Override
+	public int hashCode() {
+		int result = f0 != null ? f0.hashCode() : 0;
+		result = 31 * result + (f1 != null ? f1.hashCode() : 0);
+		result = 31 * result + (f2 != null ? f2.hashCode() : 0);
+		result = 31 * result + (f3 != null ? f3.hashCode() : 0);
+		result = 31 * result + (f4 != null ? f4.hashCode() : 0);
+		result = 31 * result + (f5 != null ? f5.hashCode() : 0);
+		result = 31 * result + (f6 != null ? f6.hashCode() : 0);
+		result = 31 * result + (f7 != null ? f7.hashCode() : 0);
+		result = 31 * result + (f8 != null ? f8.hashCode() : 0);
+		result = 31 * result + (f9 != null ? f9.hashCode() : 0);
+		result = 31 * result + (f10 != null ? f10.hashCode() : 0);
+		result = 31 * result + (f11 != null ? f11.hashCode() : 0);
+		result = 31 * result + (f12 != null ? f12.hashCode() : 0);
+		result = 31 * result + (f13 != null ? f13.hashCode() : 0);
+		result = 31 * result + (f14 != null ? f14.hashCode() : 0);
+		return result;
+	}
+	/**
+	* Shallow tuple copy.
+	* @returns A new Tuple with the same fields as this.
+	 */
+	public Tuple15<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9,T10,T11,T12,T13,T14> copy(){ 
+		return new Tuple15<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9,T10,T11,T12,T13,T14>(this.f0,
+			this.f1,
+			this.f2,
+			this.f3,
+			this.f4,
+			this.f5,
+			this.f6,
+			this.f7,
+			this.f8,
+			this.f9,
+			this.f10,
+			this.f11,
+			this.f12,
+			this.f13,
+			this.f14);
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/96e76a58/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple16.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple16.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple16.java
index ce55e36..ed4b67d 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple16.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple16.java
@@ -284,4 +284,77 @@ public class Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13,
 			+ ", " + StringUtils.arrayAwareToString(this.f15)
 			+ ")";
 	}
+
+	/**
+	 * Deep equality for tuples by calling equals() on the tuple members
+	 * @param o the object checked for equality
+	 * @return true if this is equal to o.
+	 */
+	@Override
+	public boolean equals(Object o) {
+		if(this == o) { return true; }
+		if (!(o instanceof Tuple16)) { return false; }
+		Tuple16 tuple = (Tuple16) o;
+		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
+		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }
+		if (f2 != null ? !f2.equals(tuple.f2) : tuple.f2 != null) { return false; }
+		if (f3 != null ? !f3.equals(tuple.f3) : tuple.f3 != null) { return false; }
+		if (f4 != null ? !f4.equals(tuple.f4) : tuple.f4 != null) { return false; }
+		if (f5 != null ? !f5.equals(tuple.f5) : tuple.f5 != null) { return false; }
+		if (f6 != null ? !f6.equals(tuple.f6) : tuple.f6 != null) { return false; }
+		if (f7 != null ? !f7.equals(tuple.f7) : tuple.f7 != null) { return false; }
+		if (f8 != null ? !f8.equals(tuple.f8) : tuple.f8 != null) { return false; }
+		if (f9 != null ? !f9.equals(tuple.f9) : tuple.f9 != null) { return false; }
+		if (f10 != null ? !f10.equals(tuple.f10) : tuple.f10 != null) { return false; }
+		if (f11 != null ? !f11.equals(tuple.f11) : tuple.f11 != null) { return false; }
+		if (f12 != null ? !f12.equals(tuple.f12) : tuple.f12 != null) { return false; }
+		if (f13 != null ? !f13.equals(tuple.f13) : tuple.f13 != null) { return false; }
+		if (f14 != null ? !f14.equals(tuple.f14) : tuple.f14 != null) { return false; }
+		if (f15 != null ? !f15.equals(tuple.f15) : tuple.f15 != null) { return false; }
+		return true;
+	}
+
+	@Override
+	public int hashCode() {
+		int result = f0 != null ? f0.hashCode() : 0;
+		result = 31 * result + (f1 != null ? f1.hashCode() : 0);
+		result = 31 * result + (f2 != null ? f2.hashCode() : 0);
+		result = 31 * result + (f3 != null ? f3.hashCode() : 0);
+		result = 31 * result + (f4 != null ? f4.hashCode() : 0);
+		result = 31 * result + (f5 != null ? f5.hashCode() : 0);
+		result = 31 * result + (f6 != null ? f6.hashCode() : 0);
+		result = 31 * result + (f7 != null ? f7.hashCode() : 0);
+		result = 31 * result + (f8 != null ? f8.hashCode() : 0);
+		result = 31 * result + (f9 != null ? f9.hashCode() : 0);
+		result = 31 * result + (f10 != null ? f10.hashCode() : 0);
+		result = 31 * result + (f11 != null ? f11.hashCode() : 0);
+		result = 31 * result + (f12 != null ? f12.hashCode() : 0);
+		result = 31 * result + (f13 != null ? f13.hashCode() : 0);
+		result = 31 * result + (f14 != null ? f14.hashCode() : 0);
+		result = 31 * result + (f15 != null ? f15.hashCode() : 0);
+		return result;
+	}
+	/**
+	* Shallow tuple copy.
+	* @returns A new Tuple with the same fields as this.
+	 */
+	public Tuple16<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9,T10,T11,T12,T13,T14,T15> copy(){ 
+		return new Tuple16<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9,T10,T11,T12,T13,T14,T15>(this.f0,
+			this.f1,
+			this.f2,
+			this.f3,
+			this.f4,
+			this.f5,
+			this.f6,
+			this.f7,
+			this.f8,
+			this.f9,
+			this.f10,
+			this.f11,
+			this.f12,
+			this.f13,
+			this.f14,
+			this.f15);
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/96e76a58/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple17.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple17.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple17.java
index 58a0c87..68760f5 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple17.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple17.java
@@ -296,4 +296,80 @@ public class Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13,
 			+ ", " + StringUtils.arrayAwareToString(this.f16)
 			+ ")";
 	}
+
+	/**
+	 * Deep equality for tuples by calling equals() on the tuple members
+	 * @param o the object checked for equality
+	 * @return true if this is equal to o.
+	 */
+	@Override
+	public boolean equals(Object o) {
+		if(this == o) { return true; }
+		if (!(o instanceof Tuple17)) { return false; }
+		Tuple17 tuple = (Tuple17) o;
+		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
+		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }
+		if (f2 != null ? !f2.equals(tuple.f2) : tuple.f2 != null) { return false; }
+		if (f3 != null ? !f3.equals(tuple.f3) : tuple.f3 != null) { return false; }
+		if (f4 != null ? !f4.equals(tuple.f4) : tuple.f4 != null) { return false; }
+		if (f5 != null ? !f5.equals(tuple.f5) : tuple.f5 != null) { return false; }
+		if (f6 != null ? !f6.equals(tuple.f6) : tuple.f6 != null) { return false; }
+		if (f7 != null ? !f7.equals(tuple.f7) : tuple.f7 != null) { return false; }
+		if (f8 != null ? !f8.equals(tuple.f8) : tuple.f8 != null) { return false; }
+		if (f9 != null ? !f9.equals(tuple.f9) : tuple.f9 != null) { return false; }
+		if (f10 != null ? !f10.equals(tuple.f10) : tuple.f10 != null) { return false; }
+		if (f11 != null ? !f11.equals(tuple.f11) : tuple.f11 != null) { return false; }
+		if (f12 != null ? !f12.equals(tuple.f12) : tuple.f12 != null) { return false; }
+		if (f13 != null ? !f13.equals(tuple.f13) : tuple.f13 != null) { return false; }
+		if (f14 != null ? !f14.equals(tuple.f14) : tuple.f14 != null) { return false; }
+		if (f15 != null ? !f15.equals(tuple.f15) : tuple.f15 != null) { return false; }
+		if (f16 != null ? !f16.equals(tuple.f16) : tuple.f16 != null) { return false; }
+		return true;
+	}
+
+	@Override
+	public int hashCode() {
+		int result = f0 != null ? f0.hashCode() : 0;
+		result = 31 * result + (f1 != null ? f1.hashCode() : 0);
+		result = 31 * result + (f2 != null ? f2.hashCode() : 0);
+		result = 31 * result + (f3 != null ? f3.hashCode() : 0);
+		result = 31 * result + (f4 != null ? f4.hashCode() : 0);
+		result = 31 * result + (f5 != null ? f5.hashCode() : 0);
+		result = 31 * result + (f6 != null ? f6.hashCode() : 0);
+		result = 31 * result + (f7 != null ? f7.hashCode() : 0);
+		result = 31 * result + (f8 != null ? f8.hashCode() : 0);
+		result = 31 * result + (f9 != null ? f9.hashCode() : 0);
+		result = 31 * result + (f10 != null ? f10.hashCode() : 0);
+		result = 31 * result + (f11 != null ? f11.hashCode() : 0);
+		result = 31 * result + (f12 != null ? f12.hashCode() : 0);
+		result = 31 * result + (f13 != null ? f13.hashCode() : 0);
+		result = 31 * result + (f14 != null ? f14.hashCode() : 0);
+		result = 31 * result + (f15 != null ? f15.hashCode() : 0);
+		result = 31 * result + (f16 != null ? f16.hashCode() : 0);
+		return result;
+	}
+	/**
+	* Shallow tuple copy.
+	* @returns A new Tuple with the same fields as this.
+	 */
+	public Tuple17<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9,T10,T11,T12,T13,T14,T15,T16> copy(){ 
+		return new Tuple17<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9,T10,T11,T12,T13,T14,T15,T16>(this.f0,
+			this.f1,
+			this.f2,
+			this.f3,
+			this.f4,
+			this.f5,
+			this.f6,
+			this.f7,
+			this.f8,
+			this.f9,
+			this.f10,
+			this.f11,
+			this.f12,
+			this.f13,
+			this.f14,
+			this.f15,
+			this.f16);
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/96e76a58/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple18.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple18.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple18.java
index 6f59431..33ac5d9 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple18.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple18.java
@@ -308,4 +308,83 @@ public class Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13,
 			+ ", " + StringUtils.arrayAwareToString(this.f17)
 			+ ")";
 	}
+
+	/**
+	 * Deep equality for tuples by calling equals() on the tuple members
+	 * @param o the object checked for equality
+	 * @return true if this is equal to o.
+	 */
+	@Override
+	public boolean equals(Object o) {
+		if(this == o) { return true; }
+		if (!(o instanceof Tuple18)) { return false; }
+		Tuple18 tuple = (Tuple18) o;
+		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
+		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }
+		if (f2 != null ? !f2.equals(tuple.f2) : tuple.f2 != null) { return false; }
+		if (f3 != null ? !f3.equals(tuple.f3) : tuple.f3 != null) { return false; }
+		if (f4 != null ? !f4.equals(tuple.f4) : tuple.f4 != null) { return false; }
+		if (f5 != null ? !f5.equals(tuple.f5) : tuple.f5 != null) { return false; }
+		if (f6 != null ? !f6.equals(tuple.f6) : tuple.f6 != null) { return false; }
+		if (f7 != null ? !f7.equals(tuple.f7) : tuple.f7 != null) { return false; }
+		if (f8 != null ? !f8.equals(tuple.f8) : tuple.f8 != null) { return false; }
+		if (f9 != null ? !f9.equals(tuple.f9) : tuple.f9 != null) { return false; }
+		if (f10 != null ? !f10.equals(tuple.f10) : tuple.f10 != null) { return false; }
+		if (f11 != null ? !f11.equals(tuple.f11) : tuple.f11 != null) { return false; }
+		if (f12 != null ? !f12.equals(tuple.f12) : tuple.f12 != null) { return false; }
+		if (f13 != null ? !f13.equals(tuple.f13) : tuple.f13 != null) { return false; }
+		if (f14 != null ? !f14.equals(tuple.f14) : tuple.f14 != null) { return false; }
+		if (f15 != null ? !f15.equals(tuple.f15) : tuple.f15 != null) { return false; }
+		if (f16 != null ? !f16.equals(tuple.f16) : tuple.f16 != null) { return false; }
+		if (f17 != null ? !f17.equals(tuple.f17) : tuple.f17 != null) { return false; }
+		return true;
+	}
+
+	@Override
+	public int hashCode() {
+		int result = f0 != null ? f0.hashCode() : 0;
+		result = 31 * result + (f1 != null ? f1.hashCode() : 0);
+		result = 31 * result + (f2 != null ? f2.hashCode() : 0);
+		result = 31 * result + (f3 != null ? f3.hashCode() : 0);
+		result = 31 * result + (f4 != null ? f4.hashCode() : 0);
+		result = 31 * result + (f5 != null ? f5.hashCode() : 0);
+		result = 31 * result + (f6 != null ? f6.hashCode() : 0);
+		result = 31 * result + (f7 != null ? f7.hashCode() : 0);
+		result = 31 * result + (f8 != null ? f8.hashCode() : 0);
+		result = 31 * result + (f9 != null ? f9.hashCode() : 0);
+		result = 31 * result + (f10 != null ? f10.hashCode() : 0);
+		result = 31 * result + (f11 != null ? f11.hashCode() : 0);
+		result = 31 * result + (f12 != null ? f12.hashCode() : 0);
+		result = 31 * result + (f13 != null ? f13.hashCode() : 0);
+		result = 31 * result + (f14 != null ? f14.hashCode() : 0);
+		result = 31 * result + (f15 != null ? f15.hashCode() : 0);
+		result = 31 * result + (f16 != null ? f16.hashCode() : 0);
+		result = 31 * result + (f17 != null ? f17.hashCode() : 0);
+		return result;
+	}
+	/**
+	* Shallow tuple copy.
+	* @returns A new Tuple with the same fields as this.
+	 */
+	public Tuple18<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9,T10,T11,T12,T13,T14,T15,T16,T17> copy(){ 
+		return new Tuple18<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9,T10,T11,T12,T13,T14,T15,T16,T17>(this.f0,
+			this.f1,
+			this.f2,
+			this.f3,
+			this.f4,
+			this.f5,
+			this.f6,
+			this.f7,
+			this.f8,
+			this.f9,
+			this.f10,
+			this.f11,
+			this.f12,
+			this.f13,
+			this.f14,
+			this.f15,
+			this.f16,
+			this.f17);
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/96e76a58/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple19.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple19.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple19.java
index a3ab04d..4419ca3 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple19.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple19.java
@@ -320,4 +320,86 @@ public class Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13,
 			+ ", " + StringUtils.arrayAwareToString(this.f18)
 			+ ")";
 	}
+
+	/**
+	 * Deep equality for tuples by calling equals() on the tuple members
+	 * @param o the object checked for equality
+	 * @return true if this is equal to o.
+	 */
+	@Override
+	public boolean equals(Object o) {
+		if(this == o) { return true; }
+		if (!(o instanceof Tuple19)) { return false; }
+		Tuple19 tuple = (Tuple19) o;
+		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
+		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }
+		if (f2 != null ? !f2.equals(tuple.f2) : tuple.f2 != null) { return false; }
+		if (f3 != null ? !f3.equals(tuple.f3) : tuple.f3 != null) { return false; }
+		if (f4 != null ? !f4.equals(tuple.f4) : tuple.f4 != null) { return false; }
+		if (f5 != null ? !f5.equals(tuple.f5) : tuple.f5 != null) { return false; }
+		if (f6 != null ? !f6.equals(tuple.f6) : tuple.f6 != null) { return false; }
+		if (f7 != null ? !f7.equals(tuple.f7) : tuple.f7 != null) { return false; }
+		if (f8 != null ? !f8.equals(tuple.f8) : tuple.f8 != null) { return false; }
+		if (f9 != null ? !f9.equals(tuple.f9) : tuple.f9 != null) { return false; }
+		if (f10 != null ? !f10.equals(tuple.f10) : tuple.f10 != null) { return false; }
+		if (f11 != null ? !f11.equals(tuple.f11) : tuple.f11 != null) { return false; }
+		if (f12 != null ? !f12.equals(tuple.f12) : tuple.f12 != null) { return false; }
+		if (f13 != null ? !f13.equals(tuple.f13) : tuple.f13 != null) { return false; }
+		if (f14 != null ? !f14.equals(tuple.f14) : tuple.f14 != null) { return false; }
+		if (f15 != null ? !f15.equals(tuple.f15) : tuple.f15 != null) { return false; }
+		if (f16 != null ? !f16.equals(tuple.f16) : tuple.f16 != null) { return false; }
+		if (f17 != null ? !f17.equals(tuple.f17) : tuple.f17 != null) { return false; }
+		if (f18 != null ? !f18.equals(tuple.f18) : tuple.f18 != null) { return false; }
+		return true;
+	}
+
+	@Override
+	public int hashCode() {
+		int result = f0 != null ? f0.hashCode() : 0;
+		result = 31 * result + (f1 != null ? f1.hashCode() : 0);
+		result = 31 * result + (f2 != null ? f2.hashCode() : 0);
+		result = 31 * result + (f3 != null ? f3.hashCode() : 0);
+		result = 31 * result + (f4 != null ? f4.hashCode() : 0);
+		result = 31 * result + (f5 != null ? f5.hashCode() : 0);
+		result = 31 * result + (f6 != null ? f6.hashCode() : 0);
+		result = 31 * result + (f7 != null ? f7.hashCode() : 0);
+		result = 31 * result + (f8 != null ? f8.hashCode() : 0);
+		result = 31 * result + (f9 != null ? f9.hashCode() : 0);
+		result = 31 * result + (f10 != null ? f10.hashCode() : 0);
+		result = 31 * result + (f11 != null ? f11.hashCode() : 0);
+		result = 31 * result + (f12 != null ? f12.hashCode() : 0);
+		result = 31 * result + (f13 != null ? f13.hashCode() : 0);
+		result = 31 * result + (f14 != null ? f14.hashCode() : 0);
+		result = 31 * result + (f15 != null ? f15.hashCode() : 0);
+		result = 31 * result + (f16 != null ? f16.hashCode() : 0);
+		result = 31 * result + (f17 != null ? f17.hashCode() : 0);
+		result = 31 * result + (f18 != null ? f18.hashCode() : 0);
+		return result;
+	}
+	/**
+	* Shallow tuple copy.
+	* @returns A new Tuple with the same fields as this.
+	 */
+	public Tuple19<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9,T10,T11,T12,T13,T14,T15,T16,T17,T18> copy(){ 
+		return new Tuple19<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9,T10,T11,T12,T13,T14,T15,T16,T17,T18>(this.f0,
+			this.f1,
+			this.f2,
+			this.f3,
+			this.f4,
+			this.f5,
+			this.f6,
+			this.f7,
+			this.f8,
+			this.f9,
+			this.f10,
+			this.f11,
+			this.f12,
+			this.f13,
+			this.f14,
+			this.f15,
+			this.f16,
+			this.f17,
+			this.f18);
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/96e76a58/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple2.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple2.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple2.java
index ec52f4e..d175511 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple2.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple2.java
@@ -116,4 +116,35 @@ public class Tuple2<T0, T1> extends Tuple {
 			+ ", " + StringUtils.arrayAwareToString(this.f1)
 			+ ")";
 	}
+
+	/**
+	 * Deep equality for tuples by calling equals() on the tuple members
+	 * @param o the object checked for equality
+	 * @return true if this is equal to o.
+	 */
+	@Override
+	public boolean equals(Object o) {
+		if(this == o) { return true; }
+		if (!(o instanceof Tuple2)) { return false; }
+		Tuple2 tuple = (Tuple2) o;
+		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
+		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }
+		return true;
+	}
+
+	@Override
+	public int hashCode() {
+		int result = f0 != null ? f0.hashCode() : 0;
+		result = 31 * result + (f1 != null ? f1.hashCode() : 0);
+		return result;
+	}
+	/**
+	* Shallow tuple copy.
+	* @returns A new Tuple with the same fields as this.
+	 */
+	public Tuple2<T0,T1> copy(){ 
+		return new Tuple2<T0,T1>(this.f0,
+			this.f1);
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/96e76a58/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple20.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple20.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple20.java
index 6b124df..ce6c03e 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple20.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple20.java
@@ -332,4 +332,89 @@ public class Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13,
 			+ ", " + StringUtils.arrayAwareToString(this.f19)
 			+ ")";
 	}
+
+	/**
+	 * Deep equality for tuples by calling equals() on the tuple members
+	 * @param o the object checked for equality
+	 * @return true if this is equal to o.
+	 */
+	@Override
+	public boolean equals(Object o) {
+		if(this == o) { return true; }
+		if (!(o instanceof Tuple20)) { return false; }
+		Tuple20 tuple = (Tuple20) o;
+		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
+		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }
+		if (f2 != null ? !f2.equals(tuple.f2) : tuple.f2 != null) { return false; }
+		if (f3 != null ? !f3.equals(tuple.f3) : tuple.f3 != null) { return false; }
+		if (f4 != null ? !f4.equals(tuple.f4) : tuple.f4 != null) { return false; }
+		if (f5 != null ? !f5.equals(tuple.f5) : tuple.f5 != null) { return false; }
+		if (f6 != null ? !f6.equals(tuple.f6) : tuple.f6 != null) { return false; }
+		if (f7 != null ? !f7.equals(tuple.f7) : tuple.f7 != null) { return false; }
+		if (f8 != null ? !f8.equals(tuple.f8) : tuple.f8 != null) { return false; }
+		if (f9 != null ? !f9.equals(tuple.f9) : tuple.f9 != null) { return false; }
+		if (f10 != null ? !f10.equals(tuple.f10) : tuple.f10 != null) { return false; }
+		if (f11 != null ? !f11.equals(tuple.f11) : tuple.f11 != null) { return false; }
+		if (f12 != null ? !f12.equals(tuple.f12) : tuple.f12 != null) { return false; }
+		if (f13 != null ? !f13.equals(tuple.f13) : tuple.f13 != null) { return false; }
+		if (f14 != null ? !f14.equals(tuple.f14) : tuple.f14 != null) { return false; }
+		if (f15 != null ? !f15.equals(tuple.f15) : tuple.f15 != null) { return false; }
+		if (f16 != null ? !f16.equals(tuple.f16) : tuple.f16 != null) { return false; }
+		if (f17 != null ? !f17.equals(tuple.f17) : tuple.f17 != null) { return false; }
+		if (f18 != null ? !f18.equals(tuple.f18) : tuple.f18 != null) { return false; }
+		if (f19 != null ? !f19.equals(tuple.f19) : tuple.f19 != null) { return false; }
+		return true;
+	}
+
+	@Override
+	public int hashCode() {
+		int result = f0 != null ? f0.hashCode() : 0;
+		result = 31 * result + (f1 != null ? f1.hashCode() : 0);
+		result = 31 * result + (f2 != null ? f2.hashCode() : 0);
+		result = 31 * result + (f3 != null ? f3.hashCode() : 0);
+		result = 31 * result + (f4 != null ? f4.hashCode() : 0);
+		result = 31 * result + (f5 != null ? f5.hashCode() : 0);
+		result = 31 * result + (f6 != null ? f6.hashCode() : 0);
+		result = 31 * result + (f7 != null ? f7.hashCode() : 0);
+		result = 31 * result + (f8 != null ? f8.hashCode() : 0);
+		result = 31 * result + (f9 != null ? f9.hashCode() : 0);
+		result = 31 * result + (f10 != null ? f10.hashCode() : 0);
+		result = 31 * result + (f11 != null ? f11.hashCode() : 0);
+		result = 31 * result + (f12 != null ? f12.hashCode() : 0);
+		result = 31 * result + (f13 != null ? f13.hashCode() : 0);
+		result = 31 * result + (f14 != null ? f14.hashCode() : 0);
+		result = 31 * result + (f15 != null ? f15.hashCode() : 0);
+		result = 31 * result + (f16 != null ? f16.hashCode() : 0);
+		result = 31 * result + (f17 != null ? f17.hashCode() : 0);
+		result = 31 * result + (f18 != null ? f18.hashCode() : 0);
+		result = 31 * result + (f19 != null ? f19.hashCode() : 0);
+		return result;
+	}
+	/**
+	* Shallow tuple copy.
+	* @returns A new Tuple with the same fields as this.
+	 */
+	public Tuple20<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9,T10,T11,T12,T13,T14,T15,T16,T17,T18,T19> copy(){ 
+		return new Tuple20<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9,T10,T11,T12,T13,T14,T15,T16,T17,T18,T19>(this.f0,
+			this.f1,
+			this.f2,
+			this.f3,
+			this.f4,
+			this.f5,
+			this.f6,
+			this.f7,
+			this.f8,
+			this.f9,
+			this.f10,
+			this.f11,
+			this.f12,
+			this.f13,
+			this.f14,
+			this.f15,
+			this.f16,
+			this.f17,
+			this.f18,
+			this.f19);
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/96e76a58/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple21.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple21.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple21.java
index 959fce7..7995ced 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple21.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple21.java
@@ -344,4 +344,92 @@ public class Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13,
 			+ ", " + StringUtils.arrayAwareToString(this.f20)
 			+ ")";
 	}
+
+	/**
+	 * Deep equality for tuples by calling equals() on the tuple members
+	 * @param o the object checked for equality
+	 * @return true if this is equal to o.
+	 */
+	@Override
+	public boolean equals(Object o) {
+		if(this == o) { return true; }
+		if (!(o instanceof Tuple21)) { return false; }
+		Tuple21 tuple = (Tuple21) o;
+		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
+		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }
+		if (f2 != null ? !f2.equals(tuple.f2) : tuple.f2 != null) { return false; }
+		if (f3 != null ? !f3.equals(tuple.f3) : tuple.f3 != null) { return false; }
+		if (f4 != null ? !f4.equals(tuple.f4) : tuple.f4 != null) { return false; }
+		if (f5 != null ? !f5.equals(tuple.f5) : tuple.f5 != null) { return false; }
+		if (f6 != null ? !f6.equals(tuple.f6) : tuple.f6 != null) { return false; }
+		if (f7 != null ? !f7.equals(tuple.f7) : tuple.f7 != null) { return false; }
+		if (f8 != null ? !f8.equals(tuple.f8) : tuple.f8 != null) { return false; }
+		if (f9 != null ? !f9.equals(tuple.f9) : tuple.f9 != null) { return false; }
+		if (f10 != null ? !f10.equals(tuple.f10) : tuple.f10 != null) { return false; }
+		if (f11 != null ? !f11.equals(tuple.f11) : tuple.f11 != null) { return false; }
+		if (f12 != null ? !f12.equals(tuple.f12) : tuple.f12 != null) { return false; }
+		if (f13 != null ? !f13.equals(tuple.f13) : tuple.f13 != null) { return false; }
+		if (f14 != null ? !f14.equals(tuple.f14) : tuple.f14 != null) { return false; }
+		if (f15 != null ? !f15.equals(tuple.f15) : tuple.f15 != null) { return false; }
+		if (f16 != null ? !f16.equals(tuple.f16) : tuple.f16 != null) { return false; }
+		if (f17 != null ? !f17.equals(tuple.f17) : tuple.f17 != null) { return false; }
+		if (f18 != null ? !f18.equals(tuple.f18) : tuple.f18 != null) { return false; }
+		if (f19 != null ? !f19.equals(tuple.f19) : tuple.f19 != null) { return false; }
+		if (f20 != null ? !f20.equals(tuple.f20) : tuple.f20 != null) { return false; }
+		return true;
+	}
+
+	@Override
+	public int hashCode() {
+		int result = f0 != null ? f0.hashCode() : 0;
+		result = 31 * result + (f1 != null ? f1.hashCode() : 0);
+		result = 31 * result + (f2 != null ? f2.hashCode() : 0);
+		result = 31 * result + (f3 != null ? f3.hashCode() : 0);
+		result = 31 * result + (f4 != null ? f4.hashCode() : 0);
+		result = 31 * result + (f5 != null ? f5.hashCode() : 0);
+		result = 31 * result + (f6 != null ? f6.hashCode() : 0);
+		result = 31 * result + (f7 != null ? f7.hashCode() : 0);
+		result = 31 * result + (f8 != null ? f8.hashCode() : 0);
+		result = 31 * result + (f9 != null ? f9.hashCode() : 0);
+		result = 31 * result + (f10 != null ? f10.hashCode() : 0);
+		result = 31 * result + (f11 != null ? f11.hashCode() : 0);
+		result = 31 * result + (f12 != null ? f12.hashCode() : 0);
+		result = 31 * result + (f13 != null ? f13.hashCode() : 0);
+		result = 31 * result + (f14 != null ? f14.hashCode() : 0);
+		result = 31 * result + (f15 != null ? f15.hashCode() : 0);
+		result = 31 * result + (f16 != null ? f16.hashCode() : 0);
+		result = 31 * result + (f17 != null ? f17.hashCode() : 0);
+		result = 31 * result + (f18 != null ? f18.hashCode() : 0);
+		result = 31 * result + (f19 != null ? f19.hashCode() : 0);
+		result = 31 * result + (f20 != null ? f20.hashCode() : 0);
+		return result;
+	}
+	/**
+	* Shallow tuple copy.
+	* @returns A new Tuple with the same fields as this.
+	 */
+	public Tuple21<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9,T10,T11,T12,T13,T14,T15,T16,T17,T18,T19,T20> copy(){ 
+		return new Tuple21<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9,T10,T11,T12,T13,T14,T15,T16,T17,T18,T19,T20>(this.f0,
+			this.f1,
+			this.f2,
+			this.f3,
+			this.f4,
+			this.f5,
+			this.f6,
+			this.f7,
+			this.f8,
+			this.f9,
+			this.f10,
+			this.f11,
+			this.f12,
+			this.f13,
+			this.f14,
+			this.f15,
+			this.f16,
+			this.f17,
+			this.f18,
+			this.f19,
+			this.f20);
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/96e76a58/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple22.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple22.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple22.java
index 5f26083..060f158 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple22.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple22.java
@@ -356,4 +356,95 @@ public class Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13,
 			+ ", " + StringUtils.arrayAwareToString(this.f21)
 			+ ")";
 	}
+
+	/**
+	 * Deep equality for tuples by calling equals() on the tuple members
+	 * @param o the object checked for equality
+	 * @return true if this is equal to o.
+	 */
+	@Override
+	public boolean equals(Object o) {
+		if(this == o) { return true; }
+		if (!(o instanceof Tuple22)) { return false; }
+		Tuple22 tuple = (Tuple22) o;
+		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
+		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }
+		if (f2 != null ? !f2.equals(tuple.f2) : tuple.f2 != null) { return false; }
+		if (f3 != null ? !f3.equals(tuple.f3) : tuple.f3 != null) { return false; }
+		if (f4 != null ? !f4.equals(tuple.f4) : tuple.f4 != null) { return false; }
+		if (f5 != null ? !f5.equals(tuple.f5) : tuple.f5 != null) { return false; }
+		if (f6 != null ? !f6.equals(tuple.f6) : tuple.f6 != null) { return false; }
+		if (f7 != null ? !f7.equals(tuple.f7) : tuple.f7 != null) { return false; }
+		if (f8 != null ? !f8.equals(tuple.f8) : tuple.f8 != null) { return false; }
+		if (f9 != null ? !f9.equals(tuple.f9) : tuple.f9 != null) { return false; }
+		if (f10 != null ? !f10.equals(tuple.f10) : tuple.f10 != null) { return false; }
+		if (f11 != null ? !f11.equals(tuple.f11) : tuple.f11 != null) { return false; }
+		if (f12 != null ? !f12.equals(tuple.f12) : tuple.f12 != null) { return false; }
+		if (f13 != null ? !f13.equals(tuple.f13) : tuple.f13 != null) { return false; }
+		if (f14 != null ? !f14.equals(tuple.f14) : tuple.f14 != null) { return false; }
+		if (f15 != null ? !f15.equals(tuple.f15) : tuple.f15 != null) { return false; }
+		if (f16 != null ? !f16.equals(tuple.f16) : tuple.f16 != null) { return false; }
+		if (f17 != null ? !f17.equals(tuple.f17) : tuple.f17 != null) { return false; }
+		if (f18 != null ? !f18.equals(tuple.f18) : tuple.f18 != null) { return false; }
+		if (f19 != null ? !f19.equals(tuple.f19) : tuple.f19 != null) { return false; }
+		if (f20 != null ? !f20.equals(tuple.f20) : tuple.f20 != null) { return false; }
+		if (f21 != null ? !f21.equals(tuple.f21) : tuple.f21 != null) { return false; }
+		return true;
+	}
+
+	@Override
+	public int hashCode() {
+		int result = f0 != null ? f0.hashCode() : 0;
+		result = 31 * result + (f1 != null ? f1.hashCode() : 0);
+		result = 31 * result + (f2 != null ? f2.hashCode() : 0);
+		result = 31 * result + (f3 != null ? f3.hashCode() : 0);
+		result = 31 * result + (f4 != null ? f4.hashCode() : 0);
+		result = 31 * result + (f5 != null ? f5.hashCode() : 0);
+		result = 31 * result + (f6 != null ? f6.hashCode() : 0);
+		result = 31 * result + (f7 != null ? f7.hashCode() : 0);
+		result = 31 * result + (f8 != null ? f8.hashCode() : 0);
+		result = 31 * result + (f9 != null ? f9.hashCode() : 0);
+		result = 31 * result + (f10 != null ? f10.hashCode() : 0);
+		result = 31 * result + (f11 != null ? f11.hashCode() : 0);
+		result = 31 * result + (f12 != null ? f12.hashCode() : 0);
+		result = 31 * result + (f13 != null ? f13.hashCode() : 0);
+		result = 31 * result + (f14 != null ? f14.hashCode() : 0);
+		result = 31 * result + (f15 != null ? f15.hashCode() : 0);
+		result = 31 * result + (f16 != null ? f16.hashCode() : 0);
+		result = 31 * result + (f17 != null ? f17.hashCode() : 0);
+		result = 31 * result + (f18 != null ? f18.hashCode() : 0);
+		result = 31 * result + (f19 != null ? f19.hashCode() : 0);
+		result = 31 * result + (f20 != null ? f20.hashCode() : 0);
+		result = 31 * result + (f21 != null ? f21.hashCode() : 0);
+		return result;
+	}
+	/**
+	* Shallow tuple copy.
+	* @returns A new Tuple with the same fields as this.
+	 */
+	public Tuple22<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9,T10,T11,T12,T13,T14,T15,T16,T17,T18,T19,T20,T21> copy(){ 
+		return new Tuple22<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9,T10,T11,T12,T13,T14,T15,T16,T17,T18,T19,T20,T21>(this.f0,
+			this.f1,
+			this.f2,
+			this.f3,
+			this.f4,
+			this.f5,
+			this.f6,
+			this.f7,
+			this.f8,
+			this.f9,
+			this.f10,
+			this.f11,
+			this.f12,
+			this.f13,
+			this.f14,
+			this.f15,
+			this.f16,
+			this.f17,
+			this.f18,
+			this.f19,
+			this.f20,
+			this.f21);
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/96e76a58/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple23.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple23.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple23.java
index 2e4f145..1910024 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple23.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple23.java
@@ -368,4 +368,98 @@ public class Tuple23<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13,
 			+ ", " + StringUtils.arrayAwareToString(this.f22)
 			+ ")";
 	}
+
+	/**
+	 * Deep equality for tuples by calling equals() on the tuple members
+	 * @param o the object checked for equality
+	 * @return true if this is equal to o.
+	 */
+	@Override
+	public boolean equals(Object o) {
+		if(this == o) { return true; }
+		if (!(o instanceof Tuple23)) { return false; }
+		Tuple23 tuple = (Tuple23) o;
+		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
+		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }
+		if (f2 != null ? !f2.equals(tuple.f2) : tuple.f2 != null) { return false; }
+		if (f3 != null ? !f3.equals(tuple.f3) : tuple.f3 != null) { return false; }
+		if (f4 != null ? !f4.equals(tuple.f4) : tuple.f4 != null) { return false; }
+		if (f5 != null ? !f5.equals(tuple.f5) : tuple.f5 != null) { return false; }
+		if (f6 != null ? !f6.equals(tuple.f6) : tuple.f6 != null) { return false; }
+		if (f7 != null ? !f7.equals(tuple.f7) : tuple.f7 != null) { return false; }
+		if (f8 != null ? !f8.equals(tuple.f8) : tuple.f8 != null) { return false; }
+		if (f9 != null ? !f9.equals(tuple.f9) : tuple.f9 != null) { return false; }
+		if (f10 != null ? !f10.equals(tuple.f10) : tuple.f10 != null) { return false; }
+		if (f11 != null ? !f11.equals(tuple.f11) : tuple.f11 != null) { return false; }
+		if (f12 != null ? !f12.equals(tuple.f12) : tuple.f12 != null) { return false; }
+		if (f13 != null ? !f13.equals(tuple.f13) : tuple.f13 != null) { return false; }
+		if (f14 != null ? !f14.equals(tuple.f14) : tuple.f14 != null) { return false; }
+		if (f15 != null ? !f15.equals(tuple.f15) : tuple.f15 != null) { return false; }
+		if (f16 != null ? !f16.equals(tuple.f16) : tuple.f16 != null) { return false; }
+		if (f17 != null ? !f17.equals(tuple.f17) : tuple.f17 != null) { return false; }
+		if (f18 != null ? !f18.equals(tuple.f18) : tuple.f18 != null) { return false; }
+		if (f19 != null ? !f19.equals(tuple.f19) : tuple.f19 != null) { return false; }
+		if (f20 != null ? !f20.equals(tuple.f20) : tuple.f20 != null) { return false; }
+		if (f21 != null ? !f21.equals(tuple.f21) : tuple.f21 != null) { return false; }
+		if (f22 != null ? !f22.equals(tuple.f22) : tuple.f22 != null) { return false; }
+		return true;
+	}
+
+	@Override
+	public int hashCode() {
+		int result = f0 != null ? f0.hashCode() : 0;
+		result = 31 * result + (f1 != null ? f1.hashCode() : 0);
+		result = 31 * result + (f2 != null ? f2.hashCode() : 0);
+		result = 31 * result + (f3 != null ? f3.hashCode() : 0);
+		result = 31 * result + (f4 != null ? f4.hashCode() : 0);
+		result = 31 * result + (f5 != null ? f5.hashCode() : 0);
+		result = 31 * result + (f6 != null ? f6.hashCode() : 0);
+		result = 31 * result + (f7 != null ? f7.hashCode() : 0);
+		result = 31 * result + (f8 != null ? f8.hashCode() : 0);
+		result = 31 * result + (f9 != null ? f9.hashCode() : 0);
+		result = 31 * result + (f10 != null ? f10.hashCode() : 0);
+		result = 31 * result + (f11 != null ? f11.hashCode() : 0);
+		result = 31 * result + (f12 != null ? f12.hashCode() : 0);
+		result = 31 * result + (f13 != null ? f13.hashCode() : 0);
+		result = 31 * result + (f14 != null ? f14.hashCode() : 0);
+		result = 31 * result + (f15 != null ? f15.hashCode() : 0);
+		result = 31 * result + (f16 != null ? f16.hashCode() : 0);
+		result = 31 * result + (f17 != null ? f17.hashCode() : 0);
+		result = 31 * result + (f18 != null ? f18.hashCode() : 0);
+		result = 31 * result + (f19 != null ? f19.hashCode() : 0);
+		result = 31 * result + (f20 != null ? f20.hashCode() : 0);
+		result = 31 * result + (f21 != null ? f21.hashCode() : 0);
+		result = 31 * result + (f22 != null ? f22.hashCode() : 0);
+		return result;
+	}
+	/**
+	* Shallow tuple copy.
+	* @returns A new Tuple with the same fields as this.
+	 */
+	public Tuple23<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9,T10,T11,T12,T13,T14,T15,T16,T17,T18,T19,T20,T21,T22> copy(){ 
+		return new Tuple23<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9,T10,T11,T12,T13,T14,T15,T16,T17,T18,T19,T20,T21,T22>(this.f0,
+			this.f1,
+			this.f2,
+			this.f3,
+			this.f4,
+			this.f5,
+			this.f6,
+			this.f7,
+			this.f8,
+			this.f9,
+			this.f10,
+			this.f11,
+			this.f12,
+			this.f13,
+			this.f14,
+			this.f15,
+			this.f16,
+			this.f17,
+			this.f18,
+			this.f19,
+			this.f20,
+			this.f21,
+			this.f22);
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/96e76a58/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple24.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple24.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple24.java
index 8d09321..8ed9438 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple24.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple24.java
@@ -380,4 +380,101 @@ public class Tuple24<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13,
 			+ ", " + StringUtils.arrayAwareToString(this.f23)
 			+ ")";
 	}
+
+	/**
+	 * Deep equality for tuples by calling equals() on the tuple members
+	 * @param o the object checked for equality
+	 * @return true if this is equal to o.
+	 */
+	@Override
+	public boolean equals(Object o) {
+		if(this == o) { return true; }
+		if (!(o instanceof Tuple24)) { return false; }
+		Tuple24 tuple = (Tuple24) o;
+		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
+		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }
+		if (f2 != null ? !f2.equals(tuple.f2) : tuple.f2 != null) { return false; }
+		if (f3 != null ? !f3.equals(tuple.f3) : tuple.f3 != null) { return false; }
+		if (f4 != null ? !f4.equals(tuple.f4) : tuple.f4 != null) { return false; }
+		if (f5 != null ? !f5.equals(tuple.f5) : tuple.f5 != null) { return false; }
+		if (f6 != null ? !f6.equals(tuple.f6) : tuple.f6 != null) { return false; }
+		if (f7 != null ? !f7.equals(tuple.f7) : tuple.f7 != null) { return false; }
+		if (f8 != null ? !f8.equals(tuple.f8) : tuple.f8 != null) { return false; }
+		if (f9 != null ? !f9.equals(tuple.f9) : tuple.f9 != null) { return false; }
+		if (f10 != null ? !f10.equals(tuple.f10) : tuple.f10 != null) { return false; }
+		if (f11 != null ? !f11.equals(tuple.f11) : tuple.f11 != null) { return false; }
+		if (f12 != null ? !f12.equals(tuple.f12) : tuple.f12 != null) { return false; }
+		if (f13 != null ? !f13.equals(tuple.f13) : tuple.f13 != null) { return false; }
+		if (f14 != null ? !f14.equals(tuple.f14) : tuple.f14 != null) { return false; }
+		if (f15 != null ? !f15.equals(tuple.f15) : tuple.f15 != null) { return false; }
+		if (f16 != null ? !f16.equals(tuple.f16) : tuple.f16 != null) { return false; }
+		if (f17 != null ? !f17.equals(tuple.f17) : tuple.f17 != null) { return false; }
+		if (f18 != null ? !f18.equals(tuple.f18) : tuple.f18 != null) { return false; }
+		if (f19 != null ? !f19.equals(tuple.f19) : tuple.f19 != null) { return false; }
+		if (f20 != null ? !f20.equals(tuple.f20) : tuple.f20 != null) { return false; }
+		if (f21 != null ? !f21.equals(tuple.f21) : tuple.f21 != null) { return false; }
+		if (f22 != null ? !f22.equals(tuple.f22) : tuple.f22 != null) { return false; }
+		if (f23 != null ? !f23.equals(tuple.f23) : tuple.f23 != null) { return false; }
+		return true;
+	}
+
+	@Override
+	public int hashCode() {
+		int result = f0 != null ? f0.hashCode() : 0;
+		result = 31 * result + (f1 != null ? f1.hashCode() : 0);
+		result = 31 * result + (f2 != null ? f2.hashCode() : 0);
+		result = 31 * result + (f3 != null ? f3.hashCode() : 0);
+		result = 31 * result + (f4 != null ? f4.hashCode() : 0);
+		result = 31 * result + (f5 != null ? f5.hashCode() : 0);
+		result = 31 * result + (f6 != null ? f6.hashCode() : 0);
+		result = 31 * result + (f7 != null ? f7.hashCode() : 0);
+		result = 31 * result + (f8 != null ? f8.hashCode() : 0);
+		result = 31 * result + (f9 != null ? f9.hashCode() : 0);
+		result = 31 * result + (f10 != null ? f10.hashCode() : 0);
+		result = 31 * result + (f11 != null ? f11.hashCode() : 0);
+		result = 31 * result + (f12 != null ? f12.hashCode() : 0);
+		result = 31 * result + (f13 != null ? f13.hashCode() : 0);
+		result = 31 * result + (f14 != null ? f14.hashCode() : 0);
+		result = 31 * result + (f15 != null ? f15.hashCode() : 0);
+		result = 31 * result + (f16 != null ? f16.hashCode() : 0);
+		result = 31 * result + (f17 != null ? f17.hashCode() : 0);
+		result = 31 * result + (f18 != null ? f18.hashCode() : 0);
+		result = 31 * result + (f19 != null ? f19.hashCode() : 0);
+		result = 31 * result + (f20 != null ? f20.hashCode() : 0);
+		result = 31 * result + (f21 != null ? f21.hashCode() : 0);
+		result = 31 * result + (f22 != null ? f22.hashCode() : 0);
+		result = 31 * result + (f23 != null ? f23.hashCode() : 0);
+		return result;
+	}
+	/**
+	* Shallow tuple copy.
+	* @returns A new Tuple with the same fields as this.
+	 */
+	public Tuple24<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9,T10,T11,T12,T13,T14,T15,T16,T17,T18,T19,T20,T21,T22,T23> copy(){ 
+		return new Tuple24<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9,T10,T11,T12,T13,T14,T15,T16,T17,T18,T19,T20,T21,T22,T23>(this.f0,
+			this.f1,
+			this.f2,
+			this.f3,
+			this.f4,
+			this.f5,
+			this.f6,
+			this.f7,
+			this.f8,
+			this.f9,
+			this.f10,
+			this.f11,
+			this.f12,
+			this.f13,
+			this.f14,
+			this.f15,
+			this.f16,
+			this.f17,
+			this.f18,
+			this.f19,
+			this.f20,
+			this.f21,
+			this.f22,
+			this.f23);
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/96e76a58/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple25.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple25.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple25.java
index 8ebe152..cf0fcdd 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple25.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple25.java
@@ -392,4 +392,104 @@ public class Tuple25<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13,
 			+ ", " + StringUtils.arrayAwareToString(this.f24)
 			+ ")";
 	}
+
+	/**
+	 * Deep equality for tuples by calling equals() on the tuple members
+	 * @param o the object checked for equality
+	 * @return true if this is equal to o.
+	 */
+	@Override
+	public boolean equals(Object o) {
+		if(this == o) { return true; }
+		if (!(o instanceof Tuple25)) { return false; }
+		Tuple25 tuple = (Tuple25) o;
+		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
+		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }
+		if (f2 != null ? !f2.equals(tuple.f2) : tuple.f2 != null) { return false; }
+		if (f3 != null ? !f3.equals(tuple.f3) : tuple.f3 != null) { return false; }
+		if (f4 != null ? !f4.equals(tuple.f4) : tuple.f4 != null) { return false; }
+		if (f5 != null ? !f5.equals(tuple.f5) : tuple.f5 != null) { return false; }
+		if (f6 != null ? !f6.equals(tuple.f6) : tuple.f6 != null) { return false; }
+		if (f7 != null ? !f7.equals(tuple.f7) : tuple.f7 != null) { return false; }
+		if (f8 != null ? !f8.equals(tuple.f8) : tuple.f8 != null) { return false; }
+		if (f9 != null ? !f9.equals(tuple.f9) : tuple.f9 != null) { return false; }
+		if (f10 != null ? !f10.equals(tuple.f10) : tuple.f10 != null) { return false; }
+		if (f11 != null ? !f11.equals(tuple.f11) : tuple.f11 != null) { return false; }
+		if (f12 != null ? !f12.equals(tuple.f12) : tuple.f12 != null) { return false; }
+		if (f13 != null ? !f13.equals(tuple.f13) : tuple.f13 != null) { return false; }
+		if (f14 != null ? !f14.equals(tuple.f14) : tuple.f14 != null) { return false; }
+		if (f15 != null ? !f15.equals(tuple.f15) : tuple.f15 != null) { return false; }
+		if (f16 != null ? !f16.equals(tuple.f16) : tuple.f16 != null) { return false; }
+		if (f17 != null ? !f17.equals(tuple.f17) : tuple.f17 != null) { return false; }
+		if (f18 != null ? !f18.equals(tuple.f18) : tuple.f18 != null) { return false; }
+		if (f19 != null ? !f19.equals(tuple.f19) : tuple.f19 != null) { return false; }
+		if (f20 != null ? !f20.equals(tuple.f20) : tuple.f20 != null) { return false; }
+		if (f21 != null ? !f21.equals(tuple.f21) : tuple.f21 != null) { return false; }
+		if (f22 != null ? !f22.equals(tuple.f22) : tuple.f22 != null) { return false; }
+		if (f23 != null ? !f23.equals(tuple.f23) : tuple.f23 != null) { return false; }
+		if (f24 != null ? !f24.equals(tuple.f24) : tuple.f24 != null) { return false; }
+		return true;
+	}
+
+	@Override
+	public int hashCode() {
+		int result = f0 != null ? f0.hashCode() : 0;
+		result = 31 * result + (f1 != null ? f1.hashCode() : 0);
+		result = 31 * result + (f2 != null ? f2.hashCode() : 0);
+		result = 31 * result + (f3 != null ? f3.hashCode() : 0);
+		result = 31 * result + (f4 != null ? f4.hashCode() : 0);
+		result = 31 * result + (f5 != null ? f5.hashCode() : 0);
+		result = 31 * result + (f6 != null ? f6.hashCode() : 0);
+		result = 31 * result + (f7 != null ? f7.hashCode() : 0);
+		result = 31 * result + (f8 != null ? f8.hashCode() : 0);
+		result = 31 * result + (f9 != null ? f9.hashCode() : 0);
+		result = 31 * result + (f10 != null ? f10.hashCode() : 0);
+		result = 31 * result + (f11 != null ? f11.hashCode() : 0);
+		result = 31 * result + (f12 != null ? f12.hashCode() : 0);
+		result = 31 * result + (f13 != null ? f13.hashCode() : 0);
+		result = 31 * result + (f14 != null ? f14.hashCode() : 0);
+		result = 31 * result + (f15 != null ? f15.hashCode() : 0);
+		result = 31 * result + (f16 != null ? f16.hashCode() : 0);
+		result = 31 * result + (f17 != null ? f17.hashCode() : 0);
+		result = 31 * result + (f18 != null ? f18.hashCode() : 0);
+		result = 31 * result + (f19 != null ? f19.hashCode() : 0);
+		result = 31 * result + (f20 != null ? f20.hashCode() : 0);
+		result = 31 * result + (f21 != null ? f21.hashCode() : 0);
+		result = 31 * result + (f22 != null ? f22.hashCode() : 0);
+		result = 31 * result + (f23 != null ? f23.hashCode() : 0);
+		result = 31 * result + (f24 != null ? f24.hashCode() : 0);
+		return result;
+	}
+	/**
+	* Shallow tuple copy.
+	* @returns A new Tuple with the same fields as this.
+	 */
+	public Tuple25<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9,T10,T11,T12,T13,T14,T15,T16,T17,T18,T19,T20,T21,T22,T23,T24> copy(){ 
+		return new Tuple25<T0,T1,T2,T3,T4,T5,T6,T7,T8,T9,T10,T11,T12,T13,T14,T15,T16,T17,T18,T19,T20,T21,T22,T23,T24>(this.f0,
+			this.f1,
+			this.f2,
+			this.f3,
+			this.f4,
+			this.f5,
+			this.f6,
+			this.f7,
+			this.f8,
+			this.f9,
+			this.f10,
+			this.f11,
+			this.f12,
+			this.f13,
+			this.f14,
+			this.f15,
+			this.f16,
+			this.f17,
+			this.f18,
+			this.f19,
+			this.f20,
+			this.f21,
+			this.f22,
+			this.f23,
+			this.f24);
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/96e76a58/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple3.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple3.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple3.java
index 5c2101c..aa6aff9 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple3.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple3.java
@@ -128,4 +128,38 @@ public class Tuple3<T0, T1, T2> extends Tuple {
 			+ ", " + StringUtils.arrayAwareToString(this.f2)
 			+ ")";
 	}
+
+	/**
+	 * Deep equality for tuples by calling equals() on the tuple members
+	 * @param o the object checked for equality
+	 * @return true if this is equal to o.
+	 */
+	@Override
+	public boolean equals(Object o) {
+		if(this == o) { return true; }
+		if (!(o instanceof Tuple3)) { return false; }
+		Tuple3 tuple = (Tuple3) o;
+		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
+		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }
+		if (f2 != null ? !f2.equals(tuple.f2) : tuple.f2 != null) { return false; }
+		return true;
+	}
+
+	@Override
+	public int hashCode() {
+		int result = f0 != null ? f0.hashCode() : 0;
+		result = 31 * result + (f1 != null ? f1.hashCode() : 0);
+		result = 31 * result + (f2 != null ? f2.hashCode() : 0);
+		return result;
+	}
+	/**
+	* Shallow tuple copy.
+	* @returns A new Tuple with the same fields as this.
+	 */
+	public Tuple3<T0,T1,T2> copy(){ 
+		return new Tuple3<T0,T1,T2>(this.f0,
+			this.f1,
+			this.f2);
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/96e76a58/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple4.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple4.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple4.java
index 135aa8b..3c165b0 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple4.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple4.java
@@ -140,4 +140,41 @@ public class Tuple4<T0, T1, T2, T3> extends Tuple {
 			+ ", " + StringUtils.arrayAwareToString(this.f3)
 			+ ")";
 	}
+
+	/**
+	 * Deep equality for tuples by calling equals() on the tuple members
+	 * @param o the object checked for equality
+	 * @return true if this is equal to o.
+	 */
+	@Override
+	public boolean equals(Object o) {
+		if(this == o) { return true; }
+		if (!(o instanceof Tuple4)) { return false; }
+		Tuple4 tuple = (Tuple4) o;
+		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
+		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }
+		if (f2 != null ? !f2.equals(tuple.f2) : tuple.f2 != null) { return false; }
+		if (f3 != null ? !f3.equals(tuple.f3) : tuple.f3 != null) { return false; }
+		return true;
+	}
+
+	@Override
+	public int hashCode() {
+		int result = f0 != null ? f0.hashCode() : 0;
+		result = 31 * result + (f1 != null ? f1.hashCode() : 0);
+		result = 31 * result + (f2 != null ? f2.hashCode() : 0);
+		result = 31 * result + (f3 != null ? f3.hashCode() : 0);
+		return result;
+	}
+	/**
+	* Shallow tuple copy.
+	* @returns A new Tuple with the same fields as this.
+	 */
+	public Tuple4<T0,T1,T2,T3> copy(){ 
+		return new Tuple4<T0,T1,T2,T3>(this.f0,
+			this.f1,
+			this.f2,
+			this.f3);
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/96e76a58/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple5.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple5.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple5.java
index 383fcb4..17d3ec7 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple5.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple5.java
@@ -152,4 +152,44 @@ public class Tuple5<T0, T1, T2, T3, T4> extends Tuple {
 			+ ", " + StringUtils.arrayAwareToString(this.f4)
 			+ ")";
 	}
+
+	/**
+	 * Deep equality for tuples by calling equals() on the tuple members
+	 * @param o the object checked for equality
+	 * @return true if this is equal to o.
+	 */
+	@Override
+	public boolean equals(Object o) {
+		if(this == o) { return true; }
+		if (!(o instanceof Tuple5)) { return false; }
+		Tuple5 tuple = (Tuple5) o;
+		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
+		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }
+		if (f2 != null ? !f2.equals(tuple.f2) : tuple.f2 != null) { return false; }
+		if (f3 != null ? !f3.equals(tuple.f3) : tuple.f3 != null) { return false; }
+		if (f4 != null ? !f4.equals(tuple.f4) : tuple.f4 != null) { return false; }
+		return true;
+	}
+
+	@Override
+	public int hashCode() {
+		int result = f0 != null ? f0.hashCode() : 0;
+		result = 31 * result + (f1 != null ? f1.hashCode() : 0);
+		result = 31 * result + (f2 != null ? f2.hashCode() : 0);
+		result = 31 * result + (f3 != null ? f3.hashCode() : 0);
+		result = 31 * result + (f4 != null ? f4.hashCode() : 0);
+		return result;
+	}
+	/**
+	* Shallow tuple copy.
+	* @returns A new Tuple with the same fields as this.
+	 */
+	public Tuple5<T0,T1,T2,T3,T4> copy(){ 
+		return new Tuple5<T0,T1,T2,T3,T4>(this.f0,
+			this.f1,
+			this.f2,
+			this.f3,
+			this.f4);
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/96e76a58/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple6.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple6.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple6.java
index 4b7fff4..0f519aa 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple6.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple6.java
@@ -164,4 +164,47 @@ public class Tuple6<T0, T1, T2, T3, T4, T5> extends Tuple {
 			+ ", " + StringUtils.arrayAwareToString(this.f5)
 			+ ")";
 	}
+
+	/**
+	 * Deep equality for tuples by calling equals() on the tuple members
+	 * @param o the object checked for equality
+	 * @return true if this is equal to o.
+	 */
+	@Override
+	public boolean equals(Object o) {
+		if(this == o) { return true; }
+		if (!(o instanceof Tuple6)) { return false; }
+		Tuple6 tuple = (Tuple6) o;
+		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
+		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }
+		if (f2 != null ? !f2.equals(tuple.f2) : tuple.f2 != null) { return false; }
+		if (f3 != null ? !f3.equals(tuple.f3) : tuple.f3 != null) { return false; }
+		if (f4 != null ? !f4.equals(tuple.f4) : tuple.f4 != null) { return false; }
+		if (f5 != null ? !f5.equals(tuple.f5) : tuple.f5 != null) { return false; }
+		return true;
+	}
+
+	@Override
+	public int hashCode() {
+		int result = f0 != null ? f0.hashCode() : 0;
+		result = 31 * result + (f1 != null ? f1.hashCode() : 0);
+		result = 31 * result + (f2 != null ? f2.hashCode() : 0);
+		result = 31 * result + (f3 != null ? f3.hashCode() : 0);
+		result = 31 * result + (f4 != null ? f4.hashCode() : 0);
+		result = 31 * result + (f5 != null ? f5.hashCode() : 0);
+		return result;
+	}
+	/**
+	* Shallow tuple copy.
+	* @returns A new Tuple with the same fields as this.
+	 */
+	public Tuple6<T0,T1,T2,T3,T4,T5> copy(){ 
+		return new Tuple6<T0,T1,T2,T3,T4,T5>(this.f0,
+			this.f1,
+			this.f2,
+			this.f3,
+			this.f4,
+			this.f5);
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/96e76a58/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple7.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple7.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple7.java
index d017246..1572083 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple7.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple7.java
@@ -176,4 +176,50 @@ public class Tuple7<T0, T1, T2, T3, T4, T5, T6> extends Tuple {
 			+ ", " + StringUtils.arrayAwareToString(this.f6)
 			+ ")";
 	}
+
+	/**
+	 * Deep equality for tuples by calling equals() on the tuple members
+	 * @param o the object checked for equality
+	 * @return true if this is equal to o.
+	 */
+	@Override
+	public boolean equals(Object o) {
+		if(this == o) { return true; }
+		if (!(o instanceof Tuple7)) { return false; }
+		Tuple7 tuple = (Tuple7) o;
+		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
+		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }
+		if (f2 != null ? !f2.equals(tuple.f2) : tuple.f2 != null) { return false; }
+		if (f3 != null ? !f3.equals(tuple.f3) : tuple.f3 != null) { return false; }
+		if (f4 != null ? !f4.equals(tuple.f4) : tuple.f4 != null) { return false; }
+		if (f5 != null ? !f5.equals(tuple.f5) : tuple.f5 != null) { return false; }
+		if (f6 != null ? !f6.equals(tuple.f6) : tuple.f6 != null) { return false; }
+		return true;
+	}
+
+	@Override
+	public int hashCode() {
+		int result = f0 != null ? f0.hashCode() : 0;
+		result = 31 * result + (f1 != null ? f1.hashCode() : 0);
+		result = 31 * result + (f2 != null ? f2.hashCode() : 0);
+		result = 31 * result + (f3 != null ? f3.hashCode() : 0);
+		result = 31 * result + (f4 != null ? f4.hashCode() : 0);
+		result = 31 * result + (f5 != null ? f5.hashCode() : 0);
+		result = 31 * result + (f6 != null ? f6.hashCode() : 0);
+		return result;
+	}
+	/**
+	* Shallow tuple copy.
+	* @returns A new Tuple with the same fields as this.
+	 */
+	public Tuple7<T0,T1,T2,T3,T4,T5,T6> copy(){ 
+		return new Tuple7<T0,T1,T2,T3,T4,T5,T6>(this.f0,
+			this.f1,
+			this.f2,
+			this.f3,
+			this.f4,
+			this.f5,
+			this.f6);
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/96e76a58/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple8.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple8.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple8.java
index 46a9656..b4fe194 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple8.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple8.java
@@ -188,4 +188,53 @@ public class Tuple8<T0, T1, T2, T3, T4, T5, T6, T7> extends Tuple {
 			+ ", " + StringUtils.arrayAwareToString(this.f7)
 			+ ")";
 	}
+
+	/**
+	 * Deep equality for tuples by calling equals() on the tuple members
+	 * @param o the object checked for equality
+	 * @return true if this is equal to o.
+	 */
+	@Override
+	public boolean equals(Object o) {
+		if(this == o) { return true; }
+		if (!(o instanceof Tuple8)) { return false; }
+		Tuple8 tuple = (Tuple8) o;
+		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
+		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }
+		if (f2 != null ? !f2.equals(tuple.f2) : tuple.f2 != null) { return false; }
+		if (f3 != null ? !f3.equals(tuple.f3) : tuple.f3 != null) { return false; }
+		if (f4 != null ? !f4.equals(tuple.f4) : tuple.f4 != null) { return false; }
+		if (f5 != null ? !f5.equals(tuple.f5) : tuple.f5 != null) { return false; }
+		if (f6 != null ? !f6.equals(tuple.f6) : tuple.f6 != null) { return false; }
+		if (f7 != null ? !f7.equals(tuple.f7) : tuple.f7 != null) { return false; }
+		return true;
+	}
+
+	@Override
+	public int hashCode() {
+		int result = f0 != null ? f0.hashCode() : 0;
+		result = 31 * result + (f1 != null ? f1.hashCode() : 0);
+		result = 31 * result + (f2 != null ? f2.hashCode() : 0);
+		result = 31 * result + (f3 != null ? f3.hashCode() : 0);
+		result = 31 * result + (f4 != null ? f4.hashCode() : 0);
+		result = 31 * result + (f5 != null ? f5.hashCode() : 0);
+		result = 31 * result + (f6 != null ? f6.hashCode() : 0);
+		result = 31 * result + (f7 != null ? f7.hashCode() : 0);
+		return result;
+	}
+	/**
+	* Shallow tuple copy.
+	* @returns A new Tuple with the same fields as this.
+	 */
+	public Tuple8<T0,T1,T2,T3,T4,T5,T6,T7> copy(){ 
+		return new Tuple8<T0,T1,T2,T3,T4,T5,T6,T7>(this.f0,
+			this.f1,
+			this.f2,
+			this.f3,
+			this.f4,
+			this.f5,
+			this.f6,
+			this.f7);
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/96e76a58/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple9.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple9.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple9.java
index 5c4b540..0a1ce0b 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple9.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple9.java
@@ -200,4 +200,56 @@ public class Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8> extends Tuple {
 			+ ", " + StringUtils.arrayAwareToString(this.f8)
 			+ ")";
 	}
+
+	/**
+	 * Deep equality for tuples by calling equals() on the tuple members
+	 * @param o the object checked for equality
+	 * @return true if this is equal to o.
+	 */
+	@Override
+	public boolean equals(Object o) {
+		if(this == o) { return true; }
+		if (!(o instanceof Tuple9)) { return false; }
+		Tuple9 tuple = (Tuple9) o;
+		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
+		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }
+		if (f2 != null ? !f2.equals(tuple.f2) : tuple.f2 != null) { return false; }
+		if (f3 != null ? !f3.equals(tuple.f3) : tuple.f3 != null) { return false; }
+		if (f4 != null ? !f4.equals(tuple.f4) : tuple.f4 != null) { return false; }
+		if (f5 != null ? !f5.equals(tuple.f5) : tuple.f5 != null) { return false; }
+		if (f6 != null ? !f6.equals(tuple.f6) : tuple.f6 != null) { return false; }
+		if (f7 != null ? !f7.equals(tuple.f7) : tuple.f7 != null) { return false; }
+		if (f8 != null ? !f8.equals(tuple.f8) : tuple.f8 != null) { return false; }
+		return true;
+	}
+
+	@Override
+	public int hashCode() {
+		int result = f0 != null ? f0.hashCode() : 0;
+		result = 31 * result + (f1 != null ? f1.hashCode() : 0);
+		result = 31 * result + (f2 != null ? f2.hashCode() : 0);
+		result = 31 * result + (f3 != null ? f3.hashCode() : 0);
+		result = 31 * result + (f4 != null ? f4.hashCode() : 0);
+		result = 31 * result + (f5 != null ? f5.hashCode() : 0);
+		result = 31 * result + (f6 != null ? f6.hashCode() : 0);
+		result = 31 * result + (f7 != null ? f7.hashCode() : 0);
+		result = 31 * result + (f8 != null ? f8.hashCode() : 0);
+		return result;
+	}
+	/**
+	* Shallow tuple copy.
+	* @returns A new Tuple with the same fields as this.
+	 */
+	public Tuple9<T0,T1,T2,T3,T4,T5,T6,T7,T8> copy(){ 
+		return new Tuple9<T0,T1,T2,T3,T4,T5,T6,T7,T8>(this.f0,
+			this.f1,
+			this.f2,
+			this.f3,
+			this.f4,
+			this.f5,
+			this.f6,
+			this.f7,
+			this.f8);
+	}
+
 }


[06/53] [abbrv] Rework the Taskmanager to a slot based model and remove legacy cloud code

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java
deleted file mode 100644
index e369613..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java
+++ /dev/null
@@ -1,248 +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.jobmanager.scheduler;
-
-import java.io.IOException;
-import java.util.ArrayDeque;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Queue;
-import java.util.Set;
-
-import eu.stratosphere.nephele.taskmanager.AbstractTaskResult;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.nephele.execution.ExecutionState;
-import eu.stratosphere.nephele.executiongraph.ExecutionEdge;
-import eu.stratosphere.nephele.executiongraph.ExecutionGate;
-import eu.stratosphere.nephele.executiongraph.ExecutionGraph;
-import eu.stratosphere.nephele.executiongraph.ExecutionVertex;
-import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
-import eu.stratosphere.nephele.instance.AbstractInstance;
-import eu.stratosphere.nephele.instance.DummyInstance;
-import eu.stratosphere.runtime.io.channels.ChannelID;
-import eu.stratosphere.nephele.taskmanager.TaskCancelResult;
-import eu.stratosphere.nephele.util.SerializableHashSet;
-import eu.stratosphere.util.StringUtils;
-
-public final class RecoveryLogic {
-
-	/**
-	 * The logger to report information and problems.
-	 */
-	private static final Log LOG = LogFactory.getLog(RecoveryLogic.class);
-
-	/**
-	 * Private constructor so class cannot be instantiated.
-	 */
-	private RecoveryLogic() {
-	}
-
-	public static boolean recover(final ExecutionVertex failedVertex,
-			final Map<ExecutionVertexID, ExecutionVertex> verticesToBeRestarted,
-			final Set<ExecutionVertex> assignedVertices) {
-
-		// Perform initial sanity check
-		if (failedVertex.getExecutionState() != ExecutionState.FAILED) {
-			LOG.error("Vertex " + failedVertex + " is requested to be recovered, but is not failed");
-			return false;
-		}
-
-		final ExecutionGraph eg = failedVertex.getExecutionGraph();
-		synchronized (eg) {
-
-			LOG.info("Starting recovery for failed vertex " + failedVertex);
-
-			final Set<ExecutionVertex> verticesToBeCanceled = new HashSet<ExecutionVertex>();
-
-			findVerticesToRestart(failedVertex, verticesToBeCanceled);
-
-			// Restart all predecessors without checkpoint
-			final Iterator<ExecutionVertex> cancelIterator = verticesToBeCanceled.iterator();
-			while (cancelIterator.hasNext()) {
-
-				final ExecutionVertex vertex = cancelIterator.next();
-
-				if (vertex.compareAndUpdateExecutionState(ExecutionState.FINISHED, getStateToUpdate(vertex))) {
-					LOG.info("Vertex " + vertex + " has already finished and will not be canceled");
-					if (vertex.getExecutionState() == ExecutionState.ASSIGNED) {
-						assignedVertices.add(vertex);
-					}
-					continue;
-				}
-
-				LOG.info(vertex + " is canceled by recovery logic");
-				verticesToBeRestarted.put(vertex.getID(), vertex);
-				final TaskCancelResult cancelResult = vertex.cancelTask();
-
-				if (cancelResult.getReturnCode() != AbstractTaskResult.ReturnCode.SUCCESS
-						&& cancelResult.getReturnCode() != AbstractTaskResult.ReturnCode.TASK_NOT_FOUND) {
-
-					verticesToBeRestarted.remove(vertex.getID());
-					LOG.error("Unable to cancel vertex" + cancelResult.getDescription());
-					return false;
-				}
-			}
-
-			LOG.info("Starting cache invalidation");
-
-			// Invalidate the lookup caches
-			if (!invalidateReceiverLookupCaches(failedVertex, verticesToBeCanceled)) {
-				return false;
-			}
-
-			LOG.info("Cache invalidation complete");
-
-			// Restart failed vertex
-			failedVertex.updateExecutionState(getStateToUpdate(failedVertex));
-			if (failedVertex.getExecutionState() == ExecutionState.ASSIGNED) {
-				assignedVertices.add(failedVertex);
-			}
-		}
-
-		return true;
-	}
-
-	static boolean hasInstanceAssigned(final ExecutionVertex vertex) {
-
-		return !(vertex.getAllocatedResource().getInstance() instanceof DummyInstance);
-	}
-
-	private static ExecutionState getStateToUpdate(final ExecutionVertex vertex) {
-
-		if (hasInstanceAssigned(vertex)) {
-			return ExecutionState.ASSIGNED;
-		}
-
-		return ExecutionState.CREATED;
-	}
-
-	private static void findVerticesToRestart(final ExecutionVertex failedVertex,
-			final Set<ExecutionVertex> verticesToBeCanceled) {
-
-		final Queue<ExecutionVertex> verticesToTest = new ArrayDeque<ExecutionVertex>();
-		final Set<ExecutionVertex> visited = new HashSet<ExecutionVertex>();
-		verticesToTest.add(failedVertex);
-
-		while (!verticesToTest.isEmpty()) {
-
-			final ExecutionVertex vertex = verticesToTest.poll();
-
-			// Predecessors must be either checkpoints or need to be restarted, too
-			for (int j = 0; j < vertex.getNumberOfPredecessors(); j++) {
-				final ExecutionVertex predecessor = vertex.getPredecessor(j);
-
-				if (hasInstanceAssigned(predecessor)) {
-					verticesToBeCanceled.add(predecessor);
-				}
-
-				if (!visited.contains(predecessor)) {
-					verticesToTest.add(predecessor);
-				}
-			}
-			visited.add(vertex);
-		}
-	}
-
-	private static final boolean invalidateReceiverLookupCaches(final ExecutionVertex failedVertex,
-			final Set<ExecutionVertex> verticesToBeCanceled) {
-
-		final Map<AbstractInstance, Set<ChannelID>> entriesToInvalidate = new HashMap<AbstractInstance, Set<ChannelID>>();
-
-		collectCacheEntriesToInvalidate(failedVertex, entriesToInvalidate);
-		for (final Iterator<ExecutionVertex> it = verticesToBeCanceled.iterator(); it.hasNext();) {
-			collectCacheEntriesToInvalidate(it.next(), entriesToInvalidate);
-		}
-
-		final Iterator<Map.Entry<AbstractInstance, Set<ChannelID>>> it = entriesToInvalidate.entrySet().iterator();
-
-		while (it.hasNext()) {
-
-			final Map.Entry<AbstractInstance, Set<ChannelID>> entry = it.next();
-			final AbstractInstance instance = entry.getKey();
-
-			try {
-				instance.invalidateLookupCacheEntries(entry.getValue());
-			} catch (IOException ioe) {
-				LOG.error(StringUtils.stringifyException(ioe));
-				return false;
-			}
-		}
-
-		return true;
-	}
-
-	private static void collectCacheEntriesToInvalidate(final ExecutionVertex vertex,
-			final Map<AbstractInstance, Set<ChannelID>> entriesToInvalidate) {
-
-		final int numberOfOutputGates = vertex.getNumberOfOutputGates();
-		for (int i = 0; i < numberOfOutputGates; ++i) {
-
-			final ExecutionGate outputGate = vertex.getOutputGate(i);
-			for (int j = 0; j < outputGate.getNumberOfEdges(); ++j) {
-
-				final ExecutionEdge outputChannel = outputGate.getEdge(j);
-
-				final ExecutionVertex connectedVertex = outputChannel.getInputGate().getVertex();
-				if (connectedVertex == null) {
-					LOG.error("Connected vertex is null");
-					continue;
-				}
-
-				final AbstractInstance instance = connectedVertex.getAllocatedResource().getInstance();
-				if (instance instanceof DummyInstance) {
-					continue;
-				}
-
-				Set<ChannelID> channelIDs = entriesToInvalidate.get(instance);
-				if (channelIDs == null) {
-					channelIDs = new SerializableHashSet<ChannelID>();
-					entriesToInvalidate.put(instance, channelIDs);
-				}
-
-				channelIDs.add(outputChannel.getInputChannelID());
-			}
-		}
-
-		for (int i = 0; i < vertex.getNumberOfInputGates(); ++i) {
-
-			final ExecutionGate inputGate = vertex.getInputGate(i);
-			for (int j = 0; j < inputGate.getNumberOfEdges(); ++j) {
-
-				final ExecutionEdge inputChannel = inputGate.getEdge(j);
-
-				final ExecutionVertex connectedVertex = inputChannel.getOutputGate().getVertex();
-				if (connectedVertex == null) {
-					LOG.error("Connected vertex is null");
-					continue;
-				}
-
-				final AbstractInstance instance = connectedVertex.getAllocatedResource().getInstance();
-				if (instance instanceof DummyInstance) {
-					continue;
-				}
-
-				Set<ChannelID> channelIDs = entriesToInvalidate.get(instance);
-				if (channelIDs == null) {
-					channelIDs = new SerializableHashSet<ChannelID>();
-					entriesToInvalidate.put(instance, channelIDs);
-				}
-
-				channelIDs.add(inputChannel.getOutputChannelID());
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/local/LocalExecutionListener.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/local/LocalExecutionListener.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/local/LocalExecutionListener.java
deleted file mode 100644
index 9ae5635..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/local/LocalExecutionListener.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 eu.stratosphere.nephele.jobmanager.scheduler.local;
-
-import eu.stratosphere.nephele.executiongraph.ExecutionVertex;
-import eu.stratosphere.nephele.jobmanager.scheduler.AbstractExecutionListener;
-
-/**
- * This is a wrapper class for the {@link LocalScheduler} to receive
- * notifications about state changes of vertices belonging
- * to scheduled jobs.
- * <p>
- * This class is thread-safe.
- * 
- */
-public class LocalExecutionListener extends AbstractExecutionListener {
-
-	public LocalExecutionListener(final LocalScheduler scheduler, final ExecutionVertex executionVertex) {
-		super(scheduler, executionVertex);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/local/LocalScheduler.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/local/LocalScheduler.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/local/LocalScheduler.java
deleted file mode 100644
index b731965..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/local/LocalScheduler.java
+++ /dev/null
@@ -1,213 +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.jobmanager.scheduler.local;
-
-import java.util.ArrayDeque;
-import java.util.Deque;
-import java.util.Iterator;
-import java.util.Map;
-
-import eu.stratosphere.nephele.execution.ExecutionState;
-import eu.stratosphere.nephele.executiongraph.ExecutionGraph;
-import eu.stratosphere.nephele.executiongraph.ExecutionGraphIterator;
-import eu.stratosphere.nephele.executiongraph.ExecutionStage;
-import eu.stratosphere.nephele.executiongraph.ExecutionStageListener;
-import eu.stratosphere.nephele.executiongraph.ExecutionVertex;
-import eu.stratosphere.nephele.executiongraph.InternalJobStatus;
-import eu.stratosphere.nephele.executiongraph.JobStatusListener;
-import eu.stratosphere.nephele.instance.InstanceException;
-import eu.stratosphere.nephele.instance.InstanceManager;
-import eu.stratosphere.nephele.instance.InstanceRequestMap;
-import eu.stratosphere.nephele.instance.InstanceType;
-import eu.stratosphere.nephele.instance.InstanceTypeDescription;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.jobmanager.DeploymentManager;
-import eu.stratosphere.nephele.jobmanager.scheduler.AbstractScheduler;
-import eu.stratosphere.nephele.jobmanager.scheduler.SchedulingException;
-import eu.stratosphere.util.StringUtils;
-
-public class LocalScheduler extends AbstractScheduler implements JobStatusListener, ExecutionStageListener {
-
-	/**
-	 * The job queue of the scheduler
-	 */
-	private Deque<ExecutionGraph> jobQueue = new ArrayDeque<ExecutionGraph>();
-
-	/**
-	 * Constructs a new local scheduler.
-	 * 
-	 * @param deploymentManager
-	 *        the deployment manager assigned to this scheduler
-	 * @param instanceManager
-	 *        the instance manager to be used with this scheduler
-	 */
-	public LocalScheduler(final DeploymentManager deploymentManager, final InstanceManager instanceManager) {
-		super(deploymentManager, instanceManager);
-	}
-
-	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();
-				// Field jobID of executionGraph is immutable, so no synchronization needed
-				if (executionGraph.getJobID().equals(executionGraphToRemove.getJobID())) {
-					removedFromQueue = true;
-					it.remove();
-					break;
-				}
-
-			}
-		}
-
-		if (!removedFromQueue) {
-			LOG.error("Cannot find job " + executionGraphToRemove.getJobName() + " ("
-				+ executionGraphToRemove.getJobID() + ") to remove");
-		}
-
-		// TODO: Remove vertices from restart map
-	}
-
-
-	@Override
-	public void schedulJob(final ExecutionGraph executionGraph) throws SchedulingException {
-
-		// Get Map of all available Instance types
-		final Map<InstanceType, InstanceTypeDescription> availableInstances = getInstanceManager()
-				.getMapOfAvailableInstanceTypes();
-
-		final Iterator<ExecutionStage> stageIt = executionGraph.iterator();
-		while (stageIt.hasNext()) {
-
-			final InstanceRequestMap instanceRequestMap = new InstanceRequestMap();
-			final ExecutionStage stage = stageIt.next();
-			stage.collectRequiredInstanceTypes(instanceRequestMap, ExecutionState.CREATED);
-
-			// Iterator over required Instances
-			final Iterator<Map.Entry<InstanceType, Integer>> it = instanceRequestMap.getMinimumIterator();
-			while (it.hasNext()) {
-
-				final Map.Entry<InstanceType, Integer> entry = it.next();
-
-				final InstanceTypeDescription descr = availableInstances.get(entry.getKey());
-				if (descr == null) {
-					throw new SchedulingException("Unable to schedule job: No instance of type " + entry.getKey()
-							+ " available");
-				}
-
-				if (descr.getMaximumNumberOfAvailableInstances() != -1
-						&& descr.getMaximumNumberOfAvailableInstances() < entry.getValue().intValue()) {
-					throw new SchedulingException("Unable to schedule job: " + entry.getValue().intValue()
-							+ " instances of type " + entry.getKey() + " required, but only "
-							+ descr.getMaximumNumberOfAvailableInstances() + " are available");
-				}
-			}
-		}
-
-		// Subscribe to job status notifications
-		executionGraph.registerJobStatusListener(this);
-
-		// Set state of each vertex for scheduled
-		final ExecutionGraphIterator it2 = new ExecutionGraphIterator(executionGraph, true);
-		while (it2.hasNext()) {
-
-			final ExecutionVertex vertex = it2.next();
-			vertex.registerExecutionListener(new LocalExecutionListener(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);
-		}
-	}
-
-
-	@Override
-	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;
-	}
-
-
-	@Override
-	public void shutdown() {
-
-		synchronized (this.jobQueue) {
-			this.jobQueue.clear();
-		}
-
-	}
-
-
-	@Override
-	public void jobStatusHasChanged(final ExecutionGraph executionGraph, final InternalJobStatus newJobStatus,
-			final String optionalMessage) {
-
-		if (newJobStatus == InternalJobStatus.FAILED || newJobStatus == InternalJobStatus.FINISHED
-			|| newJobStatus == InternalJobStatus.CANCELED) {
-			removeJobFromSchedule(executionGraph);
-		}
-	}
-
-
-	@Override
-	public void nextExecutionStageEntered(final JobID jobID, final ExecutionStage executionStage) {
-
-		// Request new instances if necessary
-		try {
-			requestInstances(executionStage);
-		} catch (InstanceException e) {
-			// TODO: Handle this error correctly
-			LOG.error(StringUtils.stringifyException(e));
-		}
-
-		// Deploy the assigned vertices
-		deployAssignedInputVertices(executionStage.getExecutionGraph());
-
-		// Initialize the replay of the previous stage's checkpoints
-		replayCheckpointsFromPreviousStage(executionStage.getExecutionGraph());
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueExecutionListener.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueExecutionListener.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueExecutionListener.java
deleted file mode 100644
index 1d37edc..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueExecutionListener.java
+++ /dev/null
@@ -1,40 +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.jobmanager.scheduler.queue;
-
-import eu.stratosphere.nephele.executiongraph.ExecutionVertex;
-import eu.stratosphere.nephele.jobmanager.scheduler.AbstractExecutionListener;
-
-/**
- * This is a wrapper class for the {@link QueueScheduler} to receive
- * notifications about state changes of vertices belonging
- * to scheduled jobs.
- * <p>
- * This class is thread-safe.
- * 
- */
-public final class QueueExecutionListener extends AbstractExecutionListener {
-
-	/**
-	 * Constructs a new queue execution listener.
-	 * 
-	 * @param scheduler
-	 *        the scheduler this listener is connected with
-	 * @param executionVertex
-	 *        the execution vertex this listener is created for
-	 */
-	public QueueExecutionListener(final QueueScheduler scheduler, final ExecutionVertex executionVertex) {
-		super(scheduler, executionVertex);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueScheduler.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueScheduler.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueScheduler.java
deleted file mode 100644
index cd76f04..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueScheduler.java
+++ /dev/null
@@ -1,216 +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.jobmanager.scheduler.queue;
-
-import java.util.ArrayDeque;
-import java.util.Deque;
-import java.util.Iterator;
-import java.util.Map;
-
-import eu.stratosphere.nephele.execution.ExecutionState;
-import eu.stratosphere.nephele.executiongraph.ExecutionGraph;
-import eu.stratosphere.nephele.executiongraph.ExecutionGraphIterator;
-import eu.stratosphere.nephele.executiongraph.ExecutionStage;
-import eu.stratosphere.nephele.executiongraph.ExecutionStageListener;
-import eu.stratosphere.nephele.executiongraph.ExecutionVertex;
-import eu.stratosphere.nephele.executiongraph.InternalJobStatus;
-import eu.stratosphere.nephele.executiongraph.JobStatusListener;
-import eu.stratosphere.nephele.instance.InstanceException;
-import eu.stratosphere.nephele.instance.InstanceManager;
-import eu.stratosphere.nephele.instance.InstanceRequestMap;
-import eu.stratosphere.nephele.instance.InstanceType;
-import eu.stratosphere.nephele.instance.InstanceTypeDescription;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.jobmanager.DeploymentManager;
-import eu.stratosphere.nephele.jobmanager.scheduler.AbstractScheduler;
-import eu.stratosphere.nephele.jobmanager.scheduler.SchedulingException;
-import eu.stratosphere.util.StringUtils;
-
-/**
- * The queue scheduler mains of queue of all submitted jobs and executes one job at a time.
- * 
- */
-public class QueueScheduler extends AbstractScheduler implements JobStatusListener, ExecutionStageListener {
-
-	/**
-	 * The job queue where all submitted jobs go to.
-	 */
-	private Deque<ExecutionGraph> jobQueue = new ArrayDeque<ExecutionGraph>();
-
-	/**
-	 * Constructs a new queue scheduler.
-	 * 
-	 * @param deploymentManager
-	 *        the deployment manager assigned to this scheduler
-	 * @param instanceManager
-	 *        the instance manager to be used with this scheduler
-	 */
-	public QueueScheduler(final DeploymentManager deploymentManager, final InstanceManager instanceManager) {
-		super(deploymentManager, 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");
-		}
-	}
-
-
-	@Override
-	public void schedulJob(final ExecutionGraph executionGraph) throws SchedulingException {
-
-		// Get Map of all available Instance types
-		final Map<InstanceType, InstanceTypeDescription> availableInstances = getInstanceManager()
-				.getMapOfAvailableInstanceTypes();
-
-		final Iterator<ExecutionStage> stageIt = executionGraph.iterator();
-		while (stageIt.hasNext()) {
-
-			final InstanceRequestMap instanceRequestMap = new InstanceRequestMap();
-			final ExecutionStage stage = stageIt.next();
-			stage.collectRequiredInstanceTypes(instanceRequestMap, ExecutionState.CREATED);
-
-			// Iterator over required Instances
-			final Iterator<Map.Entry<InstanceType, Integer>> it = instanceRequestMap.getMinimumIterator();
-			while (it.hasNext()) {
-
-				final Map.Entry<InstanceType, Integer> entry = it.next();
-
-				final InstanceTypeDescription descr = availableInstances.get(entry.getKey());
-				if (descr == null) {
-					throw new SchedulingException("Unable to schedule job: No instance of type " + entry.getKey()
-							+ " available");
-				}
-
-				if (descr.getMaximumNumberOfAvailableInstances() != -1
-						&& descr.getMaximumNumberOfAvailableInstances() < entry.getValue().intValue()) {
-					throw new SchedulingException("Unable to schedule job: " + entry.getValue().intValue()
-							+ " instances of type " + entry.getKey() + " required, but only "
-							+ descr.getMaximumNumberOfAvailableInstances() + " are available");
-				}
-			}
-		}
-
-		// 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 QueueExecutionListener(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);
-		}
-	}
-
-
-	@Override
-	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;
-	}
-
-
-	@Override
-	public void shutdown() {
-
-		synchronized (this.jobQueue) {
-			this.jobQueue.clear();
-		}
-
-	}
-
-
-	@Override
-	public void jobStatusHasChanged(final ExecutionGraph executionGraph, final InternalJobStatus newJobStatus,
-			final String optionalMessage) {
-
-		if (newJobStatus == InternalJobStatus.FAILED || newJobStatus == InternalJobStatus.FINISHED
-			|| newJobStatus == InternalJobStatus.CANCELED) {
-			removeJobFromSchedule(executionGraph);
-		}
-	}
-
-
-	@Override
-	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());
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitManager.java
index eea78d8..bbef991 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitManager.java
@@ -37,7 +37,7 @@ import eu.stratosphere.util.StringUtils;
 
 /**
  * The input split manager is responsible for serving input splits to {@link AbstractInputTask} objects at runtime.
- * Before passed on to the {@link AbstractScheduler}, an {@link ExecutionGraph} is registered with the input split
+ * Before passed on to the {@link eu.stratosphere.nephele.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.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/LocatableInputSplitAssigner.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/LocatableInputSplitAssigner.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/LocatableInputSplitAssigner.java
index 85df81a..3717fbf 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/LocatableInputSplitAssigner.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/LocatableInputSplitAssigner.java
@@ -16,6 +16,7 @@ package eu.stratosphere.nephele.jobmanager.splitassigner;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
+import eu.stratosphere.nephele.instance.Instance;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
@@ -23,7 +24,6 @@ import eu.stratosphere.core.io.InputSplit;
 import eu.stratosphere.core.io.LocatableInputSplit;
 import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertex;
 import eu.stratosphere.nephele.executiongraph.ExecutionVertex;
-import eu.stratosphere.nephele.instance.AbstractInstance;
 import eu.stratosphere.nephele.template.AbstractInputTask;
 import eu.stratosphere.nephele.template.AbstractInvokable;
 
@@ -115,7 +115,7 @@ public final class LocatableInputSplitAssigner implements InputSplitAssigner {
 			return null;
 		}
 
-		final AbstractInstance instance = vertex.getAllocatedResource().getInstance();
+		final Instance instance = vertex.getAllocatedResource().getInstance();
 		if (instance == null) {
 			LOG.error("Instance is null, returning random split");
 			return null;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/LocatableInputSplitList.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/LocatableInputSplitList.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/LocatableInputSplitList.java
index c830a6f..7647fae 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/LocatableInputSplitList.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/LocatableInputSplitList.java
@@ -21,16 +21,16 @@ import java.util.PriorityQueue;
 import java.util.Queue;
 import java.util.Set;
 
+import eu.stratosphere.nephele.instance.Instance;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
 import eu.stratosphere.core.io.LocatableInputSplit;
-import eu.stratosphere.nephele.instance.AbstractInstance;
 
 /**
  * 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 AbstractInstance} objects which request an input split and its nearest storage location with respect to the
+ * {@link eu.stratosphere.nephele.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>
@@ -50,13 +50,13 @@ public final class LocatableInputSplitList {
 	private Set<LocatableInputSplit> masterSet = new HashSet<LocatableInputSplit>();
 
 	/**
-	 * The map caching the specific file input split lists for each {@link AbstractInstance}.
+	 * The map caching the specific file input split lists for each {@link eu.stratosphere.nephele.instance.Instance}.
 	 */
-	private Map<AbstractInstance, Queue<QueueElem>> instanceMap = new HashMap<AbstractInstance, Queue<QueueElem>>();
+	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 AbstractInstance}.
+	 * {@link eu.stratosphere.nephele.instance.Instance}.
 	 * 
 	 */
 	private final class QueueElem implements Comparable<QueueElem> {
@@ -120,7 +120,7 @@ public final class LocatableInputSplitList {
 	/**
 	 * 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 AbstractInstance} is as
+	 * way that the distance between the split's storage location and the requesting {@link eu.stratosphere.nephele.instance.Instance} is as
 	 * short as possible.
 	 * 
 	 * @param instance
@@ -128,7 +128,7 @@ public final class LocatableInputSplitList {
 	 * @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 AbstractInstance instance) {
+	synchronized LocatableInputSplit getNextInputSplit(final Instance instance) {
 
 		final Queue<QueueElem> instanceSplitList = getInstanceSplitList(instance);
 
@@ -157,16 +157,16 @@ public final class LocatableInputSplitList {
 	}
 
 	/**
-	 * Returns a list of locatable input splits specifically ordered for the given {@link AbstractInstance}. When the
+	 * Returns a list of locatable input splits specifically ordered for the given {@link eu.stratosphere.nephele.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 AbstractInstance}.
+	 * by the minimum distance between the input splits' storage locations and the given {@link eu.stratosphere.nephele.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 AbstractInstance instance) {
+	private Queue<QueueElem> getInstanceSplitList(final Instance instance) {
 
 		Queue<QueueElem> instanceSplitList = this.instanceMap.get(instance);
 		if (instanceSplitList == null) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/file/FileInputSplitAssigner.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/file/FileInputSplitAssigner.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/file/FileInputSplitAssigner.java
index 938fb48..7894334 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/file/FileInputSplitAssigner.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/file/FileInputSplitAssigner.java
@@ -16,6 +16,7 @@ package eu.stratosphere.nephele.jobmanager.splitassigner.file;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
+import eu.stratosphere.nephele.instance.Instance;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
@@ -23,7 +24,6 @@ import eu.stratosphere.core.fs.FileInputSplit;
 import eu.stratosphere.core.io.InputSplit;
 import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertex;
 import eu.stratosphere.nephele.executiongraph.ExecutionVertex;
-import eu.stratosphere.nephele.instance.AbstractInstance;
 import eu.stratosphere.nephele.jobmanager.splitassigner.InputSplitAssigner;
 import eu.stratosphere.nephele.template.AbstractInputTask;
 import eu.stratosphere.nephele.template.AbstractInvokable;
@@ -117,7 +117,7 @@ public final class FileInputSplitAssigner implements InputSplitAssigner {
 			return null;
 		}
 
-		final AbstractInstance instance = vertex.getAllocatedResource().getInstance();
+		final Instance instance = vertex.getAllocatedResource().getInstance();
 		if (instance == null) {
 			LOG.error("Instance is null, returning random split");
 			return null;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/file/FileInputSplitList.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/file/FileInputSplitList.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/file/FileInputSplitList.java
index db84a91..ae9898a 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/file/FileInputSplitList.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/file/FileInputSplitList.java
@@ -21,15 +21,15 @@ import java.util.PriorityQueue;
 import java.util.Queue;
 import java.util.Set;
 
+import eu.stratosphere.nephele.instance.Instance;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
 import eu.stratosphere.core.fs.FileInputSplit;
-import eu.stratosphere.nephele.instance.AbstractInstance;
 
 /**
  * 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 AbstractInstance}
+ * Besides simply storing the splits, the file input split list also computes the distance all {@link eu.stratosphere.nephele.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.
@@ -50,13 +50,13 @@ public final class FileInputSplitList {
 	private Set<FileInputSplit> masterSet = new HashSet<FileInputSplit>();
 
 	/**
-	 * The map caching the specific file input split lists for each {@link AbstractInstance}.
+	 * The map caching the specific file input split lists for each {@link eu.stratosphere.nephele.instance.Instance}.
 	 */
-	private Map<AbstractInstance, Queue<QueueElem>> instanceMap = new HashMap<AbstractInstance, Queue<QueueElem>>();
+	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 AbstractInstance}.
+	 * {@link eu.stratosphere.nephele.instance.Instance}.
 	 * 
 	 */
 	private final class QueueElem implements Comparable<QueueElem> {
@@ -119,7 +119,7 @@ public final class FileInputSplitList {
 
 	/**
 	 * 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 AbstractInstance} is as
+	 * way that the distance between the split's storage location and the requesting {@link eu.stratosphere.nephele.instance.Instance} is as
 	 * short as possible.
 	 * 
 	 * @param instance
@@ -127,7 +127,7 @@ public final class FileInputSplitList {
 	 * @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 AbstractInstance instance) {
+	synchronized FileInputSplit getNextInputSplit(final Instance instance) {
 
 		final Queue<QueueElem> instanceSplitList = getInstanceSplitList(instance);
 
@@ -156,15 +156,15 @@ public final class FileInputSplitList {
 	}
 
 	/**
-	 * Returns a list of file input splits specifically ordered for the given {@link AbstractInstance}. When the list is
+	 * Returns a list of file input splits specifically ordered for the given {@link eu.stratosphere.nephele.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 AbstractInstance}.
+	 * the minimum distance between the input splits' storage locations and the given {@link eu.stratosphere.nephele.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 AbstractInstance instance) {
+	private Queue<QueueElem> getInstanceSplitList(final Instance instance) {
 
 		Queue<QueueElem> instanceSplitList = this.instanceMap.get(instance);
 		if (instanceSplitList == null) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGraph.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGraph.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGraph.java
index 374656b..fab720d 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGraph.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGraph.java
@@ -445,9 +445,8 @@ public final class ManagementGraph extends ManagementAttachment implements IORea
 			groupVertexID.read(in);
 			final ManagementGroupVertex groupVertex = this.getGroupVertexByID(groupVertexID);
 			final String instanceName = StringRecord.readString(in);
-			final String instanceType = StringRecord.readString(in);
 			final int indexInGroup = in.readInt();
-			final ManagementVertex vertex = new ManagementVertex(groupVertex, vertexID, instanceName, instanceType, indexInGroup);
+			final ManagementVertex vertex = new ManagementVertex(groupVertex, vertexID, instanceName, indexInGroup);
 			vertex.read(in);
 		}
 
@@ -523,7 +522,6 @@ public final class ManagementGraph extends ManagementAttachment implements IORea
 			managementVertex.getID().write(out);
 			managementVertex.getGroupVertex().getID().write(out);
 			StringRecord.writeString(out, managementVertex.getInstanceName());
-			StringRecord.writeString(out, managementVertex.getInstanceType());
 			out.writeInt(managementVertex.getIndexInGroup());
 			managementVertex.write(out);
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertex.java
index 639b1e9..eaececc 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertex.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertex.java
@@ -65,11 +65,6 @@ public final class ManagementVertex extends ManagementAttachment implements IORe
 	private String instanceName;
 
 	/**
-	 * The type of the instance the vertex represented by this management vertex currently runs on.
-	 */
-	private String instanceType;
-
-	/**
 	 * The index of this vertex in the management group vertex it belongs to.
 	 */
 	private final int indexInGroup;
@@ -88,19 +83,14 @@ public final class ManagementVertex extends ManagementAttachment implements IORe
 	 *        the ID of the new management vertex
 	 * @param instanceName
 	 *        the name of the instance the vertex represented by this new management vertex currently runs on
-	 * @param instanceType
-	 *        the type of the instance the vertex represented by this new management vertex currently runs on
-	 * @param checkpointState
-	 *        the state of the vertex's checkpoint
 	 * @param indexInGroup
 	 *        the index of this vertex in the management group vertex it belongs to
 	 */
 	public ManagementVertex(final ManagementGroupVertex groupVertex, final ManagementVertexID id,
-			final String instanceName, final String instanceType, final int indexInGroup) {
+			final String instanceName, final int indexInGroup) {
 		this.groupVertex = groupVertex;
 		this.id = id;
 		this.instanceName = instanceName;
-		this.instanceType = instanceType;
 
 		this.indexInGroup = indexInGroup;
 
@@ -132,15 +122,6 @@ public final class ManagementVertex extends ManagementAttachment implements IORe
 	}
 
 	/**
-	 * Returns the type of the instance the vertex represented by this management vertex currently runs on.
-	 * 
-	 * @return the type of the instance the vertex represented by this management vertex currently runs on
-	 */
-	public String getInstanceType() {
-		return this.instanceType;
-	}
-
-	/**
 	 * Returns the number of input gates this management vertex contains.
 	 * 
 	 * @return the number of input gates this management vertex contains
@@ -276,16 +257,6 @@ public final class ManagementVertex extends ManagementAttachment implements IORe
 		this.instanceName = instanceName;
 	}
 
-	/**
-	 * Sets the type of instance this vertex currently runs on.
-	 * 
-	 * @param instanceType
-	 *        the type of instance this vertex currently runs on
-	 */
-	public void setInstanceType(final String instanceType) {
-		this.instanceType = instanceType;
-	}
-
 	public void setOptMessage(final String optMessage) {
 		this.optMessage = optMessage;
 	}
@@ -294,7 +265,6 @@ public final class ManagementVertex extends ManagementAttachment implements IORe
 		return this.optMessage;
 	}
 
-
 	@Override
 	public void read(final DataInput in) throws IOException {
 
@@ -314,7 +284,6 @@ public final class ManagementVertex extends ManagementAttachment implements IORe
 		}
 
 		this.instanceName = StringRecord.readString(in);
-		this.instanceType = StringRecord.readString(in);
 	}
 
 
@@ -331,7 +300,6 @@ public final class ManagementVertex extends ManagementAttachment implements IORe
 		out.writeInt(this.outputGates.size());
 
 		StringRecord.writeString(out, this.instanceName);
-		StringRecord.writeString(out, this.instanceType);
 	}
 	
 	@Override
@@ -351,7 +319,6 @@ public final class ManagementVertex extends ManagementAttachment implements IORe
 		json.append("\"vertexname\": \"" + StringUtils.escapeHtml(this.toString()) + "\",");
 		json.append("\"vertexstatus\": \"" + this.getExecutionState() + "\",");
 		json.append("\"vertexinstancename\": \"" + this.getInstanceName() + "\",");
-		json.append("\"vertexinstancetype\": \"" + this.getInstanceType() + "\"");
 		json.append("}");
 		return json.toString();
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/net/NetUtils.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/net/NetUtils.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/net/NetUtils.java
index 35979dd..cb08c3a 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/net/NetUtils.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/net/NetUtils.java
@@ -221,6 +221,7 @@ public class NetUtils {
 	 * @return InputStream for reading from the socket.
 	 * @throws IOException
 	 */
+	@SuppressWarnings("resource")
 	public static InputStream getInputStream(Socket socket, long timeout) throws IOException {
 		return (socket.getChannel() == null) ? socket.getInputStream() : new SocketInputStream(socket, timeout);
 	}
@@ -266,6 +267,7 @@ public class NetUtils {
 	 * @return OutputStream for writing to the socket.
 	 * @throws IOException
 	 */
+	@SuppressWarnings("resource")
 	public static OutputStream getOutputStream(Socket socket, long timeout) throws IOException {
 		return (socket.getChannel() == null) ? socket.getOutputStream() : new SocketOutputStream(socket, timeout);
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/JobProfilingData.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/JobProfilingData.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/JobProfilingData.java
index 7d0c980..81b4134 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/JobProfilingData.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/JobProfilingData.java
@@ -23,7 +23,7 @@ import eu.stratosphere.nephele.executiongraph.ExecutionGraph;
 import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertex;
 import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertexIterator;
 import eu.stratosphere.nephele.executiongraph.ExecutionVertex;
-import eu.stratosphere.nephele.instance.AbstractInstance;
+import eu.stratosphere.nephele.instance.Instance;
 import eu.stratosphere.nephele.instance.DummyInstance;
 import eu.stratosphere.nephele.instance.InstanceConnectionInfo;
 import eu.stratosphere.nephele.profiling.impl.types.InternalInstanceProfilingData;
@@ -75,7 +75,7 @@ public class JobProfilingData {
 
 	public InstanceSummaryProfilingEvent getInstanceSummaryProfilingData(long timestamp) {
 
-		final Set<AbstractInstance> tempSet = new HashSet<AbstractInstance>();
+		final Set<Instance> tempSet = new HashSet<Instance>();
 		// First determine the number of allocated instances in the current stage
 		final ExecutionGroupVertexIterator it = new ExecutionGroupVertexIterator(this.executionGraph, true,
 			this.executionGraph.getIndexOfCurrentExecutionStage());
@@ -84,7 +84,7 @@ public class JobProfilingData {
 			final ExecutionGroupVertex groupVertex = it.next();
 			for (int i = 0; i < groupVertex.getCurrentNumberOfGroupMembers(); i++) {
 				final ExecutionVertex executionVertex = groupVertex.getGroupMember(i);
-				final AbstractInstance instance = executionVertex.getAllocatedResource().getInstance();
+				final Instance instance = executionVertex.getAllocatedResource().getInstance();
 				if(!(instance instanceof DummyInstance)) {
 					tempSet.add(instance);
 				}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/ExtendedManagementProtocol.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/ExtendedManagementProtocol.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/ExtendedManagementProtocol.java
index 59ec15d..c731285 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/ExtendedManagementProtocol.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/ExtendedManagementProtocol.java
@@ -15,13 +15,10 @@ package eu.stratosphere.nephele.protocols;
 
 import java.io.IOException;
 import java.util.List;
-import java.util.Map;
 
 import eu.stratosphere.core.io.StringRecord;
 import eu.stratosphere.nephele.event.job.AbstractEvent;
 import eu.stratosphere.nephele.event.job.RecentJobEvent;
-import eu.stratosphere.nephele.instance.InstanceType;
-import eu.stratosphere.nephele.instance.InstanceTypeDescription;
 import eu.stratosphere.nephele.jobgraph.JobID;
 import eu.stratosphere.nephele.managementgraph.ManagementGraph;
 import eu.stratosphere.nephele.managementgraph.ManagementVertexID;
@@ -104,19 +101,6 @@ public interface ExtendedManagementProtocol extends JobManagementProtocol {
 	void killInstance(StringRecord instanceName) throws IOException;
 
 	/**
-	 * Returns a map of all instance types which are currently available to Nephele. The map contains a description of
-	 * the hardware characteristics for each instance type as provided in the configuration file. Moreover, it contains
-	 * the actual hardware description as reported by task managers running on the individual instances. If available,
-	 * the map also contains the maximum number instances Nephele can allocate of each instance type (i.e. if no other
-	 * job occupies instances).
-	 * 
-	 * @return a list of all instance types available to Nephele
-	 * @throws IOException
-	 *         thrown if an error occurs while transmitting the list
-	 */
-	Map<InstanceType, InstanceTypeDescription> getMapOfAvailableInstanceTypes() 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.
 	 * 
@@ -126,4 +110,11 @@ public interface ExtendedManagementProtocol extends JobManagementProtocol {
 	 *         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
+	 */
+	int getAvailableSlots() throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/JobManagerProtocol.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/JobManagerProtocol.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/JobManagerProtocol.java
index 8cd5e26..5070b51 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/JobManagerProtocol.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/JobManagerProtocol.java
@@ -19,6 +19,8 @@ import eu.stratosphere.core.protocols.VersionedProtocol;
 import eu.stratosphere.nephele.instance.HardwareDescription;
 import eu.stratosphere.nephele.instance.InstanceConnectionInfo;
 import eu.stratosphere.nephele.taskmanager.TaskExecutionState;
+import eu.stratosphere.nephele.taskmanager.transferenvelope.RegisterTaskManagerResult;
+import eu.stratosphere.nephele.types.IntegerRecord;
 
 /**
  * The job manager protocol is implemented by the job manager and offers functionality
@@ -33,12 +35,23 @@ public interface JobManagerProtocol extends VersionedProtocol {
 	 * 
 	 * @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
 	 *         thrown if an error occurs during this remote procedure call
 	 */
-	void sendHeartbeat(InstanceConnectionInfo instanceConnectionInfo, HardwareDescription hardwareDescription)
+	void sendHeartbeat(InstanceConnectionInfo instanceConnectionInfo)
+			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
+	 *
+	 * @return whether the task manager was successfully registered
+	 */
+	RegisterTaskManagerResult registerTaskManager(InstanceConnectionInfo instanceConnectionInfo,
+						HardwareDescription hardwareDescription,IntegerRecord numberOfSlots)
 			throws IOException;
 
 	/**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/iomanager/ChannelAccess.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/iomanager/ChannelAccess.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/iomanager/ChannelAccess.java
index ccbc64a..85432eb 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/iomanager/ChannelAccess.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/iomanager/ChannelAccess.java
@@ -75,6 +75,7 @@ public abstract class ChannelAccess<T, R extends IORequest>
 		this.requestQueue = requestQueue;
 		
 		try {
+			@SuppressWarnings("resource")
 			RandomAccessFile file = new RandomAccessFile(id.getPath(), writeEnabled ? "rw" : "r");
 			this.fileChannel = file.getChannel();
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/memorymanager/MemoryManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/memorymanager/MemoryManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/memorymanager/MemoryManager.java
index a8fe096..8b20c75 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/memorymanager/MemoryManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/memorymanager/MemoryManager.java
@@ -69,16 +69,29 @@ public interface MemoryManager {
 	 * @return The size of the pages handled by the memory manager.
 	 */
 	int getPageSize();
+
+	/**
+	 * Returns the total size of memory.
+	 * @return
+	 */
+	long getMemorySize();
 	
 	/**
 	 * Computes to how many pages the given number of bytes corresponds. If the given number of bytes is not an
 	 * exact multiple of a page size, the result is rounded down, such that a portion of the memory (smaller
 	 * than the page size) is not included.
 	 * 
-	 * @param numBytes The number of bytes to convert to a page count.
+	 * @param fraction the fraction of the total memory per slot
 	 * @return The number of pages to which 
 	 */
-	int computeNumberOfPages(long numBytes);
+	int computeNumberOfPages(double fraction);
+
+	/**
+	 * Computes the memory size of the fraction per slot.
+	 * @param fraction
+	 * @return
+	 */
+	long computeMemorySize(double fraction);
 	
 	/**
 	 * Rounds the given value down to a multiple of the memory manager's page size.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/memorymanager/spi/DefaultMemoryManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/memorymanager/spi/DefaultMemoryManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/memorymanager/spi/DefaultMemoryManager.java
index 8bc7b13..d4a2b36 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/memorymanager/spi/DefaultMemoryManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/memorymanager/spi/DefaultMemoryManager.java
@@ -66,6 +66,13 @@ public class DefaultMemoryManager implements MemoryManager {
 	
 	private boolean isShutDown;				// flag whether the close() has already been invoked.
 
+	/**
+	 * Number of slots of the task manager
+	 */
+	private final int numberOfSlots;
+
+	private final long memorySize;
+
 	// ------------------------------------------------------------------------
 	// Constructors / Destructors
 	// ------------------------------------------------------------------------
@@ -75,8 +82,8 @@ public class DefaultMemoryManager implements MemoryManager {
 	 * 
 	 * @param memorySize The total size of the memory to be managed by this memory manager.
 	 */
-	public DefaultMemoryManager(long memorySize) {
-		this(memorySize, DEFAULT_PAGE_SIZE);
+	public DefaultMemoryManager(long memorySize, int numberOfSlots) {
+		this(memorySize, numberOfSlots, DEFAULT_PAGE_SIZE);
 	}
 
 	/**
@@ -85,7 +92,7 @@ public class DefaultMemoryManager implements MemoryManager {
 	 * @param memorySize The total size of the memory to be managed by this memory manager.
 	 * @param pageSize The size of the pages handed out by the memory manager.
 	 */
-	public DefaultMemoryManager(long memorySize, int pageSize) {
+	public DefaultMemoryManager(long memorySize, int numberOfSlots, int pageSize) {
 		// sanity checks
 		if (memorySize <= 0) {
 			throw new IllegalArgumentException("Size of total memory must be positive.");
@@ -97,6 +104,10 @@ public class DefaultMemoryManager implements MemoryManager {
 			// not a power of two
 			throw new IllegalArgumentException("The given page size is not a power of two.");
 		}
+
+		this.memorySize = memorySize;
+
+		this.numberOfSlots = numberOfSlots;
 		
 		// assign page size and bit utilities
 		this.pageSize = pageSize;
@@ -348,8 +359,18 @@ public class DefaultMemoryManager implements MemoryManager {
 	}
 
 	@Override
-	public int computeNumberOfPages(long numBytes) {
-		return getNumPages(numBytes);
+	public long getMemorySize() {
+		return this.memorySize;
+	}
+
+	@Override
+	public int computeNumberOfPages(double fraction) {
+		return getRelativeNumPages(fraction);
+	}
+
+	@Override
+	public long computeMemorySize(double fraction) {
+		return this.pageSize*computeNumberOfPages(fraction);
 	}
 
 	@Override
@@ -371,6 +392,14 @@ public class DefaultMemoryManager implements MemoryManager {
 			throw new IllegalArgumentException("The given number of bytes correstponds to more than MAX_INT pages.");
 		}
 	}
+
+	private final int getRelativeNumPages(double fraction){
+		if(fraction < 0){
+			throw new IllegalArgumentException("The fraction of memory to allocate must not be negative.");
+		}
+
+		return (int)(this.totalNumPages * fraction / this.numberOfSlots);
+	}
 	
 	// ------------------------------------------------------------------------
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
index ef0f6ab..5966cf9 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
@@ -45,6 +45,10 @@ import eu.stratosphere.nephele.ExecutionMode;
 import eu.stratosphere.runtime.io.network.LocalConnectionManager;
 import eu.stratosphere.runtime.io.network.NetworkConnectionManager;
 import eu.stratosphere.runtime.io.network.netty.NettyConnectionManager;
+import eu.stratosphere.nephele.instance.Hardware;
+import eu.stratosphere.nephele.taskmanager.transferenvelope.RegisterTaskManagerResult;
+import eu.stratosphere.nephele.types.IntegerRecord;
+
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.GnuParser;
@@ -148,7 +152,9 @@ public class TaskManager implements TaskOperationProtocol {
 
 	private final IOManager ioManager;
 
-	private static HardwareDescription hardwareDescription = null;
+	private final HardwareDescription hardwareDescription;
+
+	private final int numberOfSlots;
 
 	private final Thread heartbeatThread;
 	
@@ -156,10 +162,10 @@ public class TaskManager implements TaskOperationProtocol {
 	
 	/** Stores whether the task manager has already been shut down. */
 	private volatile boolean shutdownComplete;
-
+	
 	/**
 	 * Constructs a new task manager, starts its IPC service and attempts to discover the job manager to
-	 * receive an initial configuration. All parameters are obtained from the
+	 * receive an initial configuration. All parameters are obtained from the 
 	 * {@link GlobalConfiguration}, which must be loaded prior to instantiating the task manager.
 	 */
 	public TaskManager(ExecutionMode executionMode) throws Exception {
@@ -169,30 +175,31 @@ public class TaskManager implements TaskOperationProtocol {
 		LOG.info("Execution mode: " + executionMode);
 
 		// 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) {
+			}
+			catch (UnknownHostException e) {
 				LOG.fatal("Could not resolve JobManager host name.");
 				throw new Exception("Could not resolve JobManager host name: " + e.getMessage(), e);
 			}
-
+			
 			LOG.info("Connecting to JobManager at: " + jobManagerAddress);
 		}
-
+		
 		// Create RPC connection to the JobManager
 		try {
 			this.jobManager = RPC.getProxy(JobManagerProtocol.class, jobManagerAddress, NetUtils.getSocketFactory());
@@ -200,7 +207,7 @@ public class TaskManager implements TaskOperationProtocol {
 			LOG.fatal("Could not connect to the JobManager: " + e.getMessage(), e);
 			throw new Exception("Failed to initialize connection to JobManager: " + e.getMessage(), e);
 		}
-
+		
 		int ipcPort = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, -1);
 		int dataPort = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, -1);
 		if (ipcPort == -1) {
@@ -209,16 +216,17 @@ public class TaskManager implements TaskOperationProtocol {
 		if (dataPort == -1) {
 			dataPort = getAvailablePort();
 		}
-
+		
 		// Determine our own public facing address and start the server
 		{
 			final InetAddress taskManagerAddress;
 			try {
 				taskManagerAddress = getTaskManagerAddress(jobManagerAddress);
-			} catch (Exception e) {
+			}
+			catch (Exception e) {
 				throw new RuntimeException("The TaskManager failed to determine its own network address.", e);
 			}
-
+			
 			this.localInstanceConnectionInfo = new InstanceConnectionInfo(taskManagerAddress, ipcPort, dataPort);
 			LOG.info("TaskManager connection information:" + this.localInstanceConnectionInfo);
 
@@ -231,7 +239,7 @@ public class TaskManager implements TaskOperationProtocol {
 				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());
@@ -258,21 +266,19 @@ public class TaskManager implements TaskOperationProtocol {
 
 		// Load profiler if it should be used
 		if (GlobalConfiguration.getBoolean(ProfilingUtils.ENABLE_PROFILING_KEY, false)) {
-
+			
 			final String profilerClassName = GlobalConfiguration.getString(ProfilingUtils.TASKMANAGER_CLASSNAME_KEY,
-					"eu.stratosphere.nephele.profiling.impl.TaskManagerProfilerImpl");
-
+				"eu.stratosphere.nephele.profiling.impl.TaskManagerProfilerImpl");
+			
 			this.profiler = ProfilingUtils.loadTaskManagerProfiler(profilerClassName, jobManagerAddress.getAddress(),
-					this.localInstanceConnectionInfo);
-
+				this.localInstanceConnectionInfo);
+			
 			if (this.profiler == null) {
 				LOG.error("Cannot find class name for the profiler.");
-			}
-			else {
+			} else {
 				LOG.info("Profiling of jobs is enabled.");
 			}
-		}
-		else {
+		} else {
 			this.profiler = null;
 			LOG.info("Profiling of jobs is disabled.");
 		}
@@ -282,10 +288,11 @@ public class TaskManager implements TaskOperationProtocol {
 				ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator);
 
 		checkTempDirs(tmpDirPaths);
-
+		
 		final int pageSize = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY,
-				ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_BUFFER_SIZE);
+			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_BUFFER_SIZE);
 
+		// Initialize network buffer pool
 		int numBuffers = GlobalConfiguration.getInteger(
 				ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
 				ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS);
@@ -333,6 +340,8 @@ public class TaskManager implements TaskOperationProtocol {
 
 		{
 			HardwareDescription resources = HardwareDescriptionFactory.extractFromSystem();
+			numberOfSlots = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
+					Hardware.getNumberCPUCores());
 
 			// Check whether the memory size has been explicitly configured. if so that overrides the default mechanism
 			// of taking as much as is mentioned in the hardware description
@@ -341,29 +350,30 @@ public class TaskManager implements TaskOperationProtocol {
 			if (memorySize > 0) {
 				// manually configured memory size. override the value in the hardware config
 				resources = HardwareDescriptionFactory.construct(resources.getNumberOfCPUCores(),
-						resources.getSizeOfPhysicalMemory(), memorySize * 1024L * 1024L);
+					resources.getSizeOfPhysicalMemory(), memorySize * 1024L * 1024L);
 			}
 			this.hardwareDescription = resources;
 
 			// Initialize the memory manager
 			LOG.info("Initializing memory manager with " + (resources.getSizeOfFreeMemory() >>> 20) + " megabytes of memory. " +
 					"Page size is " + pageSize + " bytes.");
-
+			
 			try {
 				@SuppressWarnings("unused")
 				final boolean lazyAllocation = GlobalConfiguration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_LAZY_ALLOCATION_KEY,
-						ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_LAZY_ALLOCATION);
-
-				this.memoryManager = new DefaultMemoryManager(resources.getSizeOfFreeMemory(), pageSize);
+					ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_LAZY_ALLOCATION);
+				
+				this.memoryManager = new DefaultMemoryManager(resources.getSizeOfFreeMemory(), this.numberOfSlots,
+						pageSize);
 			} catch (Throwable t) {
 				LOG.fatal("Unable to initialize memory manager with " + (resources.getSizeOfFreeMemory() >>> 20)
-						+ " megabytes of memory.", t);
+					+ " megabytes of memory.", t);
 				throw new Exception("Unable to initialize memory manager.", t);
 			}
 		}
 
 		this.ioManager = new IOManager(tmpDirPaths);
-
+		
 		this.heartbeatThread = new Thread() {
 			@Override
 			public void run() {
@@ -510,19 +520,33 @@ public class TaskManager implements TaskOperationProtocol {
 						ConfigConstants.TASK_MANAGER_HEARTBEAT_INTERVAL_KEY,
 						ConfigConstants.DEFAULT_TASK_MANAGER_HEARTBEAT_INTERVAL);
 
-		while (!shutdownStarted.get()) {
-			// send heart beat
-			try {
-				LOG.debug("heartbeat");
-				this.jobManager.sendHeartbeat(this.localInstanceConnectionInfo, this.hardwareDescription);
-			} catch (IOException e) {
-				if (shutdownStarted.get()) {
+		try {
+			while(!shutdownStarted.get()){
+				RegisterTaskManagerResult result  = this.jobManager.registerTaskManager(this
+								.localInstanceConnectionInfo,this.hardwareDescription,
+						new IntegerRecord(this.numberOfSlots));
+
+				if(result.getReturnCode() == RegisterTaskManagerResult.ReturnCode.SUCCESS){
 					break;
-				} else {
-					LOG.error("Sending the heart beat caused an exception: " + e.getMessage(), e);
+				}
+
+				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);
@@ -532,9 +556,22 @@ public class TaskManager implements TaskOperationProtocol {
 					LOG.error("TaskManager heart beat loop was interrupted without shutdown.");
 				}
 			}
+
+			// send heart beat
+			try {
+				LOG.debug("heartbeat");
+				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);
+				}
+			}
 		}
 	}
 
+	
 	/**
 	 * The states of address detection mechanism.
 	 * There is only a state transition if the current state failed to determine the address.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/RegisterTaskManagerResult.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/RegisterTaskManagerResult.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/RegisterTaskManagerResult.java
new file mode 100644
index 0000000..b396edd
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/RegisterTaskManagerResult.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.taskmanager.transferenvelope;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.nephele.util.EnumUtils;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+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(DataOutput out) throws IOException {
+		EnumUtils.writeEnum(out, this.returnCode);
+	}
+
+	@Override
+	public void read(DataInput in) throws IOException {
+		this.returnCode = EnumUtils.readEnum(in, ReturnCode.class);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/topology/NetworkNode.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/topology/NetworkNode.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/topology/NetworkNode.java
index 09df691..9f6542b 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/topology/NetworkNode.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/topology/NetworkNode.java
@@ -33,8 +33,6 @@ public class NetworkNode implements IOReadableWritable {
 
 	private final List<NetworkNode> childNodes = new ArrayList<NetworkNode>();
 
-	private Object attachment;
-
 	protected NetworkNode(final String name, final NetworkNode parentNode, final NetworkTopology networkTopology) {
 		this.name = name;
 		this.parentNode = parentNode;
@@ -119,14 +117,6 @@ public class NetworkNode implements IOReadableWritable {
 		return this.childNodes.size();
 	}
 
-	public void setAttachment(final Object attachment) {
-		this.attachment = attachment;
-	}
-
-	public Object getAttachment() {
-		return this.attachment;
-	}
-
 	public NetworkNode getChildNode(final int index) {
 
 		if (index < this.childNodes.size()) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/util/IOUtils.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/util/IOUtils.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/util/IOUtils.java
index 0ca490b..554bac5 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/util/IOUtils.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/util/IOUtils.java
@@ -56,6 +56,7 @@ public final class IOUtils {
 	public static void copyBytes(final InputStream in, final OutputStream out, final int buffSize, final boolean close)
 			throws IOException {
 
+		@SuppressWarnings("resource")
 		final PrintStream ps = out instanceof PrintStream ? (PrintStream) out : null;
 		final byte[] buf = new byte[buffSize];
 		try {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/cache/FileCache.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/cache/FileCache.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/cache/FileCache.java
index e4f0a4b..fe63ebe 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/cache/FileCache.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/cache/FileCache.java
@@ -14,7 +14,6 @@
 package eu.stratosphere.pact.runtime.cache;
 
 import eu.stratosphere.api.common.cache.DistributedCache;
-
 import java.io.File;
 import java.io.IOException;
 import java.util.HashMap;
@@ -138,14 +137,12 @@ public class FileCache {
 	 * Asynchronous file copy process
 	 */
 	private class CopyProcess implements Callable<Path> {
+		
 		private JobID jobID;
-		@SuppressWarnings("unused")
-		private String name;
 		private String filePath;
 		private Boolean executable;
 
 		public CopyProcess(String name, DistributedCacheEntry e, JobID jobID) {
-			this.name = name;
 			this.filePath = e.filePath;
 			this.executable = e.isExecutable;
 			this.jobID = jobID;
@@ -168,15 +165,13 @@ public class FileCache {
 	 * If no task is using this file after 5 seconds, clear it.
 	 */
 	private class DeleteProcess implements Runnable {
+		
 		private String name;
-		@SuppressWarnings("unused")
-		private String filePath;
 		private JobID jobID;
 		private int oldCount;
 
 		public DeleteProcess(String name, DistributedCacheEntry e, JobID jobID, int c) {
 			this.name = name;
-			this.filePath = e.filePath;
 			this.jobID = jobID;
 			this.oldCount = c;
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildFirstHashMatchIterator.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildFirstHashMatchIterator.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildFirstHashMatchIterator.java
index ddfa446..a060d28 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildFirstHashMatchIterator.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildFirstHashMatchIterator.java
@@ -60,7 +60,7 @@ public class BuildFirstHashMatchIterator<V1, V2, O> implements JoinTaskIterator<
 			TypeSerializer<V1> serializer1, TypeComparator<V1> comparator1,
 			TypeSerializer<V2> serializer2, TypeComparator<V2> comparator2,
 			TypePairComparator<V2, V1> pairComparator,
-			MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, long totalMemory)
+			MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, double memoryFraction)
 	throws MemoryAllocationException
 	{		
 		this.memManager = memManager;
@@ -73,7 +73,7 @@ public class BuildFirstHashMatchIterator<V1, V2, O> implements JoinTaskIterator<
 		this.probeCopy = serializer2.createInstance();
 		
 		this.hashJoin = getHashJoin(serializer1, comparator1, serializer2, comparator2, pairComparator,
-			memManager, ioManager, ownerTask, totalMemory);
+			memManager, ioManager, ownerTask, memoryFraction);
 	}
 	
 	// --------------------------------------------------------------------------------------------
@@ -152,10 +152,10 @@ public class BuildFirstHashMatchIterator<V1, V2, O> implements JoinTaskIterator<
 	public <BT, PT> MutableHashTable<BT, PT> getHashJoin(TypeSerializer<BT> buildSideSerializer, TypeComparator<BT> buildSideComparator,
 			TypeSerializer<PT> probeSideSerializer, TypeComparator<PT> probeSideComparator,
 			TypePairComparator<PT, BT> pairComparator,
-			MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, long totalMemory)
+			MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, double memoryFraction)
 	throws MemoryAllocationException
 	{
-		final int numPages = memManager.computeNumberOfPages(totalMemory);
+		final int numPages = memManager.computeNumberOfPages(memoryFraction);
 		final List<MemorySegment> memorySegments = memManager.allocatePages(ownerTask, numPages);
 		return new MutableHashTable<BT, PT>(buildSideSerializer, probeSideSerializer, buildSideComparator, probeSideComparator, pairComparator, memorySegments, ioManager);
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildFirstReOpenableHashMatchIterator.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildFirstReOpenableHashMatchIterator.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildFirstReOpenableHashMatchIterator.java
index d699462..8c2b9ca 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildFirstReOpenableHashMatchIterator.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildFirstReOpenableHashMatchIterator.java
@@ -38,21 +38,21 @@ public class BuildFirstReOpenableHashMatchIterator<V1, V2, O> extends BuildFirst
 			TypeSerializer<V2> serializer2, TypeComparator<V2> comparator2,
 			TypePairComparator<V2, V1> pairComparator,
 			MemoryManager memManager, IOManager ioManager,
-			AbstractInvokable ownerTask, long totalMemory)
+			AbstractInvokable ownerTask, double memoryFraction)
 			throws MemoryAllocationException {
 		super(firstInput, secondInput, serializer1, comparator1, serializer2,
 				comparator2, pairComparator, memManager, ioManager, ownerTask,
-				totalMemory);
+				memoryFraction);
 		reopenHashTable = (ReOpenableMutableHashTable<V1, V2>) hashJoin;
 	}
 
 	public <BT, PT> MutableHashTable<BT, PT> getHashJoin(TypeSerializer<BT> buildSideSerializer, TypeComparator<BT> buildSideComparator,
 			TypeSerializer<PT> probeSideSerializer, TypeComparator<PT> probeSideComparator,
 			TypePairComparator<PT, BT> pairComparator,
-			MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, long totalMemory)
+			MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, double memoryFraction)
 	throws MemoryAllocationException
 	{
-		final int numPages = memManager.computeNumberOfPages(totalMemory);
+		final int numPages = memManager.computeNumberOfPages(memoryFraction);
 		final List<MemorySegment> memorySegments = memManager.allocatePages(ownerTask, numPages);
 		return new ReOpenableMutableHashTable<BT, PT>(buildSideSerializer, probeSideSerializer, buildSideComparator, probeSideComparator, pairComparator, memorySegments, ioManager);
 	}


[21/53] [abbrv] git commit: Merge fix to omit input/output registering on JobManager Rework Invokable Task Hierarchy

Posted by rm...@apache.org.
Merge fix to omit input/output registering on JobManager
Rework Invokable Task Hierarchy


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

Branch: refs/heads/travis_test
Commit: 8c1d82a8ec674de6525319501c6be2674e3143f1
Parents: 2692643
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Jun 20 21:13:23 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sun Jun 22 21:07:21 2014 +0200

----------------------------------------------------------------------
 .../stratosphere/client/program/ClientTest.java |  29 +-
 .../plantranslate/NepheleJobGraphGenerator.java |  66 ++---
 .../stratosphere/api/common/PlanExecutor.java   |   2 -
 .../api/common/io/FileOutputFormat.java         | 291 +++++--------------
 .../stratosphere/api/common/io/FormatUtil.java  |   1 -
 .../api/common/io/InitializeOnMaster.java       |  35 +++
 .../api/common/io/OutputFormat.java             |  15 +-
 .../configuration/Configuration.java            |  16 +-
 .../eu/stratosphere/core/fs/FileSystem.java     |   3 +-
 .../eu/stratosphere/core/io/StringRecord.java   |   6 +-
 .../eu/stratosphere/util/IterableIterator.java  |   4 +-
 .../api/java/io/PrintingOutputFormat.java       |   3 -
 .../nephele/execution/RuntimeEnvironment.java   |   1 -
 .../nephele/executiongraph/ExecutionGraph.java  | 104 ++++---
 .../executiongraph/ExecutionGroupVertex.java    |   1 -
 .../jobgraph/AbstractJobInputVertex.java        |  19 +-
 .../jobgraph/AbstractJobOutputVertex.java       |   9 +-
 .../nephele/jobgraph/AbstractJobVertex.java     |  31 +-
 .../stratosphere/nephele/jobgraph/JobGraph.java |  31 +-
 .../nephele/jobgraph/JobInputVertex.java        | 155 ++--------
 .../nephele/jobgraph/JobOutputVertex.java       | 132 ++-------
 .../nephele/jobgraph/JobTaskVertex.java         |  51 +---
 .../nephele/jobmanager/JobManager.java          |   4 +-
 .../splitassigner/InputSplitManager.java        |   2 -
 .../LocatableInputSplitAssigner.java            |   2 -
 .../file/FileInputSplitAssigner.java            |   5 -
 .../nephele/taskmanager/TaskManager.java        |   2 +-
 .../nephele/template/AbstractInputTask.java     |  79 -----
 .../nephele/template/AbstractInvokable.java     |   1 -
 .../nephele/template/AbstractOutputTask.java    |  22 --
 .../nephele/template/AbstractTask.java          |  21 --
 .../runtime/iterative/io/FakeOutputTask.java    |   4 +-
 .../task/IterationSynchronizationSinkTask.java  |   4 +-
 .../iterative/task/IterationTailPactTask.java   |   8 +-
 .../pact/runtime/task/DataSinkTask.java         |  10 +-
 .../pact/runtime/task/DataSourceTask.java       | 109 +++----
 .../pact/runtime/task/RegularPactTask.java      |  16 +-
 .../pact/runtime/task/util/TaskConfig.java      |   6 +-
 .../runtime/io/api/MutableRecordReader.java     |  38 +--
 .../runtime/io/api/RecordReader.java            |  18 +-
 .../runtime/io/api/RecordWriter.java            |  22 +-
 .../executiongraph/ExecutionGraphTest.java      | 163 ++++++-----
 .../ForwardTask1Input1Output.java               |   4 +-
 .../ForwardTask1Input2Outputs.java              |   4 +-
 .../ForwardTask2Inputs1Output.java              |   4 +-
 .../executiongraph/SelfCrossForwardTask.java    |  13 +-
 .../nephele/jobmanager/DoubleSourceTask.java    | 132 +++++++++
 .../nephele/jobmanager/DoubleTargetTask.java    |  24 +-
 .../jobmanager/ExceptionOutputFormat.java       |  26 +-
 .../nephele/jobmanager/ExceptionTask.java       |  11 +-
 .../nephele/jobmanager/ForwardTask.java         |  16 +-
 .../nephele/jobmanager/JobManagerITCase.java    | 158 +++++-----
 .../jobmanager/RuntimeExceptionTask.java        |  13 +-
 .../nephele/jobmanager/UnionTask.java           |  22 +-
 .../scheduler/queue/DefaultSchedulerTest.java   |  68 ++---
 .../nephele/util/tasks/DoubleSourceTask.java    | 134 +++++++++
 .../nephele/util/tasks/FileLineReader.java      | 133 +++++++++
 .../nephele/util/tasks/FileLineWriter.java      |  72 +++++
 .../nephele/util/tasks/JobFileInputVertex.java  | 255 ++++++++++++++++
 .../nephele/util/tasks/JobFileOutputVertex.java | 109 +++++++
 .../runtime/hash/HashMatchIteratorITCase.java   |   4 +-
 .../runtime/hash/ReOpenableHashTableITCase.java |   3 +-
 .../pact/runtime/io/ChannelViewsTest.java       |   4 +-
 .../pact/runtime/io/SpillingBufferTest.java     |   4 +-
 .../sort/AsynchonousPartialSorterITCase.java    |  10 +-
 .../CombiningUnilateralSortMergerITCase.java    |   4 +-
 .../pact/runtime/sort/ExternalSortITCase.java   |   8 +-
 .../sort/MassiveStringSortingITCase.java        |   8 +-
 .../sort/SortMergeMatchIteratorITCase.java      |  11 +-
 .../task/util/HashVsSortMiniBenchmark.java      |   4 +-
 .../pact/runtime/test/util/DummyInvokable.java  |   6 +-
 .../pact/runtime/test/util/TaskTestBase.java    |  15 +-
 .../bufferprovider/LocalBufferPoolTest.java     |   6 +
 .../TransitiveClosureITCase.java                |   2 +-
 .../test/iterative/nephele/JobGraphUtils.java   |  13 +-
 .../recordJobs/util/DiscardingOutputFormat.java |  20 +-
 .../test/runtime/NetworkStackThroughput.java    |  47 ++-
 77 files changed, 1567 insertions(+), 1341 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-clients/src/test/java/eu/stratosphere/client/program/ClientTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-clients/src/test/java/eu/stratosphere/client/program/ClientTest.java b/stratosphere-clients/src/test/java/eu/stratosphere/client/program/ClientTest.java
index b3f8159..a948706 100644
--- a/stratosphere-clients/src/test/java/eu/stratosphere/client/program/ClientTest.java
+++ b/stratosphere-clients/src/test/java/eu/stratosphere/client/program/ClientTest.java
@@ -28,10 +28,7 @@ import org.mockito.Mock;
 import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.powermock.modules.junit4.PowerMockRunner;
 
-import eu.stratosphere.api.common.InvalidProgramException;
 import eu.stratosphere.api.common.Plan;
-import eu.stratosphere.api.java.LocalEnvironment;
-import eu.stratosphere.client.LocalExecutor;
 import eu.stratosphere.compiler.DataStatistics;
 import eu.stratosphere.compiler.PactCompiler;
 import eu.stratosphere.compiler.costs.CostEstimator;
@@ -134,16 +131,16 @@ public class ClientTest {
 		verify(this.jobClientMock).submitJob();
 	}
 	
-
-	@Test(expected=InvalidProgramException.class)
-	public void tryLocalExecution() throws Exception {
-		new Client(configMock);
-		LocalExecutor.execute(planMock);
-	}
-	
-	@Test(expected=InvalidProgramException.class)
-	public void tryLocalEnvironmentExecution() throws Exception {
-		new Client(configMock);
-		new LocalEnvironment();
-	}
-}
+//
+//	@Test(expected=InvalidProgramException.class)
+//	public void tryLocalExecution() throws Exception {
+//		new Client(configMock);
+//		LocalExecutor.execute(planMock);
+//	}
+//	
+//	@Test(expected=InvalidProgramException.class)
+//	public void tryLocalEnvironmentExecution() throws Exception {
+//		new Client(configMock);
+//		new LocalEnvironment();
+//	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plantranslate/NepheleJobGraphGenerator.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plantranslate/NepheleJobGraphGenerator.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plantranslate/NepheleJobGraphGenerator.java
index 3089cdb..3c1e9e3 100644
--- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plantranslate/NepheleJobGraphGenerator.java
+++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plantranslate/NepheleJobGraphGenerator.java
@@ -20,14 +20,7 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-
-import eu.stratosphere.api.common.io.InputFormat;
-import eu.stratosphere.api.common.io.OutputFormat;
-import eu.stratosphere.api.common.operators.util.UserCodeWrapper;
-import eu.stratosphere.core.io.InputSplit;
-import eu.stratosphere.nephele.template.AbstractInputTask;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import java.util.Map.Entry;
 
 import eu.stratosphere.api.common.aggregators.AggregatorRegistry;
 import eu.stratosphere.api.common.aggregators.AggregatorWithName;
@@ -66,7 +59,6 @@ import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException;
 import eu.stratosphere.nephele.jobgraph.JobInputVertex;
 import eu.stratosphere.nephele.jobgraph.JobOutputVertex;
 import eu.stratosphere.nephele.jobgraph.JobTaskVertex;
-import eu.stratosphere.nephele.template.AbstractInputTask;
 import eu.stratosphere.pact.runtime.iterative.convergence.WorksetEmptyConvergenceCriterion;
 import eu.stratosphere.pact.runtime.iterative.io.FakeOutputTask;
 import eu.stratosphere.pact.runtime.iterative.task.IterationHeadPactTask;
@@ -760,7 +752,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 		} else {
 			// create task vertex
 			vertex = new JobTaskVertex(taskName, this.jobGraph);
-			vertex.setTaskClass( (this.currentIteration != null && node.isOnDynamicPath()) ? IterationIntermediatePactTask.class : RegularPactTask.class);
+			vertex.setInvokableClass((this.currentIteration != null && node.isOnDynamicPath()) ? IterationIntermediatePactTask.class : RegularPactTask.class);
 			
 			config = new TaskConfig(vertex.getConfiguration());
 			config.setDriver(ds.getDriverClass());
@@ -786,7 +778,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 		final DriverStrategy ds = node.getDriverStrategy();
 		final JobTaskVertex vertex = new JobTaskVertex(taskName, this.jobGraph);
 		final TaskConfig config = new TaskConfig(vertex.getConfiguration());
-		vertex.setTaskClass( (this.currentIteration != null && node.isOnDynamicPath()) ? IterationIntermediatePactTask.class : RegularPactTask.class);
+		vertex.setInvokableClass( (this.currentIteration != null && node.isOnDynamicPath()) ? IterationIntermediatePactTask.class : RegularPactTask.class);
 		
 		// set user code
 		config.setStubWrapper(node.getPactContract().getUserCodeWrapper());
@@ -812,31 +804,29 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 
 	private JobInputVertex createDataSourceVertex(SourcePlanNode node) throws CompilerException {
 		final JobInputVertex vertex = new JobInputVertex(node.getNodeName(), this.jobGraph);
+		final TaskConfig config = new TaskConfig(vertex.getConfiguration());
 
-		// set task class
-		@SuppressWarnings("unchecked")
-		final Class<AbstractInputTask<?>> clazz = (Class<AbstractInputTask<?>>) (Class<?>) DataSourceTask
-				.class;
-		vertex.setInputClass(clazz);
+		vertex.setInvokableClass(DataSourceTask.class);
 
 		// set user code
-		vertex.setInputFormat((UserCodeWrapper<? extends InputFormat<?, InputSplit>>)node.getPactContract()
-				.getUserCodeWrapper());
-		vertex.setInputFormatParameters(node.getPactContract().getParameters());
-		vertex.setOutputSerializer(node.getSerializer());
+		config.setStubWrapper(node.getPactContract().getUserCodeWrapper());
+		config.setStubParameters(node.getPactContract().getParameters());
+
+		config.setOutputSerializer(node.getSerializer());
 		return vertex;
 	}
 
 	private AbstractJobOutputVertex createDataSinkVertex(SinkPlanNode node) throws CompilerException {
 		final JobOutputVertex vertex = new JobOutputVertex(node.getNodeName(), this.jobGraph);
+		final TaskConfig config = new TaskConfig(vertex.getConfiguration());
 
-		vertex.setOutputClass(DataSinkTask.class);
+		vertex.setInvokableClass(DataSinkTask.class);
 		vertex.getConfiguration().setInteger(DataSinkTask.DEGREE_OF_PARALLELISM_KEY, node.getDegreeOfParallelism());
 
 		// set user code
-		vertex.setOutputFormat((UserCodeWrapper<? extends OutputFormat<?>>)node.getPactContract().getUserCodeWrapper());
-		vertex.setOutputFormatParameters(node.getPactContract().getParameters());
-		
+		config.setStubWrapper(node.getPactContract().getUserCodeWrapper());
+		config.setStubParameters(node.getPactContract().getParameters());
+
 		return vertex;
 	}
 	
@@ -884,7 +874,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 			}
 			
 			// reset the vertex type to iteration head
-			headVertex.setTaskClass(IterationHeadPactTask.class);
+			headVertex.setInvokableClass(IterationHeadPactTask.class);
 			headConfig = new TaskConfig(headVertex.getConfiguration());
 			toReturn = null;
 		} else {
@@ -892,7 +882,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 			// 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.setTaskClass(IterationHeadPactTask.class);
+			headVertex.setInvokableClass(IterationHeadPactTask.class);
 			headConfig = new TaskConfig(headVertex.getConfiguration());
 			headConfig.setDriver(NoOpDriver.class);
 			toReturn = headVertex;
@@ -952,7 +942,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 			}
 			
 			// reset the vertex type to iteration head
-			headVertex.setTaskClass(IterationHeadPactTask.class);
+			headVertex.setInvokableClass(IterationHeadPactTask.class);
 			headConfig = new TaskConfig(headVertex.getConfiguration());
 			toReturn = null;
 		} else {
@@ -960,7 +950,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 			// 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.setTaskClass(IterationHeadPactTask.class);
+			headVertex.setInvokableClass(IterationHeadPactTask.class);
 			headConfig = new TaskConfig(headVertex.getConfiguration());
 			headConfig.setDriver(NoOpDriver.class);
 			toReturn = headVertex;
@@ -1144,7 +1134,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 		// --------------------------- create the sync task ---------------------------
 		final JobOutputVertex sync = new JobOutputVertex("Sync(" +
 					bulkNode.getNodeName() + ")", this.jobGraph);
-		sync.setOutputClass(IterationSynchronizationSinkTask.class);
+		sync.setInvokableClass(IterationSynchronizationSinkTask.class);
 		sync.setNumberOfSubtasks(1);
 		this.auxVertices.add(sync);
 		
@@ -1192,14 +1182,14 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 		// No following termination criterion
 		if(rootOfStepFunction.getOutgoingChannels().isEmpty()) {
 			
-			rootOfStepFunctionVertex.setTaskClass(IterationTailPactTask.class);
+			rootOfStepFunctionVertex.setInvokableClass(IterationTailPactTask.class);
 			
 			tailConfig.setOutputSerializer(bulkNode.getSerializerForIterationChannel());
 			tailConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
 			
 			// create the fake output task
 			JobOutputVertex fakeTail = new JobOutputVertex("Fake Tail", this.jobGraph);
-			fakeTail.setOutputClass(FakeOutputTask.class);
+			fakeTail.setInvokableClass(FakeOutputTask.class);
 			fakeTail.setNumberOfSubtasks(headVertex.getNumberOfSubtasks());
 			this.auxVertices.add(fakeTail);
 			
@@ -1234,14 +1224,14 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 				tailConfigOfTerminationCriterion = new TaskConfig(rootOfTerminationCriterionVertex.getConfiguration());
 			}
 			
-			rootOfTerminationCriterionVertex.setTaskClass(IterationTailPactTask.class);
+			rootOfTerminationCriterionVertex.setInvokableClass(IterationTailPactTask.class);
 			// Hack
 			tailConfigOfTerminationCriterion.setIsSolutionSetUpdate();
 			tailConfigOfTerminationCriterion.setOutputSerializer(bulkNode.getSerializerForIterationChannel());
 			tailConfigOfTerminationCriterion.addOutputShipStrategy(ShipStrategyType.FORWARD);
 			
 			JobOutputVertex fakeTailTerminationCriterion = new JobOutputVertex("Fake Tail for Termination Criterion", this.jobGraph);
-			fakeTailTerminationCriterion.setOutputClass(FakeOutputTask.class);
+			fakeTailTerminationCriterion.setInvokableClass(FakeOutputTask.class);
 			fakeTailTerminationCriterion.setNumberOfSubtasks(headVertex.getNumberOfSubtasks());
 			this.auxVertices.add(fakeTailTerminationCriterion);
 		
@@ -1309,7 +1299,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 		{
 			final JobOutputVertex sync = new JobOutputVertex("Sync (" +
 						iterNode.getNodeName() + ")", this.jobGraph);
-			sync.setOutputClass(IterationSynchronizationSinkTask.class);
+			sync.setInvokableClass(IterationSynchronizationSinkTask.class);
 			sync.setNumberOfSubtasks(1);
 			this.auxVertices.add(sync);
 			
@@ -1367,14 +1357,14 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 				worksetTailConfig.setIsWorksetUpdate();
 				
 				if (hasWorksetTail) {
-					nextWorksetVertex.setTaskClass(IterationTailPactTask.class);
+					nextWorksetVertex.setInvokableClass(IterationTailPactTask.class);
 					
 					worksetTailConfig.setOutputSerializer(iterNode.getWorksetSerializer());
 					worksetTailConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
 					
 					// create the fake output task
 					JobOutputVertex fakeTail = new JobOutputVertex("Fake Tail", this.jobGraph);
-					fakeTail.setOutputClass(FakeOutputTask.class);
+					fakeTail.setInvokableClass(FakeOutputTask.class);
 					fakeTail.setNumberOfSubtasks(headVertex.getNumberOfSubtasks());
 					this.auxVertices.add(fakeTail);
 					
@@ -1405,14 +1395,14 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 				solutionDeltaConfig.setIsSolutionSetUpdate();
 				
 				if (hasSolutionSetTail) {
-					solutionDeltaVertex.setTaskClass(IterationTailPactTask.class);
+					solutionDeltaVertex.setInvokableClass(IterationTailPactTask.class);
 					
 					solutionDeltaConfig.setOutputSerializer(iterNode.getSolutionSetSerializer());
 					solutionDeltaConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
 	
 					// create the fake output task
 					JobOutputVertex fakeTail = new JobOutputVertex("Fake Tail", this.jobGraph);
-					fakeTail.setOutputClass(FakeOutputTask.class);
+					fakeTail.setInvokableClass(FakeOutputTask.class);
 					fakeTail.setNumberOfSubtasks(headVertex.getNumberOfSubtasks());
 					this.auxVertices.add(fakeTail);
 					

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-core/src/main/java/eu/stratosphere/api/common/PlanExecutor.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/api/common/PlanExecutor.java b/stratosphere-core/src/main/java/eu/stratosphere/api/common/PlanExecutor.java
index 7caaab2..d91abf8 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/api/common/PlanExecutor.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/api/common/PlanExecutor.java
@@ -53,7 +53,6 @@ public abstract class PlanExecutor {
 	 * Creates an executor that runs the plan locally in a multi-threaded environment.
 	 * 
 	 * @return A local executor.
-	 * @see eu.stratosphere.client.LocalExecutor
 	 */
 	public static PlanExecutor createLocalExecutor() {
 		Class<? extends PlanExecutor> leClass = loadExecutorClass(LOCAL_EXECUTOR_CLASS);
@@ -75,7 +74,6 @@ public abstract class PlanExecutor {
 	 * @param jarFiles A list of jar files that contain the user-defined function (UDF) classes and all classes used
 	 *                 from within the UDFs.
 	 * @return A remote executor.
-	 * @see eu.stratosphere.client.RemoteExecutor
 	 */
 	public static PlanExecutor createRemoteExecutor(String hostname, int port, String... jarFiles) {
 		if (hostname == null) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/FileOutputFormat.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/FileOutputFormat.java b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/FileOutputFormat.java
index c4e1d5a..d43c987 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/FileOutputFormat.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/FileOutputFormat.java
@@ -31,14 +31,14 @@ import eu.stratosphere.core.fs.Path;
  * The abstract base class for all output formats that are file based. Contains the logic to open/close the target
  * file streams.
  */
-public abstract class FileOutputFormat<IT> implements OutputFormat<IT> {
+public abstract class FileOutputFormat<IT> implements OutputFormat<IT>, InitializeOnMaster {
+	
 	private static final long serialVersionUID = 1L;
 
 	// --------------------------------------------------------------------------------------------
 	
 	/**
-	 * Defines the behavior for creating output directories. 
-	 *
+	 * Behavior for creating output directories. 
 	 */
 	public static enum OutputDirectoryMode {
 		
@@ -54,7 +54,7 @@ public abstract class FileOutputFormat<IT> implements OutputFormat<IT> {
 
 	private static WriteMode DEFAULT_WRITE_MODE;
 	
-	private static  OutputDirectoryMode DEFAULT_OUTPUT_DIRECTORY_MODE;
+	private static OutputDirectoryMode DEFAULT_OUTPUT_DIRECTORY_MODE;
 	
 	
 	private static final void initDefaultsFromConfiguration() {
@@ -100,11 +100,6 @@ public abstract class FileOutputFormat<IT> implements OutputFormat<IT> {
 	 */
 	private OutputDirectoryMode outputDirectoryMode;
 	
-	/**
-	 * Stream opening timeout.
-	 */
-	private long openTimeout = -1;
-	
 	// --------------------------------------------------------------------------------------------
 	
 	/**
@@ -158,19 +153,6 @@ public abstract class FileOutputFormat<IT> implements OutputFormat<IT> {
 		return this.outputDirectoryMode;
 	}
 	
-	
-	public void setOpenTimeout(long timeout) {
-		if (timeout < 0) {
-			throw new IllegalArgumentException("The timeout must be a nonnegative numer of milliseconds (zero for infinite).");
-		}
-		
-		this.openTimeout = (timeout == 0) ? Long.MAX_VALUE : timeout;
-	}
-	
-	public long getOpenTimeout() {
-		return this.openTimeout;
-	}
-	
 	// ----------------------------------------------------------------
 
 	@Override
@@ -200,34 +182,58 @@ public abstract class FileOutputFormat<IT> implements OutputFormat<IT> {
 		if (this.outputDirectoryMode == null) {
 			this.outputDirectoryMode = DEFAULT_OUTPUT_DIRECTORY_MODE;
 		}
-		
-		if (this.openTimeout == -1) {
-			this.openTimeout = FileInputFormat.getDefaultOpeningTimeout();
-		}
 	}
 
 	
 	@Override
 	public void open(int taskNumber, int numTasks) throws IOException {
+		if (taskNumber < 0 || numTasks < 1) {
+			throw new IllegalArgumentException("TaskNumber: " + taskNumber + ", numTasks: " + numTasks);
+		}
 		
 		if (LOG.isDebugEnabled()) {
-			LOG.debug("Openint stream for output (" + (taskNumber+1) + "/" + numTasks + "). WriteMode=" + writeMode +
-					", OutputDirectoryMode=" + outputDirectoryMode + ", timeout=" + openTimeout);
+			LOG.debug("Opening stream for output (" + (taskNumber+1) + "/" + numTasks + "). WriteMode=" + writeMode +
+					", OutputDirectoryMode=" + outputDirectoryMode);
 		}
 		
-		// obtain FSDataOutputStream asynchronously, since HDFS client is vulnerable to InterruptedExceptions
-		OutputPathOpenThread opot = new OutputPathOpenThread(this, (taskNumber + 1), numTasks);
-		opot.start();
+		Path p = this.outputFilePath;
+		if (p == null) {
+			throw new IOException("The file path is null.");
+		}
 		
-		try {
-			// get FSDataOutputStream
-			this.stream = opot.waitForCompletion();
+		final FileSystem fs = p.getFileSystem();
+
+		// if this is a local file system, we need to initialize the local output directory here
+		if (!fs.isDistributedFS()) {
+			
+			if (numTasks == 1 && outputDirectoryMode == OutputDirectoryMode.PARONLY) {
+				// output should go to a single file
+				
+				// prepare local output path. checks for write mode and removes existing files in case of OVERWRITE mode
+				if(!fs.initOutPathLocalFS(p, writeMode, false)) {
+					// output preparation failed! Cancel task.
+					throw new IOException("Output path could not be initialized. Canceling task...");
+				}
+			}
+			else {
+				// numTasks > 1 || outDirMode == OutputDirectoryMode.ALWAYS
+				
+				if(!fs.initOutPathLocalFS(p, writeMode, true)) {
+					// output preparation failed! Cancel task.
+					throw new IOException("Output directory could not be created. Canceling task...");
+				}
+			}
 		}
-		catch (Exception e) {
-			throw new RuntimeException("Stream to output file could not be opened: " + e.getMessage(), e);
+			
+			
+		// Suffix the path with the parallel instance index, if needed
+		if (numTasks > 1 || outputDirectoryMode == OutputDirectoryMode.ALWAYS) {
+			p = p.suffix("/" + (taskNumber+1));
 		}
-	}
 
+		// create output file
+		this.stream = fs.create(p, writeMode == WriteMode.OVERWRITE);
+	}
 
 	@Override
 	public void close() throws IOException {
@@ -238,153 +244,37 @@ public abstract class FileOutputFormat<IT> implements OutputFormat<IT> {
 		}
 	}
 	
-	// ============================================================================================
-	
-	private static final class OutputPathOpenThread extends Thread {
-		
-		private final Path path;
-		
-		private final int taskIndex;
-		
-		private final int numTasks;
-		
-		private final WriteMode writeMode;
-		
-		private final OutputDirectoryMode outDirMode;
-		
-		private final long timeoutMillies;
-		
-		private volatile FSDataOutputStream fdos;
-
-		private volatile Throwable error;
-		
-		private volatile boolean aborted;
-
-		
-		public OutputPathOpenThread(FileOutputFormat<?> fof, int taskIndex, int numTasks) {
-			this.path = fof.getOutputFilePath();
-			this.writeMode = fof.getWriteMode();
-			this.outDirMode = fof.getOutputDirectoryMode();
-			this.timeoutMillies = fof.getOpenTimeout();
-			this.taskIndex = taskIndex;
-			this.numTasks = numTasks;
-		}
-
-		@Override
-		public void run() {
-
-			try {
-				Path p = this.path;
-				final FileSystem fs = p.getFileSystem();
-
-				// initialize output path. 
-				if(this.numTasks == 1 && outDirMode == OutputDirectoryMode.PARONLY) {
-					// output is not written in parallel and should go to a single file
-					
-					if(!fs.isDistributedFS()) {
-						// prepare local output path
-						// checks for write mode and removes existing files in case of OVERWRITE mode
-						if(!fs.initOutPathLocalFS(p, writeMode, false)) {
-							// output preparation failed! Cancel task.
-							throw new IOException("Output path could not be initialized. Canceling task.");
-						}
-					}
-					
-				} else if(this.numTasks > 1 || outDirMode == OutputDirectoryMode.ALWAYS) {
-					// output is written in parallel into a directory or should always be written to a directory
-					
-					if(!fs.isDistributedFS()) {
-						// File system is not distributed.
-						// We need to prepare the output path on each executing node.
-						if(!fs.initOutPathLocalFS(p, writeMode, true)) {
-							// output preparation failed! Cancel task.
-							throw new IOException("Output directory could not be created. Canceling task.");
-						}
-					}
-					
-					// Suffix the path with the parallel instance index
-					p = p.suffix("/" + this.taskIndex);
-					
-				} else {
-					// invalid number of subtasks (<= 0)
-					throw new IllegalArgumentException("Invalid number of subtasks. Canceling task.");
-				}
-					
-				// create output file
-				switch(writeMode) {
-				case NO_OVERWRITE: 
-					this.fdos = fs.create(p, false);
-					break;
-				case OVERWRITE:
-					this.fdos = fs.create(p, true);
-					break;
-				default:
-					throw new IllegalArgumentException("Invalid write mode: "+writeMode);
-				}
-				
-				// check for canceling and close the stream in that case, because no one will obtain it
-				if (this.aborted) {
-					final FSDataOutputStream f = this.fdos;
-					this.fdos = null;
-					f.close();
-				}
-			}
-			catch (Throwable t) {
-				this.error = t;
-			}
-		}
+	/**
+	 * Initialization of the distributed file system if it is used.
+	 *
+	 * @param parallelism The task parallelism.
+	 */
+	@Override
+	public void initializeGlobal(int parallelism) throws IOException {
+		final Path path = getOutputFilePath();
+		final FileSystem fs = path.getFileSystem();
 		
-		public FSDataOutputStream waitForCompletion() throws Exception {
-			final long start = System.currentTimeMillis();
-			long remaining = this.timeoutMillies;
+		// only distributed file systems can be initialized at start-up time.
+		if (fs.isDistributedFS()) {
 			
-			do {
-				try {
-					this.join(remaining);
-				} catch (InterruptedException iex) {
-					// we were canceled, so abort the procedure
-					abortWait();
-					throw iex;
+			final WriteMode writeMode = getWriteMode();
+			final OutputDirectoryMode outDirMode = getOutputDirectoryMode();
+
+			if (parallelism == 1 && outDirMode == OutputDirectoryMode.PARONLY) {
+				// output is not written in parallel and should be written to a single file.
+				// prepare distributed output path
+				if(!fs.initOutPathDistFS(path, writeMode, false)) {
+					// output preparation failed! Cancel task.
+					throw new IOException("Output path could not be initialized.");
 				}
-			}
-			while (this.error == null && this.fdos == null &&
-					(remaining = this.timeoutMillies + start - System.currentTimeMillis()) > 0);
-			
-			if (this.error != null) {
-				throw new IOException("Opening the file output stream failed" +
-					(this.error.getMessage() == null ? "." : ": " + this.error.getMessage()), this.error);
-			}
-			
-			if (this.fdos != null) {
-				return this.fdos;
+
 			} else {
-				// double-check that the stream has not been set by now. we don't know here whether
-				// a) the opener thread recognized the canceling and closed the stream
-				// b) the flag was set such that the stream did not see it and we have a valid stream
-				// In any case, close the stream and throw an exception.
-				abortWait();
-				
-				final boolean stillAlive = this.isAlive();
-				final StringBuilder bld = new StringBuilder(256);
-				for (StackTraceElement e : this.getStackTrace()) {
-					bld.append("\tat ").append(e.toString()).append('\n');
+				// output should be written to a directory
+
+				// only distributed file systems can be initialized at start-up time.
+				if(!fs.initOutPathDistFS(path, writeMode, true)) {
+					throw new IOException("Output directory could not be created.");
 				}
-				throw new IOException("Output opening request timed out. Opener was " + (stillAlive ? "" : "NOT ") + 
-					" alive. Stack:\n" + bld.toString());
-			}
-		}
-		
-		/**
-		 * Double checked procedure setting the abort flag and closing the stream.
-		 */
-		private final void abortWait() {
-			this.aborted = true;
-			final FSDataOutputStream outStream = this.fdos;
-			this.fdos = null;
-			if (outStream != null) {
-				try {
-					outStream.close();
-				} catch (Throwable t) {}
 			}
 		}
 	}
@@ -437,47 +327,4 @@ public abstract class FileOutputFormat<IT> implements OutputFormat<IT> {
 			super(targetConfig);
 		}
 	}
-
-	/**
-	 * Initialization of the distributed file system if it is used.
-	 *
-	 * @param configuration The task configuration
-	 */
-	@Override
-	public void initialize(Configuration configuration){
-		final Path path = this.getOutputFilePath();
-		final WriteMode writeMode = this.getWriteMode();
-		final OutputDirectoryMode outDirMode = this.getOutputDirectoryMode();
-
-		// Prepare output path and determine max DOP
-		try {
-			final FileSystem fs = path.getFileSystem();
-
-			int dop = configuration.getInteger(DEGREE_OF_PARALLELISM_KEY, -1);
-			if(dop == 1 && outDirMode == OutputDirectoryMode.PARONLY) {
-				// output is not written in parallel and should be written to a single file.
-
-				if(fs.isDistributedFS()) {
-					// prepare distributed output path
-					if(!fs.initOutPathDistFS(path, writeMode, false)) {
-						// output preparation failed! Cancel task.
-						throw new IOException("Output path could not be initialized.");
-					}
-				}
-			} else {
-				// output should be written to a directory
-
-				if(fs.isDistributedFS()) {
-					// only distributed file systems can be initialized at start-up time.
-					if(!fs.initOutPathDistFS(path, writeMode, true)) {
-						throw new IOException("Output directory could not be created.");
-					}
-				}
-			}
-		}
-		catch (IOException e) {
-			LOG.error("Could not access the file system to detemine the status of the output.", e);
-			throw new RuntimeException("I/O Error while accessing file", e);
-		}
-	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/FormatUtil.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/FormatUtil.java b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/FormatUtil.java
index f191c61..ec1033e 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/FormatUtil.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/FormatUtil.java
@@ -153,7 +153,6 @@ public class FormatUtil {
 	{
 		final F outputFormat = ReflectionUtil.newInstance(outputFormatClass);
 		outputFormat.setOutputFilePath(new Path(path));
-		outputFormat.setOpenTimeout(0);
 		outputFormat.setWriteMode(WriteMode.OVERWRITE);
 	
 		configuration = configuration == null ? new Configuration() : configuration;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/InitializeOnMaster.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/InitializeOnMaster.java b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/InitializeOnMaster.java
new file mode 100644
index 0000000..86fdee2
--- /dev/null
+++ b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/InitializeOnMaster.java
@@ -0,0 +1,35 @@
+/***********************************************************************************************************************
+ *
+ * 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.api.common.io;
+
+import java.io.IOException;
+
+/**
+ * This interface may be implemented by {@link OutputFormat}s to have the master initialize them globally.
+ * 
+ * For example, the {@link FileOutputFormat} implements this behavior for distributed file systems and
+ * creates/deletes target directories if necessary.
+ */
+public interface InitializeOnMaster {
+
+	/**
+	 * The method is invoked on the master (JobManager) before the distributed program execution starts.
+	 * 
+	 * @param parallelism The degree of parallelism with which the format or functions will be run.
+	 * @throws IOException The initialization may throw exceptions, which may cause the job to abort.
+	 */
+	void initializeGlobal(int parallelism) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/OutputFormat.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/OutputFormat.java b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/OutputFormat.java
index bdc59e4..72dddf4 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/OutputFormat.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/OutputFormat.java
@@ -20,14 +20,13 @@ import eu.stratosphere.configuration.Configuration;
 
 
 /**
- * Describes the base interface that is used describe an output that consumes records. The output format
+ * The base interface for outputs that consumes records. The output format
  * describes how to store the final records, for example in a file.
  * <p>
  * The life cycle of an output format is the following:
  * <ol>
- *   <li>After being instantiated (parameterless), it is configured with a {@link Configuration} object. 
- *       Basic fields are read from the configuration, such as for example a file path, if the format describes
- *       files as the sink for the records.</li>
+ *   <li>configure() is invoked a single time. The method can be used to implement initialization from
+ *       the parameters (configuration) that may be attached upon instantiation.</li>
  *   <li>Each parallel output task creates an instance, configures it and opens it.</li>
  *   <li>All records of its parallel instance are handed to the output format.</li>
  *   <li>The output format is closed</li>
@@ -79,13 +78,5 @@ public interface OutputFormat<IT> extends Serializable {
 	 * @throws IOException Thrown, if the input could not be closed properly.
 	 */
 	void close() throws IOException;
-
-	/**
-	 * Method which is called on the JobManager node prior to execution. It can be used to set up output format
-	 * related tasks.
-	 *
-	 * @param configuration The task configuration
-	 */
-	void initialize(Configuration configuration);
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-core/src/main/java/eu/stratosphere/configuration/Configuration.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/configuration/Configuration.java b/stratosphere-core/src/main/java/eu/stratosphere/configuration/Configuration.java
index 451577f..6b9436b 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/configuration/Configuration.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/configuration/Configuration.java
@@ -34,17 +34,19 @@ import eu.stratosphere.core.io.StringRecord;
  * This class is thread-safe.
  * 
  */
-public class Configuration implements IOReadableWritable {
+public class Configuration implements IOReadableWritable, java.io.Serializable {
+
+	private static final long serialVersionUID = 1L;
 
 	/**
 	 * Stores the concrete key/value pairs of this configuration object.
 	 */
-	private Map<String, String> confData = new HashMap<String, String>();
+	private final Map<String, String> confData = new HashMap<String, String>();
 
 	/**
 	 * The class loader to be used for the <code>getClass</code> method.
 	 */
-	private ClassLoader classLoader;
+	private transient ClassLoader classLoader;
 
 	/**
 	 * Constructs a new configuration object.
@@ -446,7 +448,6 @@ public class Configuration implements IOReadableWritable {
 	
 	// --------------------------------------------------------------------------------------------
 
-
 	@Override
 	public void read(final DataInput in) throws IOException {
 
@@ -479,6 +480,13 @@ public class Configuration implements IOReadableWritable {
 			}
 		}
 	}
+	
+	private void readObject(java.io.ObjectInputStream s) throws java.io.IOException, ClassNotFoundException {
+		s.defaultReadObject();
+		this.classLoader = getClass().getClassLoader();
+	}
+	
+	// --------------------------------------------------------------------------------------------
 
 	@Override
 	public int hashCode() {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-core/src/main/java/eu/stratosphere/core/fs/FileSystem.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/core/fs/FileSystem.java b/stratosphere-core/src/main/java/eu/stratosphere/core/fs/FileSystem.java
index 11c7007..8e65636 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/core/fs/FileSystem.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/core/fs/FileSystem.java
@@ -459,7 +459,7 @@ public abstract class FileSystem {
 			// path exists, check write mode
 			switch (writeMode) {
 			case NO_OVERWRITE:
-				if (status.isDir()) {
+				if (status.isDir() && createDirectory) {
 					return true;
 				} else {
 					// file may not be overwritten
@@ -467,6 +467,7 @@ public abstract class FileSystem {
 							WriteMode.NO_OVERWRITE.name() + " mode. Use " + WriteMode.OVERWRITE.name() + 
 							" mode to overwrite existing files and directories.");
 				}
+
 			case OVERWRITE:
 				if (status.isDir()) {
 					if (createDirectory) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-core/src/main/java/eu/stratosphere/core/io/StringRecord.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/core/io/StringRecord.java b/stratosphere-core/src/main/java/eu/stratosphere/core/io/StringRecord.java
index 50c2599..de2358b 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/core/io/StringRecord.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/core/io/StringRecord.java
@@ -34,6 +34,8 @@ import java.text.CharacterIterator;
 import java.text.StringCharacterIterator;
 import java.util.Arrays;
 
+import eu.stratosphere.types.Value;
+
 /**
  * This class stores text using standard UTF8 encoding. It provides methods to
  * serialize, deserialize, and compare texts at byte level. The type of length
@@ -44,7 +46,9 @@ import java.util.Arrays;
  * Also includes utilities for serializing/deserialing a string, coding/decoding a string, checking if a byte array
  * contains valid UTF8 code, calculating the length of an encoded string.
  */
-public class StringRecord implements IOReadableWritable {
+public class StringRecord implements Value {
+	
+	private static final long serialVersionUID = 1L;
 
 	private static final ThreadLocal<CharsetEncoder> ENCODER_FACTORY = new ThreadLocal<CharsetEncoder>() {
 		protected CharsetEncoder initialValue() {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-core/src/main/java/eu/stratosphere/util/IterableIterator.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/util/IterableIterator.java b/stratosphere-core/src/main/java/eu/stratosphere/util/IterableIterator.java
index 16f610a..b59e2e6 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/util/IterableIterator.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/util/IterableIterator.java
@@ -18,9 +18,9 @@ package eu.stratosphere.util;
 import java.util.Iterator;
 
 /**
- * An {@link Iterator] that is also {@link Iterable} (often by returning itself).
+ * An {@link Iterator} that is also {@link Iterable} (often by returning itself).
  * 
- * @param <T> The iterated elements' type.
+ * @param <E> The iterated elements' type.
  */
 public interface IterableIterator<E> extends Iterator<E>, Iterable<E> {
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-java/src/main/java/eu/stratosphere/api/java/io/PrintingOutputFormat.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/io/PrintingOutputFormat.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/io/PrintingOutputFormat.java
index d1736d4..5c09439 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/io/PrintingOutputFormat.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/io/PrintingOutputFormat.java
@@ -95,7 +95,4 @@ public class PrintingOutputFormat<T> implements OutputFormat<T> {
 	public String toString() {
 		return "Print to " + (target == STD_OUT ? "System.out" : "System.err");
 	}
-
-	@Override
-	public void initialize(Configuration configuration){}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java
index 2416b07..ae5198a 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java
@@ -43,7 +43,6 @@ import org.apache.commons.logging.LogFactory;
 import java.io.IOException;
 import java.util.ArrayDeque;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java
index 1c4a820..18395fb 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java
@@ -31,11 +31,13 @@ import java.util.concurrent.atomic.AtomicReference;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
+import eu.stratosphere.api.common.io.InitializeOnMaster;
+import eu.stratosphere.api.common.io.OutputFormat;
 import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.configuration.IllegalConfigurationException;
 import eu.stratosphere.core.io.InputSplit;
 import eu.stratosphere.nephele.execution.ExecutionListener;
 import eu.stratosphere.nephele.execution.ExecutionState;
+import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
 import eu.stratosphere.nephele.instance.AllocatedResource;
 import eu.stratosphere.nephele.instance.DummyInstance;
 import eu.stratosphere.nephele.jobgraph.DistributionPattern;
@@ -45,11 +47,11 @@ import eu.stratosphere.runtime.io.channels.ChannelType;
 import eu.stratosphere.nephele.jobgraph.AbstractJobInputVertex;
 import eu.stratosphere.nephele.jobgraph.AbstractJobVertex;
 import eu.stratosphere.nephele.jobgraph.JobEdge;
-import eu.stratosphere.nephele.jobgraph.JobFileOutputVertex;
 import eu.stratosphere.nephele.jobgraph.JobGraph;
 import eu.stratosphere.nephele.jobgraph.JobID;
+import eu.stratosphere.nephele.jobgraph.JobInputVertex;
+import eu.stratosphere.nephele.jobgraph.JobOutputVertex;
 import eu.stratosphere.nephele.taskmanager.ExecutorThreadFactory;
-import eu.stratosphere.nephele.template.AbstractInputTask;
 import eu.stratosphere.nephele.template.AbstractInvokable;
 import eu.stratosphere.util.StringUtils;
 
@@ -462,42 +464,68 @@ public class ExecutionGraph implements ExecutionListener {
 					: false, jobVertex.getNumberOfExecutionRetries(), jobVertex.getConfiguration(), signature,
 				invokableClass);
 		} catch (Throwable t) {
-			throw new GraphConversionException(StringUtils.stringifyException(t));
+			throw new GraphConversionException(t);
 		}
 
 		// Register input and output vertices separately
 		if (jobVertex instanceof AbstractJobInputVertex) {
 
-			final InputSplit[] inputSplits;
-
+			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;
 
-			try{
+			try {
 				inputSplits = jobInputVertex.getInputSplits(jobVertex.getNumberOfSubtasks());
-			}catch(Exception e) {
-				throw new GraphConversionException("Cannot compute input splits for " + groupVertex.getName() + ": "
-						+ StringUtils.stringifyException(e));
+			}
+			catch (Throwable t) {
+				throw new GraphConversionException("Cannot compute input splits for " + groupVertex.getName(), t);
 			}
 
 			if (inputSplits == null) {
-				LOG.info("Job input vertex " + jobVertex.getName() + " generated 0 input splits");
-			} else {
-				LOG.info("Job input vertex " + jobVertex.getName() + " generated " + inputSplits.length
-					+ " input splits");
+				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){
+		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);
+			}
 
-			final OutputFormat<?> outputFormat = jobOutputVertex.getOutputFormat();
-
-			if(outputFormat != null){
-				outputFormat.initialize(groupVertex.getConfiguration());
+			OutputFormat<?> outputFormat = jobOutputVertex.getOutputFormat();
+			if (outputFormat != null && outputFormat instanceof InitializeOnMaster){
+				try {
+					((InitializeOnMaster) outputFormat).initializeGlobal(jobVertex.getNumberOfSubtasks());
+				}
+				catch (Throwable t) {
+					throw new GraphConversionException(t);
+				}
 			}
 		}
 
@@ -519,7 +547,6 @@ public class ExecutionGraph implements ExecutionListener {
 	 * @return the number of input vertices registered with this execution graph
 	 */
 	public int getNumberOfInputVertices() {
-
 		return this.stages.get(0).getNumberOfInputExecutionVertices();
 	}
 
@@ -531,7 +558,6 @@ public class ExecutionGraph implements ExecutionListener {
 	 * @return the number of input vertices for the given stage
 	 */
 	public int getNumberOfInputVertices(int stage) {
-
 		if (stage >= this.stages.size()) {
 			return 0;
 		}
@@ -545,7 +571,6 @@ public class ExecutionGraph implements ExecutionListener {
 	 * @return the number of output vertices registered with this execution graph
 	 */
 	public int getNumberOfOutputVertices() {
-
 		return this.stages.get(0).getNumberOfOutputExecutionVertices();
 	}
 
@@ -557,7 +582,6 @@ public class ExecutionGraph implements ExecutionListener {
 	 * @return the number of input vertices for the given stage
 	 */
 	public int getNumberOfOutputVertices(final int stage) {
-
 		if (stage >= this.stages.size()) {
 			return 0;
 		}
@@ -574,7 +598,6 @@ public class ExecutionGraph implements ExecutionListener {
 	 *         exists
 	 */
 	public ExecutionVertex getInputVertex(final int index) {
-
 		return this.stages.get(0).getInputExecutionVertex(index);
 	}
 
@@ -587,7 +610,6 @@ public class ExecutionGraph implements ExecutionListener {
 	 *         exists
 	 */
 	public ExecutionVertex getOutputVertex(final int index) {
-
 		return this.stages.get(0).getOutputExecutionVertex(index);
 	}
 
@@ -602,7 +624,6 @@ public class ExecutionGraph implements ExecutionListener {
 	 *         exists in that stage
 	 */
 	public ExecutionVertex getInputVertex(final int stage, final int index) {
-
 		try {
 			final ExecutionStage s = this.stages.get(stage);
 			if (s == null) {
@@ -627,7 +648,6 @@ public class ExecutionGraph implements ExecutionListener {
 	 *         exists in that stage
 	 */
 	public ExecutionVertex getOutputVertex(final int stage, final int index) {
-
 		try {
 			final ExecutionStage s = this.stages.get(stage);
 			if (s == null) {
@@ -649,7 +669,6 @@ public class ExecutionGraph implements ExecutionListener {
 	 * @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) {
@@ -663,7 +682,6 @@ public class ExecutionGraph implements ExecutionListener {
 	 * @return the number of execution stages in the execution graph
 	 */
 	public int getNumberOfStages() {
-
 		return this.stages.size();
 	}
 
@@ -676,7 +694,6 @@ public class ExecutionGraph implements ExecutionListener {
 	 *         exists in the execution graph
 	 */
 	public ExecutionVertex getVertexByChannelID(final ChannelID id) {
-
 		final ExecutionEdge edge = this.edgeMap.get(id);
 		if (edge == null) {
 			return null;
@@ -697,7 +714,6 @@ public class ExecutionGraph implements ExecutionListener {
 	 * @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);
 	}
 
@@ -708,7 +724,6 @@ public class ExecutionGraph implements ExecutionListener {
 	 *        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");
@@ -724,7 +739,6 @@ public class ExecutionGraph implements ExecutionListener {
 	 *         found
 	 */
 	public ExecutionVertex getVertexByID(final ExecutionVertexID id) {
-
 		return this.vertexMap.get(id);
 	}
 
@@ -735,7 +749,6 @@ public class ExecutionGraph implements ExecutionListener {
 	 * @return <code>true</code> if stage is completed, <code>false</code> otherwise
 	 */
 	private boolean isCurrentStageCompleted() {
-
 		if (this.indexToCurrentExecutionStage >= this.stages.size()) {
 			return true;
 		}
@@ -758,7 +771,6 @@ public class ExecutionGraph implements ExecutionListener {
 	 * @return <code>true</code> if the execution of the graph is finished, <code>false</code> otherwise
 	 */
 	public boolean isExecutionFinished() {
-
 		return (getJobStatus() == InternalJobStatus.FINISHED);
 	}
 
@@ -1307,4 +1319,26 @@ public class ExecutionGraph implements ExecutionListener {
 			}
 		}
 	}
+	
+	/**
+	 * 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/8c1d82a8/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java
index 91e9e53..dceeb90 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java
@@ -15,7 +15,6 @@ package eu.stratosphere.nephele.executiongraph;
 
 import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.core.io.InputSplit;
-import eu.stratosphere.nephele.execution.RuntimeEnvironment;
 import eu.stratosphere.nephele.instance.AllocatedResource;
 import eu.stratosphere.nephele.instance.DummyInstance;
 import eu.stratosphere.nephele.jobgraph.JobVertexID;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobInputVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobInputVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobInputVertex.java
index e4d3b9d..b901742 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobInputVertex.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobInputVertex.java
@@ -15,11 +15,8 @@ package eu.stratosphere.nephele.jobgraph;
 
 import eu.stratosphere.core.io.InputSplit;
 
-import java.io.IOException;
-
 /**
- * An abstract base class for input vertices in Nephele.
- * 
+ * An abstract base class for input vertices.
  */
 public abstract class AbstractJobInputVertex extends AbstractJobVertex {
 
@@ -28,12 +25,24 @@ public abstract class AbstractJobInputVertex extends AbstractJobVertex {
 	 * 
 	 * @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(final String name, final JobVertexID id, final JobGraph jobGraph) {
+	protected AbstractJobInputVertex(String name, JobVertexID id, JobGraph jobGraph) {
 		super(name, id, jobGraph);
 
 		jobGraph.addVertex(this);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobOutputVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobOutputVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobOutputVertex.java
index 849df4b..6020f24 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobOutputVertex.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobOutputVertex.java
@@ -24,14 +24,15 @@ public abstract class AbstractJobOutputVertex extends AbstractJobVertex {
 	 * 
 	 * @param name
 	 *        the name of the new job output vertex
-	 * @param id
-	 *        the ID of this vertex
 	 * @param jobGraph
 	 *        the job graph this vertex belongs to
 	 */
-	protected AbstractJobOutputVertex(final String name, final JobVertexID id, final JobGraph jobGraph) {
+	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/8c1d82a8/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java
index cdadd3c..cc936d9 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java
@@ -18,8 +18,9 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.util.ArrayList;
 
+import org.apache.commons.lang.Validate;
+
 import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.configuration.IllegalConfigurationException;
 import eu.stratosphere.core.io.IOReadableWritable;
 import eu.stratosphere.core.io.StringRecord;
 import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
@@ -29,8 +30,7 @@ import eu.stratosphere.nephele.util.EnumUtils;
 import eu.stratosphere.util.StringUtils;
 
 /**
- * An abstract base class for a job vertex in Nephele.
- * 
+ * An abstract base class for a job vertex.
  */
 public abstract class AbstractJobVertex implements IOReadableWritable {
 
@@ -86,19 +86,30 @@ public abstract class AbstractJobVertex implements IOReadableWritable {
 	 */
 	protected Class<? extends AbstractInvokable> invokableClass = null;
 
+	
 	/**
 	 * Constructs a new job vertex and assigns it with the given name.
 	 * 
 	 * @param name
 	 *        the name of the new job vertex
-	 * @param id
-	 *        the ID of this vertex
 	 * @param jobGraph
 	 *        the job graph this vertex belongs to
 	 */
-	protected AbstractJobVertex(final String name, final JobVertexID id, final JobGraph jobGraph) {
+	protected AbstractJobVertex(String name, JobGraph jobGraph) {
+		this(name, null, jobGraph);
+	}
+	
+	/**
+	 * 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
+	 */
+	protected AbstractJobVertex(String name, JobVertexID id, JobGraph jobGraph) {
 		this.name = name == null ? DEFAULT_NAME : name;
-		this.id = (id == null) ? new JobVertexID() : id;
+		this.id = id == null ? new JobVertexID() : id;
 		this.jobGraph = jobGraph;
 	}
 
@@ -572,13 +583,17 @@ public abstract class AbstractJobVertex implements IOReadableWritable {
 		return this.configuration;
 	}
 
+	public void setInvokableClass(Class<? extends AbstractInvokable> invokable) {
+		Validate.notNull(invokable);
+		this.invokableClass = invokable;
+	}
+	
 	/**
 	 * Returns the invokable class which represents the task of this vertex
 	 * 
 	 * @return the invokable class, <code>null</code> if it is not set
 	 */
 	public Class<? extends AbstractInvokable> getInvokableClass() {
-
 		return this.invokableClass;
 	}
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobGraph.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobGraph.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobGraph.java
index f048b0d..3d14d0a 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobGraph.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobGraph.java
@@ -26,8 +26,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.Stack;
-import java.util.Vector;
 
 import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.core.fs.FSDataInputStream;
@@ -77,11 +75,6 @@ public class JobGraph implements IOReadableWritable {
 	private Configuration jobConfiguration = new Configuration();
 
 	/**
-	 * The configuration which should be applied to the task managers involved in processing this job.
-	 */
-	private final Configuration taskManagerConfiguration = new Configuration();
-
-	/**
 	 * List of JAR files required to run this job.
 	 */
 	private final ArrayList<Path> userJars = new ArrayList<Path>();
@@ -134,24 +127,12 @@ public class JobGraph implements IOReadableWritable {
 	}
 
 	/**
-	 * Returns the configuration object distributed among the task managers
-	 * before they start processing this job.
-	 * 
-	 * @return the configuration object for the task managers, or <code>null</code> if it is not set
-	 */
-	public Configuration getTaskmanagerConfiguration() {
-
-		return this.taskManagerConfiguration;
-	}
-
-	/**
 	 * 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(final AbstractJobInputVertex inputVertex) {
-
+	public void addVertex(AbstractJobInputVertex inputVertex) {
 		if (!inputVertices.containsKey(inputVertex.getID())) {
 			inputVertices.put(inputVertex.getID(), inputVertex);
 		}
@@ -163,8 +144,7 @@ public class JobGraph implements IOReadableWritable {
 	 * @param taskVertex
 	 *        the new task vertex to be added
 	 */
-	public void addVertex(final JobTaskVertex taskVertex) {
-
+	public void addVertex(JobTaskVertex taskVertex) {
 		if (!taskVertices.containsKey(taskVertex.getID())) {
 			taskVertices.put(taskVertex.getID(), taskVertex);
 		}
@@ -176,8 +156,7 @@ public class JobGraph implements IOReadableWritable {
 	 * @param outputVertex
 	 *        the new output vertex to be added
 	 */
-	public void addVertex(final AbstractJobOutputVertex outputVertex) {
-
+	public void addVertex(AbstractJobOutputVertex outputVertex) {
 		if (!outputVertices.containsKey(outputVertex.getID())) {
 			outputVertices.put(outputVertex.getID(), outputVertex);
 		}
@@ -570,9 +549,6 @@ public class JobGraph implements IOReadableWritable {
 		// Re-instantiate the job configuration object and read the configuration
 		this.jobConfiguration = new Configuration(cl);
 		this.jobConfiguration.read(in);
-
-		// Read the task manager configuration
-		this.taskManagerConfiguration.read(in);
 	}
 
 
@@ -610,7 +586,6 @@ public class JobGraph implements IOReadableWritable {
 
 		// Write out configuration objects
 		this.jobConfiguration.write(out);
-		this.taskManagerConfiguration.write(out);
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobInputVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobInputVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobInputVertex.java
index 29f98d9..bf8f544 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobInputVertex.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobInputVertex.java
@@ -13,41 +13,21 @@
 
 package eu.stratosphere.nephele.jobgraph;
 
+import java.io.IOException;
+
 import eu.stratosphere.api.common.io.InputFormat;
-import eu.stratosphere.api.common.operators.util.UserCodeObjectWrapper;
 import eu.stratosphere.api.common.operators.util.UserCodeWrapper;
-import eu.stratosphere.api.common.typeutils.TypeSerializerFactory;
-import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.core.io.InputSplit;
-import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
-import eu.stratosphere.nephele.template.AbstractInputTask;
 import eu.stratosphere.pact.runtime.task.util.TaskConfig;
 
-import java.io.DataInput;
-import java.io.IOException;
-
 public class JobInputVertex extends AbstractJobInputVertex {
-	/**
-	 * Input format associated to this JobInputVertex. It is either directly set or reconstructed from the task
-	 * configuration. Every job input vertex requires an input format to compute the input splits and the input split
-	 * type.
-	 */
-	private volatile InputFormat<?, ? extends InputSplit> inputFormat = null;
 
-	/**
-	 * Creates a new job input vertex with the specified name.
-	 * 
-	 * @param name
-	 *        The name of the new job file input vertex.
-	 * @param id
-	 *        The ID of this vertex.
-	 * @param jobGraph
-	 *        The job graph this vertex belongs to.
-	 */
-	public JobInputVertex(final String name, final JobVertexID id, final JobGraph jobGraph) {
+	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.
 	 * 
@@ -56,8 +36,8 @@ public class JobInputVertex extends AbstractJobInputVertex {
 	 * @param jobGraph
 	 *        The job graph this vertex belongs to.
 	 */
-	public JobInputVertex(final String name, final JobGraph jobGraph) {
-		super(name, null, jobGraph);
+	public JobInputVertex(String name, JobGraph jobGraph) {
+		this(name, null, jobGraph);
 	}
 
 	/**
@@ -66,112 +46,23 @@ public class JobInputVertex extends AbstractJobInputVertex {
 	 * @param jobGraph
 	 *        The job graph this vertex belongs to.
 	 */
-	public JobInputVertex(final JobGraph jobGraph) {
-		super(null, null, jobGraph);
-	}
-
-	/**
-	 * Sets the class of the vertex's input task.
-	 *
-	 * @param inputClass
-	 *        The class of the vertex's input task.
-	 */
-	public void setInputClass(final Class<? extends AbstractInputTask<?>> inputClass) {
-		this.invokableClass = inputClass;
-	}
-
-	/**
-	 * Returns the class of the vertex's input task.
-	 * 
-	 * @return the class of the vertex's input task or <code>null</code> if no task has yet been set
-	 */
-	@SuppressWarnings("unchecked")
-	public Class<? extends AbstractInputTask<?>> getInputClass() {
-		return (Class<? extends AbstractInputTask<?>>) this.invokableClass;
-	}
-
-	/**
-	 * Sets the input format and writes it to the task configuration. It extracts it from the UserCodeWrapper.
-	 *
-	 * @param inputFormatWrapper Wrapped input format
-	 */
-	public void setInputFormat(UserCodeWrapper<? extends InputFormat<?, ? extends InputSplit>> inputFormatWrapper) {
-		TaskConfig config = new TaskConfig(this.getConfiguration());
-		config.setStubWrapper(inputFormatWrapper);
-
-		inputFormat = inputFormatWrapper.getUserCodeObject();
-	}
-
-	/**
-	 * Sets the input format and writes it to the task configuration.
-	 *
-	 * @param inputFormat Input format
-	 */
-	public void setInputFormat(InputFormat<?, ? extends InputSplit> inputFormat) {
-		this.inputFormat = inputFormat;
-
-		UserCodeWrapper<? extends InputFormat<?, ? extends InputSplit>> wrapper = new
-				UserCodeObjectWrapper<InputFormat<?, ? extends InputSplit>>(inputFormat);
-		TaskConfig config = new TaskConfig(this.getConfiguration());
-		config.setStubWrapper(wrapper);
+	public JobInputVertex(JobGraph jobGraph) {
+		this(null, jobGraph);
 	}
-
-	/**
-	 * Sets the input format parameters.
-	 *
-	 * @param inputFormatParameters Input format parameters
-	 */
-	public void setInputFormatParameters(Configuration inputFormatParameters){
-		TaskConfig config = new TaskConfig(this.getConfiguration());
-		config.setStubParameters(inputFormatParameters);
-
-		if(inputFormat == null){
-			throw new RuntimeException("There is no input format set in job vertex: " + this.getID());
-		}
-
-		inputFormat.configure(inputFormatParameters);
+	
+	public void setInputFormat(InputFormat<?, ?> format) {
+		this.inputFormat = format;
 	}
-
-	/**
-	 * Sets the output serializer for the task associated to this vertex.
-	 *
-	 * @param factory Type serializer factory
-	 */
-	public void setOutputSerializer(TypeSerializerFactory<?> factory){
-		TaskConfig config = new TaskConfig(this.getConfiguration());
-		config.setOutputSerializer(factory);
-	}
-
-	/**
-	 * Deserializes the input format from the deserialized task configuration. It then configures the input format by
-	 * calling the configure method with the current configuration.
-	 *
-	 * @param input
-	 * @throws IOException
-	 */
-	@Override
-	public void read(final DataInput input) throws IOException{
-		super.read(input);
-
-		// load input format wrapper from the config
-		ClassLoader cl = null;
-
-		try{
-			cl = LibraryCacheManager.getClassLoader(this.getJobGraph().getJobID());
+	
+	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());
 		}
-		catch (IOException ioe) {
-			throw new RuntimeException("Usercode ClassLoader could not be obtained for job: " +
-					this.getJobGraph().getJobID(), ioe);
-		}
-
-		final Configuration config = this.getConfiguration();
-		config.setClassLoader(cl);
-		final TaskConfig taskConfig = new TaskConfig(config);
-
-		inputFormat = taskConfig.<InputFormat<?, InputSplit>>getStubWrapper(cl).getUserCodeObject(InputFormat.class,
-				cl);
-
-		inputFormat.configure(taskConfig.getStubParameters());
 	}
 
 	/**
@@ -197,7 +88,7 @@ public class JobInputVertex extends AbstractJobInputVertex {
 	 */
 	@Override
 	public InputSplit[] getInputSplits(int minNumSplits) throws IOException {
-		if(inputFormat == null){
+		if (inputFormat == null){
 			throw new RuntimeException("No input format has been set for job vertex: "+ this.getID());
 		}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobOutputVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobOutputVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobOutputVertex.java
index cf937a0..abe6be9 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobOutputVertex.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobOutputVertex.java
@@ -14,41 +14,20 @@
 package eu.stratosphere.nephele.jobgraph;
 
 import eu.stratosphere.api.common.io.OutputFormat;
-import eu.stratosphere.api.common.operators.util.UserCodeObjectWrapper;
 import eu.stratosphere.api.common.operators.util.UserCodeWrapper;
-import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
-import eu.stratosphere.nephele.template.AbstractOutputTask;
 import eu.stratosphere.pact.runtime.task.util.TaskConfig;
 
-import java.io.DataInput;
-import java.io.IOException;
-
 /**
- * A JobOutputVertex is a specific subtype of a {@link AbstractJobOutputVertex} and is designed
+ * 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 volatile OutputFormat<?> outputFormat = null;
+	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 id
-	 *        the ID of this vertex
-	 * @param jobGraph
-	 *        the job graph this vertex belongs to
-	 */
-	public JobOutputVertex(final String name, final JobVertexID id, final JobGraph jobGraph) {
-		super(name, id, jobGraph);
-	}
 
 	/**
 	 * Creates a new job file output vertex with the specified name.
@@ -58,8 +37,12 @@ public class JobOutputVertex extends AbstractJobOutputVertex {
 	 * @param jobGraph
 	 *        the job graph this vertex belongs to
 	 */
-	public JobOutputVertex(final String name, final JobGraph jobGraph) {
-		super(name, null, jobGraph);
+	public JobOutputVertex(String name, JobGraph jobGraph) {
+		this(name, null, jobGraph);
+	}
+	
+	public JobOutputVertex(String name, JobVertexID id, JobGraph jobGraph) {
+		super(name, id, jobGraph);
 	}
 
 	/**
@@ -68,94 +51,21 @@ public class JobOutputVertex extends AbstractJobOutputVertex {
 	 * @param jobGraph
 	 *        the job graph this vertex belongs to
 	 */
-	public JobOutputVertex(final JobGraph jobGraph) {
-		super(null, null, jobGraph);
-	}
-
-	/**
-	 * Sets the class of the vertex's output task.
-	 * 
-	 * @param outputClass
-	 *        The class of the vertex's output task.
-	 */
-	public void setOutputClass(final Class<? extends AbstractOutputTask> outputClass) {
-		this.invokableClass = outputClass;
+	public JobOutputVertex(JobGraph jobGraph) {
+		this(null, jobGraph);
 	}
-
-	/**
-	 * Returns the class of the vertex's output task.
-	 * 
-	 * @return The class of the vertex's output task or <code>null</code> if no task has yet been set.
-	 */
-	@SuppressWarnings("unchecked")
-	public Class<? extends AbstractOutputTask> getOutputClass() {
-		return (Class<? extends AbstractOutputTask>) this.invokableClass;
+	
+	public void setOutputFormat(OutputFormat<?> format) {
+		this.outputFormat = format;
 	}
-
-	/**
-	 * Sets the output format and writes it to the task configuration.
-	 *
-	 * @param outputFormatWrapper Wrapped output format
-	 */
-	public void setOutputFormat(UserCodeWrapper<? extends OutputFormat<?>> outputFormatWrapper){
-		TaskConfig config = new TaskConfig(this.getConfiguration());
-		config.setStubWrapper(outputFormatWrapper);
-		outputFormat = outputFormatWrapper.getUserCodeObject();
-	}
-
-	/**
-	 * Sets the output format and writes it to the task configuration.
-	 *
-	 * @param outputFormat Output format
-	 */
-	public void setOutputFormat(OutputFormat<?> outputFormat){
-		this.outputFormat = outputFormat;
-		UserCodeWrapper<? extends OutputFormat<?>> wrapper = new UserCodeObjectWrapper<OutputFormat<?>>
-				(outputFormat);
-		TaskConfig config = new TaskConfig(this.getConfiguration());
-		config.setStubWrapper(wrapper);
-	}
-
-	/**
-	 * Sets the output format parameters for the output format by writing it to the task configuration.
-	 *
-	 * @param parameters Output format parameters
-	 */
-	public void setOutputFormatParameters(Configuration parameters){
-		TaskConfig config = new TaskConfig(this.getConfiguration());
-		config.setStubParameters(parameters);
-
-		outputFormat.configure(parameters);
-	}
-
-	/**
-	 * Deserializes the output format from the deserialized configuration if it contains an output format. The output
-	 * format is always stored in the stub wrapper. If the task configuration contains an output format,
-	 * then it is configured after deserialization.
-	 *
-	 * @param input
-	 * @throws IOException
-	 */
-	@Override
-	public void read(final DataInput input) throws IOException{
-		super.read(input);
-
-		ClassLoader cl = null;
-		try{
-			cl = LibraryCacheManager.getClassLoader(this.getJobGraph().getJobID());
-		}
-		catch (IOException ioe) {
-			throw new RuntimeException("Usercode ClassLoader could not be obtained for job: " +
-					this.getJobGraph().getJobID(), ioe);
-		}
-
-		final Configuration config = this.getConfiguration();
-		config.setClassLoader(cl);
-		final TaskConfig taskConfig = new TaskConfig(config);
-
-		if(taskConfig.hasStubWrapper()){
-			outputFormat = taskConfig.<OutputFormat<?> >getStubWrapper(cl).getUserCodeObject(OutputFormat.class,cl);
-			outputFormat.configure(taskConfig.getStubParameters());
+	
+	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());
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobTaskVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobTaskVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobTaskVertex.java
index 8672aeb..d16286c 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobTaskVertex.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobTaskVertex.java
@@ -13,8 +13,6 @@
 
 package eu.stratosphere.nephele.jobgraph;
 
-import eu.stratosphere.nephele.template.AbstractTask;
-
 /**
  * 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.
@@ -27,28 +25,15 @@ public class JobTaskVertex extends AbstractJobVertex {
 	 * 
 	 * @param name
 	 *        the name for the new job task vertex
-	 * @param id
-	 *        the ID of this vertex
 	 * @param jobGraph
 	 *        the job graph this vertex belongs to
 	 */
-	public JobTaskVertex(final String name, final JobVertexID id, final JobGraph jobGraph) {
-		super(name, id, jobGraph);
-
-		jobGraph.addVertex(this);
+	public JobTaskVertex(String name, JobGraph jobGraph) {
+		this(name, null, jobGraph);
 	}
-
-	/**
-	 * 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(final String name, final JobGraph jobGraph) {
-		super(name, null, jobGraph);
-
+	
+	public JobTaskVertex(String name, JobVertexID id, JobGraph jobGraph) {
+		super(name, id, jobGraph);
 		jobGraph.addVertex(this);
 	}
 
@@ -58,29 +43,7 @@ public class JobTaskVertex extends AbstractJobVertex {
 	 * @param jobGraph
 	 *        the job graph this vertex belongs to
 	 */
-	public JobTaskVertex(final JobGraph jobGraph) {
-		super(null, null, jobGraph);
-
-		jobGraph.addVertex(this);
-	}
-
-	/**
-	 * Sets the class of the vertex's task.
-	 * 
-	 * @param taskClass
-	 *        the class of the vertex's task
-	 */
-	public void setTaskClass(final Class<? extends AbstractTask> taskClass) {
-		this.invokableClass = taskClass;
-	}
-
-	/**
-	 * Returns the class of the vertex's task.
-	 * 
-	 * @return the class of the vertex's task or <code>null</code> if the class has not yet been set
-	 */
-	@SuppressWarnings("unchecked")
-	public Class<? extends AbstractTask> getTaskClass() {
-		return (Class<? extends AbstractTask>) this.invokableClass;
+	public JobTaskVertex(JobGraph jobGraph) {
+		this(null, jobGraph);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java
index 8a3cba4..f3cf3a3 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java
@@ -393,9 +393,7 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 		try {
 			// First check if job is null
 			if (job == null) {
-				JobSubmissionResult result = new JobSubmissionResult(AbstractJobResult.ReturnCode.ERROR,
-					"Submitted job is null!");
-				return result;
+				return new JobSubmissionResult(AbstractJobResult.ReturnCode.ERROR, "Submitted job is null!");
 			}
 	
 			if (LOG.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitManager.java
index 790aca9..da63bf2 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitManager.java
@@ -31,8 +31,6 @@ import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertexIterator;
 import eu.stratosphere.nephele.executiongraph.ExecutionVertex;
 import eu.stratosphere.nephele.jobgraph.JobID;
 import eu.stratosphere.nephele.jobmanager.splitassigner.file.FileInputSplitAssigner;
-import eu.stratosphere.nephele.template.AbstractInputTask;
-import eu.stratosphere.nephele.template.AbstractInvokable;
 import eu.stratosphere.util.StringUtils;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/LocatableInputSplitAssigner.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/LocatableInputSplitAssigner.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/LocatableInputSplitAssigner.java
index 1e6929d..dc52911 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/LocatableInputSplitAssigner.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/LocatableInputSplitAssigner.java
@@ -24,8 +24,6 @@ import eu.stratosphere.core.io.InputSplit;
 import eu.stratosphere.core.io.LocatableInputSplit;
 import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertex;
 import eu.stratosphere.nephele.executiongraph.ExecutionVertex;
-import eu.stratosphere.nephele.template.AbstractInputTask;
-import eu.stratosphere.nephele.template.AbstractInvokable;
 
 /**
  * The locatable input split assigner is a specific implementation of the {@link InputSplitAssigner} interface for

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/file/FileInputSplitAssigner.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/file/FileInputSplitAssigner.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/file/FileInputSplitAssigner.java
index 048562c..3580fda 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/file/FileInputSplitAssigner.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/file/FileInputSplitAssigner.java
@@ -25,8 +25,6 @@ import eu.stratosphere.core.io.InputSplit;
 import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertex;
 import eu.stratosphere.nephele.executiongraph.ExecutionVertex;
 import eu.stratosphere.nephele.jobmanager.splitassigner.InputSplitAssigner;
-import eu.stratosphere.nephele.template.AbstractInputTask;
-import eu.stratosphere.nephele.template.AbstractInvokable;
 
 /**
  * The file input split assigner is a specific implementation of the {@link InputSplitAssigner} interface for
@@ -89,14 +87,11 @@ public final class FileInputSplitAssigner implements InputSplitAssigner {
 		}
 	}
 
-
 	@Override
 	public void unregisterGroupVertex(final ExecutionGroupVertex groupVertex) {
-		
 		this.vertexMap.remove(groupVertex);
 	}
 
-
 	@Override
 	public InputSplit getNextInputSplit(final ExecutionVertex vertex) {
 


[24/53] [abbrv] [FLINK-962] Initial import of documentation from website into source code (closes #34)

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/java_api_quickstart.md
----------------------------------------------------------------------
diff --git a/docs/java_api_quickstart.md b/docs/java_api_quickstart.md
new file mode 100644
index 0000000..75f4c7c
--- /dev/null
+++ b/docs/java_api_quickstart.md
@@ -0,0 +1,126 @@
+---
+title: "Quickstart: Java API"
+---
+
+<p class="lead">Start working on your Stratosphere Java program in a few simple steps.</p>
+
+<section id="requirements">
+  <div class="page-header"><h2>Requirements</h2></div>
+  <p class="lead">The only requirements are working <strong>Maven 3.0.4</strong> (or higher) and <strong>Java 6.x</strong> (or higher) installations.</p>
+</section>
+
+<section id="create_project">
+  <div class="page-header"><h2>Create Project</h2></div>
+
+  <p class="lead">Use one of the following commands to <strong>create a project</strong>:</p>
+
+  <ul class="nav nav-tabs" style="border-bottom: none;">
+      <li class="active"><a href="#quickstart-script" data-toggle="tab">Run the <strong>quickstart script</strong></a></li>
+      <li><a href="#maven-archetype" data-toggle="tab">Use <strong>Maven archetypes</strong></a></li>
+  </ul>
+  <div class="tab-content">
+      <div class="tab-pane active" id="quickstart-script">
+{% highlight bash %}
+$ curl https://raw.githubusercontent.com/stratosphere/stratosphere-quickstart/master/quickstart.sh | bash
+{% endhighlight %}
+      </div>
+      <div class="tab-pane" id="maven-archetype">
+{% highlight bash %}
+$ mvn archetype:generate                             \
+    -DarchetypeGroupId=eu.stratosphere               \
+    -DarchetypeArtifactId=quickstart-java            \
+    -DarchetypeVersion={{site.current_stable}}
+{% endhighlight %}
+      This allows you to <strong>name your newly created project</strong>. It will interactively ask you for the groupId, artifactId, and package name.
+      </div>
+  </div>
+</section>
+
+<section id="inspect_project">
+  <div class="page-header"><h2>Inspect Project</h2></div>
+  <p class="lead">There will be a <strong>new directory in your working directory</strong>. If you've used the <em>curl</em> approach, the directory is called <code>quickstart</code>. Otherwise, it has the name of your artifactId.</p>
+  <p class="lead">The sample project is a <strong>Maven project</strong>, which contains two classes. <em>Job</em> is a basic skeleton program and <em>WordCountJob</em> a working example. Please note that the <em>main</em> method of both classes allow you to start Stratosphere in a development/testing mode.</p>
+  <p class="lead">We recommend to <strong>import this project into your IDE</strong> to develop and test it. If you use Eclipse, the <a href="http://www.eclipse.org/m2e/">m2e plugin</a> allows to <a href="http://books.sonatype.com/m2eclipse-book/reference/creating-sect-importing-projects.html#fig-creating-import">import Maven projects</a>. Some Eclipse bundles include that plugin by default, other require you to install it manually. The IntelliJ IDE also supports Maven projects out of the box.</p>
+</section>
+
+<section id="build_project">
+<div class="alert alert-danger">A note to Mac OS X users: The default JVM heapsize for Java is too small for Stratosphere. You have to manually increase it. Choose "Run Configurations" -> Arguments and write into the "VM Arguments" box: "-Xmx800m" in Eclipse.</div>
+  <div class="page-header"><h2>Build Project</h2></div>
+  <p class="lead">If you want to <strong>build your project</strong>, go to your project directory and issue the <code>mvn clean package</code> command. You will <strong>find a jar</strong> that runs on every Stratosphere cluster in <code>target/stratosphere-project-0.1-SNAPSHOT.jar</code>.</p>
+</section>
+
+<section id="next_steps">
+  <div class="page-header"><h2>Next Steps</h2></div>
+  <p class="lead"><strong>Write your application!</strong></p>
+  <p>The quickstart project contains a WordCount implementation, the "Hello World" of Big Data processing systems. The goal of WordCount is to determine the frequencies of words in a text, e.g., how often do the terms "the" or "house" occurs in all Wikipedia texts.</p>
+ <br>
+<b>Sample Input:</b> <br>
+{% highlight bash %}
+big data is big
+{% endhighlight %}
+<b>Sample Output:</b> <br>
+{% highlight bash %}
+big 2
+data 1
+is 1
+{% endhighlight %}
+
+<p>The following code shows the WordCount implementation from the Quickstart which processes some text lines with two operators (FlatMap and Reduce), and writes the prints the resulting words and counts to std-out.</p>
+
+{% highlight java %}
+public class WordCount {
+  
+  public static void main(String[] args) throws Exception {
+    
+    // set up the execution environment
+    final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+    
+    // get input data
+    DataSet<String> text = env.fromElements(
+        "To be, or not to be,--that is the question:--",
+        "Whether 'tis nobler in the mind to suffer",
+        "The slings and arrows of outrageous fortune",
+        "Or to take arms against a sea of troubles,"
+        );
+    
+    DataSet<Tuple2<String, Integer>> counts = 
+        // split up the lines in pairs (2-tuples) containing: (word,1)
+        text.flatMap(new LineSplitter())
+        // group by the tuple field "0" and sum up tuple field "1"
+        .groupBy(0)
+        .aggregate(Aggregations.SUM, 1);
+
+    // emit result
+    counts.print();
+    
+    // execute program
+    env.execute("WordCount Example");
+  }
+}
+{% endhighlight %}
+
+<p>The operations are defined by specialized classes, here the LineSplitter class.</p>
+
+{% highlight java %}
+public class LineSplitter extends FlatMapFunction<String, Tuple2<String, Integer>> {
+
+  @Override
+  public void flatMap(String value, Collector<Tuple2<String, Integer>> out) {
+    // normalize and split the line into words
+    String[] tokens = value.toLowerCase().split("\\W+");
+    
+    // emit the pairs
+    for (String token : tokens) {
+      if (token.length() > 0) {
+        out.collect(new Tuple2<String, Integer>(token, 1));
+      }
+    }
+  }
+}
+
+{% endhighlight %}
+
+<p><a href="https://github.com/stratosphere/stratosphere/blob/release-{{site.current_stable}}/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCount.java">Check GitHub</a> for the full example code.</p>
+
+<p class="lead">For a complete overview over our Java API, have a look at the <a href="{{ site.baseurl }}/docs/{{site.current_stable_documentation}}/programming_guides/java.html">Stratosphere Documentation</a> and <a href="{{ site.baseurl }}/docs/{{site.current_stable_documentation}}/programming_guides/examples_java.html">further example programs</a>. If you have any trouble, ask on our <a href="https://groups.google.com/forum/#!forum/stratosphere-dev">Mailing list</a>. We are happy to provide help.</p>
+</section>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/local_execution.md
----------------------------------------------------------------------
diff --git a/docs/local_execution.md b/docs/local_execution.md
new file mode 100644
index 0000000..cd60f62
--- /dev/null
+++ b/docs/local_execution.md
@@ -0,0 +1,106 @@
+---
+title:  "Local Execution"
+---
+
+# Local Execution/Debugging
+
+Stratosphere can run on a single machine, even in a single Java Virtual Machine. This allows users to test and debug Stratosphere programs locally. This section gives an overview of the local execution mechanisms.
+
+**NOTE:** Please also refer to the [debugging section]({{site.baseurl}}/docs/0.5/programming_guides/java.html#debugging) in the Java API documentation for a guide to testing and local debugging utilities in the Java API.
+
+The local environments and executors allow you to run Stratosphere programs in local Java Virtual Machine, or with within any JVM as part of existing programs. Most examples can be launched locally by simply hitting the "Run" button of your IDE.
+
+If you are running Stratosphere programs locally, you can also debug your program like any other Java program. You can either use `System.out.println()` to write out some internal variables or you can use the debugger. It is possible to set breakpoints within `map()`, `reduce()` and all the other methods.
+
+The `JobExecutionResult` object, which is returned after the execution finished, contains the program runtime and the accumulator results.
+
+*Note:* The local execution environments do not start any web frontend to monitor the execution.
+
+
+# Maven Dependency
+
+If you are developing your program in a Maven project, you have to add the `stratosphere-clients` module using this dependency:
+
+```xml
+<dependency>
+  <groupId>eu.stratosphere</groupId>
+  <artifactId>stratosphere-clients</artifactId>
+  <version>{{site.current_stable}}</version>
+</dependency>
+```
+
+# Local Environment
+
+The `LocalEnvironment` is a handle to local execution for Stratosphere programs. Use it to run a program within a local JVM - standalone or embedded in other programs.
+
+The local environment is instantiated via the method `ExecutionEnvironment.createLocalEnvironment()`. By default, it will use as many local threads for execution as your machine has CPU cores (hardware contexts). You can alternatively specify the desired parallelism. The local environment can be configured to log to the console using `enableLogging()`/`disableLogging()`.
+
+In most cases, calling `ExecutionEnvironment.getExecutionEnvironment()` is the even better way to go. That method returns a `LocalEnvironment` when the program is started locally (outside the command line interface), and it returns a pre-configured environment for cluster execution, when the program is invoked by the [command line interface]({{ site.baseurl }}/docs/0.5/program_execution/cli_client.html).
+
+```java
+public static void main(String[] args) throws Exception {
+    ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
+
+    DataSet<String> data = env.readTextFile("file:///path/to/file");
+
+    data
+        .filter(new FilterFunction<String>() {
+            public boolean filter(String value) {
+                return value.startsWith("http://");
+            }
+        })
+        .writeAsText("file:///path/to/result");
+
+    env.execute();
+}
+```
+
+
+# Local Executor
+
+The *LocalExecutor* is similar to the local environment, but it takes a *Plan* object, which describes the program as a single executable unit. The *LocalExecutor* is typically used with the Scala API. 
+
+The following code shows how you would use the `LocalExecutor` with the Wordcount example for Scala Programs:
+
+```scala
+public static void main(String[] args) throws Exception {
+    val input = TextFile("hdfs://path/to/file")
+
+    val words = input flatMap { _.toLowerCase().split("""\W+""") filter { _ != "" } }
+    val counts = words groupBy { x => x } count()
+
+    val output = counts.write(wordsOutput, CsvOutputFormat())
+  
+    val plan = new ScalaPlan(Seq(output), "Word Count")
+    LocalExecutor.executePlan(p);
+}
+```
+
+
+# LocalDistributedExecutor
+
+Stratosphere also offers a `LocalDistributedExecutor` which starts multiple TaskManagers within one JVM. The standard `LocalExecutor` starts one JobManager and one TaskManager in one JVM.
+With the `LocalDistributedExecutor` you can define the number of TaskManagers to start. This is useful for debugging network related code and more of a developer tool than a user tool.
+
+```java
+public static void main(String[] args) throws Exception {
+    ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+    DataSet<String> data = env.readTextFile("hdfs://path/to/file");
+
+    data
+        .filter(new FilterFunction<String>() {
+            public boolean filter(String value) {
+                return value.startsWith("http://");
+            }
+        })
+        .writeAsText("hdfs://path/to/result");
+
+    Plan p = env.createProgramPlan();
+    LocalDistributedExecutor lde = new LocalDistributedExecutor();
+    lde.startNephele(2); // start two TaskManagers
+    lde.run(p);
+}
+```
+
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/local_setup.md
----------------------------------------------------------------------
diff --git a/docs/local_setup.md b/docs/local_setup.md
new file mode 100644
index 0000000..b49118a
--- /dev/null
+++ b/docs/local_setup.md
@@ -0,0 +1,108 @@
+---
+title:  "Local Setup"
+---
+
+This documentation is intended to provide instructions on how to run Stratosphere locally on a single machine.
+
+# Download
+
+Go to the [downloads page]({{site.baseurl}}/downloads/) and get the ready to run package. If you want to interact with Hadoop (e.g. HDFS or HBase), make sure to pick the Stratosphere package **matching your Hadoop version**. When in doubt or you plan to just work with the local file system pick the package for Hadoop 1.2.x.
+
+# Requirements
+
+Stratosphere runs on **Linux**, **Mac OS X** and **Windows**. The only requirement for a local setup is **Java 1.6.x** or higher. The following manual assumes a *UNIX-like environment*, for Windows see [Stratosphere on Windows](#windows).
+
+You can check the correct installation of Java by issuing the following command:
+
+```bash
+java -version
+```
+
+The command should output something comparable to the following:
+
+```bash
+java version "1.6.0_22"
+Java(TM) SE Runtime Environment (build 1.6.0_22-b04)
+Java HotSpot(TM) 64-Bit Server VM (build 17.1-b03, mixed mode)
+```
+
+# Configuration
+
+**For local mode Stratosphere is ready to go out of the box and you don't need to change the default configuration.**
+
+The out of the box configuration will use your default Java installation. You can manually set the environment variable `JAVA_HOME` or the configuration key `env.java.home` in `conf/stratosphere-conf.yaml` if you want to manually override the Java runtime to use. Consult the [configuration page]({{site.baseurl}}/docs/0.4/setup/config.html) for further details about configuring Stratosphere.
+
+# Starting Stratosphere
+
+**You are now ready to start Stratosphere.** Unpack the downloaded archive and change to the newly created `stratosphere` directory. There you can start Stratosphere in local mode:
+
+```bash
+$ tar xzf stratosphere-*.tgz
+$ cd stratosphere
+$ bin/start-local.sh
+Starting job manager
+```
+
+You can check that the system is running by checking the log files in the `logs` directory:
+
+```bash
+$ tail log/stratosphere-*-jobmanager-*.log
+INFO ... - Initializing memory manager with 409 megabytes of memory
+INFO ... - Trying to load eu.stratosphere.nephele.jobmanager.scheduler.local.LocalScheduler as scheduler
+INFO ... - Setting up web info server, using web-root directory ...
+INFO ... - Web info server will display information about nephele job-manager on localhost, port 8081.
+INFO ... - Starting web info server for JobManager on port 8081
+```
+
+The JobManager will also start a web frontend on port 8081, which you can check with your browser at `http://localhost:8081`.
+
+# Stratosphere on Windows
+
+If you want to run Stratosphere on Windows you need to download, unpack and configure the Stratosphere archive as mentioned above. After that you can either use the **Windows Batch** file (`.bat`) or use **Cygwin**  to run the Stratosphere Jobmanager.
+
+To start Stratosphere in local mode from the *Windows Batch*, open the command window, navigate to the `bin/` directory of Stratosphere and run `start-local.bat`.
+
+```bash
+$ cd stratosphere
+$ cd bin
+$ start-local.bat
+Starting Stratosphere job manager. Webinterface by default on http://localhost:8081/.
+Do not close this batch window. Stop job manager by pressing Ctrl+C.
+```
+
+After that, you need to open a second terminal to run jobs using `stratosphere.bat`.
+
+With *Cygwin* you need to start the Cygwin Terminal, navigate to your Stratosphere directory and run the `start-local.sh` script:
+
+```bash
+$ cd stratosphere
+$ bin/start-local.sh
+Starting Nephele job manager
+```
+
+If you are installing Stratosphere from the git repository and you are using the Windows git shell, Cygwin can produce a failure similiar to this one:
+
+```bash
+c:/stratosphere/bin/start-local.sh: line 30: $'\r': command not found
+```
+
+This error occurs, because git is automatically transforming UNIX line endings to Windows style line endings when running in Windows. The problem is, that Cygwin can only deal with UNIX style line endings. The solution is to adjust the Cygwin settings to deal with the correct line endings by following these three steps:
+
+1. Start a Cygwin shell.
+
+2. Determine your home directory by entering
+
+```bash
+cd; pwd
+```
+
+It will return a path under the Cygwin root path.
+
+2.  Using NotePad, WordPad or a different text editor open the file `.bash_profile` in the home directory and append the following: (If the file does not exist you have to create it)
+
+```bash
+export SHELLOPTS
+set -o igncr
+```
+
+Save the file and open a new bash shell.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/quickstart/plotPoints.py
----------------------------------------------------------------------
diff --git a/docs/quickstart/plotPoints.py b/docs/quickstart/plotPoints.py
new file mode 100755
index 0000000..fa04d31
--- /dev/null
+++ b/docs/quickstart/plotPoints.py
@@ -0,0 +1,82 @@
+#!/usr/bin/python
+import sys
+import matplotlib.pyplot as plt
+import csv
+import os
+
+if len(sys.argv) < 4 or not sys.argv[1] in ['points', 'result']:
+  print "Usage: plot-clusters.py (points|result) <src-file> <pdf-file-prefix>"
+  sys.exit(1)
+
+inFile = sys.argv[1]
+inFile = sys.argv[2]
+outFilePx = sys.argv[3]
+
+inFileName = os.path.splitext(os.path.basename(inFile))[0]
+outFile = os.path.join(".", outFilePx+"-plot.pdf")
+
+########### READ DATA
+
+cs = []
+xs = []
+ys = []
+
+minX = None
+maxX = None
+minY = None
+maxY = None
+
+if sys.argv[1] == 'points':
+
+  with open(inFile, 'rb') as file:
+    for line in file:
+      # parse data
+      csvData = line.strip().split(' ')
+
+      x = float(csvData[0])
+      y = float(csvData[1])
+
+      if not minX or minX > x:
+        minX = x
+      if not maxX or maxX < x:
+        maxX = x
+      if not minY or minY > y:
+        minY = y
+      if not maxY or maxY < y:
+        maxY = y
+
+      xs.append(x)
+      ys.append(y)
+
+    # plot data
+    plt.clf()
+    plt.scatter(xs, ys, s=25, c="#999999", edgecolors='None', alpha=1.0)
+    plt.ylim([minY,maxY])
+    plt.xlim([minX,maxX])
+
+elif sys.argv[1] == 'result':
+
+  with open(inFile, 'rb') as file:
+    for line in file:
+      # parse data
+      csvData = line.strip().split(' ')
+
+      c = int(csvData[0])
+      x = float(csvData[1])
+      y = float(csvData[2])
+
+      cs.append(c)
+      xs.append(x)
+      ys.append(y)
+
+    # plot data
+    plt.clf()
+    plt.scatter(xs, ys, s=25, c=cs, edgecolors='None', alpha=1.0)
+    plt.ylim([minY,maxY])
+    plt.xlim([minX,maxX])
+
+
+plt.savefig(outFile, dpi=600)
+print "\nPlotted file: %s" % outFile
+
+sys.exit(0)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/run_example_quickstart.md
----------------------------------------------------------------------
diff --git a/docs/run_example_quickstart.md b/docs/run_example_quickstart.md
new file mode 100644
index 0000000..600e3fd
--- /dev/null
+++ b/docs/run_example_quickstart.md
@@ -0,0 +1,154 @@
+---
+title: "Quick Start: Run K-Means Example"
+---
+
+
+<p class="lead">
+	This guide will demonstrate Stratosphere's features by example. You will see how you can leverage Stratosphere's Iteration-feature to find clusters in a dataset using <a href="http://en.wikipedia.org/wiki/K-means_clustering">K-Means clustering</a>. 
+	On the way, you will see the compiler, the status interface and the result of the algorithm.
+</p>
+
+
+<section id="data">
+  <div class="page-header">
+  	<h2>Generate Input Data</h2>
+  </div>
+  <p>Stratosphere contains a data generator for K-Means.</p>
+  {% highlight bash %}
+# Download Stratosphere
+wget {{ site.current_stable_dl }}
+tar xzf stratosphere-*.tgz 
+cd stratosphere-*
+mkdir kmeans
+cd kmeans
+# run data generator
+java -cp  ../examples/stratosphere-java-examples-{{ site.current_stable }}-KMeans.jar eu.stratosphere.example.java.clustering.util.KMeansDataGenerator 500 10 0.08
+cp /tmp/points .
+cp /tmp/centers .
+  {% endhighlight %}
+The generator has the following arguments:
+{% highlight bash %}
+KMeansDataGenerator <numberOfDataPoints> <numberOfClusterCenters> [<relative stddev>] [<centroid range>] [<seed>]
+{% endhighlight %}
+The <i>relative standard deviation</i> is an interesting tuning parameter: it determines the closeness of the points to the centers.
+<p>The <code>kmeans/</code> directory should now contain two files: <code>centers</code> and <code>points</code>.</p>
+
+
+<h2>Review Input Data</h2>
+Use the <code>plotPoints.py</code> tool to review the result of the data generator. <a href="{{site.baseurl}}/quickstart/example-data/plotPoints.py">Download Python Script</a>
+{% highlight bash %}
+python2.7 plotPoints.py points points input
+{% endhighlight %}
+
+
+Note: You might have to install <a href="http://matplotlib.org/">matplotlib</a> (<code>python-matplotlib</code> package on Ubuntu) to use the Python script.
+
+
+The following overview presents the impact of the different standard deviations on the input data.
+<div class="row" style="padding-top:15px">
+	<div class="col-md-4">
+		<div class="text-center" style="font-weight:bold;">relative stddev = 0.03</div>
+		<a data-lightbox="inputs" href="{{site.baseurl}}/img/quickstart-example/kmeans003.png" data-lightbox="example-1"><img class="img-responsive" src="{{site.baseurl}}/img/quickstart-example/kmeans003.png" /></a>
+	</div>
+	<div class="col-md-4">
+		<div class="text-center" style="font-weight:bold;padding-bottom:2px">relative stddev = 0.08</div>
+		<a data-lightbox="inputs" href="{{site.baseurl}}/img/quickstart-example/kmeans008.png" data-lightbox="example-1"><img class="img-responsive" src="{{site.baseurl}}/img/quickstart-example/kmeans008.png" /></a>
+	</div>
+	<div class="col-md-4">
+		<div class="text-center" style="font-weight:bold;">relative stddev = 0.15</div>
+		<a data-lightbox="inputs" href="{{site.baseurl}}/img/quickstart-example/kmeans015.png" data-lightbox="example-1"><img class="img-responsive" src="{{site.baseurl}}/img/quickstart-example/kmeans015.png" /></a>
+	</div>
+</div>
+</section>
+
+<section id="run">
+ <div class="page-header">
+  	<h2>Run Clustering</h2>
+  </div>
+We are using the generated input data to run the clustering using a Stratosphere job.
+{% highlight bash %}
+# go to the Stratosphere-root directory
+cd stratosphere
+# start Stratosphere (use ./bin/start-cluster.sh if you're on a cluster)
+./bin/start-local.sh
+# Start Stratosphere web client
+./bin/start-webclient.sh
+{% endhighlight %}
+
+<h2>Review Stratosphere Compiler</h2>
+
+The Stratosphere webclient allows to submit Stratosphere programs using a graphical user interface.
+
+<div class="row" style="padding-top:15px">
+	<div class="col-md-6">
+		<a data-lightbox="compiler" href="{{site.baseurl}}/img/quickstart-example/run-webclient.png" data-lightbox="example-1"><img class="img-responsive" src="{{site.baseurl}}/img/quickstart-example/run-webclient.png" /></a>
+	</div>
+	<div class="col-md-6">
+		1. <a href="http://localhost:8080/launch.html">Open webclient on localhost:8080</a> <br>
+		2. Upload the 
+{% highlight bash %}
+examples/stratosphere-java-examples-0.5-SNAPSHOT-KMeansIterative.jar
+{% endhighlight %} file.<br>
+		3. Select it in the left box to see how the operators in the plan are connected to each other. <br>
+		4. Enter the arguments in the lower left box:
+{% highlight bash %}
+file://<pathToGenerated>points file://<pathToGenerated>centers file://<pathToGenerated>result 10
+{% endhighlight %}
+For example:
+{% highlight bash %}
+file:///tmp/stratosphere/kmeans/points file:///tmp/stratosphere/kmeans/centers file:///tmp/stratosphere/kmeans/result 20
+{% endhighlight %}
+	</div>
+</div>
+<hr>
+<div class="row" style="padding-top:15px">
+	<div class="col-md-6">
+		<a data-lightbox="compiler" href="{{site.baseurl}}/img/quickstart-example/compiler-webclient-new.png" data-lightbox="example-1"><img class="img-responsive" src="{{site.baseurl}}/img/quickstart-example/compiler-webclient-new.png" /></a>
+	</div>
+
+	<div class="col-md-6">
+		1. Press the <b>RunJob</b> to see the optimzer plan. <br>
+		2. Inspect the operators and see the properties (input sizes, cost estimation) determined by the optimizer.
+	</div>
+</div>
+<hr>
+<div class="row" style="padding-top:15px">
+	<div class="col-md-6">
+		<a data-lightbox="compiler" href="{{site.baseurl}}/img/quickstart-example/jobmanager-running-new.png" data-lightbox="example-1"><img class="img-responsive" src="{{site.baseurl}}/img/quickstart-example/jobmanager-running-new.png" /></a>
+	</div>
+	<div class="col-md-6">
+		1. Press the <b>Continue</b> button to start executing the job. <br>
+		2. <a href="http://localhost:8080/launch.html">Open Stratosphere's monitoring interface</a> to see the job's progress.<br>
+		3. Once the job has finished, you can analyize the runtime of the individual operators.
+	</div>
+</div>
+</section>
+
+<section id="result">
+ <div class="page-header">
+  	<h2>Analyze the Result</h2>
+  </div>
+Use the <a href="{{site.baseurl}}/quickstart/example-data/plotPoints.py">Python Script</a> again to visualize the result
+
+{% highlight bash %}
+python2.7 plotPoints.py result result result-pdf
+{% endhighlight %}
+
+The following three pictures show the results for the sample input above. Play around with the parameters (number of iterations, number of clusters) to see how they affect the result.
+
+<div class="row" style="padding-top:15px">
+	<div class="col-md-4">
+		<div class="text-center" style="font-weight:bold;">relative stddev = 0.03</div>
+		<a data-lightbox="results" href="{{site.baseurl}}/img/quickstart-example/result003.png" data-lightbox="example-1"><img class="img-responsive" src="{{site.baseurl}}/img/quickstart-example/result003.png" /></a>
+	</div>
+	<div class="col-md-4">
+		<div class="text-center" style="font-weight:bold;">relative stddev = 0.08</div>
+		<a data-lightbox="results" href="{{site.baseurl}}/img/quickstart-example/result008.png" data-lightbox="example-1"><img class="img-responsive" src="{{site.baseurl}}/img/quickstart-example/result008.png" /></a>
+	</div>
+	<div class="col-md-4">
+		<div class="text-center" style="font-weight:bold;">relative stddev = 0.15</div>
+		<a data-lightbox="results" href="{{site.baseurl}}/img/quickstart-example/result015.png" data-lightbox="example-1"><img class="img-responsive" src="{{site.baseurl}}/img/quickstart-example/result015.png" /></a>
+	</div>
+</div>
+
+</section>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/scala_api_examples.md
----------------------------------------------------------------------
diff --git a/docs/scala_api_examples.md b/docs/scala_api_examples.md
new file mode 100644
index 0000000..ac930b3
--- /dev/null
+++ b/docs/scala_api_examples.md
@@ -0,0 +1,195 @@
+---
+title:  "Scala API Examples"
+---
+
+The following example programs showcase different applications of Stratosphere from simple word counting to graph algorithms.
+The code samples illustrate the use of **[Stratosphere's Scala API]({{site.baseurl}}/docs/{{site.current_stable}}/programming_guides/scala.html)**. 
+
+The full source code of the following and more examples can be found in the **[stratosphere-scala-examples](https://github.com/stratosphere/stratosphere/tree/release-{{site.current_stable}}/stratosphere-examples/stratosphere-scala-examples)** module.
+
+# Word Count
+
+WordCount is the "Hello World" of Big Data processing systems. It computes the frequency of words in a text collection. The algorithm works in two steps: First, the texts are splits the text to individual words. Second, the words are grouped and counted.
+
+```scala
+// read input data
+val input = TextFile(textInput)
+
+// tokenize words
+val words = input.flatMap { _.split(" ") map { (_, 1) } }
+
+// count by word
+val counts = words.groupBy { case (word, _) => word }
+  .reduce { (w1, w2) => (w1._1, w1._2 + w2._2) }
+
+val output = counts.write(wordsOutput, CsvOutputFormat()))
+```
+
+The [WordCount example](https://github.com/stratosphere/stratosphere/blob/release-{{site.current_stable}}/stratosphere-examples/stratosphere-scala-examples/src/main/scala/eu/stratosphere/examples/scala/wordcount/WordCount.scala) implements the above described algorithm with input parameters: `<degree of parallelism>, <text input path>, <output path>`. As test data, any text file will do.
+
+# Page Rank
+
+The PageRank algorithm computes the "importance" of pages in a graph defined by links, which point from one pages to another page. It is an iterative graph algorithm, which means that it repeatedly applies the same computation. In each iteration, each page distributes its current rank over all its neighbors, and compute its new rank as a taxed sum of the ranks it received from its neighbors. The PageRank algorithm was popularized by the Google search engine which uses the importance of webpages to rank the results of search queries.
+
+In this simple example, PageRank is implemented with a [bulk iteration]({{site.baseurl}}/docs/{{site.current_stable}}/programming_guides/java.html#iterations) and a fixed number of iterations.
+
+```scala
+// cases classes so we have named fields
+case class PageWithRank(pageId: Long, rank: Double)
+case class Edge(from: Long, to: Long, transitionProbability: Double)
+
+// constants for the page rank formula
+val dampening = 0.85
+val randomJump = (1.0 - dampening) / NUM_VERTICES
+val initialRank = 1.0 / NUM_VERTICES
+  
+// read inputs
+val pages = DataSource(verticesPath, CsvInputFormat[Long]())
+val edges = DataSource(edgesPath, CsvInputFormat[Edge]())
+
+// assign initial rank
+val pagesWithRank = pages map { p => PageWithRank(p, initialRank) }
+
+// the iterative computation
+def computeRank(ranks: DataSet[PageWithRank]) = {
+
+    // send rank to neighbors
+    val ranksForNeighbors = ranks join edges
+        where { _.pageId } isEqualTo { _.from }
+        map { (p, e) => (e.to, p.rank * e.transitionProbability) }
+    
+    // gather ranks per vertex and apply page rank formula
+    ranksForNeighbors .groupBy { case (node, rank) => node }
+                      .reduce { (a, b) => (a._1, a._2 + b._2) }
+                      .map {case (node, rank) => PageWithRank(node, rank * dampening + randomJump) }
+}
+
+// invoke iteratively
+val finalRanks = pagesWithRank.iterate(numIterations, computeRank)
+val output = finalRanks.write(outputPath, CsvOutputFormat())
+```
+
+
+
+The [PageRank program](https://github.com/stratosphere/stratosphere/blob/release-{{site.current_stable}}/stratosphere-examples/stratosphere-scala-examples/src/main/scala/eu/stratosphere/examples/scala/graph/PageRank.scala) implements the above example.
+It requires the following parameters to run: `<pages input path>, <link input path>, <output path>, <num pages>, <num iterations>`.
+
+Input files are plain text files and must be formatted as follows:
+- Pages represented as an (long) ID separated by new-line characters.
+    * For example `"1\n2\n12\n42\n63\n"` gives five pages with IDs 1, 2, 12, 42, and 63.
+- Links are represented as pairs of page IDs which are separated by space characters. Links are separated by new-line characters:
+    * For example `"1 2\n2 12\n1 12\n42 63\n"` gives four (directed) links (1)->(2), (2)->(12), (1)->(12), and (42)->(63).
+
+For this simple implementation it is required that each page has at least one incoming and one outgoing link (a page can point to itself).
+
+# Connected Components
+
+The Connected Components algorithm identifies parts of a larger graph which are connected by assigning all vertices in the same connected part the same component ID. Similar to PageRank, Connected Components is an iterative algorithm. In each step, each vertex propagates its current component ID to all its neighbors. A vertex accepts the component ID from a neighbor, if it is smaller than its own component ID.
+
+This implementation uses a [delta iteration]({{site.baseurl}}/docs/{{site.current_stable}}/programming_guides/java.html#iterations): Vertices that have not changed their component id do not participate in the next step. This yields much better performance, because the later iterations typically deal only with a few outlier vertices.
+
+```scala
+// define case classes
+case class VertexWithComponent(vertex: Long, componentId: Long)
+case class Edge(from: Long, to: Long)
+
+// get input data
+val vertices = DataSource(verticesPath, CsvInputFormat[Long]())
+val directedEdges = DataSource(edgesPath, CsvInputFormat[Edge]())
+
+// assign each vertex its own ID as component ID
+val initialComponents = vertices map { v => VertexWithComponent(v, v) }
+val undirectedEdges = directedEdges flatMap { e => Seq(e, Edge(e.to, e.from)) }
+
+def propagateComponent(s: DataSet[VertexWithComponent], ws: DataSet[VertexWithComponent]) = {
+  val allNeighbors = ws join undirectedEdges
+        where { _.vertex } isEqualTo { _.from }
+        map { (v, e) => VertexWithComponent(e.to, v.componentId ) }
+    
+    val minNeighbors = allNeighbors groupBy { _.vertex } reduceGroup { cs => cs minBy { _.componentId } }
+
+    // updated solution elements == new workset
+    val s1 = s join minNeighbors
+        where { _.vertex } isEqualTo { _.vertex }
+        flatMap { (curr, candidate) =>
+            if (candidate.componentId < curr.componentId) Some(candidate) else None
+        }
+
+  (s1, s1)
+}
+
+val components = initialComponents.iterateWithDelta(initialComponents, { _.vertex }, propagateComponent,
+                    maxIterations)
+val output = components.write(componentsOutput, CsvOutputFormat())
+```
+
+The [ConnectedComponents program](https://github.com/stratosphere/stratosphere/blob/release-{{site.current_stable}}/stratosphere-examples/stratosphere-scala-examples/src/main/scala/eu/stratosphere/examples/scala/graph/ConnectedComponents.scala) implements the above example. It requires the following parameters to run: `<vertex input path>, <edge input path>, <output path> <max num iterations>`.
+
+Input files are plain text files and must be formatted as follows:
+- Vertices represented as IDs and separated by new-line characters.
+    * For example `"1\n2\n12\n42\n63\n"` gives five vertices with (1), (2), (12), (42), and (63).
+- Edges are represented as pairs for vertex IDs which are separated by space characters. Edges are separated by new-line characters:
+    * For example `"1 2\n2 12\n1 12\n42 63\n"` gives four (undirected) links (1)-(2), (2)-(12), (1)-(12), and (42)-(63).
+
+# Relational Query
+
+The Relational Query example assumes two tables, one with `orders` and the other with `lineitems` as specified by the [TPC-H decision support benchmark](http://www.tpc.org/tpch/). TPC-H is a standard benchmark in the database industry. See below for instructions how to generate the input data.
+
+The example implements the following SQL query.
+
+```sql
+SELECT l_orderkey, o_shippriority, sum(l_extendedprice) as revenue
+    FROM orders, lineitem
+WHERE l_orderkey = o_orderkey
+    AND o_orderstatus = "F" 
+    AND YEAR(o_orderdate) > 1993
+    AND o_orderpriority LIKE "5%"
+GROUP BY l_orderkey, o_shippriority;
+```
+
+The Stratosphere Scala program, which implements the above query looks as follows.
+
+```scala
+// --- define some custom classes to address fields by name ---
+case class Order(orderId: Int, status: Char, date: String, orderPriority: String, shipPriority: Int)
+case class LineItem(orderId: Int, extendedPrice: Double)
+case class PrioritizedOrder(orderId: Int, shipPriority: Int, revenue: Double)
+
+val orders = DataSource(ordersInputPath, DelimitedInputFormat(parseOrder))
+val lineItem2600s = DataSource(lineItemsInput, DelimitedInputFormat(parseLineItem))
+
+val filteredOrders = orders filter { o => o.status == "F" && o.date.substring(0, 4).toInt > 1993 && o.orderPriority.startsWith("5") }
+
+val prioritizedItems = filteredOrders join lineItems
+    where { _.orderId } isEqualTo { _.orderId } // join on the orderIds
+    map { (o, li) => PrioritizedOrder(o.orderId, o.shipPriority, li.extendedPrice) }
+
+val prioritizedOrders = prioritizedItems
+    groupBy { pi => (pi.orderId, pi.shipPriority) } 
+    reduce { (po1, po2) => po1.copy(revenue = po1.revenue + po2.revenue) }
+
+val output = prioritizedOrders.write(ordersOutput, CsvOutputFormat(formatOutput))
+```
+
+The [Relational Query program](https://github.com/stratosphere/stratosphere/blob/release-{{site.current_stable}}/stratosphere-examples/stratosphere-scala-examples/src/main/scala/eu/stratosphere/examples/scala/relational/RelationalQuery.scala) implements the above query. It requires the following parameters to run: `<orders input path>, <lineitem input path>, <output path>, <degree of parallelism>`.
+
+The orders and lineitem files can be generated using the [TPC-H benchmark](http://www.tpc.org/tpch/) suite's data generator tool (DBGEN). 
+Take the following steps to generate arbitrary large input files for the provided Stratosphere programs:
+
+1.  Download and unpack DBGEN
+2.  Make a copy of *makefile.suite* called *Makefile* and perform the following changes:
+
+```bash
+DATABASE = DB2
+MACHINE  = LINUX
+WORKLOAD = TPCH
+CC       = gcc
+```
+
+1.  Build DBGEN using *make*
+2.  Generate lineitem and orders relations using dbgen. A scale factor
+    (-s) of 1 results in a generated data set with about 1 GB size.
+
+```bash
+./dbgen -T o -s 1
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/scala_api_guide.md
----------------------------------------------------------------------
diff --git a/docs/scala_api_guide.md b/docs/scala_api_guide.md
new file mode 100644
index 0000000..4b43938
--- /dev/null
+++ b/docs/scala_api_guide.md
@@ -0,0 +1,1008 @@
+---
+title: "Scala API Programming Guide"
+---
+
+
+Scala Programming Guide
+=======================
+
+This guide explains how to develop Stratosphere programs with the Scala
+programming interface. It assumes you are familiar with the general concepts of
+Stratosphere's [Programming Model](pmodel.html "Programming Model"). We
+recommend to learn about the basic concepts first, before continuing with the
+[Java](java.html "Java Programming Guide") or this Scala programming guide.
+
+Here we will look at the general structure of a Scala job. You will learn how to
+write data sources, data sinks, and operators to create data flows that can be
+executed using the Stratosphere system.
+
+Writing Scala jobs requires an understanding of Scala, there is excellent
+documentation available [here](http://scala-lang.org/documentation/). Most
+of the examples can be understood by someone with a good understanding
+of programming in general, though.
+
+<section id="intro-example">
+Word Count Example
+------------------
+
+To start, let's look at a Word Count job implemented in Scala. This program is
+very simple but it will give you a basic idea of what a Scala job looks like.
+
+```scala
+import eu.stratosphere.client.LocalExecutor
+
+import eu.stratosphere.api.scala._
+import eu.stratosphere.api.scala.operators._
+
+object WordCount {
+  def main(args: Array[String]) {
+    val input = TextFile(textInput)
+
+    val words = input.flatMap { _.split(" ") map { (_, 1) } }
+
+    val counts = words.groupBy { case (word, _) => word }
+      .reduce { (w1, w2) => (w1._1, w1._2 + w2._2) }
+
+    val output = counts.write(wordsOutput, CsvOutputFormat())
+    val plan = new ScalaPlan(Seq(output))
+
+    LocalExecutor.execute(plan)
+  }
+}
+``` 
+
+Same as any Stratosphere job a Scala job consists of one or several data
+sources, one or several data sinks and operators in between these that transform
+data. Together these parts are referred to as the data flow graph. It dictates
+the way data is passed when a job is executed.
+
+When using Scala in Stratosphere an important concept to grasp is that of the
+`DataSet`. `DataSet` is an abstract concept that represents actual data sets at
+runtime and which has operations that transform data to create a new transformed
+data set. In this example the `TextFile("/some/input")` call creates a
+`DataSet[String]` that represents the lines of text from the input. The
+`flatMap` operation that looks like a regular Scala flatMap is in fact an
+operation on `DataSet` that passes (at runtime) the data items through the
+provided anonymous function to transform them. The result of the `flatMap`
+operation is a new `DataSet` that represents the transformed data. On this other
+operations be performed. Another such operation are `groupBy` and `reduce`, but
+we will go into details of those later in this guide.
+
+The `write` operation of `DataSet` is used to create a data sink. You provide it
+with a path where the data is to be written to and an output format. This is
+enough for now but we will discuss data formats (for sources and sinks) later.
+
+To execute a data flow graph one or several sinks have to wrapped in a `Plan`
+which can then be executed on a cluster using `RemoteExecutor`. Here, the
+`LocalExecutor` is used to run the flow on the local computer. This is useful
+for debugging your job before running it on an actual cluster.
+
+<section id="intro-example">
+Project Setup
+-------------
+
+We will only cover maven here but the concepts should work equivalently with
+other build systems such as Gradle or sbt. When wanting to develop a Scala job
+all that is needed as dependency is is `stratosphere-scala` (and `stratosphere-clients`, if
+you want to execute your jobs). So all that needs to be done is to add the
+following lines to your POM.
+
+
+```xml
+<dependencies>
+  <dependency>
+    <groupId>eu.stratosphere</groupId>
+    <artifactId>stratosphere-scala</artifactId>
+    <version>{{site.current_stable}}</version>
+  </dependency>
+  <dependency>
+    <groupId>eu.stratosphere</groupId>
+    <artifactId>stratosphere-clients</artifactId>
+    <version>{{site.current_stable}}</version>
+  </dependency>
+</dependencies>
+```
+
+To quickly get started you can use the Stratosphere Scala quickstart available
+[here]({{site.baseurl}}/quickstart/scala.html). This will give you a
+completeMaven project with some working example code that you can use to explore
+the system or as basis for your own projects.
+
+These imports are normally enough for any project:
+
+```scala
+import eu.stratosphere.api.scala._
+import eu.stratosphere.api.scala.operators._
+
+import eu.stratosphere.client.LocalExecutor
+import eu.stratosphere.client.RemoteExecutor
+```
+
+The first two imports contain things like `DataSet`, `Plan`, data sources, data
+sinks, and the operations. The last two imports are required if you want to run
+a data flow on your local machine, respectively cluster.
+
+<section id="dataset">
+The DataSet Abstraction
+-----------------------
+
+As already alluded to in the introductory example you write Scala jobs by using
+operations on a `DataSet` to create new transformed `DataSet`. This concept is
+the core of the Stratosphere Scala API so it merits some more explanation. A
+`DataSet` can look and behave like a regular Scala collection in your code but
+it does not contain any actual data but only represents data. For example: when
+you use `TextFile()` you get back a `DataSource[String]` that represents each
+line of text in the input as a `String`. No data is actually loaded or available
+at this point. The set is only used to apply further operations which themselves
+are not executed until the data flow is executed. An operation on `DataSet`
+creates a new `DataSet` that represents the transformation and has a pointer to
+the `DataSet` that represents the data to be transformed. In this way a tree of
+data sets is created that contains both the specification of the flow of data as
+well as all the transformations. This graph can be wrapped in a `Plan` and
+executed.
+
+Working with the system is like working with lazy collections, where execution
+is postponed until the user submits the job.
+
+`DataSet` has a generic parameter, this is the type of each data item or record
+that would be processed by further transformations. This is similar to how
+`List[A]` in Scala would behave. For example in:
+
+```scala
+val input: DataSet[(String, Int)] = ...
+val mapped = input map { a => (a._1, a._2 + 1)}
+```
+
+The anonymous function would retrieve in `a` tuples of type `(String, Int)`.
+
+<section id="datatypes">
+Data Types
+----------
+
+There are some restrictions regarding the data types that can be used in Scala
+jobs (basically the generic parameter of `DataSet`). The usable types are
+the primitive Scala types, case classes (which includes tuples), and custom
+data types.
+
+Custom data types must implement the interface
+[Value](https://github.com/stratosphere/stratosphere/blob/release-{{site.current_stable}}/stratosphere-core/src/main/java/eu/stratosphere/types/Value.java).
+For custom data types that should also be used as a grouping key or join key
+the [Key](https://github.com/stratosphere/stratosphere/blob/release-{{site.current_stable}}/stratosphere-core/src/main/java/eu/stratosphere/types/Key.java)
+interface must be implemented.
+
+
+
+<section id="data-sources">
+Creating Data Sources
+---------------------
+
+To get an initial `DataSet` on which to perform operations to build a data flow
+graph the following construct is used:
+
+```scala
+val input = DataSource("<file-path>", <input-format>)
+```
+
+The value `input` is now a `DataSet` with the generic type depending on the
+input format.
+
+The file path can be on of either `file:///some/file` to acces files on the
+local machine or `hdfs://some/path` to read files from HDFS. The input
+format can be one of our builtin formats or a custom input format. The builtin
+formats are:
+
+* [TextInputFormat](#text-input-format)
+* [CsvInputFormat](#csv-input-format)
+* [DelimitedInputFormat](#delimited-input-format)
+* [BinaryInputFormat](#binary-input-format)
+* [BinarySerializedInputFormat](#binary-serialized-input-format)
+* [FixedLengthInputFormat](#fixed-length-input-format)
+
+We will now have a look at each of them and show how they are employed and in
+which situations.
+
+<section id="text-input-format">
+#### TextInputFormat
+
+This input format simply reads a text file line wise and creates a `String`
+for each line. It is used as:
+
+```scala
+TextInputFormat()
+```
+
+As you have already seen in the Word Count Example there is a shortcut for this.
+Instead of using a `DataSource` with `TextInputFormat` you can simply write:
+
+```scala
+val input = TextFile("<file-path>")
+```
+
+The `input` would then be a `DataSet[String]`.
+
+<section id="csv-input-format">
+#### CsvInputFormat
+
+This input format is mainly used to read Csv-Files, as the name suggests. Input
+files must be text files. You can specify the `String` that should be used
+as the separator between individual records (this would often be newline) and
+also the separator between fields of a record (this would often be a comma).
+The `CsvInputFormat` will automatically read the records and create
+Scala tuples or custom case class objects for you. The format can be used
+in one of the following ways:
+
+```scala
+CsvInputFormat[Out]()
+CsvInputFormat[Out](recordDelim: String)
+CsvInputFormat[Out](recordDelim: String, fieldDelim: Char)
+
+CsvInputFormat[Out](fieldIndices: Seq[Int])
+CsvInputFormat[Out](fieldIndices: Seq[Int], recordDelim: String)
+CsvInputFormat[Out](fieldIndices: Seq[Int], recordDelim: String, fieldDelim: Char)
+```
+
+The default record delimiter is a newline, the default field delimiter is a
+comma. The type parameter `Out` must be a case class type, which also includes
+tuple types since they are internally case classes.
+
+Normally, all the fields of a record are read. If you want to explicitly
+specify which fields of the record should be read you can use one of the
+tree variants with a `fieldIndices` parameter. Here you give a list
+of the fields that should be read. Field indices start from zero.
+
+An example usage could look as follows:
+
+```scala
+val input = DataSource("file:///some/file", CsvInputFormat[(Int, Int, String)](Seq(1, 17, 42), "\n", ','))
+```
+
+Here only the specified fields would be read and 3-tuples created for you.
+The type of input would be `DataSet[(Int, Int, String)]`.
+
+<section id="delimited-input-format">
+#### DelimitedInputFormat
+
+This input format is meant for textual records that are separated by
+some delimiter. The delimiter could be a newline, for example. It is used like
+this:
+
+```scala
+DelimitedInputFormat[Out](parseFunction: String => Out, delim: String = "\n")
+```
+
+The input files will be split on the supplied delimiter (or the default newline)
+and the supplied parse function must parse the textual representation in the
+`String` and return an object. The type of this object will then also be the
+type of the `DataSet` created by the `DataSource` operation.
+
+Just as with `BinaryInputFormat` the function can be an anonymous function, so
+you could have:
+
+```scala
+val input = DataSource("file:///some/file", BinaryInputFormat( { line =>
+  line match {
+    case EdgeInputPattern(from, to) => Path(from.toInt, to.toInt, 1)
+  }
+}))
+```
+
+In this example EdgeInputPattern is some regular expression used for parsing
+a line of text and `Path` is a custom case class that is used to represent
+the data. The type of input would in this case be `DataSet[Path]`.
+
+<section id="binary-input-format">
+#### BinaryInputFormat
+
+This input format is best used when you have a custom binary format that
+you store the data in. It is created using one of the following:
+
+```scala
+BinaryInputFormat[Out](readFunction: DataInput => Out)
+BinaryInputFormat[Out](readFunction: DataInput => Out, blocksize: Long)
+```
+
+So you have to provide a function that gets a
+[java.io.DataInput](http://docs.oracle.com/javase/7/docs/api/java/io/DataInput.html)
+and returns the object that
+contains the data. The type of this object will then also be the type of the
+`DataSet` created by the `DataSource` operation.
+
+The provided function can also be an anonymous function, so you could
+have something like this:
+
+```scala
+val input = DataSource("file:///some/file", BinaryInputFormat( { input =>
+  val one = input.readInt
+  val two = input.readDouble
+  (one, two)  
+}))
+```
+
+Here `input` would be of type `DataSet[(Int, Double)]`.
+
+<section id="binary-serialized-input-format">
+#### BinarySerializedInputFormat
+
+This input format is only meant to be used in conjunction with
+`BinarySerializedOutputFormat`. You can use these to write elements to files using a
+Stratosphere-internal format that can efficiently be read again. You should only
+use this when output is only meant to be consumed by other Stratosphere jobs.
+The format can be used on one of two ways:
+
+```scala
+BinarySerializedInputFormat[Out]()
+BinarySerializedInputFormat[Out](blocksize: Long)
+```
+
+So if input files contain elements of type `(String, Int)` (a tuple type) you
+could use:
+
+```scala
+val input = DataSource("file:///some/file", BinarySerializedInputFormat[(String, Int)]())
+```
+
+<section id="fixed-length-input-format">
+#### FixedLengthInputFormat
+
+This input format is for cases where you want to read binary blocks
+of a fixed size. The size of a block must be specified and you must
+provide code that reads elements from a byte array.
+
+The format is used like this:
+
+```scala
+FixedLengthInputFormat[Out](readFunction: (Array[Byte], Int) => Out, recordLength: Int)
+```
+
+The specified function gets an array and a position at which it must start
+reading the array and returns the element read from the binary data.
+
+
+<section id="operations">
+Operations on DataSet
+---------------------
+
+As explained in [Programming Model](pmodel.html#operators),
+a Stratosphere job is a graph of operators that process data coming from
+sources that is finally written to sinks. When you use the Scala front end
+these operators as well as the graph is created behind the scenes. For example,
+when you write code like this:
+
+```scala
+val input = TextFile("file:///some/file")
+
+val words = input.map { x => (x, 1) }
+
+val output = counts.write(words, CsvOutputFormat()))
+
+val plan = new ScalaPlan(Seq(output))
+```
+
+What you get is a graph that has a data source, a map operator (that contains
+the code written inside the anonymous function block), and a data sink. You 
+do not have to know about this to be able to use the Scala front end but
+it helps to remember, that when you are using Scala you are building
+a data flow graph that processes data only when executed.
+
+There are operations on `DataSet` that correspond to all the types of operators
+that the Stratosphere system supports. We will shortly go trough all of them with
+some examples.
+
+<section id="operator-templates">
+#### Basic Operator Templates
+
+Most of the operations have three similar versions and we will
+explain them here for all of the operators together. The three versions are `map`,
+`flatMap`, and `filter`. All of them accept an anonymous function that
+defines what the operation does but the semantics are different.
+
+The `map` version is a simple one to one mapping. Take a look at the following
+code:
+
+```scala
+val input: DataSet[(String, Int)]
+
+val mapped = input.map { x => (x._1, x._2 + 3) }
+```
+
+This defines a map operator that operates on tuples of String and Int and just
+adds three to the Int (the second fields of the tuple). So, if the input set had
+the tuples (a, 1), (b, 2), and (c, 3) the result after the operator would be
+(a, 4), (b, 5), and (c, 6).
+
+The `flatMap` version works a bit differently,
+here you return something iterable from the anonymous function. The iterable
+could be a list or an array. The elements in this iterable are unnested.
+So for every element in the input data you get a list of elements. The
+concatenation of those is the result of the operator. If you had
+the following code:
+
+```scala
+val input: DataSet[(String, Int)]
+
+val mapped = input.flatMap { x => List( (x._1, x._2), (x._1, x._2 + 1) ) }
+```
+
+and as input the tuples (a, 1) and (b, 1) you would get (a, 1), (a, 2), (b, 1),
+and (b, 2) as result. It is one flat list, and not the individual lists returned
+from the anonymous function.
+
+The third template is `filter`. Here you give an anonymous function that
+returns a Boolean. The elements for which this Boolean is true are part of the
+result of the operation, the others are culled. An example for a filter is this
+code:
+
+
+```scala
+val input: DataSet[(String, Int)]
+
+val mapped = input.filter { x => x._2 >= 3 }
+```
+
+<section id="key-selectors">
+#### Field/Key Selectors
+
+For some operations (group, join, and cogroup) it is necessary to specify which
+parts of a data type are to be considered the key. This key is used for grouping
+elements together for reduce and for joining in case of a join or cogroup.
+In Scala the key is specified using a special anonymous function called
+a key selector. The key selector has as input an element of the type of
+the `DataSet` and must return a single value or a tuple of values that should
+be considered the key. This will become clear with some examples: (Note that
+we use the reduce operation here as an example, we will have a look at
+that further down):
+
+```scala
+val input: DataSet[(String, Int)]
+val reduced = input groupBy { x => (x._1) } reduce { ... }
+val reduced2 = input groupBy { case (w, c) => w } reduce { ... }
+
+case class Test(a: String, b: Int, c: Int)
+val input2: DataSet[Test]
+val reduced3 = input2 groupBy { x => (x.a, x.b) } reduce { ... }
+val reduced4 = input2 groupBy { case Test(x,y,z) => (x,y) } reduce { ... }
+```
+
+The anonymous function block passed to `groupBy` is the key selector. The first
+two examples both specify the `String` field of the tuple as key. In the second
+set of examples we see a custom case class and here we select the first two
+fields as a compound key.
+
+It is worth noting that the key selector function is not actually executed 
+at runtime but it is parsed at job creation time where the key information is
+extracted and stored for efficient computation at runtime.
+
+#### Map Operation
+
+Map is an operation that gets one element at a time and can output one or
+several elements. The operations that result in a `MapOperator` in the graph are exactly
+those mentioned in the previous section. For completeness' sake we will mention
+their signatures here (in this and the following such lists `In` is the
+type of the input data set, `DataSet[In]`):
+
+```scala
+def map[Out](fun: In => Out): DataSet[Out]
+def flatMap[Out](fun: In => Iterator[Out]): DataSet[Out]
+def filter(fun: In => Boolean): DataSet[Out]
+```
+
+#### Reduce Operation
+
+As explained [here](pmodel.html#operators) Reduce is an operation that looks
+at groups of elements at a time and can, for one group, output one or several
+elements. To specify how elements should be grouped you need to give
+a key selection function, as explained [above](#key-selectors).
+
+The basic template of the reduce operation is:
+
+```scala
+input groupBy { <key selector> } reduce { <reduce function> }
+```
+
+The signature of the reduce function depends on the variety of reduce operation
+selected. There are right now three different versions:
+
+```scala
+def reduce(fun: (In, In) => In): DataSet[In]
+
+def reduceGroup[Out](fun: Iterator[In] => Out): DataSet[Out]
+def combinableReduceGroup(fun: Iterator[In] => In): DataSet[In]
+```
+
+The `reduce` variant is like a `reduceLeft` on a Scala collection with
+the limitation that the output data type must be the same as the input data
+type. You specify how to elements of the selection should be combined,
+this is then used to reduce the elements in one group (of the same key)
+down to one element. This can be used to implement aggregation operators,
+for example:
+
+```scala
+val words: DataSet[(String, Int)]
+val counts = words.groupBy { case (word, count) => word}
+  .reduce { (w1, w1) => (w1._1, w1._2 + w2._2) }
+```
+
+This would add up the Int fields of those tuples that have the same String
+in the first fields. As is for example required in Word Count.
+
+The `reduceGroup` variant can be used when more control is required. Here
+your reduce function gets an `Iterator` that can be used to iterate over
+all the elements in a group. With this type or reduce operation the
+output data type can be different from the input data type. An example
+of this kind of operation is this:
+
+```scala
+val words: DataSet[(String, Int)]
+val minCounts = words.groupBy { case (word, count) => word}
+  .reduceGroup { words => words.minBy { _._2 } }
+```
+
+Here we use the minBy function of Scala collections to determine the
+element with the minimum count in a group.
+
+The `combinableGroupReduce` works like the `groupReduce` with the difference
+that the reduce operation is combinable. This is an optimization one can use,
+please have a look at [Programming Model](pmodel.html "Programming Model") for
+the details.
+
+#### Join Operation
+
+The join operation is similar to a database equi-join. It is a two input
+iteration where you have to specify a key selector for each of the inputs
+and then the anonymous function is called for every pair of matching
+elements from the two input sides.
+
+The basic template is:
+
+```scala
+input1 join input2 where { <key selector 1> } isEqualTo { <key selector 2>} map { <join function> }
+```
+
+or, because lines will get to long fast:
+```scala
+input1.join(input2)
+  .where { <key selector 1> }
+  .isEqualTo { <key selector 2>}
+  .map { <join function> }
+```
+
+(Scala can sometimes be quite finicky about where you can omit dots and
+parentheses, so it's best to use dots in multi-line code like this.)
+
+As mentioned in [here](#operator-templates) there are three versions of
+this operator, so you can use one of these in the last position:
+
+```scala
+def map[Out](fun: (LeftIn, RightIn) => Out): DataSet[Out]
+def flatMap[Out](fun: (LeftIn, RightIn) => Iterator[Out]): DataSet[Out]
+def filter(fun: (LeftIn, RightIn) => Boolean): DataSet[(LeftIn, RightIn)]
+```
+
+One example where this can be used is database-style joining with projection:
+
+```scala
+input1.join(input2)
+  .where { case (a, b, c) => (a, b) }
+  .isEqualTo { case (a, b, c, d) => (c, d) }
+  .map { (left, right) => (left._3, right._1) }
+```
+
+Here the join key for the left input is a compound of the first two tuple fields
+while the key for the second input is a compound of the last two fields. We then
+pick one field each from both sides as the result of the operation.
+
+#### CoGroup Operation
+
+The cogroup operation is a cross between join and reduce. It has two inputs
+and you have to specify a key selector for each of them. This is where the
+similarities with join stop. Instead of having one invocation of your user
+code per pair of matching elements all elements from the left and from the right
+are grouped together for one single invocation. In your function you get
+an `Iterator` for the elements from the left input and another `Iterator`
+for the elements from the right input.
+
+The basic template is:
+
+```scala
+input1 cogroup input2 where { <key selector 1> } isEqualTo { <key selector 2>} map { <cogroup function> }
+```
+
+or, because lines will get to long fast:
+```scala
+input1.cogroup(input2)
+  .where { <key selector 1> }
+  .isEqualTo { <key selector 2>}
+  .map { <cogroup function> }
+```
+
+There are to variants you can use, with the semantics explained
+[here](#operator-templates).
+
+```scala
+def map[Out](fun: (Iterator[LeftIn], Iterator[RightIn]) => Out): DataSet[Out]
+def flatMap[Out](fun: (Iterator[LeftIn], Iterator[RightIn]) => Iterator[Out]): DataSet[Out]
+```
+
+#### Cross Operation
+
+The cross operation is used to form the Cartesian product of the elements
+from two inputs. The basic template is:
+
+```scala
+input1 cross input2 map { <cogroup function> }
+```
+
+Again there are three variants, with the semantics explained
+[here](#operator-templates).
+
+```scala
+def map[Out](fun: (LeftIn, RightIn) => Out): DataSet[Out]
+def flatMap[Out](fun: (LeftIn, RightIn) => Iterator[Out]): DataSet[Out]
+def filter(fun: (LeftIn, RightIn) => Boolean): DataSet[(LeftIn, RightIn)]
+```
+
+#### Union
+
+When you want to have the combination of several data sets as the input of
+an operation you can use a union to combine them. It is used like this
+
+```scala
+val input1: DataSet[String]
+val input2: DataSet[String]
+val unioned = input1.union(input2)
+```
+
+The signature of union is:
+
+```scala
+def union(secondInput: DataSet[A])
+```
+
+Where `A` is the generic type of the `DataSet` on which you execute the `union`.
+
+<section id="iterations">
+Iterations
+----------
+
+Iterations allow you to implement *loops* in Stratosphere programs.
+[This page](iterations.html) gives a
+general introduction to iterations. This section here provides quick examples
+of how to use the concepts using the Scala API.
+The iteration operators encapsulate a part of the program and execute it
+repeatedly, feeding back the result of one iteration (the partial solution) into
+the next iteration. Stratosphere has two different types of iterations,
+*Bulk Iteration* and *Delta Iteration*.
+
+For both types of iterations you provide the iteration body as a function
+that has data sets as input and returns a new data set. The difference is
+that bulk iterations map from one data set two one new data set while
+delta iterations map two data sets to two new data sets.
+
+#### Bulk Iteration
+
+The signature of the bulk iterate method is this:
+
+```scala
+def iterate(n: Int, stepFunction: DataSet[A] => DataSet[A])
+```
+
+where `A` is the type of the `DataSet` on which `iterate` is called. The number
+of steps is given in `n`. This is how you use it in practice:
+
+```scala
+val dataPoints = DataSource(dataPointInput, DelimitedInputFormat(parseInput))
+val clusterPoints = DataSource(clusterInput, DelimitedInputFormat(parseInput))
+
+def kMeansStep(centers: DataSet[(Int, Point)]) = {
+
+  val distances = dataPoints cross centers map computeDistance
+  val nearestCenters = distances.groupBy { case (pid, _) => pid }
+    .reduceGroup { ds => ds.minBy(_._2.distance) } map asPointSum.tupled
+  val newCenters = nearestCenters.groupBy { case (cid, _) => cid }
+    .reduceGroup sumPointSums map { case (cid, pSum) => cid -> pSum.toPoint() }
+
+  newCenters
+}
+
+val finalCenters = clusterPoints.iterate(numIterations, kMeansStep)
+
+val output = finalCenters.write(clusterOutput, DelimitedOutputFormat(formatOutput.tupled))
+```
+
+Not that we use some functions here which we don't show. If you want, you
+can check out the complete code in our KMeans example.
+
+#### Delta Iteration
+
+The signature of the delta iterate method is this:
+
+```scala
+def iterateWithDelta(workset: DataSet[W], solutionSetKey: A => K, stepFunction: (DataSet[A], DataSet[W]) => (DataSet[A], DataSet[W]), maxIterations: Int)
+```
+
+where `A` is the type of the `DataSet` on which `iterateWithDelta` is called,
+`W` is the type of the `DataSet` that represents the workset and `K` is the
+key type. The maximum number of iterations must always be given.
+
+For information on how delta iterations in general work on our system, please
+refer to [iterations](iterations.html). A working example job is
+available here:
+[Scala Connected Components Example](examples_scala.html#connected_components) 
+
+
+<section id="data-sinks">
+Creating Data Sinks
+-------------------
+
+The creation of data sinks is analog to the creation of data sources. `DataSet`
+has a `write` method that is used to create a sink that writes the output
+of the operation to a file in the local file system or HDFS. The general pattern
+is this:
+
+```scala
+val sink = out.write("<file-path>", <output-format>)
+```
+
+Where `out` is some `DataSet`. Just as for data sources, the file path can be
+on of either `file:///some/file` to acces files on the local machine or
+`hdfs://some/path` to read files from HDFS. The output format can be one of our
+builtin formats or a custom output format. The builtin formats are:
+
+* [DelimitedOutputFormat](#delimited-output-format)
+* [CsvOutputFormat](#csv-output-format)
+* [RawOutputFormat](#raw-output-format)
+* [BinaryOutputFormat](#binary-output-format)
+* [BinarySerializedOutputFormat](#binary-serialized-output-format)
+
+We will now have a look at each of them and show how they are employed and in
+which situations.
+
+<section id="delimited-output-format">
+#### DelimitedOutputFormat
+
+This output format is meant for writing textual records that are separated by
+some delimiter. The delimiter could be a newline, for example. It is used like
+this:
+
+```scala
+DelimitedOutputFormat[In](formatFunction: In => String, delim: String = "\n")
+```
+
+For every element in the `DataSet` the formatting function is called and
+the result of that is appended to the output file. In between the elements
+the `delim` string is inserted.
+
+An example would be:
+
+```scala
+val out: DataSet[(String, Int)]
+val sink = out.write("file:///some/file", DelimitedOutputFormat( { elem =>
+  "%s|%d".format(elem._1, elem._2)
+}))
+```
+
+Here we use Scala String formatting to write the two fields of the tuple
+separated by a pipe character. The default newline delimiter will be inserted
+between the elements in the output files.
+
+<section id="csv-output-format">
+#### CsvOutputFormat
+
+This output format can be used to automatically write fields of tuple
+elements or case classes to CSV files. You can specify what separator should
+be used between fields of an element and also the separator between elements.
+
+```scala
+CsvOutputFormat[In]()
+CsvOutputFormat[In](recordDelim: String)
+CsvOutputFormat[In](recordDelim: String, fieldDelim: Char)
+```
+
+The default record delimiter is a newline, the default field delimiter is a
+comma. 
+
+An example usage could look as follows:
+
+```scala
+val out: DataSet[(String, Int)]
+val sink = out.write("file:///some/file", CsvOutputFormat())
+```
+
+Notice how we don't need to specify the generic type here, it is inferred.
+
+<section id="raw-output-format">
+#### RawOutputFormat
+
+This input format can be used when you want to have complete control over
+what gets written. You get an
+[OutputStream](http://docs.oracle.com/javase/7/docs/api/java/io/OutputStream.html)
+and can write the elements of the `DataSet` exactly as you see fit.
+
+A `RawOutputFormat` is created like this:
+
+```scala
+RawOutputFormat[In](writeFunction: (In, OutputStream) => Unit)
+```
+
+The function you pass in gets one element from the `DataSet` and must
+write it to the given `OutputStream`. An example would be the following:
+
+```scala
+val out: DataSet[(String, Int)]
+val sink = out.write("file:///some/file", RawOutputFormat( { (elem, output) =>
+  /* write elem._1 and elem._2 to output */ 
+}))
+```
+
+<section id="binary-output-format">
+#### BinaryOutputFormat
+
+This format is very similar to the RawOutputFormat. The difference is that
+instead of an [OutputStream](http://docs.oracle.com/javase/7/docs/api/java/io/OutputStream.html)
+you get a [DataOutput](http://docs.oracle.com/javase/7/docs/api/java/io/DataOutput.html)
+to which you can write binary data. You can also specify the block size for
+the binary output file. When you don't specify a block size some default
+is used.
+
+A `BinaryOutputFormat` is created like this:
+
+```scala
+BinaryOutputFormat[In](writeFunction: (In, DataOutput) => Unit)
+BinaryOutputFormat[In](writeFunction: (In, DataOutput) => Unit, blockSize: Long)
+```
+
+<section id="binary-serialized-output-format">
+#### BinarySerializedOutputFormat
+
+This output format is only meant to be used in conjunction with
+`BinarySerializedInputFormat`. You can use these to write elements to files using a
+Stratosphere-internal format that can efficiently be read again. You should only
+use this when output is only meant to be consumed by other Stratosphere jobs.
+The output format can be used on one of two ways:
+
+```scala
+BinarySerializedOutputFormat[In]()
+BinarySerializedOutputFormat[In](blocksize: Long)
+```
+
+So to write elements of some `DataSet[A]` to a binary file you could use:
+
+```scala
+val out: DataSet[(String, Int)]
+val sink = out.write("file:///some/file", BinarySerializedInputFormat())
+```
+
+As you can see the type of the elements need not be specified, it is inferred
+by Scala.
+
+<section id="execution">
+Executing Jobs
+--------------
+
+To execute a data flow graph the sinks need to be wrapped in a
+[ScalaPlan](https://github.com/stratosphere/stratosphere/blob/release-{{site.current_stable}}/stratosphere-scala/src/main/scala/eu/stratosphere/api/scala/ScalaPlan.scala)
+object like this:
+
+```scala
+val out: DataSet[(String, Int)]
+val sink = out.write("file:///some/file", CsvOutputFormat())
+
+val plan = new ScalaPlan(Seq(sink))
+```
+
+You can put several sinks into the `Seq` that is passed to the constructor.
+
+There are two ways one can execute a data flow plan: local execution and
+remote/cluster execution. When using local execution the plan is executed on
+the local computer. This is handy while developing jobs because you can
+easily debug your code and iterate quickly. When a job is ready to be
+used on bigger data sets it can be executed on a cluster. We will
+now give an example for each of the two execution modes.
+
+First up is local execution:
+
+```scala
+import eu.stratosphere.client.LocalExecutor
+
+...
+
+val plan: ScalaPlan = ...
+LocalExecutor.execute(plan)
+```
+
+This is all there is to it.
+
+Remote (or cluster) execution is a bit more complicated because you have
+to package your code in a jar file so that it can be distributed on the cluster.
+Have a look at the [scala quickstart](/quickstart/scala.html) to see how you
+can set up a maven project that does the packaging. Remote execution is done
+using the [RemoteExecutor](https://github.com/stratosphere/stratosphere/blob/release-{{site.current_stable}}/stratosphere-clients/src/main/java/eu/stratosphere/client/RemoteExecutor.java), like this:
+
+```scala
+import eu.stratosphere.client.RemoteExecutor
+
+...
+
+val plan: ScalaPlan = ...
+val ex = new RemoteExecutor("<job manager ip address>", <job manager port>, "your.jar");
+ex.executePlan(plan);
+```
+
+The IP address and the port of the Stratosphere job manager depend on your
+setup. Have a look at [cluster quickstart](/quickstart/setup.html) for a quick
+guide about how to set up a cluster. The default cluster port is 6123, so
+if you run a job manger on your local computer you can give this and "localhost"
+as the first to parameters to the `RemoteExecutor` constructor.
+
+<section id="rich-functions">
+Rich Functions
+--------------
+
+Sometimes having a single function that is passed to an operation is not enough.
+Using Rich Functions it is possible to have state inside your operations and
+have code executed before the first element is processed and after the last
+element is processed. For example, instead of a simple function as in this
+example:
+
+```scala
+val mapped = input map { x => x + 1 }
+```
+
+you can have a rich function like this:
+
+```scala
+val mapped = input map( new MapFunction[(String, Int), (String, Int)] {
+  val someState: SomeType = ...
+  override def open(config: Configuration) = {
+    // one-time initialization code
+  }
+
+  override def close() = {
+    // one-time clean-up code
+  }
+
+  override def apply(in: (String, Int)) = {
+    // do complex stuff
+    val result = ...
+    result
+  }
+})
+```
+
+You could also create a custom class that derives from `MapFunction`
+instead of the anonymous class we used here.
+
+There are rich functions for all the various operator types. The basic
+template is the some, though. The common interface that they implement 
+is [Function](https://github.com/stratosphere/stratosphere/blob/release-{{site.current_stable}}/stratosphere-core/src/main/java/eu/stratosphere/api/common/functions/Function.java). The `open` and `close` methods can be overridden to run set-up
+and tear-down code. The other methods can be used in a rich function to
+work with the runtime context which gives information about the context
+of the operator. Your operation code must now reside in an `apply` method
+that has the same signature as the anonymous function you would normally
+supply.
+
+The rich functions reside in the package `eu.stratosphere.api.scala.functions`.
+This is a list of all the rich functions can can be used instead of
+simple functions in the respective operations:
+
+```scala
+abstract class MapFunction[In, Out] 
+abstract class FlatMapFunction[In, Out] 
+abstract class FilterFunction[In, Out] 
+
+abstract class ReduceFunction[In]
+abstract class GroupReduceFunction[In, Out]
+abstract class CombinableGroupReduceFunction[In, Out]
+
+abstract class JoinFunction[LeftIn, RightIn, Out]
+abstract class FlatJoinFunction[LeftIn, RightIn, Out]
+
+abstract class CoGroupFunction[LeftIn, RightIn, Out]
+abstract class FlatCoGroupFunction[LeftIn, RightIn, Out]
+
+abstract class CrossFunction[LeftIn, RightIn, Out]
+abstract class FlatCrossFunction[LeftIn, RightIn, Out]
+```
+
+Note that for all the rich stubs, you need to specify the generic type of
+the input (or inputs) and the output type.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/scala_api_quickstart.md
----------------------------------------------------------------------
diff --git a/docs/scala_api_quickstart.md b/docs/scala_api_quickstart.md
new file mode 100644
index 0000000..e15eed0
--- /dev/null
+++ b/docs/scala_api_quickstart.md
@@ -0,0 +1,71 @@
+---
+title: "Quick Start: Scala API"
+---
+
+<p class="lead">Start working on your Stratosphere Scala program in a few simple steps.</p>
+
+<section id="requirements">
+  <div class="page-header"><h2>Requirements</h2></div>
+  <p class="lead">The only requirements are working <strong>Maven 3.0.4</strong> (or higher) and <strong>Java 6.x</strong> (or higher) installations.</p>
+</section>
+
+<section id="create_project">
+  <div class="page-header"><h2>Create Project</h2></div>
+  <p class="lead">Use one of the following commands to <strong>create a project</strong>:</p>
+
+  <ul class="nav nav-tabs" style="border-bottom: none;">
+      <li class="active"><a href="#quickstart-script" data-toggle="tab">Run the <strong>quickstart script</strong></a></li>
+      <li><a href="#maven-archetype" data-toggle="tab">Use <strong>Maven archetypes</strong></a></li>
+  </ul>
+  <div class="tab-content">
+      <div class="tab-pane active" id="quickstart-script">
+{% highlight bash %}
+$ curl https://raw.githubusercontent.com/stratosphere/stratosphere-quickstart/master/quickstart-scala.sh | bash
+{% endhighlight %}
+      </div>
+      <div class="tab-pane" id="maven-archetype">
+{% highlight bash %}
+$ mvn archetype:generate                             \
+    -DarchetypeGroupId=eu.stratosphere               \
+    -DarchetypeArtifactId=quickstart-scala           \
+    -DarchetypeVersion={{site.current_stable}}                  
+{% endhighlight %}
+      This allows you to <strong>name your newly created project</strong>. It will interactively ask you for the groupId, artifactId, and package name.
+      </div>
+  </div>
+</section>
+
+<section id="inspect_project">
+  <div class="page-header"><h2>Inspect Project</h2></div>
+  <p class="lead">There will be a <strong>new directory in your working directory</strong>. If you've used the <em>curl</em> approach, the directory is called <code>quickstart</code>. Otherwise, it has the name of your artifactId.</p>
+  <p class="lead">The sample project is a <strong>Maven project</strong>, which contains a sample scala <em>Job</em> that implements Word Count. Please note that the <em>RunJobLocal</em> and <em>RunJobRemote</em> objects allow you to start Stratosphere in a development/testing mode.</p>
+  <p class="lead">We recommend to <strong>import this project into your IDE</strong>. For Eclipse, you need the following plugins, which you can install from the provided Eclipse Update Sites:
+    <ul>
+      <li class="lead"><strong>Eclipse 4.x</strong>:
+        <ul>
+          <li><strong>Scala IDE</strong> <small>(http://download.scala-ide.org/sdk/e38/scala210/stable/site)</small></li>
+          <li><strong>m2eclipse-scala</strong> <small>(http://alchim31.free.fr/m2e-scala/update-site)</small></li>
+          <li><strong>Build Helper Maven Plugin</strong> <small>(https://repository.sonatype.org/content/repositories/forge-sites/m2e-extras/0.15.0/N/0.15.0.201206251206/)</small></li>
+        </ul>
+      </li>
+      <li class="lead"><strong>Eclipse 3.7</strong>:
+        <ul>
+          <li><strong>Scala IDE</strong> <small>(http://download.scala-ide.org/sdk/e37/scala210/stable/site)</small></li>
+          <li><strong>m2eclipse-scala</strong> <small>(http://alchim31.free.fr/m2e-scala/update-site)</small></li>
+          <li><strong>Build Helper Maven Plugin</strong> <small>(https://repository.sonatype.org/content/repositories/forge-sites/m2e-extras/0.14.0/N/0.14.0.201109282148/)</small></li>
+        </ul>
+      </li>
+    </ul>
+  </p>
+  <p class="lead">The IntelliJ IDE also supports Maven and offers a plugin for Scala development.</p>
+</section>
+
+<section id="build_project">
+  <div class="page-header"><h2>Build Project</h2></div>
+  <p class="lead">If you want to <strong>build your project</strong>, go to your project directory and issue the <code>mvn clean package</code> command. You will <strong>find a jar</strong> that runs on every Stratosphere cluster in <code>target/stratosphere-project-0.1-SNAPSHOT.jar</code>.</p>
+</section>
+
+<section id="next_steps">
+  <div class="page-header"><h2>Next Steps</h2></div>
+  <p class="lead"><strong>Write your application!</strong> If you have any trouble, ask on our <a href="https://github.com/stratosphere/stratosphere/issues">GitHub page</a> (open an issue) or on our <a href="https://groups.google.com/forum/#!forum/stratosphere-dev">Mailing list</a>. We are happy to provide help.</p>
+</p>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/setup_quickstart.md
----------------------------------------------------------------------
diff --git a/docs/setup_quickstart.md b/docs/setup_quickstart.md
new file mode 100644
index 0000000..debe21c
--- /dev/null
+++ b/docs/setup_quickstart.md
@@ -0,0 +1,132 @@
+---
+title: "Quickstart: Setup"
+---
+
+<p class="lead">Get Stratosphere up and running in a few simple steps.</p>
+
+<section id="requirements">
+  <div class="page-header"><h2>Requirements</h2></div>
+  <p class="lead">Stratosphere runs on all <em>UNIX-like</em> environments: <strong>Linux</strong>, <strong>Mac OS X</strong>, <strong>Cygwin</strong>. The only requirement is to have a working <strong>Java 6.x</strong> (or higher) installation.</p>
+</section>
+
+<section id="download">
+  <div class="page-header"><h2>Download</h2></div>
+  <p class="lead">Download the ready to run binary package. Choose the Stratosphere distribution that <strong>matches your Hadoop version</strong>. If you are unsure which version to choose or you just want to run locally, pick the package for Hadoop 1.2.</p>
+  <p>
+  	<ul class="nav nav-tabs">
+  		<li class="active"><a href="#bin-hadoop1" data-toggle="tab">Hadoop 1.2</a></li>
+      <li><a href="#bin-hadoop2" data-toggle="tab">Hadoop 2 (YARN)</a></li>
+		</ul>
+		<div class="tab-content text-center">
+			<div class="tab-pane active" id="bin-hadoop1">
+				<a class="btn btn-info btn-lg" onclick="_gaq.push(['_trackEvent','Action','download-quickstart-setup-1',this.href]);" href="{{site.current_stable_dl}}"><i class="icon-download"> </i> Download Stratosphere for Hadoop 1.2</a>
+	    </div>
+			<div class="tab-pane" id="bin-hadoop2">
+	      <a class="btn btn-info btn-lg" onclick="_gaq.push(['_trackEvent','Action','download-quickstart-setup-2',this.href]);" href="{{site.current_stable_dl_yarn}}"><i class="icon-download"> </i> Download Stratosphere for Hadoop 2 (YARN)</a>
+	    </div>
+	  </div>
+	</p>
+</section>
+
+<section id="start">
+  <div class="page-header"><h2>Start</h2></div> 
+  <p class="lead">You are almost done.</p>
+  <ol>
+  	<li class="lead"><strong>Go to the download directory</strong>,</li>
+  	<li class="lead"><strong>Unpack the downloaded archive</strong>, and</li>
+  	<li class="lead"><strong>Start Stratosphere</strong>.</li>
+  </ol>
+
+{% highlight bash %}
+$ cd ~/Downloads              # Go to download directory
+$ tar xzf stratosphere-*.tgz  # Unpack the downloaded archive
+$ cd stratosphere
+$ bin/start-local.sh          # Start Stratosphere
+{% endhighlight %}
+
+  <p class="lead">Check the <strong>JobManager's web frontend</strong> at <a href="http://localhost:8081">http://localhost:8081</a> and make sure everything is up and running.</p>
+</section>
+
+<section id="example">
+  <div class="page-header"><h2>Run Example</h2></div>
+  <p class="lead">Run the <strong>Word Count example</strong> to see Stratosphere at work.</p>
+
+  <ol>
+  	<li class="lead"><strong>Download test data:</strong>
+{% highlight bash %}
+$ wget -O hamlet.txt http://www.gutenberg.org/cache/epub/1787/pg1787.txt
+{% endhighlight %}
+		  You now have a text file called <em>hamlet.txt</em> in your working directory.
+		</li>
+  	<li class="lead"><strong>Start the example program</strong>:
+{% highlight bash %}
+$ bin/stratosphere run \
+    --jarfile ./examples/stratosphere-java-examples-{{site.current_stable}}-WordCount.jar \
+    --arguments file://`pwd`/hamlet.txt file://`pwd`/wordcount-result.txt
+{% endhighlight %}
+      You will find a file called <strong>wordcount-result.txt</strong> in your current directory.
+  	</li>
+  </ol>
+
+</section>
+
+<section id="cluster">
+  <div class="page-header"><h2>Cluster Setup</h2></div>
+  <p class="lead"><strong>Running Stratosphere on a cluster</strong> is as easy as running it locally. Having <strong>passwordless SSH</strong> and <strong>the same directory structure</strong> on all your cluster nodes lets you use our scripts to control everything.</p>
+  <ol>
+  	<li class="lead">Copy the unpacked <strong>stratosphere</strong> directory from the downloaded archive to the same file system path on each node of your setup.</li>
+  	<li class="lead">Choose a <strong>master node</strong> (JobManager) and set the <code>jobmanager.rpc.address</code> key in <code>conf/stratosphere-conf.yaml</code> to its IP or hostname. Make sure that all nodes in your cluster have the same <code>jobmanager.rpc.address</code> configured.</li>
+  	<li class="lead">Add the IPs or hostnames (one per line) of all <strong>worker nodes</strong> (TaskManager) to the slaves files in <code>conf/slaves</code>.</li>
+  </ol>
+  <p class="lead">You can now <strong>start the cluster</strong> at your master node with <code>bin/start-cluster.sh</code>.</p>
+  <p class="lead">
+    The following <strong>example</strong> illustrates the setup with three nodes (with IP addresses from <em>10.0.0.1</em> to <em>10.0.0.3</em> and hostnames <em>master</em>, <em>worker1</em>, <em>worker2</em>) and shows the contents of the configuration files, which need to be accessible at the same path on all machines:
+  </p>
+  <div class="row">
+    <div class="col-md-6 text-center">
+      <img src="{{ site.baseurl }}/img/quickstart_cluster.png" style="width: 85%">
+    </div>
+    <div class="col-md-6">
+      <div class="row">
+        <p class="lead text-center">
+        /path/to/<strong>stratosphere/conf/<br>stratosphere-conf.yaml</strong>
+<pre>
+jobmanager.rpc.address: 10.0.0.1
+</pre>
+        </p>
+      </div>
+      <div class="row" style="margin-top: 1em;">
+        <p class="lead text-center">
+        /path/to/<strong>stratosphere/<br>conf/slaves</strong>
+<pre>
+10.0.0.2
+10.0.0.3
+</pre>
+        </p>
+      </div>
+    </div>
+  </div>
+</section>
+
+<section id="yarn">
+  <div class="page-header"><h2>Stratosphere on YARN</h2></div>
+  <p class="lead">You can easily deploy Stratosphere on your existing <strong>YARN cluster</strong>. 
+    <ol>
+    <li class="lead">Download the <strong>Stratosphere YARN package</strong> with the YARN client:
+      <div class="text-center" style="padding: 1em;">
+      <a style="padding-left:10px" onclick="_gaq.push(['_trackEvent','Action','download-quickstart-yarn',this.href]);" class="btn btn-info btn-lg" href="{{site.current_stable_uberjar}}"><i class="icon-download"> </i> Stratosphere {{ site.current_stable }} for YARN</a>
+      </div>
+    </li>
+    <li class="lead">Make sure your <strong>HADOOP_HOME</strong> (or <em>YARN_CONF_DIR</em> or <em>HADOOP_CONF_DIR</em>) <strong>environment variable</strong> is set to read your YARN and HDFS configuration.</li>
+    <li class="lead">Run the <strong>YARN client</strong> with:
+      <div class="text-center" style="padding:1em;">
+        <code>./bin/yarn-session.sh</code>
+      </div>
+      
+      You can run the client with options <code>-n 10 -tm 8192</code> to allocate 10 TaskManagers with 8GB of memory each.</li>
+  </ol>
+  </p>
+</section>
+
+<hr />
+<p class="lead">For <strong>more detailed instructions</strong>, check out the <a href="{{site.baseurl}}/docs/{{site.current_stable_documentation}}">Documentation</a>.</p>
\ No newline at end of file


[13/53] [abbrv] git commit: Replaced Tarjan's algorithm with a simpler depth-first traversal cycle detection algorithm. By doing so, one gets rid off a possible linear time check whether a node is contained in the stack.

Posted by rm...@apache.org.
Replaced Tarjan's algorithm with a simpler depth-first traversal cycle detection algorithm. By doing so, one gets rid off a possible linear time check whether a node is contained in the stack.


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

Branch: refs/heads/travis_test
Commit: fba44a94b9e3dd7f87c391278f17ae55393dd51b
Parents: 93bc0b9
Author: Till Rohrmann <ti...@gmail.com>
Authored: Wed Apr 9 14:32:55 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sun Jun 22 21:07:20 2014 +0200

----------------------------------------------------------------------
 .../stratosphere/nephele/jobgraph/JobGraph.java | 76 ++++++++------------
 1 file changed, 28 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/fba44a94/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobGraph.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobGraph.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobGraph.java
index 41fd907..804a258 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobGraph.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobGraph.java
@@ -424,18 +424,14 @@ public class JobGraph implements IOReadableWritable {
 	 */
 	public boolean isAcyclic() {
 
-		// Tarjan's algorithm to detect strongly connected componenent of a graph
 		final AbstractJobVertex[] reachable = getAllReachableJobVertices();
-		final HashMap<AbstractJobVertex, Integer> indexMap = new HashMap<AbstractJobVertex, Integer>();
-		final HashMap<AbstractJobVertex, Integer> lowLinkMap = new HashMap<AbstractJobVertex, Integer>();
-		final Stack<AbstractJobVertex> stack = new Stack<AbstractJobVertex>();
-		final Integer index = Integer.valueOf(0);
 
-		for (int i = 0; i < reachable.length; i++) {
-			if (!indexMap.containsKey(reachable[i])) {
-				if (!tarjan(reachable[i], index, indexMap, lowLinkMap, stack)) {
-					return false;
-				}
+		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;
 			}
 		}
 
@@ -443,51 +439,35 @@ public class JobGraph implements IOReadableWritable {
 	}
 
 	/**
-	 * Auxiliary method implementing Tarjan's algorithm for strongly-connected components to determine whether the job
-	 * graph is acyclic.
+	 * 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 tarjan(final AbstractJobVertex jv, Integer index,
-			final HashMap<AbstractJobVertex, Integer> indexMap, final HashMap<AbstractJobVertex, Integer> lowLinkMap,
-			final Stack<AbstractJobVertex> stack) {
-
-		indexMap.put(jv, Integer.valueOf(index));
-		lowLinkMap.put(jv, Integer.valueOf(index));
-		index = Integer.valueOf(index.intValue() + 1);
-		stack.push(jv);
-
-		for (int i = 0; i < jv.getNumberOfForwardConnections(); i++) {
+	private boolean detectCycle(final AbstractJobVertex jv, final HashSet<JobVertexID> temporarilyMarked,
+								final HashSet<JobVertexID> permanentlyMarked){
+		JobVertexID vertexID = jv.getID();
 
-			final AbstractJobVertex jv2 = jv.getForwardConnection(i).getConnectedVertex();
-			if (!indexMap.containsKey(jv2) || stack.contains(jv2)) {
-				if (!indexMap.containsKey(jv2)) {
-					if (!tarjan(jv2, index, indexMap, lowLinkMap, stack)) {
-						return false;
-					}
-				}
-				if (lowLinkMap.get(jv) > lowLinkMap.get(jv2)) {
-					lowLinkMap.put(jv, Integer.valueOf(lowLinkMap.get(jv2)));
-				}
-			}
-		}
-
-		if (lowLinkMap.get(jv).equals(indexMap.get(jv))) {
+		if(permanentlyMarked.contains(vertexID)){
+			return false;
+		}else if(temporarilyMarked.contains(vertexID)){
+			return true;
+		}else{
+			temporarilyMarked.add(vertexID);
 
-			int count = 0;
-			while (stack.size() > 0) {
-				final AbstractJobVertex jv2 = stack.pop();
-				if (jv == jv2) {
-					break;
+			for(int i = 0; i < jv.getNumberOfForwardConnections(); i++){
+				if(detectCycle(jv.getForwardConnection(i).getConnectedVertex(), temporarilyMarked, permanentlyMarked)){
+					return true;
 				}
-
-				count++;
 			}
 
-			if (count > 0) {
-				return false;
-			}
+			permanentlyMarked.add(vertexID);
+			return false;
 		}
-
-		return true;
 	}
 
 	/**


[03/53] [abbrv] Rework the Taskmanager to a slot based model and remove legacy cloud code

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/CombineTaskExternalITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/CombineTaskExternalITCase.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/CombineTaskExternalITCase.java
index 6fd366e..2d8113b 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/CombineTaskExternalITCase.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/CombineTaskExternalITCase.java
@@ -33,6 +33,8 @@ import eu.stratosphere.types.Record;
 public class CombineTaskExternalITCase extends DriverTestBase<GenericGroupReduce<Record, ?>> {
 	
 	private static final long COMBINE_MEM = 3 * 1024 * 1024;
+
+	private final double combine_frac;
 	
 	private final ArrayList<Record> outList = new ArrayList<Record>();
 	
@@ -42,6 +44,8 @@ public class CombineTaskExternalITCase extends DriverTestBase<GenericGroupReduce
 
 	public CombineTaskExternalITCase() {
 		super(COMBINE_MEM, 0);
+
+		combine_frac = (double)COMBINE_MEM/this.getMemoryManager().getMemorySize();
 	}
 
 	
@@ -55,7 +59,7 @@ public class CombineTaskExternalITCase extends DriverTestBase<GenericGroupReduce
 		setOutput(this.outList);
 		
 		getTaskConfig().setDriverStrategy(DriverStrategy.SORTED_GROUP_COMBINE);
-		getTaskConfig().setMemoryDriver(COMBINE_MEM);
+		getTaskConfig().setRelativeMemoryDriver(combine_frac);
 		getTaskConfig().setFilehandlesDriver(2);
 		
 		final GroupReduceCombineDriver<Record> testTask = new GroupReduceCombineDriver<Record>();
@@ -108,7 +112,7 @@ public class CombineTaskExternalITCase extends DriverTestBase<GenericGroupReduce
 		setOutput(this.outList);
 		
 		getTaskConfig().setDriverStrategy(DriverStrategy.SORTED_GROUP_COMBINE);
-		getTaskConfig().setMemoryDriver(COMBINE_MEM);
+		getTaskConfig().setRelativeMemoryDriver(combine_frac);
 		getTaskConfig().setFilehandlesDriver(2);
 		
 		final GroupReduceCombineDriver<Record> testTask = new GroupReduceCombineDriver<Record>();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/CombineTaskTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/CombineTaskTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/CombineTaskTest.java
index 0917051..98e7003 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/CombineTaskTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/CombineTaskTest.java
@@ -39,6 +39,8 @@ import eu.stratosphere.util.Collector;
 public class CombineTaskTest extends DriverTestBase<GenericGroupReduce<Record, ?>>
 {
 	private static final long COMBINE_MEM = 3 * 1024 * 1024;
+
+	private final double combine_frac;
 	
 	private final ArrayList<Record> outList = new ArrayList<Record>();
 	
@@ -48,6 +50,8 @@ public class CombineTaskTest extends DriverTestBase<GenericGroupReduce<Record, ?
 
 	public CombineTaskTest() {
 		super(COMBINE_MEM, 0);
+
+		combine_frac = (double)COMBINE_MEM/this.getMemoryManager().getMemorySize();
 	}
 	
 	@Test
@@ -60,7 +64,7 @@ public class CombineTaskTest extends DriverTestBase<GenericGroupReduce<Record, ?
 		setOutput(this.outList);
 		
 		getTaskConfig().setDriverStrategy(DriverStrategy.SORTED_GROUP_COMBINE);
-		getTaskConfig().setMemoryDriver(COMBINE_MEM);
+		getTaskConfig().setRelativeMemoryDriver(combine_frac);
 		getTaskConfig().setFilehandlesDriver(2);
 		
 		final GroupReduceCombineDriver<Record> testTask = new GroupReduceCombineDriver<Record>();
@@ -96,7 +100,7 @@ public class CombineTaskTest extends DriverTestBase<GenericGroupReduce<Record, ?
 		setOutput(new DiscardingOutputCollector<Record>());
 		
 		getTaskConfig().setDriverStrategy(DriverStrategy.SORTED_GROUP_COMBINE);
-		getTaskConfig().setMemoryDriver(COMBINE_MEM);
+		getTaskConfig().setRelativeMemoryDriver(combine_frac);
 		getTaskConfig().setFilehandlesDriver(2);
 		
 		final GroupReduceCombineDriver<Record> testTask = new GroupReduceCombineDriver<Record>();
@@ -120,7 +124,7 @@ public class CombineTaskTest extends DriverTestBase<GenericGroupReduce<Record, ?
 		setOutput(new DiscardingOutputCollector<Record>());
 		
 		getTaskConfig().setDriverStrategy(DriverStrategy.SORTED_GROUP_COMBINE);
-		getTaskConfig().setMemoryDriver(COMBINE_MEM);
+		getTaskConfig().setRelativeMemoryDriver(combine_frac);
 		getTaskConfig().setFilehandlesDriver(2);
 		
 		final GroupReduceCombineDriver<Record> testTask = new GroupReduceCombineDriver<Record>();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/CrossTaskExternalITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/CrossTaskExternalITCase.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/CrossTaskExternalITCase.java
index 19e6209..fdf1941 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/CrossTaskExternalITCase.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/CrossTaskExternalITCase.java
@@ -27,11 +27,14 @@ import eu.stratosphere.types.Record;
 public class CrossTaskExternalITCase extends DriverTestBase<GenericCrosser<Record, Record, Record>>
 {
 	private static final long CROSS_MEM = 1024 * 1024;
+
+	private final double cross_frac;
 	
 	private final CountingOutputCollector output = new CountingOutputCollector();
 
 	public CrossTaskExternalITCase() {
 		super(CROSS_MEM, 0);
+		cross_frac = (double)CROSS_MEM/this.getMemoryManager().getMemorySize();
 	}
 	
 	@Test
@@ -52,7 +55,7 @@ public class CrossTaskExternalITCase extends DriverTestBase<GenericCrosser<Recor
 		addInput(new UniformRecordGenerator(keyCnt2, valCnt2, false));
 				
 		getTaskConfig().setDriverStrategy(DriverStrategy.NESTEDLOOP_BLOCKED_OUTER_FIRST);
-		getTaskConfig().setMemoryDriver(CROSS_MEM);
+		getTaskConfig().setRelativeMemoryDriver(cross_frac);
 		
 		final CrossDriver<Record, Record, Record> testTask = new CrossDriver<Record, Record, Record>();
 		
@@ -84,7 +87,7 @@ public class CrossTaskExternalITCase extends DriverTestBase<GenericCrosser<Recor
 		addInput(new UniformRecordGenerator(keyCnt2, valCnt2, false));
 				
 		getTaskConfig().setDriverStrategy(DriverStrategy.NESTEDLOOP_STREAMED_OUTER_FIRST);
-		getTaskConfig().setMemoryDriver(CROSS_MEM);
+		getTaskConfig().setRelativeMemoryDriver(cross_frac);
 		
 		final CrossDriver<Record, Record, Record> testTask = new CrossDriver<Record, Record, Record>();
 		

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/CrossTaskTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/CrossTaskTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/CrossTaskTest.java
index 8667a6f..baa9589 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/CrossTaskTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/CrossTaskTest.java
@@ -32,11 +32,15 @@ import eu.stratosphere.util.Collector;
 public class CrossTaskTest extends DriverTestBase<GenericCrosser<Record, Record, Record>>
 {
 	private static final long CROSS_MEM = 1024 * 1024;
+
+	private final double cross_frac;
 	
 	private final CountingOutputCollector output = new CountingOutputCollector();
 
 	public CrossTaskTest() {
 		super(CROSS_MEM, 0);
+
+		cross_frac = (double)CROSS_MEM/this.getMemoryManager().getMemorySize();
 	}
 	
 	@Test
@@ -56,7 +60,7 @@ public class CrossTaskTest extends DriverTestBase<GenericCrosser<Record, Record,
 		addInput(new UniformRecordGenerator(keyCnt2, valCnt2, false));
 				
 		getTaskConfig().setDriverStrategy(DriverStrategy.NESTEDLOOP_BLOCKED_OUTER_FIRST);
-		getTaskConfig().setMemoryDriver(CROSS_MEM);
+		getTaskConfig().setRelativeMemoryDriver(cross_frac);
 		
 		final CrossDriver<Record, Record, Record> testTask = new CrossDriver<Record, Record, Record>();
 		
@@ -86,7 +90,7 @@ public class CrossTaskTest extends DriverTestBase<GenericCrosser<Record, Record,
 		addInput(new UniformRecordGenerator(keyCnt2, valCnt2, false));
 				
 		getTaskConfig().setDriverStrategy(DriverStrategy.NESTEDLOOP_BLOCKED_OUTER_SECOND);
-		getTaskConfig().setMemoryDriver(CROSS_MEM);
+		getTaskConfig().setRelativeMemoryDriver(cross_frac);
 		
 		final CrossDriver<Record, Record, Record> testTask = new CrossDriver<Record, Record, Record>();
 		
@@ -114,7 +118,7 @@ public class CrossTaskTest extends DriverTestBase<GenericCrosser<Record, Record,
 		addInput(new UniformRecordGenerator(keyCnt2, valCnt2, false));
 				
 		getTaskConfig().setDriverStrategy(DriverStrategy.NESTEDLOOP_BLOCKED_OUTER_FIRST);
-		getTaskConfig().setMemoryDriver(CROSS_MEM);
+		getTaskConfig().setRelativeMemoryDriver(cross_frac);
 		
 		final CrossDriver<Record, Record, Record> testTask = new CrossDriver<Record, Record, Record>();
 		
@@ -144,7 +148,7 @@ public class CrossTaskTest extends DriverTestBase<GenericCrosser<Record, Record,
 		addInput(new UniformRecordGenerator(keyCnt2, valCnt2, false));
 				
 		getTaskConfig().setDriverStrategy(DriverStrategy.NESTEDLOOP_BLOCKED_OUTER_SECOND);
-		getTaskConfig().setMemoryDriver(CROSS_MEM);
+		getTaskConfig().setRelativeMemoryDriver(cross_frac);
 		
 		final CrossDriver<Record, Record, Record> testTask = new CrossDriver<Record, Record, Record>();
 		
@@ -175,7 +179,7 @@ public class CrossTaskTest extends DriverTestBase<GenericCrosser<Record, Record,
 		addInput(new UniformRecordGenerator(keyCnt2, valCnt2, false));
 				
 		getTaskConfig().setDriverStrategy(DriverStrategy.NESTEDLOOP_STREAMED_OUTER_FIRST);
-		getTaskConfig().setMemoryDriver(CROSS_MEM);
+		getTaskConfig().setRelativeMemoryDriver(cross_frac);
 		
 		final CrossDriver<Record, Record, Record> testTask = new CrossDriver<Record, Record, Record>();
 		
@@ -206,7 +210,7 @@ public class CrossTaskTest extends DriverTestBase<GenericCrosser<Record, Record,
 		addInput(new UniformRecordGenerator(keyCnt2, valCnt2, false));
 				
 		getTaskConfig().setDriverStrategy(DriverStrategy.NESTEDLOOP_STREAMED_OUTER_SECOND);
-		getTaskConfig().setMemoryDriver(CROSS_MEM);
+		getTaskConfig().setRelativeMemoryDriver(cross_frac);
 		
 		final CrossDriver<Record, Record, Record> testTask = new CrossDriver<Record, Record, Record>();
 		
@@ -234,7 +238,7 @@ public class CrossTaskTest extends DriverTestBase<GenericCrosser<Record, Record,
 		addInput(new UniformRecordGenerator(keyCnt2, valCnt2, false));
 				
 		getTaskConfig().setDriverStrategy(DriverStrategy.NESTEDLOOP_STREAMED_OUTER_FIRST);
-		getTaskConfig().setMemoryDriver(CROSS_MEM);
+		getTaskConfig().setRelativeMemoryDriver(cross_frac);
 		
 		final CrossDriver<Record, Record, Record> testTask = new CrossDriver<Record, Record, Record>();
 		
@@ -263,7 +267,7 @@ public class CrossTaskTest extends DriverTestBase<GenericCrosser<Record, Record,
 		addInput(new UniformRecordGenerator(keyCnt2, valCnt2, false));
 				
 		getTaskConfig().setDriverStrategy(DriverStrategy.NESTEDLOOP_STREAMED_OUTER_SECOND);
-		getTaskConfig().setMemoryDriver(CROSS_MEM);
+		getTaskConfig().setRelativeMemoryDriver(cross_frac);
 		
 		final CrossDriver<Record, Record, Record> testTask = new CrossDriver<Record, Record, Record>();
 		
@@ -294,7 +298,7 @@ public class CrossTaskTest extends DriverTestBase<GenericCrosser<Record, Record,
 		addInput(new UniformRecordGenerator(keyCnt2, valCnt2, false));
 				
 		getTaskConfig().setDriverStrategy(DriverStrategy.NESTEDLOOP_STREAMED_OUTER_FIRST);
-		getTaskConfig().setMemoryDriver(CROSS_MEM);
+		getTaskConfig().setRelativeMemoryDriver(cross_frac);
 		
 		final CrossDriver<Record, Record, Record> testTask = new CrossDriver<Record, Record, Record>();
 		
@@ -324,7 +328,7 @@ public class CrossTaskTest extends DriverTestBase<GenericCrosser<Record, Record,
 		addInput(new UniformRecordGenerator(keyCnt2, valCnt2, false));
 				
 		getTaskConfig().setDriverStrategy(DriverStrategy.NESTEDLOOP_STREAMED_OUTER_SECOND);
-		getTaskConfig().setMemoryDriver(CROSS_MEM);
+		getTaskConfig().setRelativeMemoryDriver(cross_frac);
 		
 		final CrossDriver<Record, Record, Record> testTask = new CrossDriver<Record, Record, Record>();
 		
@@ -354,7 +358,7 @@ public class CrossTaskTest extends DriverTestBase<GenericCrosser<Record, Record,
 		addInput(new UniformRecordGenerator(keyCnt2, valCnt2, false));
 				
 		getTaskConfig().setDriverStrategy(DriverStrategy.NESTEDLOOP_BLOCKED_OUTER_FIRST);
-		getTaskConfig().setMemoryDriver(CROSS_MEM);
+		getTaskConfig().setRelativeMemoryDriver(cross_frac);
 		
 		final CrossDriver<Record, Record, Record> testTask = new CrossDriver<Record, Record, Record>();
 		
@@ -384,7 +388,7 @@ public class CrossTaskTest extends DriverTestBase<GenericCrosser<Record, Record,
 		addInput(new UniformRecordGenerator(keyCnt2, valCnt2, false));
 				
 		getTaskConfig().setDriverStrategy(DriverStrategy.NESTEDLOOP_BLOCKED_OUTER_SECOND);
-		getTaskConfig().setMemoryDriver(CROSS_MEM);
+		getTaskConfig().setRelativeMemoryDriver(cross_frac);
 		
 		final CrossDriver<Record, Record, Record> testTask = new CrossDriver<Record, Record, Record>();
 		
@@ -411,7 +415,7 @@ public class CrossTaskTest extends DriverTestBase<GenericCrosser<Record, Record,
 		addInput(new DelayingInfinitiveInputIterator(100));
 		
 		getTaskConfig().setDriverStrategy(DriverStrategy.NESTEDLOOP_BLOCKED_OUTER_FIRST);
-		getTaskConfig().setMemoryDriver(CROSS_MEM);
+		getTaskConfig().setRelativeMemoryDriver(cross_frac);
 		
 		final CrossDriver<Record, Record, Record> testTask = new CrossDriver<Record, Record, Record>();
 		
@@ -454,7 +458,7 @@ public class CrossTaskTest extends DriverTestBase<GenericCrosser<Record, Record,
 		addInput(new DelayingInfinitiveInputIterator(100));
 		
 		getTaskConfig().setDriverStrategy(DriverStrategy.NESTEDLOOP_BLOCKED_OUTER_SECOND);
-		getTaskConfig().setMemoryDriver(CROSS_MEM);
+		getTaskConfig().setRelativeMemoryDriver(cross_frac);
 		
 		final CrossDriver<Record, Record, Record> testTask = new CrossDriver<Record, Record, Record>();
 		
@@ -497,7 +501,7 @@ public class CrossTaskTest extends DriverTestBase<GenericCrosser<Record, Record,
 		addInput(new DelayingInfinitiveInputIterator(100));
 		
 		getTaskConfig().setDriverStrategy(DriverStrategy.NESTEDLOOP_STREAMED_OUTER_FIRST);
-		getTaskConfig().setMemoryDriver(CROSS_MEM);
+		getTaskConfig().setRelativeMemoryDriver(cross_frac);
 		
 		final CrossDriver<Record, Record, Record> testTask = new CrossDriver<Record, Record, Record>();
 		
@@ -540,7 +544,7 @@ public class CrossTaskTest extends DriverTestBase<GenericCrosser<Record, Record,
 		addInput(new DelayingInfinitiveInputIterator(100));
 		
 		getTaskConfig().setDriverStrategy(DriverStrategy.NESTEDLOOP_STREAMED_OUTER_SECOND);
-		getTaskConfig().setMemoryDriver(CROSS_MEM);
+		getTaskConfig().setRelativeMemoryDriver(cross_frac);
 		
 		final CrossDriver<Record, Record, Record> testTask = new CrossDriver<Record, Record, Record>();
 		

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/DataSinkTaskTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/DataSinkTaskTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/DataSinkTaskTest.java
index bfd0d42..67f9fe8 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/DataSinkTaskTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/DataSinkTaskTest.java
@@ -42,13 +42,13 @@ import eu.stratosphere.types.IntValue;
 import eu.stratosphere.types.Key;
 import eu.stratosphere.types.Record;
 
-public class DataSinkTaskTest extends TaskTestBase {
-
-	private static final int MEMORY_MANAGER_SIZE = 1024 * 1024;
+public class DataSinkTaskTest extends TaskTestBase
+{
+	private static final Log LOG = LogFactory.getLog(DataSinkTaskTest.class);
+	
+	private static final int MEMORY_MANAGER_SIZE = 3 * 1024 * 1024;
 
 	private static final int NETWORK_BUFFER_SIZE = 1024;
-
-	private static final Log LOG = LogFactory.getLog(DataSinkTaskTest.class);
 	
 	private final String tempTestPath = Path.constructTestPath("dst_test");
 	
@@ -65,7 +65,7 @@ public class DataSinkTaskTest extends TaskTestBase {
 
 		int keyCnt = 100;
 		int valCnt = 20;
-
+		
 		super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
 		super.addInput(new UniformRecordGenerator(keyCnt, valCnt, false), 0);
 		
@@ -131,7 +131,7 @@ public class DataSinkTaskTest extends TaskTestBase {
 
 		int keyCnt = 100;
 		int valCnt = 20;
-
+		
 		super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
 		super.addInput(new UniformRecordGenerator(keyCnt, valCnt, 0, 0, false), 0);
 		super.addInput(new UniformRecordGenerator(keyCnt, valCnt, keyCnt, 0, false), 0);
@@ -201,8 +201,9 @@ public class DataSinkTaskTest extends TaskTestBase {
 
 		int keyCnt = 100;
 		int valCnt = 20;
-
-		super.initEnvironment(MEMORY_MANAGER_SIZE * 4, NETWORK_BUFFER_SIZE);
+		double memoryFraction = 1.0;
+		
+		super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
 		super.addInput(new UniformRecordGenerator(keyCnt, valCnt, true), 0);
 		
 		DataSinkTask<Record> testTask = new DataSinkTask<Record>();
@@ -210,8 +211,9 @@ public class DataSinkTaskTest extends TaskTestBase {
 		// set sorting
 		super.getTaskConfig().setInputLocalStrategy(0, LocalStrategy.SORT);
 		super.getTaskConfig().setInputComparator(
-				new RecordComparatorFactory(new int[]{1},((Class<? extends Key<?>>[])new Class[]{IntValue.class})), 0);
-		super.getTaskConfig().setMemoryInput(0, 4 * 1024 * 1024);
+				new RecordComparatorFactory(new int[]{1},((Class<? extends Key<?>>[])new Class[]{IntValue.class})),
+				0);
+		super.getTaskConfig().setRelativeMemoryInput(0, memoryFraction);
 		super.getTaskConfig().setFilehandlesInput(0, 8);
 		super.getTaskConfig().setSpillingThresholdInput(0, 0.8f);
 
@@ -279,7 +281,7 @@ public class DataSinkTaskTest extends TaskTestBase {
 
 		int keyCnt = 100;
 		int valCnt = 20;
-
+		
 		super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
 		super.addInput(new UniformRecordGenerator(keyCnt, valCnt, false), 0);
 
@@ -310,9 +312,10 @@ public class DataSinkTaskTest extends TaskTestBase {
 	public void testFailingSortingDataSinkTask() {
 
 		int keyCnt = 100;
-		int valCnt = 20;
-
-		super.initEnvironment(MEMORY_MANAGER_SIZE * 4, NETWORK_BUFFER_SIZE);
+		int valCnt = 20;;
+		double memoryFraction = 1.0;
+		
+		super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
 		super.addInput(new UniformRecordGenerator(keyCnt, valCnt, true), 0);
 
 		DataSinkTask<Record> testTask = new DataSinkTask<Record>();
@@ -322,8 +325,9 @@ public class DataSinkTaskTest extends TaskTestBase {
 		// set sorting
 		super.getTaskConfig().setInputLocalStrategy(0, LocalStrategy.SORT);
 		super.getTaskConfig().setInputComparator(
-				new RecordComparatorFactory(new int[]{1},((Class<? extends Key<?>>[])new Class[]{IntValue.class})), 0);
-		super.getTaskConfig().setMemoryInput(0, 4 * 1024 * 1024);
+				new RecordComparatorFactory(new int[]{1},((Class<? extends Key<?>>[])new Class[]{IntValue.class})),
+				0);
+		super.getTaskConfig().setRelativeMemoryInput(0, memoryFraction);
 		super.getTaskConfig().setFilehandlesInput(0, 8);
 		super.getTaskConfig().setSpillingThresholdInput(0, 0.8f);
 		
@@ -347,7 +351,7 @@ public class DataSinkTaskTest extends TaskTestBase {
 	
 	@Test
 	public void testCancelDataSinkTask() {
-
+		
 		super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
 		super.addInput(new InfiniteInputIterator(), 0);
 		
@@ -389,8 +393,9 @@ public class DataSinkTaskTest extends TaskTestBase {
 	@Test
 	@SuppressWarnings("unchecked")
 	public void testCancelSortingDataSinkTask() {
-
-		super.initEnvironment(MEMORY_MANAGER_SIZE * 4, NETWORK_BUFFER_SIZE);
+		double memoryFraction = 1.0;
+		
+		super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
 		super.addInput(new InfiniteInputIterator(), 0);
 		
 		final DataSinkTask<Record> testTask = new DataSinkTask<Record>();
@@ -402,7 +407,7 @@ public class DataSinkTaskTest extends TaskTestBase {
 		super.getTaskConfig().setInputComparator(
 				new RecordComparatorFactory(new int[]{1},((Class<? extends Key<?>>[])new Class[]{IntValue.class})), 
 				0);
-		super.getTaskConfig().setMemoryInput(0, 4 * 1024 * 1024);
+		super.getTaskConfig().setRelativeMemoryInput(0, memoryFraction);
 		super.getTaskConfig().setFilehandlesInput(0, 8);
 		super.getTaskConfig().setSpillingThresholdInput(0, 0.8f);
 		

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/MatchTaskExternalITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/MatchTaskExternalITCase.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/MatchTaskExternalITCase.java
index 7ea8ea4..dd77059 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/MatchTaskExternalITCase.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/MatchTaskExternalITCase.java
@@ -35,7 +35,11 @@ public class MatchTaskExternalITCase extends DriverTestBase<GenericJoiner<Record
 	private static final long SORT_MEM = 3*1024*1024;
 	
 	private static final long BNLJN_MEM = 10 * PAGE_SIZE;
-	
+
+	private final double bnljn_frac;
+
+	private final double hash_frac;
+
 	@SuppressWarnings("unchecked")
 	private final RecordComparator comparator1 = new RecordComparator(
 		new int[]{0}, (Class<? extends Key<?>>[])new Class[]{ IntValue.class });
@@ -48,6 +52,8 @@ public class MatchTaskExternalITCase extends DriverTestBase<GenericJoiner<Record
 	
 	public MatchTaskExternalITCase() {
 		super(HASH_MEM, 2, SORT_MEM);
+		bnljn_frac = (double)BNLJN_MEM/this.getMemoryManager().getMemorySize();
+		hash_frac = (double)HASH_MEM/this.getMemoryManager().getMemorySize();
 	}
 	
 	@Test
@@ -65,7 +71,7 @@ public class MatchTaskExternalITCase extends DriverTestBase<GenericJoiner<Record
 		addInputComparator(this.comparator2);
 		getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get());
 		getTaskConfig().setDriverStrategy(DriverStrategy.MERGE);
-		getTaskConfig().setMemoryDriver(BNLJN_MEM);
+		getTaskConfig().setRelativeMemoryDriver(bnljn_frac);
 		setNumFileHandlesForSort(4);
 		
 		final MatchDriver<Record, Record, Record> testTask = new MatchDriver<Record, Record, Record>();
@@ -99,7 +105,7 @@ public class MatchTaskExternalITCase extends DriverTestBase<GenericJoiner<Record
 		getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get());
 		setOutput(this.output);
 		getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_FIRST);
-		getTaskConfig().setMemoryDriver(HASH_MEM);
+		getTaskConfig().setRelativeMemoryDriver(hash_frac);
 		
 		MatchDriver<Record, Record, Record> testTask = new MatchDriver<Record, Record, Record>();
 		
@@ -130,7 +136,7 @@ public class MatchTaskExternalITCase extends DriverTestBase<GenericJoiner<Record
 		getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get());
 		setOutput(this.output);
 		getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND);
-		getTaskConfig().setMemoryDriver(HASH_MEM);
+		getTaskConfig().setRelativeMemoryDriver(hash_frac);
 		
 		MatchDriver<Record, Record, Record> testTask = new MatchDriver<Record, Record, Record>();
 		

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/MatchTaskTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/MatchTaskTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/MatchTaskTest.java
index ce5a8c5..de56c0b 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/MatchTaskTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/MatchTaskTest.java
@@ -40,8 +40,14 @@ public class MatchTaskTest extends DriverTestBase<GenericJoiner<Record, Record,
 	private static final long HASH_MEM = 6*1024*1024;
 	
 	private static final long SORT_MEM = 3*1024*1024;
+
+	private static final int NUM_SORTER = 2;
 	
 	private static final long BNLJN_MEM = 10 * PAGE_SIZE;
+
+	private final double bnljn_frac;
+
+	private final double hash_frac;
 	
 	@SuppressWarnings("unchecked")
 	private final RecordComparator comparator1 = new RecordComparator(
@@ -55,7 +61,9 @@ public class MatchTaskTest extends DriverTestBase<GenericJoiner<Record, Record,
 	
 	
 	public MatchTaskTest() {
-		super(HASH_MEM, 2, SORT_MEM);
+		super(HASH_MEM, NUM_SORTER, SORT_MEM);
+		bnljn_frac = (double)BNLJN_MEM/this.getMemoryManager().getMemorySize();
+		hash_frac = (double)HASH_MEM/this.getMemoryManager().getMemorySize();
 	}
 	
 	
@@ -72,7 +80,7 @@ public class MatchTaskTest extends DriverTestBase<GenericJoiner<Record, Record,
 		addInputComparator(this.comparator2);
 		getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get());
 		getTaskConfig().setDriverStrategy(DriverStrategy.MERGE);
-		getTaskConfig().setMemoryDriver(BNLJN_MEM);
+		getTaskConfig().setRelativeMemoryDriver(bnljn_frac);
 		setNumFileHandlesForSort(4);
 		
 		final MatchDriver<Record, Record, Record> testTask = new MatchDriver<Record, Record, Record>();
@@ -106,7 +114,7 @@ public class MatchTaskTest extends DriverTestBase<GenericJoiner<Record, Record,
 		addInputComparator(this.comparator2);
 		getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get());
 		getTaskConfig().setDriverStrategy(DriverStrategy.MERGE);
-		getTaskConfig().setMemoryDriver(BNLJN_MEM);
+		getTaskConfig().setRelativeMemoryDriver(bnljn_frac);
 		setNumFileHandlesForSort(4);
 		
 		final MatchDriver<Record, Record, Record> testTask = new MatchDriver<Record, Record, Record>();
@@ -142,7 +150,7 @@ public class MatchTaskTest extends DriverTestBase<GenericJoiner<Record, Record,
 		addInputComparator(this.comparator2);
 		getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get());
 		getTaskConfig().setDriverStrategy(DriverStrategy.MERGE);
-		getTaskConfig().setMemoryDriver(BNLJN_MEM);
+		getTaskConfig().setRelativeMemoryDriver(bnljn_frac);
 		setNumFileHandlesForSort(4);
 		
 		final MatchDriver<Record, Record, Record> testTask = new MatchDriver<Record, Record, Record>();
@@ -178,7 +186,7 @@ public class MatchTaskTest extends DriverTestBase<GenericJoiner<Record, Record,
 		addInputComparator(this.comparator2);
 		getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get());
 		getTaskConfig().setDriverStrategy(DriverStrategy.MERGE);
-		getTaskConfig().setMemoryDriver(BNLJN_MEM);
+		getTaskConfig().setRelativeMemoryDriver(bnljn_frac);
 		setNumFileHandlesForSort(4);
 		
 		final MatchDriver<Record, Record, Record> testTask = new MatchDriver<Record, Record, Record>();
@@ -214,7 +222,7 @@ public class MatchTaskTest extends DriverTestBase<GenericJoiner<Record, Record,
 		addInputComparator(this.comparator2);
 		getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get());
 		getTaskConfig().setDriverStrategy(DriverStrategy.MERGE);
-		getTaskConfig().setMemoryDriver(BNLJN_MEM);
+		getTaskConfig().setRelativeMemoryDriver(bnljn_frac);
 		setNumFileHandlesForSort(4);
 		
 		final MatchDriver<Record, Record, Record> testTask = new MatchDriver<Record, Record, Record>();
@@ -250,7 +258,7 @@ public class MatchTaskTest extends DriverTestBase<GenericJoiner<Record, Record,
 		addInputComparator(this.comparator2);
 		getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get());
 		getTaskConfig().setDriverStrategy(DriverStrategy.MERGE);
-		getTaskConfig().setMemoryDriver(BNLJN_MEM);
+		getTaskConfig().setRelativeMemoryDriver(bnljn_frac);
 		setNumFileHandlesForSort(4);
 		
 		final MatchDriver<Record, Record, Record> testTask = new MatchDriver<Record, Record, Record>();
@@ -286,7 +294,7 @@ public class MatchTaskTest extends DriverTestBase<GenericJoiner<Record, Record,
 		addInputComparator(this.comparator2);
 		getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get());
 		getTaskConfig().setDriverStrategy(DriverStrategy.MERGE);
-		getTaskConfig().setMemoryDriver(BNLJN_MEM);
+		getTaskConfig().setRelativeMemoryDriver(bnljn_frac);
 		setNumFileHandlesForSort(4);
 		
 		final MatchDriver<Record, Record, Record> testTask = new MatchDriver<Record, Record, Record>();
@@ -321,7 +329,7 @@ public class MatchTaskTest extends DriverTestBase<GenericJoiner<Record, Record,
 		addInputComparator(this.comparator2);
 		getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get());
 		getTaskConfig().setDriverStrategy(DriverStrategy.MERGE);
-		getTaskConfig().setMemoryDriver(BNLJN_MEM);
+		getTaskConfig().setRelativeMemoryDriver(bnljn_frac);
 		setNumFileHandlesForSort(4);
 		
 		final MatchDriver<Record, Record, Record> testTask = new MatchDriver<Record, Record, Record>();
@@ -357,7 +365,7 @@ public class MatchTaskTest extends DriverTestBase<GenericJoiner<Record, Record,
 		addInputComparator(this.comparator2);
 		getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get());
 		getTaskConfig().setDriverStrategy(DriverStrategy.MERGE);
-		getTaskConfig().setMemoryDriver(BNLJN_MEM);
+		getTaskConfig().setRelativeMemoryDriver(bnljn_frac);
 		setNumFileHandlesForSort(4);
 		
 		final MatchDriver<Record, Record, Record> testTask = new MatchDriver<Record, Record, Record>();
@@ -386,7 +394,7 @@ public class MatchTaskTest extends DriverTestBase<GenericJoiner<Record, Record,
 		addInputComparator(this.comparator2);
 		getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get());
 		getTaskConfig().setDriverStrategy(DriverStrategy.MERGE);
-		getTaskConfig().setMemoryDriver(BNLJN_MEM);
+		getTaskConfig().setRelativeMemoryDriver(bnljn_frac);
 		setNumFileHandlesForSort(4);
 		
 		final MatchDriver<Record, Record, Record> testTask = new MatchDriver<Record, Record, Record>();
@@ -437,7 +445,7 @@ public class MatchTaskTest extends DriverTestBase<GenericJoiner<Record, Record,
 		addInputComparator(this.comparator2);
 		getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get());
 		getTaskConfig().setDriverStrategy(DriverStrategy.MERGE);
-		getTaskConfig().setMemoryDriver(BNLJN_MEM);
+		getTaskConfig().setRelativeMemoryDriver(bnljn_frac);
 		setNumFileHandlesForSort(4);
 		
 		final MatchDriver<Record, Record, Record> testTask = new MatchDriver<Record, Record, Record>();
@@ -488,7 +496,7 @@ public class MatchTaskTest extends DriverTestBase<GenericJoiner<Record, Record,
 		addInputComparator(this.comparator2);
 		getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get());
 		getTaskConfig().setDriverStrategy(DriverStrategy.MERGE);
-		getTaskConfig().setMemoryDriver(BNLJN_MEM);
+		getTaskConfig().setRelativeMemoryDriver(bnljn_frac);
 		setNumFileHandlesForSort(4);
 		
 		final MatchDriver<Record, Record, Record> testTask = new MatchDriver<Record, Record, Record>();
@@ -539,7 +547,7 @@ public class MatchTaskTest extends DriverTestBase<GenericJoiner<Record, Record,
 		getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get());
 		setOutput(this.outList);
 		getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_FIRST);
-		getTaskConfig().setMemoryDriver(HASH_MEM);
+		getTaskConfig().setRelativeMemoryDriver(hash_frac);
 		
 		MatchDriver<Record, Record, Record> testTask = new MatchDriver<Record, Record, Record>();
 		
@@ -570,7 +578,7 @@ public class MatchTaskTest extends DriverTestBase<GenericJoiner<Record, Record,
 		getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get());
 		setOutput(this.outList);
 		getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND);
-		getTaskConfig().setMemoryDriver(HASH_MEM);
+		getTaskConfig().setRelativeMemoryDriver(hash_frac);
 		
 		MatchDriver<Record, Record, Record> testTask = new MatchDriver<Record, Record, Record>();
 		
@@ -601,7 +609,7 @@ public class MatchTaskTest extends DriverTestBase<GenericJoiner<Record, Record,
 		getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get());
 		setOutput(this.outList);
 		getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_FIRST);
-		getTaskConfig().setMemoryDriver(HASH_MEM);
+		getTaskConfig().setRelativeMemoryDriver(hash_frac);
 		
 		MatchDriver<Record, Record, Record> testTask = new MatchDriver<Record, Record, Record>();
 		
@@ -632,7 +640,7 @@ public class MatchTaskTest extends DriverTestBase<GenericJoiner<Record, Record,
 		getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get());
 		setOutput(this.outList);
 		getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND);
-		getTaskConfig().setMemoryDriver(HASH_MEM);
+		getTaskConfig().setRelativeMemoryDriver(hash_frac);
 		
 		MatchDriver<Record, Record, Record> testTask = new MatchDriver<Record, Record, Record>();
 		
@@ -663,7 +671,7 @@ public class MatchTaskTest extends DriverTestBase<GenericJoiner<Record, Record,
 		getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get());
 		setOutput(this.outList);
 		getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_FIRST);
-		getTaskConfig().setMemoryDriver(HASH_MEM);
+		getTaskConfig().setRelativeMemoryDriver(hash_frac);
 		
 		MatchDriver<Record, Record, Record> testTask = new MatchDriver<Record, Record, Record>();
 		
@@ -694,7 +702,7 @@ public class MatchTaskTest extends DriverTestBase<GenericJoiner<Record, Record,
 		getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get());
 		setOutput(new NirvanaOutputList());
 		getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_FIRST);
-		getTaskConfig().setMemoryDriver(HASH_MEM);
+		getTaskConfig().setRelativeMemoryDriver(hash_frac);
 		
 		MatchDriver<Record, Record, Record> testTask = new MatchDriver<Record, Record, Record>();
 		
@@ -724,7 +732,7 @@ public class MatchTaskTest extends DriverTestBase<GenericJoiner<Record, Record,
 		getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get());
 		setOutput(new NirvanaOutputList());
 		getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND);
-		getTaskConfig().setMemoryDriver(HASH_MEM);
+		getTaskConfig().setRelativeMemoryDriver(hash_frac);
 		
 		MatchDriver<Record, Record, Record> testTask = new MatchDriver<Record, Record, Record>();
 		
@@ -755,7 +763,7 @@ public class MatchTaskTest extends DriverTestBase<GenericJoiner<Record, Record,
 		setOutput(new NirvanaOutputList());
 		
 		getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_FIRST);
-		getTaskConfig().setMemoryDriver(HASH_MEM);
+		getTaskConfig().setRelativeMemoryDriver(hash_frac);
 		
 		final MatchDriver<Record, Record, Record> testTask = new MatchDriver<Record, Record, Record>();
 		
@@ -799,7 +807,7 @@ public class MatchTaskTest extends DriverTestBase<GenericJoiner<Record, Record,
 		getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get());
 		setOutput(new NirvanaOutputList());
 		getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND);
-		getTaskConfig().setMemoryDriver(HASH_MEM);
+		getTaskConfig().setRelativeMemoryDriver(hash_frac);
 		
 		final MatchDriver<Record, Record, Record> testTask = new MatchDriver<Record, Record, Record>();
 		
@@ -843,7 +851,7 @@ public class MatchTaskTest extends DriverTestBase<GenericJoiner<Record, Record,
 		getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get());
 		setOutput(new NirvanaOutputList());
 		getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_FIRST);
-		getTaskConfig().setMemoryDriver(HASH_MEM);
+		getTaskConfig().setRelativeMemoryDriver(hash_frac);
 		
 		final MatchDriver<Record, Record, Record> testTask = new MatchDriver<Record, Record, Record>();
 		
@@ -887,7 +895,7 @@ public class MatchTaskTest extends DriverTestBase<GenericJoiner<Record, Record,
 		getTaskConfig().setDriverPairComparator(RecordPairComparatorFactory.get());
 		setOutput(new NirvanaOutputList());
 		getTaskConfig().setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND);
-		getTaskConfig().setMemoryDriver(HASH_MEM);
+		getTaskConfig().setRelativeMemoryDriver(hash_frac);
 		
 		final MatchDriver<Record, Record, Record> testTask = new MatchDriver<Record, Record, Record>();
 		

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/ReduceTaskExternalITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/ReduceTaskExternalITCase.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/ReduceTaskExternalITCase.java
index 7b14137..bd3524d 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/ReduceTaskExternalITCase.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/ReduceTaskExternalITCase.java
@@ -130,7 +130,9 @@ public class ReduceTaskExternalITCase extends DriverTestBase<GenericGroupReduce<
 		try {
 			sorter = new CombiningUnilateralSortMerger<Record>(new MockCombiningReduceStub(), 
 				getMemoryManager(), getIOManager(), new UniformRecordGenerator(keyCnt, valCnt, false), 
-				getOwningNepheleTask(), RecordSerializerFactory.get(), this.comparator.duplicate(), this.perSortMem, 2, 0.8f);
+				getOwningNepheleTask(), RecordSerializerFactory.get(), this.comparator.duplicate(),
+					this.perSortFractionMem,
+					2, 0.8f);
 			addInput(sorter.getIterator());
 			
 			GroupReduceDriver<Record, Record> testTask = new GroupReduceDriver<Record, Record>();
@@ -174,7 +176,9 @@ public class ReduceTaskExternalITCase extends DriverTestBase<GenericGroupReduce<
 		try {
 			sorter = new CombiningUnilateralSortMerger<Record>(new MockCombiningReduceStub(), 
 				getMemoryManager(), getIOManager(), new UniformRecordGenerator(keyCnt, valCnt, false), 
-				getOwningNepheleTask(), RecordSerializerFactory.get(), this.comparator.duplicate(), this.perSortMem, 2, 0.8f);
+				getOwningNepheleTask(), RecordSerializerFactory.get(), this.comparator.duplicate(),
+					this.perSortFractionMem,
+					2, 0.8f);
 			addInput(sorter.getIterator());
 			
 			GroupReduceDriver<Record, Record> testTask = new GroupReduceDriver<Record, Record>();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/ReduceTaskTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/ReduceTaskTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/ReduceTaskTest.java
index ad859f4..a968ce2 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/ReduceTaskTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/ReduceTaskTest.java
@@ -125,7 +125,8 @@ public class ReduceTaskTest extends DriverTestBase<GenericGroupReduce<Record, Re
 		try {
 			sorter = new CombiningUnilateralSortMerger<Record>(new MockCombiningReduceStub(), 
 				getMemoryManager(), getIOManager(), new UniformRecordGenerator(keyCnt, valCnt, false), 
-				getOwningNepheleTask(), RecordSerializerFactory.get(), this.comparator.duplicate(), this.perSortMem, 4, 0.8f);
+				getOwningNepheleTask(), RecordSerializerFactory.get(), this.comparator.duplicate(), this.perSortFractionMem,
+					4, 0.8f);
 			addInput(sorter.getIterator());
 			
 			GroupReduceDriver<Record, Record> testTask = new GroupReduceDriver<Record, Record>();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/chaining/ChainTaskTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/chaining/ChainTaskTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/chaining/ChainTaskTest.java
index dda215e..3e9a04e 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/chaining/ChainTaskTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/chaining/ChainTaskTest.java
@@ -43,11 +43,11 @@ import eu.stratosphere.util.LogUtils;
 
 
 public class ChainTaskTest extends TaskTestBase {
-
+	
 	private static final int MEMORY_MANAGER_SIZE = 1024 * 1024 * 3;
 
 	private static final int NETWORK_BUFFER_SIZE = 1024;
-
+	
 	private final List<Record> outList = new ArrayList<Record>();
 	
 	@SuppressWarnings("unchecked")
@@ -67,10 +67,13 @@ public class ChainTaskTest extends TaskTestBase {
 	public void testMapTask() {
 		final int keyCnt = 100;
 		final int valCnt = 20;
+
+		final double memoryFraction = 1.0;
 		
 		try {
+		
 			// environment
-			super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
+			initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
 			addInput(new UniformRecordGenerator(keyCnt, valCnt, false), 0);
 			addOutput(this.outList);
 			
@@ -89,7 +92,7 @@ public class ChainTaskTest extends TaskTestBase {
 				// driver
 				combineConfig.setDriverStrategy(DriverStrategy.SORTED_GROUP_COMBINE);
 				combineConfig.setDriverComparator(compFact, 0);
-				combineConfig.setMemoryDriver(3 * 1024 * 1024);
+				combineConfig.setRelativeMemoryDriver(memoryFraction);
 				
 				// udf
 				combineConfig.setStubWrapper(new UserCodeClassWrapper<MockReduceStub>(MockReduceStub.class));
@@ -123,10 +126,14 @@ public class ChainTaskTest extends TaskTestBase {
 	public void testFailingMapTask() {
 		int keyCnt = 100;
 		int valCnt = 20;
+
+		final long memorySize = 1024 * 1024 * 3;
+		final int bufferSize = 1014*1024;
+		final double memoryFraction = 1.0;
 		
 		try {
 			// environment
-			super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
+			initEnvironment(memorySize, bufferSize);
 			addInput(new UniformRecordGenerator(keyCnt, valCnt, false), 0);
 			addOutput(this.outList);
 	
@@ -145,7 +152,7 @@ public class ChainTaskTest extends TaskTestBase {
 				// driver
 				combineConfig.setDriverStrategy(DriverStrategy.SORTED_GROUP_COMBINE);
 				combineConfig.setDriverComparator(compFact, 0);
-				combineConfig.setMemoryDriver(3 * 1024 * 1024);
+				combineConfig.setRelativeMemoryDriver(memoryFraction);
 				
 				// udf
 				combineConfig.setStubWrapper(new UserCodeClassWrapper<MockFailingCombineStub>(MockFailingCombineStub.class));

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/drivers/ReduceCombineDriverTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/drivers/ReduceCombineDriverTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/drivers/ReduceCombineDriverTest.java
index ef88cb9..4467e30 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/drivers/ReduceCombineDriverTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/drivers/ReduceCombineDriverTest.java
@@ -43,7 +43,7 @@ public class ReduceCombineDriverTest {
 		try {
 			TestTaskContext<GenericReduce<Tuple2<String, Integer>>, Tuple2<String, Integer>> context =
 					new TestTaskContext<GenericReduce<Tuple2<String,Integer>>, Tuple2<String,Integer>>(1024 * 1024);
-			context.getTaskConfig().setMemoryDriver(512 * 1024);
+			context.getTaskConfig().setRelativeMemoryDriver(0.5);
 			
 			List<Tuple2<String, Integer>> data = DriverTestData.createReduceImmutableData();
 			Collections.shuffle(data);
@@ -80,7 +80,7 @@ public class ReduceCombineDriverTest {
 			{
 				TestTaskContext<GenericReduce<Tuple2<String, Integer>>, Tuple2<String, Integer>> context =
 						new TestTaskContext<GenericReduce<Tuple2<String,Integer>>, Tuple2<String,Integer>>(1024 * 1024);
-				context.getTaskConfig().setMemoryDriver(512 * 1024);
+				context.getTaskConfig().setRelativeMemoryDriver(0.5);
 				
 				List<Tuple2<String, Integer>> data = DriverTestData.createReduceImmutableData();
 				Collections.shuffle(data);
@@ -111,7 +111,7 @@ public class ReduceCombineDriverTest {
 			{
 				TestTaskContext<GenericReduce<Tuple2<String, Integer>>, Tuple2<String, Integer>> context =
 						new TestTaskContext<GenericReduce<Tuple2<String,Integer>>, Tuple2<String,Integer>>(1024 * 1024);
-				context.getTaskConfig().setMemoryDriver(512 * 1024);
+				context.getTaskConfig().setRelativeMemoryDriver(0.5);
 				
 				List<Tuple2<String, Integer>> data = DriverTestData.createReduceImmutableData();
 				Collections.shuffle(data);
@@ -152,7 +152,7 @@ public class ReduceCombineDriverTest {
 			{
 				TestTaskContext<GenericReduce<Tuple2<StringValue, IntValue>>, Tuple2<StringValue, IntValue>> context =
 						new TestTaskContext<GenericReduce<Tuple2<StringValue, IntValue>>, Tuple2<StringValue, IntValue>>(1024 * 1024);
-				context.getTaskConfig().setMemoryDriver(512 * 1024);
+				context.getTaskConfig().setRelativeMemoryDriver(0.5);
 				
 				List<Tuple2<StringValue, IntValue>> data = DriverTestData.createReduceMutableData();
 				TupleTypeInfo<Tuple2<StringValue, IntValue>> typeInfo = (TupleTypeInfo<Tuple2<StringValue, IntValue>>) TypeExtractor.getForObject(data.get(0));
@@ -180,7 +180,7 @@ public class ReduceCombineDriverTest {
 			{
 				TestTaskContext<GenericReduce<Tuple2<StringValue, IntValue>>, Tuple2<StringValue, IntValue>> context =
 						new TestTaskContext<GenericReduce<Tuple2<StringValue, IntValue>>, Tuple2<StringValue, IntValue>>(1024 * 1024);
-				context.getTaskConfig().setMemoryDriver(512 * 1024);
+				context.getTaskConfig().setRelativeMemoryDriver(0.5);
 				
 				List<Tuple2<StringValue, IntValue>> data = DriverTestData.createReduceMutableData();
 				TupleTypeInfo<Tuple2<StringValue, IntValue>> typeInfo = (TupleTypeInfo<Tuple2<StringValue, IntValue>>) TypeExtractor.getForObject(data.get(0));

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/drivers/TestTaskContext.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/drivers/TestTaskContext.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/drivers/TestTaskContext.java
index 78b0709..f458ae9 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/drivers/TestTaskContext.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/drivers/TestTaskContext.java
@@ -62,7 +62,7 @@ public class TestTaskContext<S, T> implements PactTaskContext<S, T> {
 	public TestTaskContext() {}
 	
 	public TestTaskContext(long memoryInBytes) {
-		this.memoryManager = new DefaultMemoryManager(memoryInBytes, 32 * 1024);
+		this.memoryManager = new DefaultMemoryManager(memoryInBytes,1 ,32 * 1024);
 	}
 	
 	// --------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/DriverTestBase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/DriverTestBase.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/DriverTestBase.java
index c1e2ea8..531382e 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/DriverTestBase.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/DriverTestBase.java
@@ -64,6 +64,8 @@ public class DriverTestBase<S extends Function> implements PactTaskContext<S, Re
 	private final TaskConfig taskConfig;
 	
 	protected final long perSortMem;
+
+	protected final double perSortFractionMem;
 	
 	private Collector<Record> output;
 	
@@ -95,8 +97,9 @@ public class DriverTestBase<S extends Function> implements PactTaskContext<S, Re
 		final long totalMem = Math.max(memory, 0) + (Math.max(maxNumSorters, 0) * perSortMemory);
 		
 		this.perSortMem = perSortMemory;
+		this.perSortFractionMem = (double)perSortMemory/totalMem;
 		this.ioManager = new IOManager();
-		this.memManager = totalMem > 0 ? new DefaultMemoryManager(totalMem) : null;
+		this.memManager = totalMem > 0 ? new DefaultMemoryManager(totalMem,1) : null;
 		
 		this.inputs = new ArrayList<MutableObjectIterator<Record>>();
 		this.comparators = new ArrayList<TypeComparator<Record>>();
@@ -115,7 +118,8 @@ public class DriverTestBase<S extends Function> implements PactTaskContext<S, Re
 	
 	public void addInputSorted(MutableObjectIterator<Record> input, RecordComparator comp) throws Exception {
 		UnilateralSortMerger<Record> sorter = new UnilateralSortMerger<Record>(
-				this.memManager, this.ioManager, input, this.owner, RecordSerializerFactory.get(), comp, this.perSortMem, 32, 0.8f);
+				this.memManager, this.ioManager, input, this.owner, RecordSerializerFactory.get(), comp,
+				this.perSortFractionMem, 32, 0.8f);
 		this.sorters.add(sorter);
 		this.inputs.add(null);
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java
index 2585a74..ab1d4e4 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java
@@ -74,7 +74,7 @@ public class MockEnvironment implements Environment, BufferProvider, LocalBuffer
 		this.inputs = new LinkedList<InputGate<Record>>();
 		this.outputs = new LinkedList<OutputGate>();
 
-		this.memManager = new DefaultMemoryManager(memorySize);
+		this.memManager = new DefaultMemoryManager(memorySize, 1);
 		this.ioManager = new IOManager(System.getProperty("java.io.tmpdir"));
 		this.inputSplitProvider = inputSplitProvider;
 		this.mockBuffer = new Buffer(new MemorySegment(new byte[bufferSize]), bufferSize, null);
@@ -309,14 +309,13 @@ public class MockEnvironment implements Environment, BufferProvider, LocalBuffer
 	}
 
 	@Override
-	public OutputGate createAndRegisterOutputGate()
-	{
+	public OutputGate createAndRegisterOutputGate() {
 		return this.outputs.remove(0);
 	}
 
+	@SuppressWarnings("unchecked")
 	@Override
-	public <T extends IOReadableWritable> InputGate<T> createAndRegisterInputGate()
-	{
+	public <T extends IOReadableWritable> InputGate<T> createAndRegisterInputGate() {
 		return (InputGate<T>) this.inputs.remove(0);
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoderTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoderTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoderTest.java
index f695979..1ee9293 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoderTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoderTest.java
@@ -354,7 +354,7 @@ public class InboundEnvelopeDecoderTest {
 		buf.readerIndex(0);
 		ByteBuf[] slices = randomSlices(buf);
 
-		ch.writeInbound(slices);
+		ch.writeInbound((Object) slices);
 
 		for (ByteBuf slice : slices) {
 			Assert.assertEquals(1, slice.refCnt());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-test-utils/src/main/java/eu/stratosphere/test/compiler/util/CompilerTestBase.java
----------------------------------------------------------------------
diff --git a/stratosphere-test-utils/src/main/java/eu/stratosphere/test/compiler/util/CompilerTestBase.java b/stratosphere-test-utils/src/main/java/eu/stratosphere/test/compiler/util/CompilerTestBase.java
index 7c37bec..8d07163 100644
--- a/stratosphere-test-utils/src/main/java/eu/stratosphere/test/compiler/util/CompilerTestBase.java
+++ b/stratosphere-test-utils/src/main/java/eu/stratosphere/test/compiler/util/CompilerTestBase.java
@@ -12,7 +12,6 @@
  **********************************************************************************************************************/
 package eu.stratosphere.test.compiler.util;
 
-import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -32,12 +31,6 @@ import eu.stratosphere.compiler.costs.DefaultCostEstimator;
 import eu.stratosphere.compiler.plan.OptimizedPlan;
 import eu.stratosphere.compiler.plan.PlanNode;
 import eu.stratosphere.compiler.plan.SingleInputPlanNode;
-import eu.stratosphere.nephele.instance.HardwareDescription;
-import eu.stratosphere.nephele.instance.HardwareDescriptionFactory;
-import eu.stratosphere.nephele.instance.InstanceType;
-import eu.stratosphere.nephele.instance.InstanceTypeDescription;
-import eu.stratosphere.nephele.instance.InstanceTypeDescriptionFactory;
-import eu.stratosphere.nephele.instance.InstanceTypeFactory;
 import eu.stratosphere.util.OperatingSystem;
 
 /**
@@ -63,37 +56,28 @@ public abstract class CompilerTestBase {
 	
 	protected PactCompiler noStatsCompiler;
 	
-	protected InstanceTypeDescription instanceType;
-	
 	private int statCounter;
 	
 	// ------------------------------------------------------------------------	
 	
 	@Before
-	public void setup() {
-		InetSocketAddress dummyAddr = new InetSocketAddress("localhost", 12345);
-		
+	public void setup() {		
 		this.dataStats = new DataStatistics();
-		this.withStatsCompiler = new PactCompiler(this.dataStats, new DefaultCostEstimator(), dummyAddr);
+		this.withStatsCompiler = new PactCompiler(this.dataStats, new DefaultCostEstimator());
 		this.withStatsCompiler.setDefaultDegreeOfParallelism(DEFAULT_PARALLELISM);
 		
-		this.noStatsCompiler = new PactCompiler(null, new DefaultCostEstimator(), dummyAddr);
+		this.noStatsCompiler = new PactCompiler(null, new DefaultCostEstimator());
 		this.noStatsCompiler.setDefaultDegreeOfParallelism(DEFAULT_PARALLELISM);
-		
-		// create the instance type description
-		InstanceType iType = InstanceTypeFactory.construct("standard", 6, 2, 4096, 100, 0);
-		HardwareDescription hDesc = HardwareDescriptionFactory.construct(2, 4096 * 1024 * 1024, 2000 * 1024 * 1024);
-		this.instanceType = InstanceTypeDescriptionFactory.construct(iType, hDesc, DEFAULT_PARALLELISM * 2);
 	}
 	
 	// ------------------------------------------------------------------------
 	
 	public OptimizedPlan compileWithStats(Plan p) {
-		return this.withStatsCompiler.compile(p, this.instanceType);
+		return this.withStatsCompiler.compile(p);
 	}
 	
 	public OptimizedPlan compileNoStats(Plan p) {
-		return this.noStatsCompiler.compile(p, this.instanceType);
+		return this.noStatsCompiler.compile(p);
 	}
 	
 	public void setSourceStatistics(GenericDataSourceBase<?, ?> source, long size, float recordWidth) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-test-utils/src/main/java/eu/stratosphere/test/util/AbstractTestBase.java
----------------------------------------------------------------------
diff --git a/stratosphere-test-utils/src/main/java/eu/stratosphere/test/util/AbstractTestBase.java b/stratosphere-test-utils/src/main/java/eu/stratosphere/test/util/AbstractTestBase.java
index 28a2417..2873d86 100644
--- a/stratosphere-test-utils/src/main/java/eu/stratosphere/test/util/AbstractTestBase.java
+++ b/stratosphere-test-utils/src/main/java/eu/stratosphere/test/util/AbstractTestBase.java
@@ -45,26 +45,29 @@ import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.util.LogUtils;
 
 public abstract class AbstractTestBase {
-	private static final int DEFAULT_NUM_TASK_MANAGER = 1;
+	protected static final int MINIMUM_HEAP_SIZE_MB = 192;
 	
-	private static final int MINIMUM_HEAP_SIZE_MB = 192;
-	
-	private static final long MEMORY_SIZE = 80;
+	protected static final long TASK_MANAGER_MEMORY_SIZE = 80;
+
+	protected static final int DEFAULT_TASK_MANAGER_NUM_SLOTS = 1;
+
+	protected static final int DEFAULT_NUM_TASK_TRACKER = 1;
 
-	private int numTaskManager = DEFAULT_NUM_TASK_MANAGER;
-	
 	protected final Configuration config;
 	
 	protected NepheleMiniCluster executor;
 	
 	private final List<File> tempFiles;
-	
-		
+
+	protected int taskManagerNumSlots = DEFAULT_TASK_MANAGER_NUM_SLOTS;
+
+	protected int numTaskTracker = DEFAULT_NUM_TASK_TRACKER;
+
 	public AbstractTestBase(Configuration config) {
 		verifyJvmOptions();
 		this.config = config;
 		this.tempFiles = new ArrayList<File>();
-		
+
 		LogUtils.initializeDefaultConsoleLogger(Level.WARN);
 	}
 
@@ -73,15 +76,6 @@ public abstract class AbstractTestBase {
 		Assert.assertTrue("Insufficient java heap space " + heap + "mb - set JVM option: -Xmx" + MINIMUM_HEAP_SIZE_MB
 				+ "m", heap > MINIMUM_HEAP_SIZE_MB - 50);
 	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Getter/Setter
-	// --------------------------------------------------------------------------------------------
-
-	public int getNumTaskManager() { return numTaskManager; }
-
-	public void setNumTaskManager(int numTaskManager) { this.numTaskManager = numTaskManager; }
-
 	// --------------------------------------------------------------------------------------------
 	//  Local Test Cluster Life Cycle
 	// --------------------------------------------------------------------------------------------
@@ -91,8 +85,9 @@ public abstract class AbstractTestBase {
 		this.executor = new NepheleMiniCluster();
 		this.executor.setDefaultOverwriteFiles(true);
 		this.executor.setLazyMemoryAllocation(true);
-		this.executor.setMemorySize(MEMORY_SIZE);
-		this.executor.setNumTaskManager(this.numTaskManager);
+		this.executor.setMemorySize(TASK_MANAGER_MEMORY_SIZE);
+		this.executor.setTaskManagerNumSlots(taskManagerNumSlots);
+		this.executor.setNumTaskTracker(this.numTaskTracker);
 		this.executor.start();
 	}
 
@@ -109,6 +104,19 @@ public abstract class AbstractTestBase {
 			deleteAllTempFiles();
 		}
 	}
+
+	//------------------
+	// Accessors
+	//------------------
+
+	public int getTaskManagerNumSlots() { return taskManagerNumSlots; }
+
+	public void setTaskManagerNumSlots(int taskManagerNumSlots) { this.taskManagerNumSlots = taskManagerNumSlots; }
+
+	public int getNumTaskTracker() { return numTaskTracker; }
+
+	public void setNumTaskTracker(int numTaskTracker) { this.numTaskTracker = numTaskTracker; }
+
 	
 	// --------------------------------------------------------------------------------------------
 	//  Temporary File Utilities

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-test-utils/src/main/java/eu/stratosphere/test/util/JavaProgramTestBase.java
----------------------------------------------------------------------
diff --git a/stratosphere-test-utils/src/main/java/eu/stratosphere/test/util/JavaProgramTestBase.java b/stratosphere-test-utils/src/main/java/eu/stratosphere/test/util/JavaProgramTestBase.java
index dc83a56..2aa000a 100644
--- a/stratosphere-test-utils/src/main/java/eu/stratosphere/test/util/JavaProgramTestBase.java
+++ b/stratosphere-test-utils/src/main/java/eu/stratosphere/test/util/JavaProgramTestBase.java
@@ -45,11 +45,13 @@ public abstract class JavaProgramTestBase extends AbstractTestBase {
 	
 	public JavaProgramTestBase(Configuration config) {
 		super(config);
+		setTaskManagerNumSlots(degreeOfParallelism);
 	}
 	
 	
 	public void setDegreeOfParallelism(int degreeOfParallelism) {
 		this.degreeOfParallelism = degreeOfParallelism;
+		setTaskManagerNumSlots(degreeOfParallelism);
 	}
 	
 	public JobExecutionResult getLatestExecutionResult() {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-test-utils/src/main/java/eu/stratosphere/test/util/RecordAPITestBase.java
----------------------------------------------------------------------
diff --git a/stratosphere-test-utils/src/main/java/eu/stratosphere/test/util/RecordAPITestBase.java b/stratosphere-test-utils/src/main/java/eu/stratosphere/test/util/RecordAPITestBase.java
index 45be660..23e7c2b 100644
--- a/stratosphere-test-utils/src/main/java/eu/stratosphere/test/util/RecordAPITestBase.java
+++ b/stratosphere-test-utils/src/main/java/eu/stratosphere/test/util/RecordAPITestBase.java
@@ -28,6 +28,8 @@ import eu.stratosphere.nephele.client.JobClient;
 import eu.stratosphere.nephele.jobgraph.JobGraph;
 
 public abstract class RecordAPITestBase extends AbstractTestBase {
+
+	protected static final int DOP = 4;
 	
 	protected JobExecutionResult jobExecutionResult;
 	
@@ -40,6 +42,7 @@ public abstract class RecordAPITestBase extends AbstractTestBase {
 	
 	public RecordAPITestBase(Configuration config) {
 		super(config);
+		setTaskManagerNumSlots(DOP);
 	}
 	
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/accumulators/AccumulatorITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/accumulators/AccumulatorITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/accumulators/AccumulatorITCase.java
index a14ee03..18bc3e9 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/accumulators/AccumulatorITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/accumulators/AccumulatorITCase.java
@@ -71,13 +71,14 @@ public class AccumulatorITCase extends RecordAPITestBase {
 	private static final String INPUT = "one\n" + "two two\n" + "three three three\n";
 	private static final String EXPECTED = "one 1\ntwo 2\nthree 3\n";
 	
-	private static final int NUM_SUBTASKS = 2;
+	private static final int DOP = 2;
 
 	protected String dataPath;
 	protected String resultPath;
 	
 	public AccumulatorITCase(Configuration config) {
 		super(config);
+		setTaskManagerNumSlots(DOP);
 	}
 
 	@Override
@@ -97,7 +98,7 @@ public class AccumulatorITCase extends RecordAPITestBase {
 		
 		Assert.assertEquals(new Integer(3), (Integer) res.getAccumulatorResult("num-lines"));
 
-		Assert.assertEquals(new Double(NUM_SUBTASKS), (Double)res.getAccumulatorResult("open-close-counter"));
+		Assert.assertEquals(new Double(DOP), (Double)res.getAccumulatorResult("open-close-counter"));
 		
 		// Test histogram (words per line distribution)
 		Map<Integer, Integer> dist = Maps.newHashMap();
@@ -121,7 +122,7 @@ public class AccumulatorITCase extends RecordAPITestBase {
 	@Parameters
 	public static Collection<Object[]> getConfigurations() {
 		Configuration config1 = new Configuration();
-		config1.setInteger("IterationAllReducer#NoSubtasks", NUM_SUBTASKS);
+		config1.setInteger("IterationAllReducer#NoSubtasks", DOP);
 		return toParameterList(config1);
 	}
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/BroadcastVarsNepheleITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/BroadcastVarsNepheleITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/BroadcastVarsNepheleITCase.java
index b80810b..75c50fc 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/BroadcastVarsNepheleITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/BroadcastVarsNepheleITCase.java
@@ -62,13 +62,17 @@ public class BroadcastVarsNepheleITCase extends RecordAPITestBase {
 
 	private static final int NUM_FEATURES = 3;
 
+	private static final int DOP = 4;
+
 	protected String pointsPath;
 
 	protected String modelsPath;
 
 	protected String resultPath;
 
-
+	public BroadcastVarsNepheleITCase(){
+		setTaskManagerNumSlots(DOP);
+	}
 	
 
 	public static final String getInputPoints(int numPoints, int numDimensions, long seed) {
@@ -122,7 +126,7 @@ public class BroadcastVarsNepheleITCase extends RecordAPITestBase {
 
 	@Override
 	protected JobGraph getJobGraph() throws Exception {
-		return createJobGraphV1(this.pointsPath, this.modelsPath, this.resultPath, 4);
+		return createJobGraphV1(this.pointsPath, this.modelsPath, this.resultPath, DOP);
 	}
 
 	@Override
@@ -222,7 +226,7 @@ public class BroadcastVarsNepheleITCase extends RecordAPITestBase {
 	@SuppressWarnings("unchecked")
 	private static JobInputVertex 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, numSubTasks);
+		JobInputVertex pointsInput = JobGraphUtils.createInput(pointsInFormat, pointsPath, "Input[Points]", jobGraph, numSubTasks);
 
 		{
 			TaskConfig taskConfig = new TaskConfig(pointsInput.getConfiguration());
@@ -236,7 +240,7 @@ public class BroadcastVarsNepheleITCase extends RecordAPITestBase {
 	@SuppressWarnings("unchecked")
 	private static JobInputVertex 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, numSubTasks);
+		JobInputVertex modelsInput = JobGraphUtils.createInput(modelsInFormat, pointsPath, "Input[Models]", jobGraph, numSubTasks);
 
 		{
 			TaskConfig taskConfig = new TaskConfig(modelsInput.getConfiguration());
@@ -248,7 +252,7 @@ public class BroadcastVarsNepheleITCase extends RecordAPITestBase {
 	}
 
 	private static JobTaskVertex createMapper(JobGraph jobGraph, int numSubTasks, TypeSerializerFactory<?> serializer) {
-		JobTaskVertex pointsInput = JobGraphUtils.createTask(RegularPactTask.class, "Map[DotProducts]", jobGraph, numSubTasks, numSubTasks);
+		JobTaskVertex pointsInput = JobGraphUtils.createTask(RegularPactTask.class, "Map[DotProducts]", jobGraph, numSubTasks);
 
 		{
 			TaskConfig taskConfig = new TaskConfig(pointsInput.getConfiguration());
@@ -272,7 +276,7 @@ public class BroadcastVarsNepheleITCase extends RecordAPITestBase {
 	}
 
 	private static JobOutputVertex createOutput(JobGraph jobGraph, String resultPath, int numSubTasks, TypeSerializerFactory<?> serializer) {
-		JobOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "Output", numSubTasks, numSubTasks);
+		JobOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "Output", numSubTasks);
 
 		{
 			TaskConfig taskConfig = new TaskConfig(output.getConfiguration());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/KMeansIterativeNepheleITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/KMeansIterativeNepheleITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/KMeansIterativeNepheleITCase.java
index 8cda32f..5e86af5 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/KMeansIterativeNepheleITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/KMeansIterativeNepheleITCase.java
@@ -59,7 +59,11 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 	private static final int ITERATION_ID = 42;
 	
 	private static final int MEMORY_PER_CONSUMER = 2;
-	
+
+	private static final int DOP = 4;
+
+	private static final double MEMORY_FRACTION_PER_CONSUMER = (double)MEMORY_PER_CONSUMER/TASK_MANAGER_MEMORY_SIZE*DOP;
+
 	protected String dataPath;
 	protected String clusterPath;
 	protected String resultPath;
@@ -67,6 +71,7 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 	
 	public KMeansIterativeNepheleITCase() {
 		LogUtils.initializeDefaultConsoleLogger(Level.ERROR);
+		setTaskManagerNumSlots(DOP);
 	}
 	
 	@Override
@@ -83,7 +88,7 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 
 	@Override
 	protected JobGraph getJobGraph() throws Exception {
-		return createJobGraph(dataPath, clusterPath, this.resultPath, 4, 20);
+		return createJobGraph(dataPath, clusterPath, this.resultPath, DOP, 20);
 	}
 
 	// -------------------------------------------------------------------------------------------------------------
@@ -93,7 +98,7 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 	private static JobInputVertex 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, numSubTasks);
+		JobInputVertex pointsInput = JobGraphUtils.createInput(pointsInFormat, pointsPath, "[Points]", jobGraph, numSubTasks);
 		{
 			TaskConfig taskConfig = new TaskConfig(pointsInput.getConfiguration());
 			taskConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
@@ -114,7 +119,7 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 	private static JobInputVertex 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, numSubTasks);
+		JobInputVertex modelsInput = JobGraphUtils.createInput(modelsInFormat, centersPath, "[Models]", jobGraph, numSubTasks);
 
 		{
 			TaskConfig taskConfig = new TaskConfig(modelsInput.getConfiguration());
@@ -135,7 +140,7 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 
 	private static JobOutputVertex createOutput(JobGraph jobGraph, String resultPath, int numSubTasks, TypeSerializerFactory<?> serializer) {
 		
-		JobOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "Output", numSubTasks, numSubTasks);
+		JobOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "Output", numSubTasks);
 
 		{
 			TaskConfig taskConfig = new TaskConfig(output.getConfiguration());
@@ -152,7 +157,7 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 	}
 	
 	private static JobTaskVertex createIterationHead(JobGraph jobGraph, int numSubTasks, TypeSerializerFactory<?> serializer) {
-		JobTaskVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "Iteration Head", jobGraph, numSubTasks, numSubTasks);
+		JobTaskVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "Iteration Head", jobGraph, numSubTasks);
 
 		TaskConfig headConfig = new TaskConfig(head.getConfiguration());
 		headConfig.setIterationId(ITERATION_ID);
@@ -163,7 +168,7 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 		headConfig.setInputSerializer(serializer, 0);
 		
 		// back channel / iterations
-		headConfig.setBackChannelMemory(MEMORY_PER_CONSUMER * JobGraphUtils.MEGABYTE);
+		headConfig.setRelativeBackChannelMemory(MEMORY_FRACTION_PER_CONSUMER);
 		
 		// output into iteration. broadcasting the centers
 		headConfig.setOutputSerializer(serializer);
@@ -190,7 +195,7 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 			TypeComparatorFactory<?> outputComparator)
 	{
 		JobTaskVertex mapper = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
-			"Map (Select nearest center)", jobGraph, numSubTasks, numSubTasks);
+			"Map (Select nearest center)", jobGraph, numSubTasks);
 		
 		TaskConfig intermediateConfig = new TaskConfig(mapper.getConfiguration());
 		intermediateConfig.setIterationId(ITERATION_ID);
@@ -220,7 +225,7 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 		// ---------------- the tail (co group) --------------------
 		
 		JobTaskVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "Reduce / Iteration Tail", jobGraph,
-			numSubTasks, numSubTasks);
+			numSubTasks);
 		
 		TaskConfig tailConfig = new TaskConfig(tail.getConfiguration());
 		tailConfig.setIterationId(ITERATION_ID);
@@ -235,7 +240,7 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 
 		tailConfig.setInputLocalStrategy(0, LocalStrategy.SORT);
 		tailConfig.setInputComparator(inputComparator, 0);
-		tailConfig.setMemoryInput(0, MEMORY_PER_CONSUMER * JobGraphUtils.MEGABYTE);
+		tailConfig.setRelativeMemoryInput(0, MEMORY_FRACTION_PER_CONSUMER);
 		tailConfig.setFilehandlesInput(0, 128);
 		tailConfig.setSpillingThresholdInput(0, 0.9f);
 		
@@ -279,7 +284,7 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 		
 		JobTaskVertex reducer = createReducer(jobGraph, numSubTasks, serializer, int0Comparator, serializer);
 		
-		JobOutputVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput", numSubTasks, numSubTasks);
+		JobOutputVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput", numSubTasks);
 		
 		JobOutputVertex sync = createSync(jobGraph, numIterations, numSubTasks);
 		
@@ -293,7 +298,8 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 		JobGraphUtils.connect(head, mapper, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
 		new TaskConfig(mapper.getConfiguration()).setBroadcastGateIterativeWithNumberOfEventsUntilInterrupt(0, numSubTasks);
 		new TaskConfig(mapper.getConfiguration()).setInputCached(0, true);
-		new TaskConfig(mapper.getConfiguration()).setInputMaterializationMemory(0, MEMORY_PER_CONSUMER * JobGraphUtils.MEGABYTE);
+		new TaskConfig(mapper.getConfiguration()).setRelativeInputMaterializationMemory(0,
+				MEMORY_FRACTION_PER_CONSUMER);
 
 		JobGraphUtils.connect(mapper, reducer, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
 		new TaskConfig(reducer.getConfiguration()).setGateIterativeWithNumberOfEventsUntilInterrupt(0, numSubTasks);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/cancelling/CancellingTestBase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/cancelling/CancellingTestBase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/cancelling/CancellingTestBase.java
index 8ce656e..1fc289d 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/cancelling/CancellingTestBase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/cancelling/CancellingTestBase.java
@@ -60,9 +60,13 @@ public abstract class CancellingTestBase {
 	 */
 	private static final int DEFAULT_CANCEL_FINISHED_INTERVAL = 10 * 1000;
 
+	private static final int DEFAULT_TASK_MANAGER_NUM_SLOTS = 1;
+
 	// --------------------------------------------------------------------------------------------
 	
 	protected NepheleMiniCluster executor;
+
+	protected int taskManagerNumSlots = DEFAULT_TASK_MANAGER_NUM_SLOTS;
 	
 	// --------------------------------------------------------------------------------------------
 	
@@ -83,7 +87,7 @@ public abstract class CancellingTestBase {
 		verifyJvmOptions();
 		this.executor = new NepheleMiniCluster();
 		this.executor.setDefaultOverwriteFiles(true);
-		
+		this.executor.setTaskManagerNumSlots(taskManagerNumSlots);
 		this.executor.start();
 	}
 
@@ -231,4 +235,8 @@ public abstract class CancellingTestBase {
 		final NepheleJobGraphGenerator jgg = new NepheleJobGraphGenerator();
 		return jgg.compileJobGraph(op);
 	}
+
+	public void setTaskManagerNumSlots(int taskManagerNumSlots) { this.taskManagerNumSlots = taskManagerNumSlots; }
+
+	public int getTaskManagerNumSlots() { return this.taskManagerNumSlots; }
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/cancelling/MapCancelingITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/cancelling/MapCancelingITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/cancelling/MapCancelingITCase.java
index 1aeb229..7d48ae8 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/cancelling/MapCancelingITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/cancelling/MapCancelingITCase.java
@@ -27,6 +27,11 @@ import eu.stratosphere.types.Record;
 import eu.stratosphere.util.Collector;
 
 public class MapCancelingITCase extends CancellingTestBase {
+	private static final int DOP = 4;
+
+	public MapCancelingITCase() {
+		setTaskManagerNumSlots(DOP);
+	}
 	
 //	@Test
 	public void testMapCancelling() throws Exception {
@@ -40,7 +45,7 @@ public class MapCancelingITCase extends CancellingTestBase {
 		
 		
 		Plan p = new Plan(sink);
-		p.setDefaultParallelism(4);
+		p.setDefaultParallelism(DOP);
 		
 		runAndCancelJob(p, 5 * 1000, 10 * 1000);
 	}
@@ -57,7 +62,7 @@ public class MapCancelingITCase extends CancellingTestBase {
 		
 		
 		Plan p = new Plan(sink);
-		p.setDefaultParallelism(4);
+		p.setDefaultParallelism(DOP);
 		
 		runAndCancelJob(p, 5 * 1000, 10 * 1000);
 	}
@@ -74,7 +79,7 @@ public class MapCancelingITCase extends CancellingTestBase {
 		
 		
 		Plan p = new Plan(sink);
-		p.setDefaultParallelism(4);
+		p.setDefaultParallelism(DOP);
 		
 		runAndCancelJob(p, 10 * 1000, 10 * 1000);
 	}
@@ -91,7 +96,7 @@ public class MapCancelingITCase extends CancellingTestBase {
 		
 		
 		Plan p = new Plan(sink);
-		p.setDefaultParallelism(4);
+		p.setDefaultParallelism(DOP);
 		
 		runAndCancelJob(p, 10 * 1000, 10 * 1000);
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/cancelling/MatchJoinCancelingITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/cancelling/MatchJoinCancelingITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/cancelling/MatchJoinCancelingITCase.java
index 09413f0..82e2ace 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/cancelling/MatchJoinCancelingITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/cancelling/MatchJoinCancelingITCase.java
@@ -30,6 +30,11 @@ import eu.stratosphere.types.Record;
 import eu.stratosphere.util.Collector;
 
 public class MatchJoinCancelingITCase extends CancellingTestBase {
+	private static final int DOP = 4;
+
+	public MatchJoinCancelingITCase(){
+		setTaskManagerNumSlots(DOP);
+	}
 	
 	// --------------- Test Sort Matches that are canceled while still reading / sorting -----------------
 //	@Test
@@ -48,7 +53,7 @@ public class MatchJoinCancelingITCase extends CancellingTestBase {
 		GenericDataSink sink = new GenericDataSink(new DiscardingOutputFormat(), matcher, "Sink");
 		
 		Plan p = new Plan(sink);
-		p.setDefaultParallelism(4);
+		p.setDefaultParallelism(DOP);
 		
 		runAndCancelJob(p, 3000, 10*1000);
 	}
@@ -69,7 +74,7 @@ public class MatchJoinCancelingITCase extends CancellingTestBase {
 		GenericDataSink sink = new GenericDataSink(new DiscardingOutputFormat(), matcher, "Sink");
 		
 		Plan p = new Plan(sink);
-		p.setDefaultParallelism(4);
+		p.setDefaultParallelism(DOP);
 		
 		runAndCancelJob(p, 5000, 10*1000);
 	}
@@ -90,7 +95,7 @@ public class MatchJoinCancelingITCase extends CancellingTestBase {
 		GenericDataSink sink = new GenericDataSink(new DiscardingOutputFormat(), matcher, "Sink");
 		
 		Plan p = new Plan(sink);
-		p.setDefaultParallelism(4);
+		p.setDefaultParallelism(DOP);
 		
 		runAndCancelJob(p, 5000);
 		
@@ -117,7 +122,7 @@ public class MatchJoinCancelingITCase extends CancellingTestBase {
 		GenericDataSink sink = new GenericDataSink(new DiscardingOutputFormat(), matcher, "Sink");
 		
 		Plan p = new Plan(sink);
-		p.setDefaultParallelism(4);
+		p.setDefaultParallelism(DOP);
 		
 		runAndCancelJob(p, 30 * 1000, 30 * 1000);
 	}
@@ -145,7 +150,7 @@ public class MatchJoinCancelingITCase extends CancellingTestBase {
 		GenericDataSink sink = new GenericDataSink(new DiscardingOutputFormat(), matcher, "Sink");
 		
 		Plan p = new Plan(sink);
-		p.setDefaultParallelism(4);
+		p.setDefaultParallelism(DOP);
 		
 		runAndCancelJob(p, 10 * 1000, 20 * 1000);
 	}
@@ -171,7 +176,7 @@ public class MatchJoinCancelingITCase extends CancellingTestBase {
 		GenericDataSink sink = new GenericDataSink(new DiscardingOutputFormat(), matcher, "Sink");
 		
 		Plan p = new Plan(sink);
-		p.setDefaultParallelism(4);
+		p.setDefaultParallelism(DOP);
 		
 		runAndCancelJob(p, 10 * 1000, 10 * 1000);
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/clients/examples/LocalExecutorITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/clients/examples/LocalExecutorITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/clients/examples/LocalExecutorITCase.java
index 984ecc2..b198d99 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/clients/examples/LocalExecutorITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/clients/examples/LocalExecutorITCase.java
@@ -23,8 +23,11 @@ import eu.stratosphere.client.LocalExecutor;
 import eu.stratosphere.test.recordJobs.wordcount.WordCount;
 import eu.stratosphere.test.testdata.WordCountData;
 
+
 public class LocalExecutorITCase {
 
+	private static final int DOP = 4;
+
 	@Test
 	public void testLocalExecutorWithWordCount() {
 		try {
@@ -40,14 +43,15 @@ public class LocalExecutorITCase {
 			
 			// run WordCount
 			WordCount wc = new WordCount();
-			wc.getPlan("4", inFile.toURI().toString(), outFile.toURI().toString());
-			
+
 			LocalExecutor executor = new LocalExecutor();
 			LocalExecutor.setLoggingLevel(Level.WARN);
 			executor.setDefaultOverwriteFiles(true);
+			executor.setTaskManagerNumSlots(DOP);
 			executor.start();
 			
-			executor.executePlan(wc.getPlan("4", inFile.toURI().toString(), outFile.toURI().toString()));
+			executor.executePlan(wc.getPlan(new Integer(DOP).toString(), inFile.toURI().toString(),
+					outFile.toURI().toString()));
 			executor.stop();
 		} catch (Exception e) {
 			e.printStackTrace();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/WordCountITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/WordCountITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/WordCountITCase.java
index 4a60836..272bce6 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/WordCountITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/WordCountITCase.java
@@ -25,7 +25,9 @@ public class WordCountITCase extends JavaProgramTestBase {
 	protected String resultPath;
 
 	public WordCountITCase(){
-		setNumTaskManager(2);
+		setDegreeOfParallelism(4);
+		setNumTaskTracker(2);
+		setTaskManagerNumSlots(2);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/ComputeEdgeDegreesITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/ComputeEdgeDegreesITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/ComputeEdgeDegreesITCase.java
index d2caeb7..831a9ae 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/ComputeEdgeDegreesITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/ComputeEdgeDegreesITCase.java
@@ -27,7 +27,7 @@ public class ComputeEdgeDegreesITCase extends eu.stratosphere.test.recordJobTest
 	protected Plan getTestJob() {
 		ComputeEdgeDegrees computeDegrees = new ComputeEdgeDegrees();
 		return computeDegrees.getScalaPlan(
-				config.getInteger("ComputeEdgeDegreesTest#NumSubtasks", 4),
+				config.getInteger("ComputeEdgeDegreesTest#NumSubtasks", DOP),
 				edgesPath, resultPath);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/ConnectedComponentsITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/ConnectedComponentsITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/ConnectedComponentsITCase.java
index 6725bde..40f95af 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/ConnectedComponentsITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/ConnectedComponentsITCase.java
@@ -22,7 +22,7 @@ public class ConnectedComponentsITCase extends eu.stratosphere.test.iterative.Co
 	protected Plan getTestJob() {
 		ConnectedComponents cc = new ConnectedComponents();
 		Plan plan = cc.getScalaPlan(verticesPath, edgesPath, resultPath, 100);
-		plan.setDefaultParallelism(4);
+		plan.setDefaultParallelism(DOP);
 		return plan;
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/EnumTrianglesOnEdgesWithDegreesITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/EnumTrianglesOnEdgesWithDegreesITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/EnumTrianglesOnEdgesWithDegreesITCase.java
index 5801d59..81b5c2a 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/EnumTrianglesOnEdgesWithDegreesITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/EnumTrianglesOnEdgesWithDegreesITCase.java
@@ -27,7 +27,7 @@ public class EnumTrianglesOnEdgesWithDegreesITCase extends eu.stratosphere.test.
 	protected Plan getTestJob() {
 		EnumTrianglesOnEdgesWithDegrees enumTriangles = new EnumTrianglesOnEdgesWithDegrees();
 		return enumTriangles.getScalaPlan(
-				config.getInteger("EnumTrianglesTest#NumSubtasks", 4),
+				config.getInteger("EnumTrianglesTest#NumSubtasks", DOP),
 				edgesPath, resultPath);
 	}
 }


[49/53] [abbrv] git commit: Fix generic type warnings in generated tuples

Posted by rm...@apache.org.
Fix generic type warnings in generated tuples


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

Branch: refs/heads/travis_test
Commit: 49cd35a7632f32fd70eeefffc129ac764c1df262
Parents: 35438ec
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Jun 25 16:55:20 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Jun 25 16:55:20 2014 +0200

----------------------------------------------------------------------
 .../api/java/operators/CrossOperator.java       | 220 +++++++++----------
 .../eu/stratosphere/api/java/tuple/Tuple1.java  |   1 +
 .../eu/stratosphere/api/java/tuple/Tuple10.java |   1 +
 .../eu/stratosphere/api/java/tuple/Tuple11.java |   1 +
 .../eu/stratosphere/api/java/tuple/Tuple12.java |   1 +
 .../eu/stratosphere/api/java/tuple/Tuple13.java |   1 +
 .../eu/stratosphere/api/java/tuple/Tuple14.java |   1 +
 .../eu/stratosphere/api/java/tuple/Tuple15.java |   1 +
 .../eu/stratosphere/api/java/tuple/Tuple16.java |   1 +
 .../eu/stratosphere/api/java/tuple/Tuple17.java |   1 +
 .../eu/stratosphere/api/java/tuple/Tuple18.java |   1 +
 .../eu/stratosphere/api/java/tuple/Tuple19.java |   1 +
 .../eu/stratosphere/api/java/tuple/Tuple2.java  |   1 +
 .../eu/stratosphere/api/java/tuple/Tuple20.java |   1 +
 .../eu/stratosphere/api/java/tuple/Tuple21.java |   1 +
 .../eu/stratosphere/api/java/tuple/Tuple22.java |   1 +
 .../eu/stratosphere/api/java/tuple/Tuple23.java |   1 +
 .../eu/stratosphere/api/java/tuple/Tuple24.java |   1 +
 .../eu/stratosphere/api/java/tuple/Tuple25.java |   1 +
 .../eu/stratosphere/api/java/tuple/Tuple3.java  |   1 +
 .../eu/stratosphere/api/java/tuple/Tuple4.java  |   1 +
 .../eu/stratosphere/api/java/tuple/Tuple5.java  |   1 +
 .../eu/stratosphere/api/java/tuple/Tuple6.java  |   1 +
 .../eu/stratosphere/api/java/tuple/Tuple7.java  |   1 +
 .../eu/stratosphere/api/java/tuple/Tuple8.java  |   1 +
 .../eu/stratosphere/api/java/tuple/Tuple9.java  |   1 +
 .../api/java/tuple/TupleGenerator.java          |   1 +
 27 files changed, 136 insertions(+), 110 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/49cd35a7/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CrossOperator.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CrossOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CrossOperator.java
index 3566224..b497407 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CrossOperator.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CrossOperator.java
@@ -469,12 +469,12 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 	// GENERATED FROM eu.stratosphere.api.java.tuple.TupleGenerator.
 
 		/**
-		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields.
-		 * Requires the classes of the fields of the resulting tuples.
-		 *
+		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. 
+		 * Requires the classes of the fields of the resulting tuples. 
+		 * 
 		 * @param type0 The class of field '0' of the result tuples.
 		 * @return The projected data set.
-		 *
+		 * 
 		 * @see Tuple
 		 * @see DataSet
 		 */
@@ -483,7 +483,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 			if(types.length != this.fieldIndexes.length) {
 				throw new IllegalArgumentException("Numbers of projected fields and types do not match.");
 			}
-
+			
 			TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, types);
 			TupleTypeInfo<Tuple1<T0>> tType = new TupleTypeInfo<Tuple1<T0>>(fTypes);
 
@@ -491,13 +491,13 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		}
 
 		/**
-		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields.
-		 * Requires the classes of the fields of the resulting tuples.
-		 *
+		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. 
+		 * Requires the classes of the fields of the resulting tuples. 
+		 * 
 		 * @param type0 The class of field '0' of the result tuples.
 		 * @param type1 The class of field '1' of the result tuples.
 		 * @return The projected data set.
-		 *
+		 * 
 		 * @see Tuple
 		 * @see DataSet
 		 */
@@ -506,7 +506,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 			if(types.length != this.fieldIndexes.length) {
 				throw new IllegalArgumentException("Numbers of projected fields and types do not match.");
 			}
-
+			
 			TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, types);
 			TupleTypeInfo<Tuple2<T0, T1>> tType = new TupleTypeInfo<Tuple2<T0, T1>>(fTypes);
 
@@ -514,14 +514,14 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		}
 
 		/**
-		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields.
-		 * Requires the classes of the fields of the resulting tuples.
-		 *
+		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. 
+		 * Requires the classes of the fields of the resulting tuples. 
+		 * 
 		 * @param type0 The class of field '0' of the result tuples.
 		 * @param type1 The class of field '1' of the result tuples.
 		 * @param type2 The class of field '2' of the result tuples.
 		 * @return The projected data set.
-		 *
+		 * 
 		 * @see Tuple
 		 * @see DataSet
 		 */
@@ -530,7 +530,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 			if(types.length != this.fieldIndexes.length) {
 				throw new IllegalArgumentException("Numbers of projected fields and types do not match.");
 			}
-
+			
 			TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, types);
 			TupleTypeInfo<Tuple3<T0, T1, T2>> tType = new TupleTypeInfo<Tuple3<T0, T1, T2>>(fTypes);
 
@@ -538,15 +538,15 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		}
 
 		/**
-		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields.
-		 * Requires the classes of the fields of the resulting tuples.
-		 *
+		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. 
+		 * Requires the classes of the fields of the resulting tuples. 
+		 * 
 		 * @param type0 The class of field '0' of the result tuples.
 		 * @param type1 The class of field '1' of the result tuples.
 		 * @param type2 The class of field '2' of the result tuples.
 		 * @param type3 The class of field '3' of the result tuples.
 		 * @return The projected data set.
-		 *
+		 * 
 		 * @see Tuple
 		 * @see DataSet
 		 */
@@ -555,7 +555,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 			if(types.length != this.fieldIndexes.length) {
 				throw new IllegalArgumentException("Numbers of projected fields and types do not match.");
 			}
-
+			
 			TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, types);
 			TupleTypeInfo<Tuple4<T0, T1, T2, T3>> tType = new TupleTypeInfo<Tuple4<T0, T1, T2, T3>>(fTypes);
 
@@ -563,16 +563,16 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		}
 
 		/**
-		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields.
-		 * Requires the classes of the fields of the resulting tuples.
-		 *
+		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. 
+		 * Requires the classes of the fields of the resulting tuples. 
+		 * 
 		 * @param type0 The class of field '0' of the result tuples.
 		 * @param type1 The class of field '1' of the result tuples.
 		 * @param type2 The class of field '2' of the result tuples.
 		 * @param type3 The class of field '3' of the result tuples.
 		 * @param type4 The class of field '4' of the result tuples.
 		 * @return The projected data set.
-		 *
+		 * 
 		 * @see Tuple
 		 * @see DataSet
 		 */
@@ -581,7 +581,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 			if(types.length != this.fieldIndexes.length) {
 				throw new IllegalArgumentException("Numbers of projected fields and types do not match.");
 			}
-
+			
 			TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, types);
 			TupleTypeInfo<Tuple5<T0, T1, T2, T3, T4>> tType = new TupleTypeInfo<Tuple5<T0, T1, T2, T3, T4>>(fTypes);
 
@@ -589,9 +589,9 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		}
 
 		/**
-		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields.
-		 * Requires the classes of the fields of the resulting tuples.
-		 *
+		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. 
+		 * Requires the classes of the fields of the resulting tuples. 
+		 * 
 		 * @param type0 The class of field '0' of the result tuples.
 		 * @param type1 The class of field '1' of the result tuples.
 		 * @param type2 The class of field '2' of the result tuples.
@@ -599,7 +599,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		 * @param type4 The class of field '4' of the result tuples.
 		 * @param type5 The class of field '5' of the result tuples.
 		 * @return The projected data set.
-		 *
+		 * 
 		 * @see Tuple
 		 * @see DataSet
 		 */
@@ -608,7 +608,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 			if(types.length != this.fieldIndexes.length) {
 				throw new IllegalArgumentException("Numbers of projected fields and types do not match.");
 			}
-
+			
 			TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, types);
 			TupleTypeInfo<Tuple6<T0, T1, T2, T3, T4, T5>> tType = new TupleTypeInfo<Tuple6<T0, T1, T2, T3, T4, T5>>(fTypes);
 
@@ -616,9 +616,9 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		}
 
 		/**
-		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields.
-		 * Requires the classes of the fields of the resulting tuples.
-		 *
+		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. 
+		 * Requires the classes of the fields of the resulting tuples. 
+		 * 
 		 * @param type0 The class of field '0' of the result tuples.
 		 * @param type1 The class of field '1' of the result tuples.
 		 * @param type2 The class of field '2' of the result tuples.
@@ -627,7 +627,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		 * @param type5 The class of field '5' of the result tuples.
 		 * @param type6 The class of field '6' of the result tuples.
 		 * @return The projected data set.
-		 *
+		 * 
 		 * @see Tuple
 		 * @see DataSet
 		 */
@@ -636,7 +636,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 			if(types.length != this.fieldIndexes.length) {
 				throw new IllegalArgumentException("Numbers of projected fields and types do not match.");
 			}
-
+			
 			TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, types);
 			TupleTypeInfo<Tuple7<T0, T1, T2, T3, T4, T5, T6>> tType = new TupleTypeInfo<Tuple7<T0, T1, T2, T3, T4, T5, T6>>(fTypes);
 
@@ -644,9 +644,9 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		}
 
 		/**
-		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields.
-		 * Requires the classes of the fields of the resulting tuples.
-		 *
+		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. 
+		 * Requires the classes of the fields of the resulting tuples. 
+		 * 
 		 * @param type0 The class of field '0' of the result tuples.
 		 * @param type1 The class of field '1' of the result tuples.
 		 * @param type2 The class of field '2' of the result tuples.
@@ -656,7 +656,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		 * @param type6 The class of field '6' of the result tuples.
 		 * @param type7 The class of field '7' of the result tuples.
 		 * @return The projected data set.
-		 *
+		 * 
 		 * @see Tuple
 		 * @see DataSet
 		 */
@@ -665,7 +665,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 			if(types.length != this.fieldIndexes.length) {
 				throw new IllegalArgumentException("Numbers of projected fields and types do not match.");
 			}
-
+			
 			TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, types);
 			TupleTypeInfo<Tuple8<T0, T1, T2, T3, T4, T5, T6, T7>> tType = new TupleTypeInfo<Tuple8<T0, T1, T2, T3, T4, T5, T6, T7>>(fTypes);
 
@@ -673,9 +673,9 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		}
 
 		/**
-		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields.
-		 * Requires the classes of the fields of the resulting tuples.
-		 *
+		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. 
+		 * Requires the classes of the fields of the resulting tuples. 
+		 * 
 		 * @param type0 The class of field '0' of the result tuples.
 		 * @param type1 The class of field '1' of the result tuples.
 		 * @param type2 The class of field '2' of the result tuples.
@@ -686,7 +686,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		 * @param type7 The class of field '7' of the result tuples.
 		 * @param type8 The class of field '8' of the result tuples.
 		 * @return The projected data set.
-		 *
+		 * 
 		 * @see Tuple
 		 * @see DataSet
 		 */
@@ -695,7 +695,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 			if(types.length != this.fieldIndexes.length) {
 				throw new IllegalArgumentException("Numbers of projected fields and types do not match.");
 			}
-
+			
 			TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, types);
 			TupleTypeInfo<Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8>> tType = new TupleTypeInfo<Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8>>(fTypes);
 
@@ -703,9 +703,9 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		}
 
 		/**
-		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields.
-		 * Requires the classes of the fields of the resulting tuples.
-		 *
+		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. 
+		 * Requires the classes of the fields of the resulting tuples. 
+		 * 
 		 * @param type0 The class of field '0' of the result tuples.
 		 * @param type1 The class of field '1' of the result tuples.
 		 * @param type2 The class of field '2' of the result tuples.
@@ -717,7 +717,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		 * @param type8 The class of field '8' of the result tuples.
 		 * @param type9 The class of field '9' of the result tuples.
 		 * @return The projected data set.
-		 *
+		 * 
 		 * @see Tuple
 		 * @see DataSet
 		 */
@@ -726,7 +726,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 			if(types.length != this.fieldIndexes.length) {
 				throw new IllegalArgumentException("Numbers of projected fields and types do not match.");
 			}
-
+			
 			TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, types);
 			TupleTypeInfo<Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9>> tType = new TupleTypeInfo<Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9>>(fTypes);
 
@@ -734,9 +734,9 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		}
 
 		/**
-		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields.
-		 * Requires the classes of the fields of the resulting tuples.
-		 *
+		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. 
+		 * Requires the classes of the fields of the resulting tuples. 
+		 * 
 		 * @param type0 The class of field '0' of the result tuples.
 		 * @param type1 The class of field '1' of the result tuples.
 		 * @param type2 The class of field '2' of the result tuples.
@@ -749,7 +749,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		 * @param type9 The class of field '9' of the result tuples.
 		 * @param type10 The class of field '10' of the result tuples.
 		 * @return The projected data set.
-		 *
+		 * 
 		 * @see Tuple
 		 * @see DataSet
 		 */
@@ -758,7 +758,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 			if(types.length != this.fieldIndexes.length) {
 				throw new IllegalArgumentException("Numbers of projected fields and types do not match.");
 			}
-
+			
 			TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, types);
 			TupleTypeInfo<Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>> tType = new TupleTypeInfo<Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>>(fTypes);
 
@@ -766,9 +766,9 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		}
 
 		/**
-		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields.
-		 * Requires the classes of the fields of the resulting tuples.
-		 *
+		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. 
+		 * Requires the classes of the fields of the resulting tuples. 
+		 * 
 		 * @param type0 The class of field '0' of the result tuples.
 		 * @param type1 The class of field '1' of the result tuples.
 		 * @param type2 The class of field '2' of the result tuples.
@@ -782,7 +782,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		 * @param type10 The class of field '10' of the result tuples.
 		 * @param type11 The class of field '11' of the result tuples.
 		 * @return The projected data set.
-		 *
+		 * 
 		 * @see Tuple
 		 * @see DataSet
 		 */
@@ -791,7 +791,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 			if(types.length != this.fieldIndexes.length) {
 				throw new IllegalArgumentException("Numbers of projected fields and types do not match.");
 			}
-
+			
 			TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, types);
 			TupleTypeInfo<Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>> tType = new TupleTypeInfo<Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>>(fTypes);
 
@@ -799,9 +799,9 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		}
 
 		/**
-		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields.
-		 * Requires the classes of the fields of the resulting tuples.
-		 *
+		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. 
+		 * Requires the classes of the fields of the resulting tuples. 
+		 * 
 		 * @param type0 The class of field '0' of the result tuples.
 		 * @param type1 The class of field '1' of the result tuples.
 		 * @param type2 The class of field '2' of the result tuples.
@@ -816,7 +816,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		 * @param type11 The class of field '11' of the result tuples.
 		 * @param type12 The class of field '12' of the result tuples.
 		 * @return The projected data set.
-		 *
+		 * 
 		 * @see Tuple
 		 * @see DataSet
 		 */
@@ -825,7 +825,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 			if(types.length != this.fieldIndexes.length) {
 				throw new IllegalArgumentException("Numbers of projected fields and types do not match.");
 			}
-
+			
 			TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, types);
 			TupleTypeInfo<Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>> tType = new TupleTypeInfo<Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>>(fTypes);
 
@@ -833,9 +833,9 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		}
 
 		/**
-		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields.
-		 * Requires the classes of the fields of the resulting tuples.
-		 *
+		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. 
+		 * Requires the classes of the fields of the resulting tuples. 
+		 * 
 		 * @param type0 The class of field '0' of the result tuples.
 		 * @param type1 The class of field '1' of the result tuples.
 		 * @param type2 The class of field '2' of the result tuples.
@@ -851,7 +851,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		 * @param type12 The class of field '12' of the result tuples.
 		 * @param type13 The class of field '13' of the result tuples.
 		 * @return The projected data set.
-		 *
+		 * 
 		 * @see Tuple
 		 * @see DataSet
 		 */
@@ -860,7 +860,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 			if(types.length != this.fieldIndexes.length) {
 				throw new IllegalArgumentException("Numbers of projected fields and types do not match.");
 			}
-
+			
 			TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, types);
 			TupleTypeInfo<Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>> tType = new TupleTypeInfo<Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>>(fTypes);
 
@@ -868,9 +868,9 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		}
 
 		/**
-		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields.
-		 * Requires the classes of the fields of the resulting tuples.
-		 *
+		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. 
+		 * Requires the classes of the fields of the resulting tuples. 
+		 * 
 		 * @param type0 The class of field '0' of the result tuples.
 		 * @param type1 The class of field '1' of the result tuples.
 		 * @param type2 The class of field '2' of the result tuples.
@@ -887,7 +887,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		 * @param type13 The class of field '13' of the result tuples.
 		 * @param type14 The class of field '14' of the result tuples.
 		 * @return The projected data set.
-		 *
+		 * 
 		 * @see Tuple
 		 * @see DataSet
 		 */
@@ -896,7 +896,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 			if(types.length != this.fieldIndexes.length) {
 				throw new IllegalArgumentException("Numbers of projected fields and types do not match.");
 			}
-
+			
 			TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, types);
 			TupleTypeInfo<Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>> tType = new TupleTypeInfo<Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>>(fTypes);
 
@@ -904,9 +904,9 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		}
 
 		/**
-		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields.
-		 * Requires the classes of the fields of the resulting tuples.
-		 *
+		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. 
+		 * Requires the classes of the fields of the resulting tuples. 
+		 * 
 		 * @param type0 The class of field '0' of the result tuples.
 		 * @param type1 The class of field '1' of the result tuples.
 		 * @param type2 The class of field '2' of the result tuples.
@@ -924,7 +924,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		 * @param type14 The class of field '14' of the result tuples.
 		 * @param type15 The class of field '15' of the result tuples.
 		 * @return The projected data set.
-		 *
+		 * 
 		 * @see Tuple
 		 * @see DataSet
 		 */
@@ -933,7 +933,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 			if(types.length != this.fieldIndexes.length) {
 				throw new IllegalArgumentException("Numbers of projected fields and types do not match.");
 			}
-
+			
 			TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, types);
 			TupleTypeInfo<Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>> tType = new TupleTypeInfo<Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>>(fTypes);
 
@@ -941,9 +941,9 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		}
 
 		/**
-		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields.
-		 * Requires the classes of the fields of the resulting tuples.
-		 *
+		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. 
+		 * Requires the classes of the fields of the resulting tuples. 
+		 * 
 		 * @param type0 The class of field '0' of the result tuples.
 		 * @param type1 The class of field '1' of the result tuples.
 		 * @param type2 The class of field '2' of the result tuples.
@@ -962,7 +962,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		 * @param type15 The class of field '15' of the result tuples.
 		 * @param type16 The class of field '16' of the result tuples.
 		 * @return The projected data set.
-		 *
+		 * 
 		 * @see Tuple
 		 * @see DataSet
 		 */
@@ -971,7 +971,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 			if(types.length != this.fieldIndexes.length) {
 				throw new IllegalArgumentException("Numbers of projected fields and types do not match.");
 			}
-
+			
 			TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, types);
 			TupleTypeInfo<Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>> tType = new TupleTypeInfo<Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>>(fTypes);
 
@@ -979,9 +979,9 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		}
 
 		/**
-		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields.
-		 * Requires the classes of the fields of the resulting tuples.
-		 *
+		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. 
+		 * Requires the classes of the fields of the resulting tuples. 
+		 * 
 		 * @param type0 The class of field '0' of the result tuples.
 		 * @param type1 The class of field '1' of the result tuples.
 		 * @param type2 The class of field '2' of the result tuples.
@@ -1001,7 +1001,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		 * @param type16 The class of field '16' of the result tuples.
 		 * @param type17 The class of field '17' of the result tuples.
 		 * @return The projected data set.
-		 *
+		 * 
 		 * @see Tuple
 		 * @see DataSet
 		 */
@@ -1010,7 +1010,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 			if(types.length != this.fieldIndexes.length) {
 				throw new IllegalArgumentException("Numbers of projected fields and types do not match.");
 			}
-
+			
 			TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, types);
 			TupleTypeInfo<Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>> tType = new TupleTypeInfo<Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>>(fTypes);
 
@@ -1018,9 +1018,9 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		}
 
 		/**
-		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields.
-		 * Requires the classes of the fields of the resulting tuples.
-		 *
+		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. 
+		 * Requires the classes of the fields of the resulting tuples. 
+		 * 
 		 * @param type0 The class of field '0' of the result tuples.
 		 * @param type1 The class of field '1' of the result tuples.
 		 * @param type2 The class of field '2' of the result tuples.
@@ -1041,7 +1041,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		 * @param type17 The class of field '17' of the result tuples.
 		 * @param type18 The class of field '18' of the result tuples.
 		 * @return The projected data set.
-		 *
+		 * 
 		 * @see Tuple
 		 * @see DataSet
 		 */
@@ -1050,7 +1050,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 			if(types.length != this.fieldIndexes.length) {
 				throw new IllegalArgumentException("Numbers of projected fields and types do not match.");
 			}
-
+			
 			TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, types);
 			TupleTypeInfo<Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>> tType = new TupleTypeInfo<Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>>(fTypes);
 
@@ -1058,9 +1058,9 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		}
 
 		/**
-		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields.
-		 * Requires the classes of the fields of the resulting tuples.
-		 *
+		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. 
+		 * Requires the classes of the fields of the resulting tuples. 
+		 * 
 		 * @param type0 The class of field '0' of the result tuples.
 		 * @param type1 The class of field '1' of the result tuples.
 		 * @param type2 The class of field '2' of the result tuples.
@@ -1082,7 +1082,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		 * @param type18 The class of field '18' of the result tuples.
 		 * @param type19 The class of field '19' of the result tuples.
 		 * @return The projected data set.
-		 *
+		 * 
 		 * @see Tuple
 		 * @see DataSet
 		 */
@@ -1091,7 +1091,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 			if(types.length != this.fieldIndexes.length) {
 				throw new IllegalArgumentException("Numbers of projected fields and types do not match.");
 			}
-
+			
 			TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, types);
 			TupleTypeInfo<Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>> tType = new TupleTypeInfo<Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>>(fTypes);
 
@@ -1099,9 +1099,9 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		}
 
 		/**
-		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields.
-		 * Requires the classes of the fields of the resulting tuples.
-		 *
+		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. 
+		 * Requires the classes of the fields of the resulting tuples. 
+		 * 
 		 * @param type0 The class of field '0' of the result tuples.
 		 * @param type1 The class of field '1' of the result tuples.
 		 * @param type2 The class of field '2' of the result tuples.
@@ -1124,7 +1124,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		 * @param type19 The class of field '19' of the result tuples.
 		 * @param type20 The class of field '20' of the result tuples.
 		 * @return The projected data set.
-		 *
+		 * 
 		 * @see Tuple
 		 * @see DataSet
 		 */
@@ -1133,7 +1133,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 			if(types.length != this.fieldIndexes.length) {
 				throw new IllegalArgumentException("Numbers of projected fields and types do not match.");
 			}
-
+			
 			TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, types);
 			TupleTypeInfo<Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>> tType = new TupleTypeInfo<Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>>(fTypes);
 
@@ -1141,9 +1141,9 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		}
 
 		/**
-		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields.
-		 * Requires the classes of the fields of the resulting tuples.
-		 *
+		 * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. 
+		 * Requires the classes of the fields of the resulting tuples. 
+		 * 
 		 * @param type0 The class of field '0' of the result tuples.
 		 * @param type1 The class of field '1' of the result tuples.
 		 * @param type2 The class of field '2' of the result tuples.
@@ -1167,7 +1167,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		 * @param type20 The class of field '20' of the result tuples.
 		 * @param type21 The class of field '21' of the result tuples.
 		 * @return The projected data set.
-		 *
+		 * 
 		 * @see Tuple
 		 * @see DataSet
 		 */
@@ -1176,7 +1176,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 			if(types.length != this.fieldIndexes.length) {
 				throw new IllegalArgumentException("Numbers of projected fields and types do not match.");
 			}
-
+			
 			TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, types);
 			TupleTypeInfo<Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>> tType = new TupleTypeInfo<Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>>(fTypes);
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/49cd35a7/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple1.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple1.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple1.java
index 66be617..7f5b7a2 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple1.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple1.java
@@ -114,6 +114,7 @@ public class Tuple1<T0> extends Tuple {
 	public boolean equals(Object o) {
 		if(this == o) { return true; }
 		if (!(o instanceof Tuple1)) { return false; }
+		@SuppressWarnings("rawtypes")
 		Tuple1 tuple = (Tuple1) o;
 		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
 		return true;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/49cd35a7/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple10.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple10.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple10.java
index 3a8ffd9..5c06654 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple10.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple10.java
@@ -222,6 +222,7 @@ public class Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9> extends Tuple {
 	public boolean equals(Object o) {
 		if(this == o) { return true; }
 		if (!(o instanceof Tuple10)) { return false; }
+		@SuppressWarnings("rawtypes")
 		Tuple10 tuple = (Tuple10) o;
 		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
 		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/49cd35a7/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple11.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple11.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple11.java
index 9cafa32..6413d82 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple11.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple11.java
@@ -234,6 +234,7 @@ public class Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10> extends Tuple
 	public boolean equals(Object o) {
 		if(this == o) { return true; }
 		if (!(o instanceof Tuple11)) { return false; }
+		@SuppressWarnings("rawtypes")
 		Tuple11 tuple = (Tuple11) o;
 		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
 		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/49cd35a7/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple12.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple12.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple12.java
index 81204a9..0f6d366 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple12.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple12.java
@@ -246,6 +246,7 @@ public class Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11> extends T
 	public boolean equals(Object o) {
 		if(this == o) { return true; }
 		if (!(o instanceof Tuple12)) { return false; }
+		@SuppressWarnings("rawtypes")
 		Tuple12 tuple = (Tuple12) o;
 		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
 		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/49cd35a7/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple13.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple13.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple13.java
index 5a4b02a..127dde8 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple13.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple13.java
@@ -258,6 +258,7 @@ public class Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12> exte
 	public boolean equals(Object o) {
 		if(this == o) { return true; }
 		if (!(o instanceof Tuple13)) { return false; }
+		@SuppressWarnings("rawtypes")
 		Tuple13 tuple = (Tuple13) o;
 		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
 		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/49cd35a7/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple14.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple14.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple14.java
index 205e0b4..d3042ef 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple14.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple14.java
@@ -270,6 +270,7 @@ public class Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>
 	public boolean equals(Object o) {
 		if(this == o) { return true; }
 		if (!(o instanceof Tuple14)) { return false; }
+		@SuppressWarnings("rawtypes")
 		Tuple14 tuple = (Tuple14) o;
 		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
 		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/49cd35a7/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple15.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple15.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple15.java
index 6ffaa4e..4e6205c 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple15.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple15.java
@@ -282,6 +282,7 @@ public class Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13,
 	public boolean equals(Object o) {
 		if(this == o) { return true; }
 		if (!(o instanceof Tuple15)) { return false; }
+		@SuppressWarnings("rawtypes")
 		Tuple15 tuple = (Tuple15) o;
 		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
 		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/49cd35a7/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple16.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple16.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple16.java
index ed4b67d..de57619 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple16.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple16.java
@@ -294,6 +294,7 @@ public class Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13,
 	public boolean equals(Object o) {
 		if(this == o) { return true; }
 		if (!(o instanceof Tuple16)) { return false; }
+		@SuppressWarnings("rawtypes")
 		Tuple16 tuple = (Tuple16) o;
 		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
 		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/49cd35a7/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple17.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple17.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple17.java
index 68760f5..6fd2554 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple17.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple17.java
@@ -306,6 +306,7 @@ public class Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13,
 	public boolean equals(Object o) {
 		if(this == o) { return true; }
 		if (!(o instanceof Tuple17)) { return false; }
+		@SuppressWarnings("rawtypes")
 		Tuple17 tuple = (Tuple17) o;
 		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
 		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/49cd35a7/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple18.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple18.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple18.java
index 33ac5d9..f45e406 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple18.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple18.java
@@ -318,6 +318,7 @@ public class Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13,
 	public boolean equals(Object o) {
 		if(this == o) { return true; }
 		if (!(o instanceof Tuple18)) { return false; }
+		@SuppressWarnings("rawtypes")
 		Tuple18 tuple = (Tuple18) o;
 		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
 		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/49cd35a7/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple19.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple19.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple19.java
index 4419ca3..2a867e3 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple19.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple19.java
@@ -330,6 +330,7 @@ public class Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13,
 	public boolean equals(Object o) {
 		if(this == o) { return true; }
 		if (!(o instanceof Tuple19)) { return false; }
+		@SuppressWarnings("rawtypes")
 		Tuple19 tuple = (Tuple19) o;
 		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
 		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/49cd35a7/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple2.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple2.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple2.java
index d175511..d3be5cc 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple2.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple2.java
@@ -126,6 +126,7 @@ public class Tuple2<T0, T1> extends Tuple {
 	public boolean equals(Object o) {
 		if(this == o) { return true; }
 		if (!(o instanceof Tuple2)) { return false; }
+		@SuppressWarnings("rawtypes")
 		Tuple2 tuple = (Tuple2) o;
 		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
 		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/49cd35a7/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple20.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple20.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple20.java
index ce6c03e..a8cd9ee 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple20.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple20.java
@@ -342,6 +342,7 @@ public class Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13,
 	public boolean equals(Object o) {
 		if(this == o) { return true; }
 		if (!(o instanceof Tuple20)) { return false; }
+		@SuppressWarnings("rawtypes")
 		Tuple20 tuple = (Tuple20) o;
 		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
 		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/49cd35a7/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple21.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple21.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple21.java
index 7995ced..6d0e7bb 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple21.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple21.java
@@ -354,6 +354,7 @@ public class Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13,
 	public boolean equals(Object o) {
 		if(this == o) { return true; }
 		if (!(o instanceof Tuple21)) { return false; }
+		@SuppressWarnings("rawtypes")
 		Tuple21 tuple = (Tuple21) o;
 		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
 		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/49cd35a7/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple22.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple22.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple22.java
index 060f158..43d0d3f 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple22.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple22.java
@@ -366,6 +366,7 @@ public class Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13,
 	public boolean equals(Object o) {
 		if(this == o) { return true; }
 		if (!(o instanceof Tuple22)) { return false; }
+		@SuppressWarnings("rawtypes")
 		Tuple22 tuple = (Tuple22) o;
 		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
 		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/49cd35a7/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple23.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple23.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple23.java
index 1910024..a71dcf6 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple23.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple23.java
@@ -378,6 +378,7 @@ public class Tuple23<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13,
 	public boolean equals(Object o) {
 		if(this == o) { return true; }
 		if (!(o instanceof Tuple23)) { return false; }
+		@SuppressWarnings("rawtypes")
 		Tuple23 tuple = (Tuple23) o;
 		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
 		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/49cd35a7/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple24.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple24.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple24.java
index 8ed9438..92b5467 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple24.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple24.java
@@ -390,6 +390,7 @@ public class Tuple24<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13,
 	public boolean equals(Object o) {
 		if(this == o) { return true; }
 		if (!(o instanceof Tuple24)) { return false; }
+		@SuppressWarnings("rawtypes")
 		Tuple24 tuple = (Tuple24) o;
 		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
 		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/49cd35a7/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple25.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple25.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple25.java
index cf0fcdd..013188d 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple25.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple25.java
@@ -402,6 +402,7 @@ public class Tuple25<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13,
 	public boolean equals(Object o) {
 		if(this == o) { return true; }
 		if (!(o instanceof Tuple25)) { return false; }
+		@SuppressWarnings("rawtypes")
 		Tuple25 tuple = (Tuple25) o;
 		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
 		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/49cd35a7/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple3.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple3.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple3.java
index aa6aff9..293b74c 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple3.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple3.java
@@ -138,6 +138,7 @@ public class Tuple3<T0, T1, T2> extends Tuple {
 	public boolean equals(Object o) {
 		if(this == o) { return true; }
 		if (!(o instanceof Tuple3)) { return false; }
+		@SuppressWarnings("rawtypes")
 		Tuple3 tuple = (Tuple3) o;
 		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
 		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/49cd35a7/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple4.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple4.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple4.java
index 3c165b0..7007896 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple4.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple4.java
@@ -150,6 +150,7 @@ public class Tuple4<T0, T1, T2, T3> extends Tuple {
 	public boolean equals(Object o) {
 		if(this == o) { return true; }
 		if (!(o instanceof Tuple4)) { return false; }
+		@SuppressWarnings("rawtypes")
 		Tuple4 tuple = (Tuple4) o;
 		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
 		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/49cd35a7/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple5.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple5.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple5.java
index 17d3ec7..fda4ab1 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple5.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple5.java
@@ -162,6 +162,7 @@ public class Tuple5<T0, T1, T2, T3, T4> extends Tuple {
 	public boolean equals(Object o) {
 		if(this == o) { return true; }
 		if (!(o instanceof Tuple5)) { return false; }
+		@SuppressWarnings("rawtypes")
 		Tuple5 tuple = (Tuple5) o;
 		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
 		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/49cd35a7/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple6.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple6.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple6.java
index 0f519aa..1cf0ad1 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple6.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple6.java
@@ -174,6 +174,7 @@ public class Tuple6<T0, T1, T2, T3, T4, T5> extends Tuple {
 	public boolean equals(Object o) {
 		if(this == o) { return true; }
 		if (!(o instanceof Tuple6)) { return false; }
+		@SuppressWarnings("rawtypes")
 		Tuple6 tuple = (Tuple6) o;
 		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
 		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/49cd35a7/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple7.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple7.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple7.java
index 1572083..ef08ca9 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple7.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple7.java
@@ -186,6 +186,7 @@ public class Tuple7<T0, T1, T2, T3, T4, T5, T6> extends Tuple {
 	public boolean equals(Object o) {
 		if(this == o) { return true; }
 		if (!(o instanceof Tuple7)) { return false; }
+		@SuppressWarnings("rawtypes")
 		Tuple7 tuple = (Tuple7) o;
 		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
 		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/49cd35a7/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple8.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple8.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple8.java
index b4fe194..76dc3d9 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple8.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple8.java
@@ -198,6 +198,7 @@ public class Tuple8<T0, T1, T2, T3, T4, T5, T6, T7> extends Tuple {
 	public boolean equals(Object o) {
 		if(this == o) { return true; }
 		if (!(o instanceof Tuple8)) { return false; }
+		@SuppressWarnings("rawtypes")
 		Tuple8 tuple = (Tuple8) o;
 		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
 		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/49cd35a7/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple9.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple9.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple9.java
index 0a1ce0b..23ab671 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple9.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/Tuple9.java
@@ -210,6 +210,7 @@ public class Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8> extends Tuple {
 	public boolean equals(Object o) {
 		if(this == o) { return true; }
 		if (!(o instanceof Tuple9)) { return false; }
+		@SuppressWarnings("rawtypes")
 		Tuple9 tuple = (Tuple9) o;
 		if (f0 != null ? !f0.equals(tuple.f0) : tuple.f0 != null) { return false; }
 		if (f1 != null ? !f1.equals(tuple.f1) : tuple.f1 != null) { return false; }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/49cd35a7/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/TupleGenerator.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/TupleGenerator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/TupleGenerator.java
index 08e5cd3..c73e4cd 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/TupleGenerator.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/tuple/TupleGenerator.java
@@ -674,6 +674,7 @@ class TupleGenerator {
 		w.println("\tpublic boolean equals(Object o) {");
 		w.println("\t\tif(this == o) { return true; }");
 		w.println("\t\tif (!(o instanceof " + className + ")) { return false; }");
+		w.println("\t\t@SuppressWarnings(\"rawtypes\")");
 		w.println("\t\t" + className + " tuple = (" + className + ") o;");
 		for (int i = 0; i < numFields; i++) {
 			String field = "f" + i;


[34/53] [abbrv] git commit: [FLINK-979] Fix NetworkThroughput test input and output task config

Posted by rm...@apache.org.
[FLINK-979] Fix NetworkThroughput test input and output task config

- Set DummyInputFormat and DummyOutputFormat via TaskConfig to respect task
  hierarchy refactoring.
- Run test via main method instead of JUnit test runner (this was originally
  a test in order to use RecordAPITestBase for JobGraph submission).

This closes #41.


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

Branch: refs/heads/travis_test
Commit: 2f0bd8fa26f5ed13d5f116b296557df8207a8260
Parents: f3c5428
Author: uce <u....@fu-berlin.de>
Authored: Tue Jun 24 23:24:17 2014 +0200
Committer: uce <u....@fu-berlin.de>
Committed: Wed Jun 25 10:25:54 2014 +0200

----------------------------------------------------------------------
 .../test/runtime/NetworkStackThroughput.java    | 280 +++++++++++--------
 1 file changed, 168 insertions(+), 112 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2f0bd8fa/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackThroughput.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackThroughput.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackThroughput.java
index ed6f608..74d52b9 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackThroughput.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackThroughput.java
@@ -13,22 +13,12 @@
 
 package eu.stratosphere.test.runtime;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.junit.After;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
+import eu.stratosphere.api.common.io.GenericInputFormat;
+import eu.stratosphere.api.common.io.OutputFormat;
+import eu.stratosphere.api.common.operators.util.UserCodeObjectWrapper;
 import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.core.io.InputSplit;
 import eu.stratosphere.nephele.jobgraph.DistributionPattern;
 import eu.stratosphere.nephele.jobgraph.JobGraph;
 import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException;
@@ -36,14 +26,22 @@ import eu.stratosphere.nephele.jobgraph.JobInputVertex;
 import eu.stratosphere.nephele.jobgraph.JobOutputVertex;
 import eu.stratosphere.nephele.jobgraph.JobTaskVertex;
 import eu.stratosphere.nephele.template.AbstractInvokable;
+import eu.stratosphere.pact.runtime.task.util.TaskConfig;
 import eu.stratosphere.runtime.io.api.RecordReader;
 import eu.stratosphere.runtime.io.api.RecordWriter;
 import eu.stratosphere.runtime.io.channels.ChannelType;
 import eu.stratosphere.test.util.RecordAPITestBase;
+import eu.stratosphere.types.Record;
 import eu.stratosphere.util.LogUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.junit.After;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
 
-@RunWith(Parameterized.class)
-public class NetworkStackThroughput extends RecordAPITestBase {
+public class NetworkStackThroughput {
 
 	private static final Log LOG = LogFactory.getLog(NetworkStackThroughput.class);
 
@@ -62,125 +60,102 @@ public class NetworkStackThroughput extends RecordAPITestBase {
 	private static final int IS_SLOW_SLEEP_MS = 10;
 
 	private static final int IS_SLOW_EVERY_NUM_RECORDS = (2 * 32 * 1024) / SpeedTestRecord.RECORD_SIZE;
-	
-	// ------------------------------------------------------------------------
-	
-	private int dataVolumeGb;
-	private boolean useForwarder;
-	private boolean isSlowSender;
-	private boolean isSlowReceiver;
-	private int parallelism;
 
 	// ------------------------------------------------------------------------
 
-	public NetworkStackThroughput(Configuration config) {
-		super(config);
-		
-		dataVolumeGb = this.config.getInteger(DATA_VOLUME_GB_CONFIG_KEY, 1);
-		useForwarder = this.config.getBoolean(USE_FORWARDER_CONFIG_KEY, true);
-		isSlowSender = this.config.getBoolean(IS_SLOW_SENDER_CONFIG_KEY, false);
-		isSlowReceiver = this.config.getBoolean(IS_SLOW_RECEIVER_CONFIG_KEY, false);
-		parallelism = config.getInteger(PARALLELISM_CONFIG_KEY, 1);
-		
-		int numSlots = config.getInteger(NUM_SLOTS_PER_TM_CONFIG_KEY, 1);
-		
-		if (parallelism % numSlots != 0) {
-			throw new RuntimeException("The test case defines a parallelism that is not a multiple of the slots per task manager.");
-		}
-		
-		setNumTaskTracker(parallelism / numSlots);
-		setTaskManagerNumSlots(numSlots);
-		
-		LogUtils.initializeDefaultConsoleLogger();
-	}
+	// wrapper to reuse RecordAPITestBase code in runs via main()
+	private static class TestBaseWrapper extends RecordAPITestBase {
 
-	@Parameters
-	public static Collection<Object[]> getConfigurations() {
-		Object[][] configParams = new Object[][]{
-				new Object[]{1, false, false, false, 4, 2},
-				new Object[]{1, true, false, false, 4, 2},
-				new Object[]{1, true, true, false, 4, 2},
-				new Object[]{1, true, false, true, 4, 2},
-				new Object[]{2, true, false, false, 4, 2},
-				new Object[]{4, true, false, false, 4, 2},
-				new Object[]{4, true, false, false, 8, 4},
-				new Object[]{4, true, false, false, 16, 8},
-		};
+		private int dataVolumeGb;
+		private boolean useForwarder;
+		private boolean isSlowSender;
+		private boolean isSlowReceiver;
+		private int parallelism;
 
-		List<Configuration> configs = new ArrayList<Configuration>(configParams.length);
-		for (Object[] p : configParams) {
-			Configuration config = new Configuration();
-			config.setInteger(DATA_VOLUME_GB_CONFIG_KEY, (Integer) p[0]);
-			config.setBoolean(USE_FORWARDER_CONFIG_KEY, (Boolean) p[1]);
-			config.setBoolean(IS_SLOW_SENDER_CONFIG_KEY, (Boolean) p[2]);
-			config.setBoolean(IS_SLOW_RECEIVER_CONFIG_KEY, (Boolean) p[3]);
-			config.setInteger(PARALLELISM_CONFIG_KEY, (Integer) p[4]);
-			config.setInteger(NUM_SLOTS_PER_TM_CONFIG_KEY, (Integer) p[5]);
+		public TestBaseWrapper(Configuration config) {
+			super(config);
 
-			configs.add(config);
+			dataVolumeGb = config.getInteger(DATA_VOLUME_GB_CONFIG_KEY, 1);
+			useForwarder = config.getBoolean(USE_FORWARDER_CONFIG_KEY, true);
+			isSlowSender = config.getBoolean(IS_SLOW_SENDER_CONFIG_KEY, false);
+			isSlowReceiver = config.getBoolean(IS_SLOW_RECEIVER_CONFIG_KEY, false);
+			parallelism = config.getInteger(PARALLELISM_CONFIG_KEY, 1);
+
+			int numSlots = config.getInteger(NUM_SLOTS_PER_TM_CONFIG_KEY, 1);
+
+			if (parallelism % numSlots != 0) {
+				throw new RuntimeException("The test case defines a parallelism that is not a multiple of the slots per task manager.");
+			}
+
+			setNumTaskTracker(parallelism / numSlots);
+			setTaskManagerNumSlots(numSlots);
+
+			LogUtils.initializeDefaultConsoleLogger();
 		}
 
-		return toParameterList(configs);
-	}
+		@Override
+		protected JobGraph getJobGraph() throws Exception {
+			return createJobGraph(dataVolumeGb, useForwarder, isSlowSender, isSlowReceiver, parallelism);
+		}
 
-	// ------------------------------------------------------------------------
+		private JobGraph createJobGraph(int dataVolumeGb, boolean useForwarder, boolean isSlowSender,
+				boolean isSlowReceiver, int numSubtasks) throws JobGraphDefinitionException {
 
-	@Override
-	protected JobGraph getJobGraph() throws Exception {
-		return createJobGraph(dataVolumeGb, useForwarder, isSlowSender, isSlowReceiver, parallelism);
-	}
+			JobGraph jobGraph = new JobGraph("Speed Test");
 
-	@After
-	public void calculateThroughput() {
-		if (getJobExecutionResult() != null) {
-			int dataVolumeGb = this.config.getInteger(DATA_VOLUME_GB_CONFIG_KEY, 1);
+			JobInputVertex producer = new JobInputVertex("Speed Test Producer", jobGraph);
+			producer.setInvokableClass(SpeedTestProducer.class);
+			producer.setNumberOfSubtasks(numSubtasks);
+			producer.getConfiguration().setInteger(DATA_VOLUME_GB_CONFIG_KEY, dataVolumeGb);
+			producer.getConfiguration().setBoolean(IS_SLOW_SENDER_CONFIG_KEY, isSlowSender);
 
-			double dataVolumeMbit = dataVolumeGb * 8192.0;
-			double runtimeSecs = getJobExecutionResult().getNetRuntime() / 1000.0;
+			TaskConfig inputConfig = new TaskConfig(producer.getConfiguration());
+			inputConfig.setStubWrapper(new UserCodeObjectWrapper<Object>(new DummyInputFormat()));
 
-			int mbitPerSecond = (int) Math.round(dataVolumeMbit / runtimeSecs);
+			JobTaskVertex forwarder = null;
+			if (useForwarder) {
+				forwarder = new JobTaskVertex("Speed Test Forwarder", jobGraph);
+				forwarder.setInvokableClass(SpeedTestForwarder.class);
+				forwarder.setNumberOfSubtasks(numSubtasks);
+			}
 
-			LOG.info(String.format("Test finished with throughput of %d MBit/s (" +
-					"runtime [secs]: %.2f, data volume [mbits]: %.2f)", mbitPerSecond, runtimeSecs, dataVolumeMbit));
-		}
-	}
+			JobOutputVertex consumer = new JobOutputVertex("Speed Test Consumer", jobGraph);
+			consumer.setInvokableClass(SpeedTestConsumer.class);
+			consumer.setNumberOfSubtasks(numSubtasks);
+			consumer.getConfiguration().setBoolean(IS_SLOW_RECEIVER_CONFIG_KEY, isSlowReceiver);
 
-	private JobGraph createJobGraph(int dataVolumeGb, boolean useForwarder, boolean isSlowSender, boolean isSlowReceiver,
-									int numSubtasks) throws JobGraphDefinitionException {
+			TaskConfig outputConfig = new TaskConfig(consumer.getConfiguration());
+			outputConfig.setStubWrapper(new UserCodeObjectWrapper<Object>(new DummyOutputFormat()));
 
-		JobGraph jobGraph = new JobGraph("Speed Test");
+			if (useForwarder) {
+				producer.connectTo(forwarder, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
+				forwarder.connectTo(consumer, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
 
-		JobInputVertex producer = new JobInputVertex("Speed Test Producer", jobGraph);
-		producer.setInvokableClass(SpeedTestProducer.class);
-		producer.setNumberOfSubtasks(numSubtasks);
-		producer.getConfiguration().setInteger(DATA_VOLUME_GB_CONFIG_KEY, dataVolumeGb);
-		producer.getConfiguration().setBoolean(IS_SLOW_SENDER_CONFIG_KEY, isSlowSender);
+				forwarder.setVertexToShareInstancesWith(producer);
+				consumer.setVertexToShareInstancesWith(producer);
+			}
+			else {
+				producer.connectTo(consumer, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
+				producer.setVertexToShareInstancesWith(consumer);
+			}
 
-		JobTaskVertex forwarder = null;
-		if (useForwarder) {
-			forwarder = new JobTaskVertex("Speed Test Forwarder", jobGraph);
-			forwarder.setInvokableClass(SpeedTestForwarder.class);
-			forwarder.setNumberOfSubtasks(numSubtasks);
+			return jobGraph;
 		}
 
-		JobOutputVertex consumer = new JobOutputVertex("Speed Test Consumer", jobGraph);
-		consumer.setInvokableClass(SpeedTestConsumer.class);
-		consumer.setNumberOfSubtasks(numSubtasks);
-		consumer.getConfiguration().setBoolean(IS_SLOW_RECEIVER_CONFIG_KEY, isSlowReceiver);
+		@After
+		public void calculateThroughput() {
+			if (getJobExecutionResult() != null) {
+				int dataVolumeGb = this.config.getInteger(DATA_VOLUME_GB_CONFIG_KEY, 1);
 
-		if (useForwarder) {
-			producer.connectTo(forwarder, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
-			forwarder.connectTo(consumer, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
+				double dataVolumeMbit = dataVolumeGb * 8192.0;
+				double runtimeSecs = getJobExecutionResult().getNetRuntime() / 1000.0;
 
-			forwarder.setVertexToShareInstancesWith(producer);
-			consumer.setVertexToShareInstancesWith(producer);
-		}
-		else {
-			producer.connectTo(consumer, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
-			producer.setVertexToShareInstancesWith(consumer);
-		}
+				int mbitPerSecond = (int) Math.round(dataVolumeMbit / runtimeSecs);
 
-		return jobGraph;
+				LOG.info(String.format("Test finished with throughput of %d MBit/s (runtime [secs]: %.2f, " +
+								"data volume [mbits]: %.2f)", mbitPerSecond, runtimeSecs, dataVolumeMbit));
+			}
+		}
 	}
 
 	// ------------------------------------------------------------------------
@@ -293,4 +268,85 @@ public class NetworkStackThroughput extends RecordAPITestBase {
 			in.readFully(this.buf);
 		}
 	}
+
+	public static final class DummyInputFormat extends GenericInputFormat {
+
+		private static final long serialVersionUID = 6891640958330871924L;
+
+		@Override
+		public void open(InputSplit split) throws IOException {
+
+		}
+
+		@Override
+		public boolean reachedEnd() throws IOException {
+			return false;
+		}
+
+		@Override
+		public Object nextRecord(Object reuse) throws IOException {
+			return null;
+		}
+	}
+
+	public static final class DummyOutputFormat implements OutputFormat<Record> {
+
+		@Override
+		public void configure(Configuration parameters) {
+		}
+
+		@Override
+		public void open(int taskNumber, int numTasks) {
+		}
+
+		@Override
+		public void writeRecord(Record record) {
+		}
+
+		@Override
+		public void close() {
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	public void testThroughput() throws Exception {
+		Object[][] configParams = new Object[][]{
+				new Object[]{1, false, false, false, 4, 2},
+				new Object[]{1, true, false, false, 4, 2},
+				new Object[]{1, true, true, false, 4, 2},
+				new Object[]{1, true, false, true, 4, 2},
+				new Object[]{2, true, false, false, 4, 2},
+				new Object[]{4, true, false, false, 4, 2},
+				new Object[]{4, true, false, false, 8, 4},
+				new Object[]{4, true, false, false, 16, 8},
+		};
+
+		for (Object[] p : configParams) {
+			Configuration config = new Configuration();
+			config.setInteger(DATA_VOLUME_GB_CONFIG_KEY, (Integer) p[0]);
+			config.setBoolean(USE_FORWARDER_CONFIG_KEY, (Boolean) p[1]);
+			config.setBoolean(IS_SLOW_SENDER_CONFIG_KEY, (Boolean) p[2]);
+			config.setBoolean(IS_SLOW_RECEIVER_CONFIG_KEY, (Boolean) p[3]);
+			config.setInteger(PARALLELISM_CONFIG_KEY, (Integer) p[4]);
+			config.setInteger(NUM_SLOTS_PER_TM_CONFIG_KEY, (Integer) p[5]);
+
+			TestBaseWrapper test = new TestBaseWrapper(config);
+
+			test.startCluster();
+			test.testJob();
+			test.calculateThroughput();
+			test.stopCluster();
+		}
+	}
+
+	private void runAllTests() throws Exception {
+		testThroughput();
+
+		System.out.println("Done.");
+	}
+
+	public static void main(String[] args) throws Exception {
+		new NetworkStackThroughput().runAllTests();
+	}
 }


[45/53] [abbrv] git commit: [FLINK-760] Add distinct operator

Posted by rm...@apache.org.
[FLINK-760] Add distinct operator


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

Branch: refs/heads/travis_test
Commit: e2aabd90604d5880ef8e7b82ce58de218ddd48a7
Parents: 6c827fb
Author: Markus Holzemer <ma...@gmx.de>
Authored: Tue May 27 11:24:49 2014 +0200
Committer: uce <u....@fu-berlin.de>
Committed: Wed Jun 25 15:45:29 2014 +0200

----------------------------------------------------------------------
 .../java/eu/stratosphere/api/java/DataSet.java  |  45 +++-
 .../api/java/operators/DistinctOperator.java    | 109 +++++++++-
 .../api/java/operator/DistinctOperatorTest.java | 170 +++++++++++++++
 .../test/javaApiOperators/DistinctITCase.java   | 208 +++++++++++++++++++
 4 files changed, 522 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e2aabd90/stratosphere-java/src/main/java/eu/stratosphere/api/java/DataSet.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/DataSet.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/DataSet.java
index 0770f24..a077e9a 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/DataSet.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/DataSet.java
@@ -36,6 +36,7 @@ import eu.stratosphere.api.java.operators.CrossOperator;
 import eu.stratosphere.api.java.operators.CrossOperator.DefaultCross;
 import eu.stratosphere.api.java.operators.CustomUnaryOperation;
 import eu.stratosphere.api.java.operators.DataSink;
+import eu.stratosphere.api.java.operators.DistinctOperator;
 import eu.stratosphere.api.java.operators.FilterOperator;
 import eu.stratosphere.api.java.operators.FlatMapOperator;
 import eu.stratosphere.api.java.operators.Grouping;
@@ -301,13 +302,45 @@ public abstract class DataSet<T> {
 	//  distinct
 	// --------------------------------------------------------------------------------------------
 	
-//	public <K extends Comparable<K>> DistinctOperator<T> distinct(KeySelector<T, K> keyExtractor) {
-//		return new DistinctOperator<T>(this, new Keys.SelectorFunctionKeys<T, K>(keyExtractor, getType()));
-//	}
+	/**
+	 * Returns a distinct set of a {@link DataSet} using a {@link KeySelector} function.
+	 * <p/>
+	 * The KeySelector function is called for each element of the DataSet and extracts a single key value on which the
+	 * decision is made if two items are distinct or not.
+	 *  
+	 * @param keyExtractor The KeySelector function which extracts the key values from the DataSet on which the
+	 *                     distinction of the DataSet is decided.
+	 * @return A DistinctOperator that represents the distinct DataSet.
+	 */
+	public <K extends Comparable<K>> DistinctOperator<T> distinct(KeySelector<T, K> keyExtractor) {
+		return new DistinctOperator<T>(this, new Keys.SelectorFunctionKeys<T, K>(keyExtractor, getType()));
+	}
 	
-//	public DistinctOperator<T> distinct(int... fields) {
-//		return new DistinctOperator<T>(this, new Keys.FieldPositionKeys<T>(fields, getType(), true));
-//	}
+	/**
+	 * Returns a distinct set of a {@link Tuple} {@link DataSet} using field position keys.
+	 * <p/>
+	 * The field position keys specify the fields of Tuples on which the decision is made if two Tuples are distinct or
+	 * not.
+	 * <p/>
+	 * Note: Field position keys can only be specified for Tuple DataSets.
+	 *
+	 * @param fields One or more field positions on which the distinction of the DataSet is decided. 
+	 * @return A DistinctOperator that represents the distinct DataSet.
+	 */
+	public DistinctOperator<T> distinct(int... fields) {
+		return new DistinctOperator<T>(this, new Keys.FieldPositionKeys<T>(fields, getType(), true));
+	}
+	
+	/**
+	 * Returns a distinct set of a {@link Tuple} {@link DataSet} using all fields of the tuple.
+	 * <p/>
+	 * Note: This operator can only be applied to Tuple DataSets.
+	 * 
+	 * @return A DistinctOperator that represents the distinct DataSet.
+	 */
+	public DistinctOperator<T> distinct() {
+		return new DistinctOperator<T>(this, null);
+	}
 	
 	// --------------------------------------------------------------------------------------------
 	//  Grouping

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e2aabd90/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/DistinctOperator.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/DistinctOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/DistinctOperator.java
index 87c99de..ed1d1a5 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/DistinctOperator.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/DistinctOperator.java
@@ -14,8 +14,23 @@
  **********************************************************************************************************************/
 package eu.stratosphere.api.java.operators;
 
+import java.util.Iterator;
+
+import eu.stratosphere.api.common.InvalidProgramException;
+import eu.stratosphere.api.common.functions.GenericGroupReduce;
+import eu.stratosphere.api.common.functions.GenericMap;
 import eu.stratosphere.api.common.operators.Operator;
+import eu.stratosphere.api.common.operators.UnaryOperatorInformation;
+import eu.stratosphere.api.common.operators.base.GroupReduceOperatorBase;
+import eu.stratosphere.api.common.operators.base.MapOperatorBase;
 import eu.stratosphere.api.java.DataSet;
+import eu.stratosphere.api.java.functions.GroupReduceFunction;
+import eu.stratosphere.api.java.operators.translation.KeyExtractingMapper;
+import eu.stratosphere.api.java.operators.translation.PlanUnwrappingReduceGroupOperator;
+import eu.stratosphere.api.java.tuple.Tuple2;
+import eu.stratosphere.api.java.typeutils.TupleTypeInfo;
+import eu.stratosphere.types.TypeInformation;
+import eu.stratosphere.util.Collector;
 
 /**
  * This operator represents the application of a "distinct" function on a data set, and the
@@ -25,21 +40,107 @@ import eu.stratosphere.api.java.DataSet;
  */
 public class DistinctOperator<T> extends SingleInputOperator<T, T, DistinctOperator<T>> {
 	
-	@SuppressWarnings("unused")
 	private final Keys<T> keys;
 	
 	public DistinctOperator(DataSet<T> input, Keys<T> keys) {
 		super(input, input.getType());
 		
+		// if keys is null distinction is done on all tuple fields
 		if (keys == null) {
-			throw new NullPointerException();
+			if (input.getType().isTupleType()) {
+				
+				TupleTypeInfo<?> tupleType = (TupleTypeInfo<?>) input.getType();
+				int[] allFields = new int[tupleType.getArity()];
+				for(int i = 0; i < tupleType.getArity(); i++) {
+					allFields[i] = i;
+				}
+				keys = new Keys.FieldPositionKeys<T>(allFields, input.getType(), true);
+			}
+			else {
+				throw new InvalidProgramException("Distinction on all fields is only possible on tuple data types.");
+			}
+		}
+		
+		
+		// FieldPositionKeys can only be applied on Tuples
+		if (keys instanceof Keys.FieldPositionKeys && !input.getType().isTupleType()) {
+			throw new InvalidProgramException("Distinction on field positions is only possible on tuple data types.");
 		}
 		
 		this.keys = keys;
 	}
 
 	@Override
-	protected eu.stratosphere.api.common.operators.SingleInputOperator<T, T, ?> translateToDataFlow(Operator<T> input) {
-		throw new UnsupportedOperationException("NOT IMPLEMENTED");
+	protected eu.stratosphere.api.common.operators.base.GroupReduceOperatorBase<?, T, ?> translateToDataFlow(Operator<T> input) {
+		
+		GroupReduceFunction<T, T> function = new DistinctFunction<T>();
+		String name = function.getClass().getName();
+		
+		if (keys instanceof Keys.FieldPositionKeys) {
+
+			int[] logicalKeyPositions = keys.computeLogicalKeyPositions();
+			UnaryOperatorInformation<T, T> operatorInfo = new UnaryOperatorInformation<T, T>(getInputType(), getResultType());
+			GroupReduceOperatorBase<T, T, GenericGroupReduce<T, T>> po =
+					new GroupReduceOperatorBase<T, T, GenericGroupReduce<T, T>>(function, operatorInfo, logicalKeyPositions, name);
+
+			po.setCombinable(true);
+			po.setInput(input);
+			po.setDegreeOfParallelism(this.getParallelism());
+			
+			return po;
+		}
+		else if (keys instanceof Keys.SelectorFunctionKeys) {
+		
+			@SuppressWarnings("unchecked")
+			Keys.SelectorFunctionKeys<T, ?> selectorKeys = (Keys.SelectorFunctionKeys<T, ?>) keys;
+			
+			PlanUnwrappingReduceGroupOperator<T, T, ?> po = translateSelectorFunctionDistinct(
+							selectorKeys, function, getInputType(), getResultType(), name, input, true);
+			
+			po.setDegreeOfParallelism(this.getParallelism());
+			
+			return po;
+		}
+		else {
+			throw new UnsupportedOperationException("Unrecognized key type.");
+		}
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	private static <IN, OUT, K> PlanUnwrappingReduceGroupOperator<IN, OUT, K> translateSelectorFunctionDistinct(
+			Keys.SelectorFunctionKeys<IN, ?> rawKeys, GroupReduceFunction<IN, OUT> function,
+			TypeInformation<IN> inputType, TypeInformation<OUT> outputType, String name, Operator<IN> input,
+			boolean combinable)
+	{
+		@SuppressWarnings("unchecked")
+		final Keys.SelectorFunctionKeys<IN, K> keys = (Keys.SelectorFunctionKeys<IN, K>) rawKeys;
+		
+		TypeInformation<Tuple2<K, IN>> typeInfoWithKey = new TupleTypeInfo<Tuple2<K, IN>>(keys.getKeyType(), inputType);
+		
+		KeyExtractingMapper<IN, K> extractor = new KeyExtractingMapper<IN, K>(keys.getKeyExtractor());
+		
+		PlanUnwrappingReduceGroupOperator<IN, OUT, K> reducer = new PlanUnwrappingReduceGroupOperator<IN, OUT, K>(function, keys, name, outputType, typeInfoWithKey, combinable);
+		
+		MapOperatorBase<IN, Tuple2<K, IN>, GenericMap<IN, Tuple2<K, IN>>> mapper = new MapOperatorBase<IN, Tuple2<K, IN>, GenericMap<IN, Tuple2<K, IN>>>(extractor, new UnaryOperatorInformation<IN, Tuple2<K, IN>>(inputType, typeInfoWithKey), "Key Extractor");
+
+		reducer.setInput(mapper);
+		mapper.setInput(input);
+		
+		// set the mapper's parallelism to the input parallelism to make sure it is chained
+		mapper.setDegreeOfParallelism(input.getDegreeOfParallelism());
+		
+		return reducer;
+	}
+	
+	public static final class DistinctFunction<T> extends GroupReduceFunction<T, T> {
+
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void reduce(Iterator<T> values, Collector<T> out)
+				throws Exception {
+			out.collect(values.next());
+		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e2aabd90/stratosphere-java/src/test/java/eu/stratosphere/api/java/operator/DistinctOperatorTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/test/java/eu/stratosphere/api/java/operator/DistinctOperatorTest.java b/stratosphere-java/src/test/java/eu/stratosphere/api/java/operator/DistinctOperatorTest.java
new file mode 100644
index 0000000..a520812
--- /dev/null
+++ b/stratosphere-java/src/test/java/eu/stratosphere/api/java/operator/DistinctOperatorTest.java
@@ -0,0 +1,170 @@
+/***********************************************************************************************************************
+ *
+ * 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.api.java.operator;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+import junit.framework.Assert;
+
+import org.junit.Test;
+
+import eu.stratosphere.api.common.InvalidProgramException;
+import eu.stratosphere.api.java.DataSet;
+import eu.stratosphere.api.java.ExecutionEnvironment;
+import eu.stratosphere.api.java.functions.KeySelector;
+import eu.stratosphere.api.java.tuple.Tuple5;
+import eu.stratosphere.api.java.typeutils.BasicTypeInfo;
+import eu.stratosphere.api.java.typeutils.TupleTypeInfo;
+
+public class DistinctOperatorTest {
+
+	// TUPLE DATA
+	private final List<Tuple5<Integer, Long, String, Long, Integer>> emptyTupleData = 
+			new ArrayList<Tuple5<Integer, Long, String, Long, Integer>>();
+	
+	private final TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>> tupleTypeInfo = new 
+			TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>>(
+					BasicTypeInfo.INT_TYPE_INFO,
+					BasicTypeInfo.LONG_TYPE_INFO,
+					BasicTypeInfo.STRING_TYPE_INFO,
+					BasicTypeInfo.LONG_TYPE_INFO,
+					BasicTypeInfo.INT_TYPE_INFO
+			);
+	
+	// LONG DATA
+	private final List<Long> emptyLongData = new ArrayList<Long>();
+	
+	private final List<CustomType> customTypeData = new ArrayList<CustomType>();
+	
+	@Test  
+	public void testDistinctByKeyFields1() {
+		
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// should work
+		try {
+			tupleDs.distinct(0);
+		} catch(Exception e) {
+			Assert.fail();
+		}
+	}
+	
+	@Test(expected = InvalidProgramException.class)  
+	public void testDistinctByKeyFields2() {
+		
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Long> longDs = env.fromCollection(emptyLongData, BasicTypeInfo.LONG_TYPE_INFO);
+		// should not work: distinct on basic type
+		longDs.distinct(0);
+	}
+	
+	@Test(expected = InvalidProgramException.class)  
+	public void testDistinctByKeyFields3() {
+		
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		
+		this.customTypeData.add(new CustomType());
+		
+		DataSet<CustomType> customDs = env.fromCollection(customTypeData);
+		// should not work: distinct on custom type
+		customDs.distinct(0);
+		
+	}
+	
+	@Test
+	public void testDistinctByKeyFields4() {
+		
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// should work
+		tupleDs.distinct();
+	}
+	
+	@Test(expected = InvalidProgramException.class)
+	public void testDistinctByKeyFields5() {
+		
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		
+		this.customTypeData.add(new CustomType());
+		
+		DataSet<CustomType> customDs = env.fromCollection(customTypeData);
+
+		// should not work, distinct without selector on custom types
+		customDs.distinct();
+	}
+	
+	@Test(expected = IllegalArgumentException.class)
+	public void testDistinctByKeyFields6() {
+		
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
+
+		// should not work, negative field position
+		tupleDs.distinct(-1);
+	}
+	
+	@Test
+	@SuppressWarnings("serial")
+	public void testDistinctByKeySelector1() {
+		
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		this.customTypeData.add(new CustomType());
+		
+		try {
+			DataSet<CustomType> customDs = env.fromCollection(customTypeData);
+			// should work
+			customDs.distinct(
+					new KeySelector<DistinctOperatorTest.CustomType, Long>() {
+	
+						@Override
+						public Long getKey(CustomType value) {
+							return value.myLong;
+					}
+			});
+		} catch(Exception e) {
+			Assert.fail();
+		}
+		
+	}
+	
+
+	public static class CustomType implements Serializable {
+		
+		private static final long serialVersionUID = 1L;
+		
+		public int myInt;
+		public long myLong;
+		public String myString;
+		
+		public CustomType() {};
+		
+		public CustomType(int i, long l, String s) {
+			myInt = i;
+			myLong = l;
+			myString = s;
+		}
+		
+		@Override
+		public String toString() {
+			return myInt+","+myLong+","+myString;
+		}
+	}
+	
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e2aabd90/stratosphere-tests/src/test/java/eu/stratosphere/test/javaApiOperators/DistinctITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/javaApiOperators/DistinctITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/javaApiOperators/DistinctITCase.java
new file mode 100644
index 0000000..11fcf97
--- /dev/null
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/javaApiOperators/DistinctITCase.java
@@ -0,0 +1,208 @@
+/***********************************************************************************************************************
+ *
+ * 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.test.javaApiOperators;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.LinkedList;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import eu.stratosphere.api.java.DataSet;
+import eu.stratosphere.api.java.ExecutionEnvironment;
+import eu.stratosphere.api.java.functions.KeySelector;
+import eu.stratosphere.api.java.functions.MapFunction;
+import eu.stratosphere.api.java.tuple.Tuple1;
+import eu.stratosphere.api.java.tuple.Tuple3;
+import eu.stratosphere.api.java.tuple.Tuple5;
+import eu.stratosphere.configuration.Configuration;
+import eu.stratosphere.test.javaApiOperators.util.CollectionDataSets;
+import eu.stratosphere.test.javaApiOperators.util.CollectionDataSets.CustomType;
+import eu.stratosphere.test.util.JavaProgramTestBase;
+
+@SuppressWarnings("serial")
+@RunWith(Parameterized.class)
+public class DistinctITCase extends JavaProgramTestBase {
+	
+	private static int NUM_PROGRAMS = 5;
+	
+	private int curProgId = config.getInteger("ProgramId", -1);
+	private String resultPath;
+	private String expectedResult;
+	
+	public DistinctITCase(Configuration config) {
+		super(config);
+	}
+	
+	@Override
+	protected void preSubmit() throws Exception {
+		resultPath = getTempDirPath("result");
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		expectedResult = DistinctProgs.runProgram(curProgId, resultPath);
+	}
+	
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(expectedResult, resultPath);
+	}
+	
+	@Parameters
+	public static Collection<Object[]> getConfigurations() throws IOException {
+		LinkedList<Configuration> tConfigs = new LinkedList<Configuration>();
+
+		for(int i=1; i <= NUM_PROGRAMS; i++) {
+			Configuration config = new Configuration();
+			config.setInteger("ProgramId", i);
+			tConfigs.add(config);
+		}
+		
+		return toParameterList(tConfigs);
+	}
+	
+	private static class DistinctProgs {
+		
+		public static String runProgram(int progId, String resultPath) throws Exception {
+			
+			switch(progId) {
+			case 1: {
+				
+				/*
+				 * check correctness of distinct on tuples with key field selector
+				 */
+				
+				final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+				
+				DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.getSmall3TupleDataSet(env);
+				DataSet<Tuple3<Integer, Long, String>> distinctDs = ds.union(ds).distinct(0, 1, 2);
+				
+				distinctDs.writeAsCsv(resultPath);
+				env.execute();
+				
+				// return expected result
+				return "1,1,Hi\n" +
+						"2,2,Hello\n" +
+						"3,2,Hello world\n";
+			}
+			case 2: {
+				
+				/*
+				 * check correctness of distinct on tuples with key field selector with not all fields selected
+				 */
+				
+				final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+				
+				DataSet<Tuple5<Integer, Long,  Integer, String, Long>> ds = CollectionDataSets.getSmall5TupleDataSet(env);
+				DataSet<Tuple1<Integer>> distinctDs = ds.union(ds).distinct(0).project(0).types(Integer.class);
+				
+				distinctDs.writeAsCsv(resultPath);
+				env.execute();
+				
+				// return expected result
+				return "1\n" +
+						"2\n";
+			}
+			case 3: {
+				
+				/*
+				 * check correctness of distinct on tuples with key extractor
+				 */
+				
+				final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+				
+				DataSet<Tuple5<Integer, Long,  Integer, String, Long>> ds = CollectionDataSets.getSmall5TupleDataSet(env);
+				DataSet<Tuple1<Integer>> reduceDs = ds.union(ds)
+						.distinct(new KeySelector<Tuple5<Integer, Long,  Integer, String, Long>, Integer>() {
+									private static final long serialVersionUID = 1L;
+									@Override
+									public Integer getKey(Tuple5<Integer, Long,  Integer, String, Long> in) {
+										return in.f0;
+									}
+								}).project(0).types(Integer.class);
+				
+				reduceDs.writeAsCsv(resultPath);
+				env.execute();
+				
+				// return expected result
+				return "1\n" +
+						"2\n";
+								
+			}
+			case 4: {
+				
+				/*
+				 * check correctness of distinct on custom type with type extractor
+				 */
+				
+				final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+				
+				DataSet<CustomType> ds = CollectionDataSets.getCustomTypeDataSet(env);
+				DataSet<Tuple1<Integer>> reduceDs = ds
+						.distinct(new KeySelector<CustomType, Integer>() {
+									private static final long serialVersionUID = 1L;
+									@Override
+									public Integer getKey(CustomType in) {
+										return in.myInt;
+									}
+								})
+						.map(new MapFunction<CollectionDataSets.CustomType, Tuple1<Integer>>() {
+							@Override
+							public Tuple1<Integer> map(CustomType value) throws Exception {
+								return new Tuple1<Integer>(value.myInt);
+							}
+						});
+				
+				reduceDs.writeAsCsv(resultPath);
+				env.execute();
+				
+				// return expected result
+				return "1\n" +
+						"2\n" +
+						"3\n" +
+						"4\n" +
+						"5\n" +
+						"6\n";
+				
+			}
+			case 5: {
+				
+				/*
+				 * check correctness of distinct on tuples
+				 */
+				
+				final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+				
+				DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.getSmall3TupleDataSet(env);
+				DataSet<Tuple3<Integer, Long, String>> distinctDs = ds.union(ds).distinct();
+				
+				distinctDs.writeAsCsv(resultPath);
+				env.execute();
+				
+				// return expected result
+				return "1,1,Hi\n" +
+						"2,2,Hello\n" +
+						"3,2,Hello world\n";
+			}
+			default: 
+				throw new IllegalArgumentException("Invalid program id");
+			}
+		}
+	}
+}


[28/53] [abbrv] [FLINK-962] Initial import of documentation from website into source code (closes #34)

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/cogroup.svg
----------------------------------------------------------------------
diff --git a/docs/img/cogroup.svg b/docs/img/cogroup.svg
new file mode 100644
index 0000000..26ec868
--- /dev/null
+++ b/docs/img/cogroup.svg
@@ -0,0 +1,856 @@
+<?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"
+   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+   width="800"
+   height="195"
+   id="svg35289"
+   version="1.1"
+   inkscape:version="0.48.3.1 r9886"
+   sodipodi:docname="cogroup.svg">
+  <defs
+     id="defs35291">
+    <marker
+       inkscape:stockid="Arrow1Lstart"
+       orient="auto"
+       refY="0.0"
+       refX="0.0"
+       id="Arrow1Lstart"
+       style="overflow:visible">
+      <path
+         id="path3855"
+         d="M 0.0,0.0 L 5.0,-5.0 L -12.5,0.0 L 5.0,5.0 L 0.0,0.0 z "
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1.0pt"
+         transform="scale(0.8) translate(12.5,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="EmptyTriangleInM"
+       orient="auto"
+       refY="0.0"
+       refX="0.0"
+       id="EmptyTriangleInM"
+       style="overflow:visible">
+      <path
+         id="path4009"
+         d="M 5.77,0.0 L -2.88,5.0 L -2.88,-5.0 L 5.77,0.0 z "
+         style="fill-rule:evenodd;fill:#FFFFFF;stroke:#000000;stroke-width:1.0pt"
+         transform="scale(-0.4) translate(-4.5,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lstart"
+       orient="auto"
+       refY="0.0"
+       refX="0.0"
+       id="Arrow2Lstart"
+       style="overflow:visible">
+      <path
+         id="path3873"
+         style="fill-rule:evenodd;stroke-width:0.62500000;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 L -2.2072895,0.016013256 L 8.7185884,-4.0017078 C 6.9730900,-1.6296469 6.9831476,1.6157441 8.7185878,4.0337352 z "
+         transform="scale(1.1) translate(1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0.0"
+       refX="0.0"
+       id="Arrow1Mstart"
+       style="overflow:visible">
+      <path
+         id="path3861"
+         d="M 0.0,0.0 L 5.0,-5.0 L -12.5,0.0 L 5.0,5.0 L 0.0,0.0 z "
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1.0pt"
+         transform="scale(0.4) translate(10,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Sstart"
+       orient="auto"
+       refY="0.0"
+       refX="0.0"
+       id="Arrow1Sstart"
+       style="overflow:visible">
+      <path
+         id="path3867"
+         d="M 0.0,0.0 L 5.0,-5.0 L -12.5,0.0 L 5.0,5.0 L 0.0,0.0 z "
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1.0pt"
+         transform="scale(0.2) translate(6,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35250"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35252"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35254"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35256"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35258"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35260"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend-0"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456-7"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35403"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35405"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35407"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35409"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35411"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35413"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend-9"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456-75"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35572"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35574"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35576"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35578"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35580"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35582"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35584"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35586"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35588"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35590"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend-3"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456-77"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35951"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35953"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35955"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35957"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35959"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35961"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35963"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35965"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35967"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35969"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend-93"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456-0"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker36515"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path36517"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker36519"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path36521"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker36523"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path36525"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker36527"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path36529"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker36531"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path36533"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend-36"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456-73"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+  </defs>
+  <sodipodi:namedview
+     id="base"
+     pagecolor="#ffffff"
+     bordercolor="#666666"
+     borderopacity="1.0"
+     inkscape:pageopacity="0.0"
+     inkscape:pageshadow="2"
+     inkscape:zoom="0.49497475"
+     inkscape:cx="-657.23669"
+     inkscape:cy="-312.51371"
+     inkscape:document-units="px"
+     inkscape:current-layer="layer1"
+     showgrid="false"
+     fit-margin-top="0"
+     fit-margin-left="0"
+     fit-margin-right="0"
+     fit-margin-bottom="0"
+     inkscape:window-width="1916"
+     inkscape:window-height="1161"
+     inkscape:window-x="1400"
+     inkscape:window-y="18"
+     inkscape:window-maximized="0" />
+  <metadata
+     id="metadata35294">
+    <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>
+  <g
+     inkscape:label="Layer 1"
+     inkscape:groupmode="layer"
+     id="layer1"
+     transform="translate(-484.80217,-248.14373)">
+    <g
+       id="g34970"
+       transform="translate(1902.9756,-921.79345)">
+      <rect
+         y="1225.0306"
+         x="-1230.1406"
+         height="136.06552"
+         width="57.281994"
+         id="rect2994-5-9-9-1"
+         style="opacity:0.95999995;fill:#f3f3f3;stroke:#000000;stroke-width:1.5;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:25" />
+      <g
+         style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+         transform="matrix(0.40773951,0,0,0.41277147,-1308.0944,1915.8924)"
+         id="g3792-4-1-79-5">
+        <rect
+           style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65632725;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-0-0-93-9"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65632725;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-5-7-2-4"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <g
+         style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+         id="g3792-8-9-5-4-4"
+         transform="matrix(0.40773951,0,0,0.41277147,-1308.0944,1950.654)">
+        <rect
+           style="fill:#f3de4a;fill-opacity:1;stroke:#000000;stroke-width:3.65632725;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-1-46-8-3-2"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65632725;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-0-9-7-7-0"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <g
+         style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+         id="g3792-0-2-0-1-0"
+         transform="matrix(0.40773951,0,0,0.41277147,-1308.0944,2020.1768)">
+        <rect
+           style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.65632725;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-4-2-4-2-0"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65632725;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-4-4-8-2-0"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <g
+         style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+         id="g3792-8-4-7-0-0-8"
+         transform="matrix(0.40773951,0,0,0.41277147,-1308.0944,1985.4152)">
+        <rect
+           style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65632725;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-1-4-75-4-2-1"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65632725;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-0-7-4-2-1-7"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <rect
+         y="1173.7782"
+         x="-1152.9976"
+         height="32.716614"
+         width="232.52985"
+         id="rect2994-5-9-4-7-4"
+         style="opacity:0.95999995;fill:#f3f3f3;stroke:#000000;stroke-width:1.5;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:25" />
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cc"
+         id="path4066-2-0-2-4"
+         d="m -1181.1127,1241.642 148.6025,0"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.5;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="ccccc"
+         id="path4066-2-0-5-7-0"
+         d="m -1181.1127,1276.4035 19.9637,14.4355 c 5.7719,-7.3309 13.2528,-0.6632 8.304,6.0044 l 19.8863,14.3795 219.2601,-0.059"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.5;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="ccc"
+         id="path4066-2-0-7-0-9"
+         d="m -1181.1127,1311.1647 48.4175,-34.3028 100.4399,-0.4584"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.5;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cc"
+         id="path4066-2-0-3-3-5"
+         d="m -1181.1127,1345.9263 28.844,0"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.5;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cc"
+         id="path4066-2-0-9-4-1"
+         d="m -1068.1581,1200.5444 -0.4619,128.2519"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.5;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cc"
+         id="path4066-2-0-9-5-6-0"
+         d="m -948.6859,1200.4497 0.052,24.4359"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.5;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <g
+         style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+         transform="matrix(0.40773951,0,0,0.41277147,-1055.2752,1864.387)"
+         id="g3792-4-1-1-7-1">
+        <rect
+           style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65632725;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-0-0-9-5-9"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65632725;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-5-7-6-1-9"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <g
+         style="opacity:0.95999995;stroke-width:1.53030467;stroke-miterlimit:4;stroke-dasharray:none"
+         id="g3792-0-2-0-2-7-0"
+         transform="matrix(0.40271073,0,0,0.40768065,-1173.5759,1856.0714)">
+        <rect
+           style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.70198441;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-4-2-4-5-4-9"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.70198441;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-4-4-8-4-1-3"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <rect
+         y="1297.5363"
+         x="-909.70618"
+         height="28.5"
+         width="103.5"
+         id="rect2994-3-0-0-4-6"
+         style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49999988;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.49999962, 4.49999962;stroke-dashoffset:0" />
+      <g
+         id="g34054-2"
+         transform="translate(261.32072,614.33529)">
+        <g
+           transform="matrix(0.40688131,0,0,0.41130412,-1252.1926,1369.3045)"
+           id="g3792-8-5-02-5-8"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1658.4716"
+             x="211.42857"
+             height="50"
+             width="50"
+             id="rect2992-1-7-9-9-0"
+             style="fill:#f3de4a;fill-opacity:1;stroke:#000000;stroke-width:3.66670442;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1658.4716"
+             x="261.42859"
+             height="50"
+             width="50"
+             id="rect2992-8-0-4-4-7-2"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.66670442;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+        <g
+           transform="matrix(0.40688131,0,0,0.41130412,-1199.1018,1369.3045)"
+           id="g3792-0-2-0-2-3-3-5-0"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none" />
+      </g>
+      <g
+         transform="translate(-1.0606602,2.4748737)"
+         id="g34960">
+        <rect
+           style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49999988;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.49999962, 4.49999962;stroke-dashoffset:0"
+           id="rect2994-3-0-0-6-7"
+           width="103.5"
+           height="28.5"
+           x="-1146.2062"
+           y="1329.1628" />
+        <g
+           transform="translate(24.82072,575.33529)"
+           id="g34070-5">
+          <g
+             style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+             id="g3792-0-1-17-4-6-1"
+             transform="matrix(0.40475762,0,0,0.40915735,-1251.7436,1436.4245)">
+            <rect
+               style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-4-8-4-1-8-1"
+               width="50"
+               height="50"
+               x="211.42857"
+               y="-1658.4716" />
+            <rect
+               style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-8-4-5-3-2-1-5"
+               width="50"
+               height="50"
+               x="261.42859"
+               y="-1658.4716" />
+          </g>
+          <g
+             style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+             id="g3792-0-2-0-2-3-7-5-1"
+             transform="matrix(0.40475762,0,0,0.40915735,-1198.4405,1436.4245)">
+            <rect
+               style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-4-2-4-5-3-9-5-91"
+               width="50"
+               height="50"
+               x="211.42857"
+               y="-1658.4716" />
+            <rect
+               style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-8-4-4-8-4-7-60-6-0"
+               width="50"
+               height="50"
+               x="261.42859"
+               y="-1658.4716" />
+          </g>
+        </g>
+      </g>
+      <g
+         transform="translate(-400.5,92.5)"
+         id="g34945">
+        <rect
+           style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49999976;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.4999996, 4.4999996;stroke-dashoffset:0"
+           id="rect2994-3-0-0-1-4"
+           width="103.5"
+           height="62.617905"
+           x="-628.79382"
+           y="1135.4185" />
+        <g
+           transform="translate(426.32072,486.33529)"
+           id="g34078-4">
+          <g
+             style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+             transform="matrix(0.40795244,0,0,0.41238689,-1138.1239,1338.3958)"
+             id="g3792-1-9-8-4-1">
+            <rect
+               style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-7-3-3-6-10"
+               width="50"
+               height="50"
+               x="211.71547"
+               y="-1660.405" />
+            <rect
+               style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-8-8-6-83-2-5"
+               width="50"
+               height="50"
+               x="261.71548"
+               y="-1660.405" />
+          </g>
+          <g
+             style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+             transform="matrix(0.40795244,0,0,0.41238689,-1082.0231,1337.5985)"
+             id="g3792-4-1-1-4-3-15" />
+        </g>
+        <g
+           transform="translate(426.32073,519.81003)"
+           id="g34078-4-5">
+          <g
+             style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+             transform="matrix(0.40795244,0,0,0.41238689,-1138.1239,1338.3958)"
+             id="g3792-1-9-8-4-1-2">
+            <rect
+               style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-7-3-3-6-10-4"
+               width="50"
+               height="50"
+               x="211.71547"
+               y="-1660.405" />
+            <rect
+               style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-8-8-6-83-2-5-6"
+               width="50"
+               height="50"
+               x="261.71548"
+               y="-1660.405" />
+          </g>
+          <g
+             style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+             transform="matrix(0.40795244,0,0,0.41238689,-1082.0231,1337.5985)"
+             id="g3792-4-1-1-4-3-15-4">
+            <rect
+               style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-0-0-9-1-0-6-1"
+               width="50"
+               height="50"
+               x="211.42857"
+               y="-1658.4716" />
+            <rect
+               style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-8-5-7-6-0-2-3-9"
+               width="50"
+               height="50"
+               x="261.42859"
+               y="-1658.4716" />
+          </g>
+        </g>
+      </g>
+    </g>
+    <path
+       inkscape:connector-curvature="0"
+       sodipodi:nodetypes="ccc"
+       id="path4066-2-0-9-4-1-2"
+       d="m 669.67823,271.03445 22.75265,-0.035 -0.12654,35.13393"
+       style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.5;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+    <text
+       xml:space="preserve"
+       style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+       x="601.92139"
+       y="275.3765"
+       id="text39591"
+       sodipodi:linespacing="125%"><tspan
+         sodipodi:role="line"
+         id="tspan39593"
+         x="601.92139"
+         y="275.3765">Key Field</tspan></text>
+  </g>
+</svg>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/cross.svg
----------------------------------------------------------------------
diff --git a/docs/img/cross.svg b/docs/img/cross.svg
new file mode 100644
index 0000000..5161f39
--- /dev/null
+++ b/docs/img/cross.svg
@@ -0,0 +1,893 @@
+<?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"
+   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+   width="800"
+   height="195"
+   id="svg35289"
+   version="1.1"
+   inkscape:version="0.48.3.1 r9886"
+   sodipodi:docname="match.svg">
+  <defs
+     id="defs35291">
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35250"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35252"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35254"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35256"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35258"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35260"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend-0"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456-7"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35403"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35405"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35407"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35409"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35411"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35413"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend-9"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456-75"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35572"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35574"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35576"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35578"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35580"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35582"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35584"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35586"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35588"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35590"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend-3"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456-77"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35951"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35953"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35955"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35957"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35959"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35961"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35963"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35965"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35967"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35969"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+  </defs>
+  <sodipodi:namedview
+     id="base"
+     pagecolor="#ffffff"
+     bordercolor="#666666"
+     borderopacity="1.0"
+     inkscape:pageopacity="0.0"
+     inkscape:pageshadow="2"
+     inkscape:zoom="1.979899"
+     inkscape:cx="318.74501"
+     inkscape:cy="39.054602"
+     inkscape:document-units="px"
+     inkscape:current-layer="layer1"
+     showgrid="false"
+     fit-margin-top="0"
+     fit-margin-left="0"
+     fit-margin-right="0"
+     fit-margin-bottom="0"
+     inkscape:window-width="1916"
+     inkscape:window-height="570"
+     inkscape:window-x="1400"
+     inkscape:window-y="609"
+     inkscape:window-maximized="0" />
+  <metadata
+     id="metadata35294">
+    <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>
+  <g
+     inkscape:label="Layer 1"
+     inkscape:groupmode="layer"
+     id="layer1"
+     transform="translate(-484.80217,-248.14373)">
+    <g
+       id="g35082"
+       transform="translate(1984.0346,-344.20004)">
+      <rect
+         y="753.82758"
+         x="-1055.1145"
+         height="28.5"
+         width="103.5"
+         id="rect2994-3-0-0-8"
+         style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49999988;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.49999962, 4.49999962;stroke-dashoffset:0" />
+      <rect
+         y="683.20093"
+         x="-1171.0269"
+         height="28.5"
+         width="103.5"
+         id="rect2994-3-0-0-4"
+         style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49999988;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.49999962, 4.49999962;stroke-dashoffset:0" />
+      <rect
+         y="719.13367"
+         x="-1171.0267"
+         height="28.5"
+         width="103.5"
+         id="rect2994-3-0-0-9"
+         style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49999988;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.49999962, 4.49999962;stroke-dashoffset:0" />
+      <rect
+         y="753.82758"
+         x="-1171.0269"
+         height="28.5"
+         width="103.5"
+         id="rect2994-3-0-0-6"
+         style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49999988;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.49999962, 4.49999962;stroke-dashoffset:0" />
+      <rect
+         y="649.7262"
+         x="-1055.1145"
+         height="28.5"
+         width="103.5"
+         id="rect2994-3-0-0-5"
+         style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49999988;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.49999962, 4.49999962;stroke-dashoffset:0" />
+      <rect
+         y="683.20093"
+         x="-1055.1145"
+         height="28.5"
+         width="103.5"
+         id="rect2994-3-0-0-1"
+         style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49999988;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.49999962, 4.49999962;stroke-dashoffset:0" />
+      <rect
+         y="719.13367"
+         x="-1055.1145"
+         height="28.5"
+         width="103.5"
+         id="rect2994-3-0-0-3"
+         style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49999988;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.49999962, 4.49999962;stroke-dashoffset:0" />
+      <rect
+         y="595.75568"
+         x="-1175.5856"
+         height="31.868748"
+         width="227.71063"
+         id="rect2994-5-9-4-9"
+         style="opacity:0.95999995;fill:#f3f3f3;fill-opacity:1;stroke:#000000;stroke-width:1.49999988;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:25" />
+      <rect
+         y="647.87219"
+         x="-1250.59"
+         height="136.05968"
+         width="57.377071"
+         id="rect2994-5-9-7"
+         style="opacity:0.95999995;fill:#f3f3f3;fill-opacity:1;stroke:#000000;stroke-width:1.49999988;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:25" />
+      <g
+         style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+         transform="matrix(0.40795244,0,0,0.41238689,-1328.5519,1337.5985)"
+         id="g3792-4-1-0">
+        <rect
+           style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-0-0-5"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-5-7-13"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <g
+         style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+         id="g3792-8-9-5-2"
+         transform="matrix(0.40688131,0,0,0.41130412,-1328.2718,1370.405)">
+        <rect
+           style="fill:#f3de4a;fill-opacity:1;stroke:#000000;stroke-width:3.66670442;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-1-46-8-99"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.66670442;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-0-9-7-9"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <g
+         style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+         id="g3792-0-2-0-6"
+         transform="matrix(0.40475762,0,0,0.40915735,-1327.7167,1436.4245)">
+        <rect
+           style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-4-2-4-0"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-4-4-8-6"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <g
+         style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+         id="g3792-8-4-7-0-4"
+         transform="matrix(0.40581951,0,0,0.41023078,-1327.9943,1403.4839)">
+        <rect
+           style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.67629814;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-1-4-75-4-3"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.67629814;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-0-7-4-2-05"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <g
+         style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+         transform="matrix(0.40946483,0,0,0.41232107,-1082.4563,1285.2048)"
+         id="g3792-4-1-1-41">
+        <rect
+           style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65060806;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-0-0-9-45"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65060806;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-5-7-6-8"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <g
+         style="opacity:0.95999995;stroke-width:1.53030467;stroke-miterlimit:4;stroke-dasharray:none"
+         id="g3792-0-2-0-2-5"
+         transform="matrix(0.40135661,0,0,0.4041563,-1197.5462,1271.8679)">
+        <rect
+           style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.72435784;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-4-2-4-5-1"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.72435784;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-4-4-8-4-9"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <rect
+         y="649.7262"
+         x="-1171.0269"
+         height="28.5"
+         width="103.5"
+         id="rect2994-3-0-0"
+         style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49999988;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.49999962, 4.49999962;stroke-dashoffset:0" />
+      <g
+         id="g34046">
+        <g
+           id="g3792-1-9-6"
+           transform="matrix(0.40795244,0,0,0.41238689,-1252.5361,1338.3958)"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1660.405"
+             x="211.71547"
+             height="50"
+             width="50"
+             id="rect2992-7-3-7"
+             style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1660.405"
+             x="261.71548"
+             height="50"
+             width="50"
+             id="rect2992-8-8-6-9"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+        <g
+           transform="matrix(0.40795244,0,0,0.41238689,-1199.4354,1337.5985)"
+           id="g3792-0-2-0-2-3-9"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1658.4716"
+             x="211.42857"
+             height="50"
+             width="50"
+             id="rect2992-4-2-4-5-3-61"
+             style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1658.4716"
+             x="261.42859"
+             height="50"
+             width="50"
+             id="rect2992-8-4-4-8-4-7-8"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+      </g>
+      <g
+         id="g34054">
+        <g
+           transform="matrix(0.40688131,0,0,0.41130412,-1252.1926,1369.3045)"
+           id="g3792-8-5-02-5"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1658.4716"
+             x="211.42857"
+             height="50"
+             width="50"
+             id="rect2992-1-7-9-9"
+             style="fill:#f3de4a;fill-opacity:1;stroke:#000000;stroke-width:3.66670442;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1658.4716"
+             x="261.42859"
+             height="50"
+             width="50"
+             id="rect2992-8-0-4-4-7"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.66670442;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+        <g
+           transform="matrix(0.40688131,0,0,0.41130412,-1199.1018,1369.3045)"
+           id="g3792-0-2-0-2-3-3-5"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1658.4716"
+             x="211.42857"
+             height="50"
+             width="50"
+             id="rect2992-4-2-4-5-3-2-9"
+             style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.66670442;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1658.4716"
+             x="261.42859"
+             height="50"
+             width="50"
+             id="rect2992-8-4-4-8-4-7-6-6"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.66670442;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+      </g>
+      <g
+         id="g34062">
+        <g
+           id="g3792-1-9-9-9"
+           transform="matrix(0.40581951,0,0,0.41023078,-1252.0845,1404.2772)"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1660.405"
+             x="211.71547"
+             height="50"
+             width="50"
+             id="rect2992-7-3-2-2"
+             style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.67629814;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1660.405"
+             x="261.71548"
+             height="50"
+             width="50"
+             id="rect2992-8-8-6-8-9"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.67629814;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+        <g
+           transform="matrix(0.40581951,0,0,0.41023078,-1198.7711,1403.4839)"
+           id="g3792-0-2-0-2-3-2-6"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1658.4716"
+             x="211.42857"
+             height="50"
+             width="50"
+             id="rect2992-4-2-4-5-3-6-1"
+             style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.67629814;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1658.4716"
+             x="261.42859"
+             height="50"
+             width="50"
+             id="rect2992-8-4-4-8-4-7-5-7"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.67629814;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+      </g>
+      <g
+         id="g34070">
+        <g
+           transform="matrix(0.40475762,0,0,0.40915735,-1251.7436,1436.4245)"
+           id="g3792-0-1-17-4-6"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1658.4716"
+             x="211.42857"
+             height="50"
+             width="50"
+             id="rect2992-4-8-4-1-8"
+             style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1658.4716"
+             x="261.42859"
+             height="50"
+             width="50"
+             id="rect2992-8-4-5-3-2-1"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+        <g
+           transform="matrix(0.40475762,0,0,0.40915735,-1198.4405,1436.4245)"
+           id="g3792-0-2-0-2-3-7-5"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1658.4716"
+             x="211.42857"
+             height="50"
+             width="50"
+             id="rect2992-4-2-4-5-3-9-5"
+             style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1658.4716"
+             x="261.42859"
+             height="50"
+             width="50"
+             id="rect2992-8-4-4-8-4-7-60-6"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+      </g>
+      <g
+         id="g34078">
+        <g
+           id="g3792-1-9-8-4"
+           transform="matrix(0.40795244,0,0,0.41238689,-1138.1239,1338.3958)"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1660.405"
+             x="211.71547"
+             height="50"
+             width="50"
+             id="rect2992-7-3-3-6"
+             style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1660.405"
+             x="261.71548"
+             height="50"
+             width="50"
+             id="rect2992-8-8-6-83-2"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+        <g
+           id="g3792-4-1-1-4-3"
+           transform="matrix(0.40795244,0,0,0.41238689,-1082.0231,1337.5985)"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1658.4716"
+             x="211.42857"
+             height="50"
+             width="50"
+             id="rect2992-0-0-9-1-0"
+             style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1658.4716"
+             x="261.42859"
+             height="50"
+             width="50"
+             id="rect2992-8-5-7-6-0-2"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+      </g>
+      <g
+         id="g34086">
+        <g
+           transform="matrix(0.40688131,0,0,0.41130412,-1137.7804,1369.3045)"
+           id="g3792-8-5-02-8-6"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1658.4716"
+             x="211.42857"
+             height="50"
+             width="50"
+             id="rect2992-1-7-9-0-7"
+             style="fill:#f3de4a;fill-opacity:1;stroke:#000000;stroke-width:3.66670442;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1658.4716"
+             x="261.42859"
+             height="50"
+             width="50"
+             id="rect2992-8-0-4-4-4-1"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.66670442;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+        <g
+           id="g3792-4-1-1-8-59"
+           transform="matrix(0.40688131,0,0,0.41130412,-1081.6895,1369.3045)"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1658.4716"
+             x="211.42857"
+             height="50"
+             width="50"
+             id="rect2992-0-0-9-7-8"
+             style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.66670442;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1658.4716"
+             x="261.42859"
+             height="50"
+             width="50"
+             id="rect2992-8-5-7-6-08-2"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.66670442;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+      </g>
+      <g
+         id="g34094">
+        <g
+           id="g3792-1-9-9-7-0"
+           transform="matrix(0.40581951,0,0,0.41023078,-1137.6723,1404.2772)"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1660.405"
+             x="211.71547"
+             height="50"
+             width="50"
+             id="rect2992-7-3-2-9-78"
+             style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.67629814;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1660.405"
+             x="261.71548"
+             height="50"
+             width="50"
+             id="rect2992-8-8-6-8-0-6"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.67629814;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+        <g
+           id="g3792-4-1-1-2-38"
+           transform="matrix(0.40581951,0,0,0.41023078,-1081.3588,1403.4839)"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1658.4716"
+             x="211.42857"
+             height="50"
+             width="50"
+             id="rect2992-0-0-9-4-6"
+             style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.67629814;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1658.4716"
+             x="261.42859"
+             height="50"
+             width="50"
+             id="rect2992-8-5-7-6-7-1"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.67629814;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+      </g>
+      <g
+         id="g34102">
+        <g
+           transform="matrix(0.40475762,0,0,0.40915735,-1137.3314,1436.4245)"
+           id="g3792-0-1-17-4-8-1"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1658.4716"
+             x="211.42857"
+             height="50"
+             width="50"
+             id="rect2992-4-8-4-1-9-8"
+             style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1658.4716"
+             x="261.42859"
+             height="50"
+             width="50"
+             id="rect2992-8-4-5-3-2-0-6"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+        <g
+           id="g3792-4-1-1-3-6"
+           transform="matrix(0.40475762,0,0,0.40915735,-1081.0281,1436.4245)"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1658.4716"
+             x="211.42857"
+             height="50"
+             width="50"
+             id="rect2992-0-0-9-9-6"
+             style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1658.4716"
+             x="261.42859"
+             height="50"
+             width="50"
+             id="rect2992-8-5-7-6-2-1"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+      </g>
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cc"
+         id="path4066-2-0-34"
+         d="m -1201.5038,663.97626 25.9157,-0.11419"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.49999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cc"
+         id="path4066-2-0-5-0"
+         d="m -1201.5574,698.55143 25.9693,0"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.49999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cc"
+         id="path4066-2-0-7-9"
+         d="m -1201.6105,733.38363 26.0224,-0.14274"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.49999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cc"
+         id="path4066-2-0-3-6"
+         d="m -1201.6636,768.07765 26.0755,-0.1474"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.49999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cc"
+         id="path4066-2-0-9-8"
+         d="m -1092.6535,621.72237 -0.1005,25.32033"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.49999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cc"
+         id="path4066-2-0-9-5-2"
+         d="m -975.21023,621.80923 0,23.97522"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.49999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+    </g>
+  </g>
+</svg>


[29/53] [abbrv] git commit: [FLINK-962] Initial import of documentation from website into source code (closes #34)

Posted by rm...@apache.org.
[FLINK-962] Initial import of documentation from website into source code (closes #34)


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

Branch: refs/heads/travis_test
Commit: 40b94f73300788e191d32c5918e647bff748e151
Parents: b4b633e
Author: uce <u....@fu-berlin.de>
Authored: Sat Jun 21 15:14:05 2014 +0200
Committer: uce <u....@fu-berlin.de>
Committed: Mon Jun 23 14:51:07 2014 +0200

----------------------------------------------------------------------
 docs/README.md                                  |   60 +
 docs/_config.yml                                |   33 +
 docs/_layouts/docs.html                         |   92 ++
 docs/_plugins/tocify.rb                         |   10 +
 docs/build_docs.sh                              |   58 +
 docs/cli.md                                     |  129 ++
 docs/cluster_execution.md                       |  125 ++
 docs/cluster_setup.md                           |  363 +++++
 docs/config.md                                  |  171 ++
 docs/css/syntax.css                             |   60 +
 docs/faq.md                                     |  285 ++++
 docs/hadoop_compatability.md                    |    5 +
 docs/img/cogroup.svg                            |  856 ++++++++++
 docs/img/cross.svg                              |  893 +++++++++++
 docs/img/dataflow.svg                           |  979 ++++++++++++
 docs/img/datatypes.svg                          |  143 ++
 docs/img/iterations_delta_iterate_operator.png  |  Bin 0 -> 113607 bytes
 ...terations_delta_iterate_operator_example.png |  Bin 0 -> 335057 bytes
 docs/img/iterations_iterate_operator.png        |  Bin 0 -> 63465 bytes
 .../img/iterations_iterate_operator_example.png |  Bin 0 -> 102925 bytes
 docs/img/iterations_supersteps.png              |  Bin 0 -> 54098 bytes
 docs/img/japi_example_overview.png              |  Bin 0 -> 45406 bytes
 docs/img/join.svg                               |  615 ++++++++
 docs/img/map.svg                                |  295 ++++
 docs/img/operator.svg                           |  241 +++
 docs/img/recorddm.svg                           |  263 ++++
 docs/img/reduce.svg                             |  425 +++++
 docs/img/spargel_example.png                    |  Bin 0 -> 199032 bytes
 docs/img/spargel_example_input.png              |  Bin 0 -> 113478 bytes
 docs/index.md                                   |   11 +
 docs/iterations.md                              |  188 +++
 docs/java_api_examples.md                       |  304 ++++
 docs/java_api_guide.md                          | 1476 ++++++++++++++++++
 docs/java_api_quickstart.md                     |  126 ++
 docs/local_execution.md                         |  106 ++
 docs/local_setup.md                             |  108 ++
 docs/quickstart/plotPoints.py                   |   82 +
 docs/run_example_quickstart.md                  |  154 ++
 docs/scala_api_examples.md                      |  195 +++
 docs/scala_api_guide.md                         | 1008 ++++++++++++
 docs/scala_api_quickstart.md                    |   71 +
 docs/setup_quickstart.md                        |  132 ++
 docs/spargel_guide.md                           |  112 ++
 docs/web_client.md                              |   53 +
 docs/yarn_setup.md                              |  188 +++
 pom.xml                                         |    2 +
 46 files changed, 10417 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/README.md
----------------------------------------------------------------------
diff --git a/docs/README.md b/docs/README.md
new file mode 100644
index 0000000..4ecb30e
--- /dev/null
+++ b/docs/README.md
@@ -0,0 +1,60 @@
+This README gives an overview of how to build and contribute to the
+documentation of Apache Flink.
+
+The documentation is included with the source of Apache Flink in order to ensure
+that you always have docs corresponding to your checked out version. The online
+documentation at http://flink.incubator.apache.org/ is also generated from the
+files found here.
+
+# Requirements
+
+We use Markdown to write and Jekyll to translate the documentation to static
+HTML. You can install all needed software via:
+
+    gem install jekyll
+    gem install redcarpet
+    sudo easy_install Pygments
+
+Redcarpet is needed for Markdown processing and the Python based Pygments is
+used for syntax highlighting.
+
+# Build
+
+The `docs/build_docs.sh` script calls Jekyll and generates the documentation to
+`docs/target`. You can then point your browser to `docs/target/index.html` and
+start reading.
+
+If you call the script with the preview flag `build_docs.sh -p`, Jekyll will
+start a web server at `localhost:4000` and continiously generate the docs.
+This is useful to preview changes locally.
+
+# Contribute
+
+The documentation pages are written in
+[Markdown](http://daringfireball.net/projects/markdown/syntax). It is possible
+to use the [GitHub flavored syntax](http://github.github.com/github-flavored-markdown)
+and intermix plain html.
+
+In addition to Markdown, every page contains a front matter, which specifies the
+title of the page. This title is used as the top-level heading for the page.
+
+    ---
+    title: "Title of the Page"
+    ---
+
+Furthermore, you can access variables found in `docs/_config.yml` as follows:
+
+    {{ site.FLINK_VERSION }}
+
+This will be replaced with the value of the variable when generating the docs.
+
+All documents are structed with headings. From these heading, an page outline is
+automatically generated for each page.
+
+```
+# Level-1 Heading
+## Level-2 Heading
+### Level-3 heading
+#### Level-4 heading
+##### Level-5 heading
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/_config.yml
----------------------------------------------------------------------
diff --git a/docs/_config.yml b/docs/_config.yml
new file mode 100644
index 0000000..2d7c1ce
--- /dev/null
+++ b/docs/_config.yml
@@ -0,0 +1,33 @@
+#------------------------------------------------------------------------------
+# VARIABLES
+#------------------------------------------------------------------------------
+# Variables specified in this file can be used in the documentation via:
+#     {{ site.CONFIG_KEY }}
+#------------------------------------------------------------------------------
+
+FLINK_VERSION: 0.6-SNAPSHOT
+FLINK_VERSION_SHORT: 0.6
+FLINK_ISSUES_URL: https://issues.apache.org/jira/browse/FLINK
+FLINK_GITHUB_URL:  https://github.com/apache/incubator-flink
+
+#------------------------------------------------------------------------------
+# BUILD CONFIG
+#------------------------------------------------------------------------------
+# These variables configure the jekyll build (./build_docs.sh). You don't need
+# to change anything here.
+#------------------------------------------------------------------------------
+
+defaults:
+  -
+    scope:
+      path: ""
+    values:
+      layout: docs
+
+highlighter: pygments
+markdown: redcarpet
+redcarpet:
+  # https://help.github.com/articles/github-flavored-markdown
+  extensions: ["no_intra_emphasis", "fenced_code_blocks", "autolink",
+               "tables", "with_toc_data", "strikethrough", "superscript",
+               "lax_spacing"]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/_layouts/docs.html
----------------------------------------------------------------------
diff --git a/docs/_layouts/docs.html b/docs/_layouts/docs.html
new file mode 100644
index 0000000..4b99d4a
--- /dev/null
+++ b/docs/_layouts/docs.html
@@ -0,0 +1,92 @@
+<!DOCTYPE html>
+<html lang="en">
+  <head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <title>Apache Flink {{ site.FLINK_VERSION }} Documentation: {{ page.title }}</title>
+
+    <link rel="stylesheet" href="//netdna.bootstrapcdn.com/bootstrap/3.1.1/css/bootstrap.min.css">
+    <link rel="stylesheet" href="css/syntax.css">
+
+    <!-- HTML5 Shim and Respond.js IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+      <script src="https://oss.maxcdn.com/libs/html5shiv/3.7.0/html5shiv.js"></script>
+      <script src="https://oss.maxcdn.com/libs/respond.js/1.4.2/respond.min.js"></script>
+    <![endif]-->
+  </head>
+  <body>
+    <div class="container">
+        <div class="row">
+            <h1>Apache Flink {{ site.FLINK_VERSION }} Documentation</h1>
+        </div>
+        <div class="row">
+            <div class="col-md-3">
+                <ul>
+                    <li>Quickstart
+                        <ul>
+                            <li><a href="setup_quickstart.html">Install</a></li>
+                            <li><a href="run_example_quickstart.html">Run Example</a></li>
+                            <li><a href="java_api_quickstart.html">Java API</a></li>
+                            <li><a href="scala_api_quickstart.html">Scala API</a></li>
+                            <li><a href="faq.html">FAQ</a></li>
+                        </ul>
+                    </li>
+
+                    <li>Setup &amp; Configuration
+                        <ul>
+                            <li><a href="local_setup.html">Local Setup</a></li>
+                            <li><a href="cluster_setup.html">Cluster Setup</a></li>
+                            <li><a href="yarn_setup.html">YARN Setup</a></li>
+                            <li><a href="config.html">Configuration</a></li>
+                        </ul>
+                    </li>
+
+                    <li>Programming Guides
+                        <ul>
+                            <li><a href="java_api_guide.html">Java API</a></li>
+                            <li><a href="scala_api_guide.html">Scala API</a></li>
+                            <li><a href="hadoop_compatability.html">Hadoop Compatability</a></li>
+                            <li><a href="iterations.html">Iterations</a></li>
+                            <li><a href="spargel_guide.html">Spargel Graph API</a></li>
+                        </ul>
+                    </li>
+
+                    <li>Examples
+                        <ul>
+                            <li><a href="java_api_examples.html">Java API</a></li>
+                            <li><a href="scala_api_examples.html">Scala API</a></li>
+                        </ul>
+                    </li>
+
+                    <li>Execution
+                        <ul>
+                            <li><a href="local_execution.html">Local/Debugging</a></li>
+                            <li><a href="cluster_execution.html">Cluster</a></li>
+                            <li><a href="cli.html">Command-Line Interface</a></li>
+                            <li><a href="web_client.html">Web Interface</a></li>
+                        </ul>
+                    </li>
+
+                    <li>Internals
+                        <ul>
+                            <li>To be written</li>
+                        </ul>
+                    </li>
+                </ul>
+            </div>
+            <div class="col-md-9">
+                <h1>{{ page.title }}</h1>
+
+                {{ page.content | tocify }}
+
+                {{ content }}
+            </div>
+        </div>
+    </div>
+
+    <script src="https://ajax.googleapis.com/ajax/libs/jquery/1.11.0/jquery.min.js"></script>
+    <script src="//netdna.bootstrapcdn.com/bootstrap/3.1.1/js/bootstrap.min.js"></script>
+  </body>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/_plugins/tocify.rb
----------------------------------------------------------------------
diff --git a/docs/_plugins/tocify.rb b/docs/_plugins/tocify.rb
new file mode 100644
index 0000000..7df0c3d
--- /dev/null
+++ b/docs/_plugins/tocify.rb
@@ -0,0 +1,10 @@
+module Jekyll
+  module Tocify
+    def tocify(input)
+      converter = Redcarpet::Markdown.new(Redcarpet::Render::HTML_TOC)
+      converter.render(input)
+    end
+  end
+end
+
+Liquid::Template.register_filter(Jekyll::Tocify)

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/build_docs.sh
----------------------------------------------------------------------
diff --git a/docs/build_docs.sh b/docs/build_docs.sh
new file mode 100755
index 0000000..7ae3343
--- /dev/null
+++ b/docs/build_docs.sh
@@ -0,0 +1,58 @@
+#!/bin/bash
+########################################################################################################################
+# Copyright (C) 2010-2014 by the Stratos	phere 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.
+########################################################################################################################
+
+HAS_JEKYLL=true
+
+command -v jekyll > /dev/null
+if [ $? -ne 0 ]; then
+	echo -n "ERROR: Could not find jekyll. "
+	echo "Please install with 'gem install jekyll' (see http://jekyllrb.com)."
+
+	HAS_JEKYLL=false
+fi
+
+command -v redcarpet > /dev/null
+if [ $? -ne 0 ]; then
+	echo -n "WARN: Could not find redcarpet. "
+	echo -n "Please install with 'sudo gem install redcarpet' (see https://github.com/vmg/redcarpet). "
+	echo "Redcarpet is needed for Markdown parsing and table of contents generation."
+fi
+
+command -v pygmentize > /dev/null
+if [ $? -ne 0 ]; then
+	echo -n "WARN: Could not find pygments. "
+	echo -n "Please install with 'sudo easy_install Pygments' (requires Python; see http://pygments.org). "
+	echo "Pygments is needed for syntax highlighting of the code examples."
+fi
+
+DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
+
+DOCS_SRC=${DIR}
+DOCS_DST=${DOCS_SRC}/target
+
+# default jekyll command is to just build site
+JEKYLL_CMD="build"
+
+# if -p flag is provided, serve site on localhost
+while getopts ":p" opt; do
+	case $opt in
+		p)
+		JEKYLL_CMD="serve --watch"
+		;;
+	esac
+done
+
+if $HAS_JEKYLL; then
+	jekyll ${JEKYLL_CMD} --source ${DOCS_SRC} --destination ${DOCS_DST}
+fi
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/cli.md
----------------------------------------------------------------------
diff --git a/docs/cli.md b/docs/cli.md
new file mode 100644
index 0000000..0e778da
--- /dev/null
+++ b/docs/cli.md
@@ -0,0 +1,129 @@
+---
+title:  "Command-Line Interface"
+---
+
+Stratosphere provides a command-line interface to run programs that are packaged
+as JAR files, and control their execution.  The command line interface is part
+of any Stratosphere setup, available in local single node setups and in
+distributed setups. It is located under `<stratosphere-home>/bin/stratosphere`
+and connects by default to the running Stratosphere master (JobManager) that was
+started from the same installation directory.
+
+A prerequisite to using the command line interface is that the Stratosphere
+master (JobManager) has been started (via `<stratosphere-home>/bin/start-
+local.sh` or `<stratosphere-home>/bin/start-cluster.sh`).
+
+The command line can be used to
+
+- submit jobs for execution,
+- cancel a running job,
+- provide information about a job, and
+- list running and waiting jobs.
+
+# Examples
+
+-   Run example program with no arguments.
+
+        ./bin/stratosphere run ./examples/stratosphere-java-examples-{{ site.FLINK_VERSION }}-WordCount.jar
+
+-   Run example program with arguments for input and result files
+
+        ./bin/stratosphere run ./examples/stratosphere-java-examples-{{ site.FLINK_VERSION }}-WordCount.jar \
+                               file:///home/user/hamlet.txt file:///home/user/wordcount_out
+
+-   Run example program with parallelism 16 and arguments for input and result files
+
+        ./bin/stratosphere run -p 16 ./examples/stratosphere-java-examples-{{ site.FLINK_VERSION }}-WordCount.jar \
+                                file:///home/user/hamlet.txt file:///home/user/wordcount_out
+
+-   Run example program on a specific JobManager:
+
+        ./bin/stratosphere run -m myJMHost:6123 \
+                               ./examples/stratosphere-java-examples-{{ site.FLINK_VERSION }}-WordCount.jar \
+                               -file:///home/user/hamlet.txt file:///home/user/wordcount_out
+
+
+-   Display the expected arguments for the WordCount example program:
+
+        ./bin/stratosphere info -d ./examples/stratosphere-java-examples-{{ site.FLINK_VERSION }}-WordCount.jar
+
+-   Display the optimized execution plan for the WordCount example program as JSON:
+
+        ./bin/stratosphere info -e 
+                                ./examples/stratosphere-java-examples-{{ site.FLINK_VERSION }}-WordCount.jar \
+                                file:///home/user/hamlet.txt file:///home/user/wordcount_out
+
+-   List scheduled and running jobs (including their JobIDs):
+
+        ./bin/stratosphere list -s -r
+
+-   Cancel a job:
+
+        ./bin/stratosphere cancel -i <jobID>
+
+# Usage
+
+The command line syntax is as follows:
+
+```
+./stratosphere <ACTION> [OPTIONS] [ARGUMENTS]
+
+General options:
+     -h,--help      Show the help for the CLI Frontend, or a specific action.
+     -v,--verbose   Print more detailed error messages.
+
+
+Action "run" - compiles and submits a Stratosphere program that is given in the form of a JAR file.
+
+  "run" options:
+
+     -p,--parallelism <parallelism> The degree of parallelism for the execution. This value is used unless the program overrides the degree of parallelism on the execution environment or program plan. If this option is not set, then the execution will use the default parallelism specified in the stratosphere-conf.yaml file.
+
+     -c,--class <classname>         The class with the entry point (main method, or getPlan() method). Needs only be specified if the JAR file has no manifest pointing to that class. See program packaging instructions for details.
+
+     -m,--jobmanager <host:port>    Option to submit the program to a different Stratosphere master (JobManager).
+
+  "run" arguments:
+
+     - The first argument is the path to the JAR file of the program.
+     - All successive arguments are passed to the program's main method (or getPlan() method).
+
+
+Action "info" - displays information about a Stratosphere program.
+
+  "info" action arguments:
+     -d,--description               Show description of the program, if the main class implements the 'ProgramDescription' interface.
+
+     -e,--executionplan             Show the execution data flow plan of the program, in JSON representation.
+
+     -p,--parallelism <parallelism> The degree of parallelism for the execution, see above. The parallelism is relevant for the execution plan. The option is only evaluated if used together with the -e option.
+
+     -c,--class <classname>         The class with the entry point (main method, or getPlan() method). Needs only be specified if the JAR file has no manifest pointing to that class. See program packaging instructions for details.
+
+     -m,--jobmanager <host:port>    Option to connect to a different Stratosphere master (JobManager). Connecting to a master is relevant to compile the execution plan. The option is only evaluated if used together with the -e option.
+
+  "info" arguments:
+
+     - The first argument is the path to the JAR file of the program.
+     - All successive arguments are passed to the program's main method (or getPlan() method).
+
+
+Action "list" lists submitted Stratosphere programs.
+
+  "list" action arguments:
+
+     -r,--running                   Show running programs and their JobIDs
+
+     -s,--scheduled                 Show scheduled programs and their JobIDs
+
+     -m,--jobmanager <host:port>    Option to connect to a different Stratosphere master (JobManager).
+
+
+Action "cancel" cancels a submitted Stratosphere program.
+
+  "cancel" action arguments:
+
+     -i,--jobid <jobID>             JobID of program to cancel
+     
+     -m,--jobmanager <host:port>    Option to connect to a different Stratosphere master (JobManager).
+```

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/cluster_execution.md
----------------------------------------------------------------------
diff --git a/docs/cluster_execution.md b/docs/cluster_execution.md
new file mode 100644
index 0000000..a41bc0f
--- /dev/null
+++ b/docs/cluster_execution.md
@@ -0,0 +1,125 @@
+---
+title:  "Cluster Execution"
+---
+
+Stratosphere programs can run distributed on clusters of many machines. There
+are two ways to send a program to a cluster for execution:
+
+# Command Line Interface
+
+The command line interface lets you submit packaged programs (JARs) to a cluster
+(or single machine setup).
+
+Please refer to the [Command Line Interface](cli.html) documentation for
+details.
+
+# Remote Environment
+
+The remote environment lets you execute Stratosphere Java programs on a cluster
+directly. The remote environment points to the cluster on which you want to
+execute the program.
+
+## Maven Dependency
+
+If you are developing your program as a Maven project, you have to add the
+`stratosphere-clients` module using this dependency:
+
+```xml
+<dependency>
+  <groupId>eu.stratosphere</groupId>
+  <artifactId>stratosphere-clients</artifactId>
+  <version>{{ site.FLINK_VERSION }}</version>
+</dependency>
+```
+
+## Example
+
+The following illustrates the use of the `RemoteEnvironment`:
+
+```java
+public static void main(String[] args) throws Exception {
+    ExecutionEnvironment env = ExecutionEnvironment
+        .createRemoteEnvironment("strato-master", "7661", "/home/user/udfs.jar");
+
+    DataSet<String> data = env.readTextFile("hdfs://path/to/file");
+
+    data
+        .filter(new FilterFunction<String>() {
+            public boolean filter(String value) {
+                return value.startsWith("http://");
+            }
+        })
+        .writeAsText("hdfs://path/to/result");
+
+    env.execute();
+}
+```
+
+Note that the program contains custom UDFs and hence requires a JAR file with
+the classes of the code attached. The constructor of the remote environment
+takes the path(s) to the JAR file(s).
+
+# Remote Executor
+
+Similar to the RemoteEnvironment, the RemoteExecutor lets you execute
+Stratosphere programs on a cluster directly. The remote executor accepts a
+*Plan* object, which describes the program as a single executable unit.
+
+## Maven Dependency
+
+If you are developing your program in a Maven project, you have to add the
+`stratosphere-clients` module using this dependency:
+
+```xml
+<dependency>
+  <groupId>eu.stratosphere</groupId>
+  <artifactId>stratosphere-clients</artifactId>
+  <version>{{ site.FLINK_VERSION }}</version>
+</dependency>
+```
+
+## Example
+
+The following illustrates the use of the `RemoteExecutor` with the Scala API:
+
+```scala
+def main(args: Array[String]) {
+    val input = TextFile("hdfs://path/to/file")
+
+    val words = input flatMap { _.toLowerCase().split("""\W+""") filter { _ != "" } }
+    val counts = words groupBy { x => x } count()
+
+    val output = counts.write(wordsOutput, CsvOutputFormat())
+  
+    val plan = new ScalaPlan(Seq(output), "Word Count")
+    val executor = new RemoteExecutor("strato-master", 7881, "/path/to/jarfile.jar")
+    executor.executePlan(p);
+}
+```
+
+The following illustrates the use of the `RemoteExecutor` with the Java API (as
+an alternative to the RemoteEnvironment):
+
+```java
+public static void main(String[] args) throws Exception {
+    ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+    DataSet<String> data = env.readTextFile("hdfs://path/to/file");
+
+    data
+        .filter(new FilterFunction<String>() {
+            public boolean filter(String value) {
+                return value.startsWith("http://");
+            }
+        })
+        .writeAsText("hdfs://path/to/result");
+
+    Plan p = env.createProgramPlan();
+    RemoteExecutor e = new RemoteExecutor("strato-master", 7881, "/path/to/jarfile.jar");
+    e.executePlan(p);
+}
+```
+
+Note that the program contains custom UDFs and hence requires a JAR file with
+the classes of the code attached. The constructor of the remote executor takes
+the path(s) to the JAR file(s).

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/cluster_setup.md
----------------------------------------------------------------------
diff --git a/docs/cluster_setup.md b/docs/cluster_setup.md
new file mode 100644
index 0000000..3692882
--- /dev/null
+++ b/docs/cluster_setup.md
@@ -0,0 +1,363 @@
+---
+title:  "Cluster Setup"
+---
+
+This documentation is intended to provide instructions on how to run
+Stratosphere in a fully distributed fashion on a static (but possibly
+heterogeneous) cluster.
+
+This involves two steps. First, installing and configuring Stratosphere and
+second installing and configuring the [Hadoop Distributed
+Filesystem](http://hadoop.apache.org/) (HDFS).
+
+# Preparing the Cluster
+
+## Software Requirements
+
+Stratosphere runs on all *UNIX-like environments*, e.g. **Linux**, **Mac OS X**,
+and **Cygwin** (for Windows) and expects the cluster to consist of **one master
+node** and **one or more worker nodes**. Before you start to setup the system,
+make sure you have the following software installed **on each node**:
+
+- **Java 1.6.x** or higher,
+- **ssh** (sshd must be running to use the Stratosphere scripts that manage
+  remote components)
+
+If your cluster does not fulfill these software requirements you will need to
+install/upgrade it.
+
+For example, on Ubuntu Linux, type in the following commands to install Java and
+ssh:
+
+```
+sudo apt-get install ssh 
+sudo apt-get install openjdk-7-jre
+```
+
+You can check the correct installation of Java by issuing the following command:
+
+```
+java -version
+```
+
+The command should output something comparable to the following on every node of
+your cluster (depending on your Java version, there may be small differences):
+
+```
+java version "1.6.0_22"
+Java(TM) SE Runtime Environment (build 1.6.0_22-b04)
+Java HotSpot(TM) 64-Bit Server VM (build 17.1-b03, mixed mode)
+```
+
+To make sure the ssh daemon is running properly, you can use the command
+
+```
+ps aux | grep sshd
+```
+
+Something comparable to the following line should appear in the output
+of the command on every host of your cluster:
+
+```
+root       894  0.0  0.0  49260   320 ?        Ss   Jan09   0:13 /usr/sbin/sshd
+```
+
+## Configuring Remote Access with ssh
+
+In order to start/stop the remote processes, the master node requires access via
+ssh to the worker nodes. It is most convenient to use ssh's public key
+authentication for this. To setup public key authentication, log on to the
+master as the user who will later execute all the Stratosphere components. **The
+same user (i.e. a user with the same user name) must also exist on all worker
+nodes**. For the remainder of this instruction we will refer to this user as
+*stratosphere*. Using the super user *root* is highly discouraged for security
+reasons.
+
+Once you logged in to the master node as the desired user, you must generate a
+new public/private key pair. The following command will create a new
+public/private key pair into the *.ssh* directory inside the home directory of
+the user *stratosphere*. See the ssh-keygen man page for more details. Note that
+the private key is not protected by a passphrase.
+
+```
+ssh-keygen -b 2048 -P '' -f ~/.ssh/id_rsa
+```
+
+Next, copy/append the content of the file *.ssh/id_rsa.pub* to your
+authorized_keys file. The content of the authorized_keys file defines which
+public keys are considered trustworthy during the public key authentication
+process. On most systems the appropriate command is
+
+```
+cat .ssh/id_rsa.pub >> .ssh/authorized_keys
+```
+
+On some Linux systems, the authorized keys file may also be expected by the ssh
+daemon under *.ssh/authorized_keys2*. In either case, you should make sure the
+file only contains those public keys which you consider trustworthy for each
+node of cluster.
+
+Finally, the authorized keys file must be copied to every worker node of your
+cluster. You can do this by repeatedly typing in
+
+```
+scp .ssh/authorized_keys <worker>:~/.ssh/
+```
+
+and replacing *\<worker\>* with the host name of the respective worker node.
+After having finished the copy process, you should be able to log on to each
+worker node from your master node via ssh without a password.
+
+## Setting JAVA_HOME on each Node
+
+Stratosphere requires the `JAVA_HOME` environment variable to be set on the
+master and all worker nodes and point to the directory of your Java
+installation.
+
+You can set this variable in `conf/stratosphere-conf.yaml` via the
+`env.java.home` key.
+
+Alternatively, add the following line to your shell profile. If you use the
+*bash* shell (probably the most common shell), the shell profile is located in
+*\~/.bashrc*:
+
+```
+export JAVA_HOME=/path/to/java_home/
+```
+
+If your ssh daemon supports user environments, you can also add `JAVA_HOME` to
+*.\~/.ssh/environment*. As super user *root* you can enable ssh user
+environments with the following commands:
+
+```
+echo "PermitUserEnvironment yes" >> /etc/ssh/sshd_config
+/etc/init.d/ssh restart
+```
+
+# Hadoop Distributed Filesystem (HDFS) Setup
+
+The Stratosphere system currently uses the Hadoop Distributed Filesystem (HDFS)
+to read and write data in a distributed fashion.
+
+Make sure to have a running HDFS installation. The following instructions are
+just a general overview of some required settings. Please consult one of the
+many installation guides available online for more detailed instructions.
+
+**Note that the following instructions are based on Hadoop 1.2 and might differ
+**for Hadoop 2.
+
+## Downloading, Installing, and Configuring HDFS
+
+Similar to the Stratosphere system HDFS runs in a distributed fashion. HDFS
+consists of a **NameNode** which manages the distributed file system's meta
+data. The actual data is stored by one or more **DataNodes**. For the remainder
+of this instruction we assume the HDFS's NameNode component runs on the master
+node while all the worker nodes run an HDFS DataNode.
+
+To start, log on to your master node and download Hadoop (which includes  HDFS)
+from the Apache [Hadoop Releases](http://hadoop.apache.org/releases.html) page.
+
+Next, extract the Hadoop archive.
+
+After having extracted the Hadoop archive, change into the Hadoop directory and
+edit the Hadoop environment configuration file:
+
+```
+cd hadoop-*
+vi conf/hadoop-env.sh
+```
+
+Uncomment and modify the following line in the file according to the path of
+your Java installation.
+
+``` export JAVA_HOME=/path/to/java_home/ ```
+
+Save the changes and open the HDFS configuration file *conf/hdfs-site.xml*. HDFS
+offers multiple configuration parameters which affect the behavior of the
+distributed file system in various ways. The following excerpt shows a minimal
+configuration which is required to make HDFS work. More information on how to
+configure HDFS can be found in the [HDFS User
+Guide](http://hadoop.apache.org/docs/r1.2.1/hdfs_user_guide.html) guide.
+
+```xml
+<configuration>
+  <property>
+    <name>fs.default.name</name>
+    <value>hdfs://MASTER:50040/</value>
+  </property>
+  <property>
+    <name>dfs.data.dir</name>
+    <value>DATAPATH</value>
+  </property>
+</configuration>
+```
+
+Replace *MASTER* with the IP/host name of your master node which runs the
+*NameNode*. *DATAPATH* must be replaced with path to the directory in which the
+actual HDFS data shall be stored on each worker node. Make sure that the
+*stratosphere* user has sufficient permissions to read and write in that
+directory.
+
+After having saved the HDFS configuration file, open the file *conf/slaves* and
+enter the IP/host name of those worker nodes which shall act as *DataNode*s.
+Each entry must be separated by a line break.
+
+```
+<worker 1>
+<worker 2>
+.
+.
+.
+<worker n>
+```
+
+Initialize the HDFS by typing in the following command. Note that the
+command will **delete all data** which has been previously stored in the
+HDFS. However, since we have just installed a fresh HDFS, it should be
+safe to answer the confirmation with *yes*.
+
+```
+bin/hadoop namenode -format
+```
+
+Finally, we need to make sure that the Hadoop directory is available to
+all worker nodes which are intended to act as DataNodes and that all nodes
+**find the directory under the same path**. We recommend to use a shared network
+directory (e.g. an NFS share) for that. Alternatively, one can copy the
+directory to all nodes (with the disadvantage that all configuration and
+code updates need to be synced to all nodes).
+
+## Starting HDFS
+
+To start the HDFS log on to the master and type in the following
+commands
+
+```
+cd hadoop-*
+binn/start-dfs.sh
+```
+
+If your HDFS setup is correct, you should be able to open the HDFS
+status website at *http://MASTER:50070*. In a matter of a seconds,
+all DataNodes should appear as live nodes. For troubleshooting we would
+like to point you to the [Hadoop Quick
+Start](http://wiki.apache.org/hadoop/QuickStart)
+guide.
+
+# Stratosphere Setup
+
+Go to the [downloads page]({{site.baseurl}}/downloads/) and get the ready to run
+package. Make sure to pick the Stratosphere package **matching your Hadoop
+version**.
+
+After downloading the latest release, copy the archive to your master node and
+extract it:
+
+```
+tar xzf stratosphere-*.tgz
+cd stratosphere-*
+```
+
+## Configuring the Cluster
+
+After having extracted the system files, you need to configure Stratosphere for
+the cluster by editing *conf/stratosphere-conf.yaml*.
+
+Set the `jobmanager.rpc.address` key to point to your master node. Furthermode
+define the maximum amount of main memory the JVM is allowed to allocate on each
+node by setting the `jobmanager.heap.mb` and `taskmanager.heap.mb` keys.
+
+The value is given in MB. If some worker nodes have more main memory which you
+want to allocate to the Stratosphere system you can overwrite the default value
+by setting an environment variable `STRATOSPHERE_TM_HEAP` on the respective
+node.
+
+Finally you must provide a list of all nodes in your cluster which shall be used
+as worker nodes. Therefore, similar to the HDFS configuration, edit the file
+*conf/slaves* and enter the IP/host name of each worker node. Each worker node
+will later run a TaskManager.
+
+Each entry must be separated by a new line, as in the following example:
+
+```
+192.168.0.100
+192.168.0.101
+.
+.
+.
+192.168.0.150
+```
+
+The Stratosphere directory must be available on every worker under the same
+path. Similarly as for HDFS, you can use a shared NSF directory, or copy the
+entire Stratosphere directory to every worker node.
+
+## Configuring the Network Buffers
+
+Network buffers are a critical resource for the communication layers. They are
+used to buffer records before transmission over a network, and to buffer
+incoming data before dissecting it into records and handing them to the
+application. A sufficient number of network buffers are critical to achieve a
+good throughput.
+
+In general, configure the task manager to have so many buffers that each logical
+network connection on you expect to be open at the same time has a dedicated
+buffer. A logical network connection exists for each point-to-point exchange of
+data over the network, which typically happens at repartitioning- or
+broadcasting steps. In those, each parallel task inside the TaskManager has to
+be able to talk to all other parallel tasks. Hence, the required number of
+buffers on a task manager is *total-degree-of-parallelism* (number of targets)
+\* *intra-node-parallelism* (number of sources in one task manager) \* *n*.
+Here, *n* is a constant that defines how many repartitioning-/broadcasting steps
+you expect to be active at the same time.
+
+Since the *intra-node-parallelism* is typically the number of cores, and more
+than 4 repartitioning or broadcasting channels are rarely active in parallel, it
+frequently boils down to *\#cores\^2\^* \* *\#machines* \* 4. To support for
+example a cluster of 20 8-core machines, you should use roughly 5000 network
+buffers for optimal throughput.
+
+Each network buffer is by default 64 KiBytes large. In the above example, the
+system would allocate roughly 300 MiBytes for network buffers.
+
+The number and size of network buffers can be configured with the following
+parameters:
+
+- `taskmanager.network.numberOfBuffers`, and
+- `taskmanager.network.bufferSizeInBytes`.
+
+## Configuring Temporary I/O Directories
+
+Although Stratosphere aims to process as much data in main memory as possible,
+it is not uncommon that  more data needs to be processed than memory is
+available. Stratosphere's runtime is designed to  write temporary data to disk
+to handle these situations.
+
+The `taskmanager.tmp.dirs` parameter specifies a list of directories into which
+Stratosphere writes temporary files. The paths of the directories need to be
+separated by ':' (colon character).  Stratosphere will concurrently write (or
+read) one temporary file to (from) each configured directory.  This way,
+temporary I/O can be evenly distributed over multiple independent I/O devices
+such as hard disks to improve performance.  To leverage fast I/O devices (e.g.,
+SSD, RAID, NAS), it is possible to specify a directory multiple times.
+
+If the `taskmanager.tmp.dirs` parameter is not explicitly specified,
+Stratosphere writes temporary data to the temporary  directory of the operating
+system, such as */tmp* in Linux systems.
+
+Please see the [configuration page](config.html) for details and additional
+configuration options.
+
+## Starting Stratosphere
+
+The following script starts a JobManager on the local node and connects via
+SSH to all worker nodes listed in the *slaves* file to start the
+TaskManager on each node. Now your Stratosphere system is up and
+running. The JobManager running on the local node will now accept jobs
+at the configured RPC port.
+
+Assuming that you are on the master node and inside the Stratosphere directory:
+
+```
+bin/start-cluster.sh
+```

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/config.md
----------------------------------------------------------------------
diff --git a/docs/config.md b/docs/config.md
new file mode 100644
index 0000000..c11cc18
--- /dev/null
+++ b/docs/config.md
@@ -0,0 +1,171 @@
+---
+title:  "Configuration"
+---
+
+# Overview
+
+This page provides an overview of possible settings for Stratosphere. All
+configuration is done in `conf/stratosphere-conf.yaml`, which is expected to be
+a flat collection of [YAML key value pairs](http://www.yaml.org/spec/1.2/spec.html)
+with format `key: value`.
+
+The system and run scripts parse the config at startup and override the
+respective default values with the given values for every that has been set.
+This page contains a reference for all configuration keys used in the system.
+
+# Common Options
+
+- `env.java.home`: The path to the Java installation to use (DEFAULT: system's
+default Java installation).
+- `jobmanager.rpc.address`: The IP address of the JobManager (DEFAULT:
+localhost).
+- `jobmanager.rpc.port`: The port number of the JobManager (DEFAULT: 6123).
+- `jobmanager.heap.mb`: JVM heap size (in megabytes) for the JobManager
+(DEFAULT: 256).
+- `taskmanager.heap.mb`: JVM heap size (in megabytes) for the TaskManager. In
+contrast to Hadoop, Stratosphere runs operators and functions inside the
+TaskManager (including sorting/hashing/caching), so this value should be as
+large as possible (DEFAULT: 512).
+- `taskmanager.tmp.dirs`: The directory for temporary files, or a list of
+directories separated by the systems directory delimiter (for example ':'
+(colon) on Linux/Unix). If multiple directories are specified then the temporary
+files will be distributed across the directories in a round robin fashion. The
+I/O manager component will spawn one reading and one writing thread per
+directory. A directory may be listed multiple times to have the I/O manager use
+multiple threads for it (for example if it is physically stored on a very fast
+disc or RAID) (DEFAULT: The system's tmp dir).
+- `parallelization.degree.default`: The default degree of parallelism to use for
+programs that have no degree of parallelism specified. A value of -1 indicates
+no limit, in which the degree of parallelism is set to the number of available
+instances at the time of compilation (DEFAULT: -1).
+- `parallelization.intra-node.default`: The number of parallel instances of an
+operation that are assigned to each TaskManager. A value of -1 indicates no
+limit (DEFAULT: -1).
+- `taskmanager.network.numberOfBuffers`: The number of buffers available to the
+network stack. This number determines how many streaming data exchange channels
+a TaskManager can have at the same time and how well buffered the channels are.
+If a job is rejected or you get a warning that the system has not enough buffers
+available, increase this value (DEFAULT: 2048).
+- `taskmanager.memory.size`: The amount of memory (in megabytes) that the task
+manager reserves for sorting, hash tables, and caching of intermediate results.
+If unspecified (-1), the memory manager will take a fixed ratio of the heap
+memory available to the JVM after the allocation of the network buffers (0.8)
+(DEFAULT: -1).
+- `jobmanager.profiling.enable`: Flag to enable job manager's profiling
+component. This collects network/cpu utilization statistics, which are displayed
+as charts in the SWT visualization GUI (DEFAULT: false).
+
+# HDFS
+
+These parameters configure the default HDFS used by Stratosphere. If you don't
+specify a HDFS configuration, you will have to specify the full path to your
+HDFS files like `hdfs://address:port/path/to/files` and filed with be written
+with default HDFS parameters (block size, replication factor).
+
+- `fs.hdfs.hadoopconf`: The absolute path to the Hadoop configuration directory.
+The system will look for the "core-site.xml" and "hdfs-site.xml" files in that
+directory (DEFAULT: null).
+- `fs.hdfs.hdfsdefault`: The absolute path of Hadoop's own configuration file
+"hdfs-default.xml" (DEFAULT: null).
+- `fs.hdfs.hdfssite`: The absolute path of Hadoop's own configuration file
+"hdfs-site.xml" (DEFAULT: null).
+
+# JobManager &amp; TaskManager
+
+The following parameters configure Stratosphere's JobManager, TaskManager, and
+runtime channel management.
+
+- `jobmanager.rpc.address`: The hostname or IP address of the JobManager
+(DEFAULT: localhost).
+- `jobmanager.rpc.port`: The port of the JobManager (DEFAULT: 6123).
+- `jobmanager.rpc.numhandler`: The number of RPC threads for the JobManager.
+Increase those for large setups in which many TaskManagers communicate with the
+JobManager simultaneousl (DEFAULT: 8).
+- `jobmanager.profiling.enable`: Flag to enable the profiling component. This
+collects network/cpu utilization statistics, which are displayed as charts in
+the SWT visualization GUI. The profiling may add a small overhead on the
+execution (DEFAULT: false).
+- `jobmanager.web.port`: Port of the JobManager's web interface (DEFAULT: 8081).
+- `jobmanager.heap.mb`: JVM heap size (in megabytes) for the JobManager
+(DEFAULT: 256).
+- `taskmanager.heap.mb`: JVM heap size (in megabytes) for the TaskManager. In
+contrast to Hadoop, Stratosphere runs operators and functions inside the
+TaskManager (including sorting/hashing/caching), so this value should be as
+large as possible (DEFAULT: 512).
+- `taskmanager.rpc.port`: The task manager's IPC port (DEFAULT: 6122).
+- `taskmanager.data.port`: The task manager's port used for data exchange
+operations (DEFAULT: 6121).
+- `taskmanager.tmp.dirs`: The directory for temporary files, or a list of
+directories separated by the systems directory delimiter (for example ':'
+(colon) on Linux/Unix). If multiple directories are specified then the temporary
+files will be distributed across the directories in a round robin fashion. The
+I/O manager component will spawn one reading and one writing thread per
+directory. A directory may be listed multiple times to have the I/O manager use
+multiple threads for it (for example if it is physically stored on a very fast
+disc or RAID) (DEFAULT: The system's tmp dir).
+- `taskmanager.network.numberOfBuffers`: The number of buffers available to the
+network stack. This number determines how many streaming data exchange channels
+a TaskManager can have at the same time and how well buffered the channels are.
+If a job is rejected or you get a warning that the system has not enough buffers
+available, increase this value (DEFAULT: 2048).
+- `taskmanager.network.bufferSizeInBytes`: The size of the network buffers, in
+bytes (DEFAULT: 32768 (= 32 KiBytes)).
+- `taskmanager.memory.size`: The amount of memory (in megabytes) that the task
+manager reserves for sorting, hash tables, and caching of intermediate results.
+If unspecified (-1), the memory manager will take a relative amount of the heap
+memory available to the JVM after the allocation of the network buffers (0.8)
+(DEFAULT: -1).
+- `taskmanager.memory.fraction`: The fraction of memory (after allocation of the
+network buffers) that the task manager reserves for sorting, hash tables, and
+caching of intermediate results. This value is only used if
+'taskmanager.memory.size' is unspecified (-1) (DEFAULT: 0.8).
+- `jobclient.polling.interval`: The interval (in seconds) in which the client
+polls the JobManager for the status of its job (DEFAULT: 2).
+- `taskmanager.runtime.max-fan`: The maximal fan-in for external merge joins and
+fan-out for spilling hash tables. Limits the numer of file handles per operator,
+but may cause intermediate merging/partitioning, if set too small (DEFAULT: 128).
+- `taskmanager.runtime.sort-spilling-threshold`: A sort operation starts spilling
+when this fraction of its memory budget is full (DEFAULT: 0.8).
+- `taskmanager.runtime.fs_timeout`: The maximal time (in milliseconds) that the
+system waits for a response from the filesystem. Note that for HDFS, this time
+may occasionally be rather long. A value of 0 indicates infinite waiting time
+(DEFAULT: 0).
+
+# JobManager Web Frontend
+
+- `jobmanager.web.port`: Port of the JobManager's web interface that displays
+status of running jobs and execution time breakdowns of finished jobs
+(DEFAULT: 8081).
+- `jobmanager.web.history`: The number of latest jobs that the JobManager's web
+front-end in its history (DEFAULT: 5).
+
+# Webclient
+
+These parameters configure the web interface that can be used to submit jobs and
+review the compiler's execution plans.
+
+- `webclient.port`: The port of the webclient server (DEFAULT: 8080).
+- `webclient.tempdir`: The temp directory for the web server. Used for example
+for caching file fragments during file-uploads (DEFAULT: The system's temp
+directory).
+- `webclient.uploaddir`: The directory into which the web server will store
+uploaded programs (DEFAULT: ${webclient.tempdir}/webclient-jobs/).
+- `webclient.plandump`: The directory into which the web server will dump
+temporary JSON files describing the execution plans
+(DEFAULT: ${webclient.tempdir}/webclient-plans/).
+
+# Compiler/Optimizer
+
+- `compiler.delimited-informat.max-line-samples`: The maximum number of line
+samples taken by the compiler for delimited inputs. The samples are used to
+estimate the number of records. This value can be overridden for a specific
+input with the input format's parameters (DEFAULT: 10).
+- `compiler.delimited-informat.min-line-samples`: The minimum number of line
+samples taken by the compiler for delimited inputs. The samples are used to
+estimate the number of records. This value can be overridden for a specific
+input with the input format's parameters (DEFAULT: 2).
+- `compiler.delimited-informat.max-sample-len`: The maximal length of a line
+sample that the compiler takes for delimited inputs. If the length of a single
+sample exceeds this value (possible because of misconfiguration of the parser),
+the sampling aborts. This value can be overridden for a specific input with the
+input format's parameters (DEFAULT: 2097152 (= 2 MiBytes)).

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/css/syntax.css
----------------------------------------------------------------------
diff --git a/docs/css/syntax.css b/docs/css/syntax.css
new file mode 100644
index 0000000..2774b76
--- /dev/null
+++ b/docs/css/syntax.css
@@ -0,0 +1,60 @@
+.highlight  { background: #ffffff; }
+.highlight .c { color: #999988; font-style: italic } /* Comment */
+.highlight .err { color: #a61717; background-color: #e3d2d2 } /* Error */
+.highlight .k { font-weight: bold } /* Keyword */
+.highlight .o { font-weight: bold } /* Operator */
+.highlight .cm { color: #999988; font-style: italic } /* Comment.Multiline */
+.highlight .cp { color: #999999; font-weight: bold } /* Comment.Preproc */
+.highlight .c1 { color: #999988; font-style: italic } /* Comment.Single */
+.highlight .cs { color: #999999; font-weight: bold; font-style: italic } /* Comment.Special */
+.highlight .gd { color: #000000; background-color: #ffdddd } /* Generic.Deleted */
+.highlight .gd .x { color: #000000; background-color: #ffaaaa } /* Generic.Deleted.Specific */
+.highlight .ge { font-style: italic } /* Generic.Emph */
+.highlight .gr { color: #aa0000 } /* Generic.Error */
+.highlight .gh { color: #999999 } /* Generic.Heading */
+.highlight .gi { color: #000000; background-color: #ddffdd } /* Generic.Inserted */
+.highlight .gi .x { color: #000000; background-color: #aaffaa } /* Generic.Inserted.Specific */
+.highlight .go { color: #888888 } /* Generic.Output */
+.highlight .gp { color: #555555 } /* Generic.Prompt */
+.highlight .gs { font-weight: bold } /* Generic.Strong */
+.highlight .gu { color: #aaaaaa } /* Generic.Subheading */
+.highlight .gt { color: #aa0000 } /* Generic.Traceback */
+.highlight .kc { font-weight: bold } /* Keyword.Constant */
+.highlight .kd { font-weight: bold } /* Keyword.Declaration */
+.highlight .kp { font-weight: bold } /* Keyword.Pseudo */
+.highlight .kr { font-weight: bold } /* Keyword.Reserved */
+.highlight .kt { color: #445588; font-weight: bold } /* Keyword.Type */
+.highlight .m { color: #009999 } /* Literal.Number */
+.highlight .s { color: #d14 } /* Literal.String */
+.highlight .na { color: #008080 } /* Name.Attribute */
+.highlight .nb { color: #0086B3 } /* Name.Builtin */
+.highlight .nc { color: #445588; font-weight: bold } /* Name.Class */
+.highlight .no { color: #008080 } /* Name.Constant */
+.highlight .ni { color: #800080 } /* Name.Entity */
+.highlight .ne { color: #990000; font-weight: bold } /* Name.Exception */
+.highlight .nf { color: #990000; font-weight: bold } /* Name.Function */
+.highlight .nn { color: #555555 } /* Name.Namespace */
+.highlight .nt { color: #000080 } /* Name.Tag */
+.highlight .nv { color: #008080 } /* Name.Variable */
+.highlight .ow { font-weight: bold } /* Operator.Word */
+.highlight .w { color: #bbbbbb } /* Text.Whitespace */
+.highlight .mf { color: #009999 } /* Literal.Number.Float */
+.highlight .mh { color: #009999 } /* Literal.Number.Hex */
+.highlight .mi { color: #009999 } /* Literal.Number.Integer */
+.highlight .mo { color: #009999 } /* Literal.Number.Oct */
+.highlight .sb { color: #d14 } /* Literal.String.Backtick */
+.highlight .sc { color: #d14 } /* Literal.String.Char */
+.highlight .sd { color: #d14 } /* Literal.String.Doc */
+.highlight .s2 { color: #d14 } /* Literal.String.Double */
+.highlight .se { color: #d14 } /* Literal.String.Escape */
+.highlight .sh { color: #d14 } /* Literal.String.Heredoc */
+.highlight .si { color: #d14 } /* Literal.String.Interpol */
+.highlight .sx { color: #d14 } /* Literal.String.Other */
+.highlight .sr { color: #009926 } /* Literal.String.Regex */
+.highlight .s1 { color: #d14 } /* Literal.String.Single */
+.highlight .ss { color: #990073 } /* Literal.String.Symbol */
+.highlight .bp { color: #999999 } /* Name.Builtin.Pseudo */
+.highlight .vc { color: #008080 } /* Name.Variable.Class */
+.highlight .vg { color: #008080 } /* Name.Variable.Global */
+.highlight .vi { color: #008080 } /* Name.Variable.Instance */
+.highlight .il { color: #009999 } /* Literal.Number.Integer.Long */

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/faq.md
----------------------------------------------------------------------
diff --git a/docs/faq.md b/docs/faq.md
new file mode 100644
index 0000000..3ceb527
--- /dev/null
+++ b/docs/faq.md
@@ -0,0 +1,285 @@
+---
+title: "Frequently Asked Questions (FAQ)"
+---
+
+# General
+
+## Is Stratosphere a Hadoop Project?
+
+Stratosphere is a data processing system and an alternative to Hadoop's
+MapReduce component. It comes with its own runtime, rather than building on top
+of MapReduce. As such, it can work completely independently of the Hadoop
+ecosystem. However, Stratosphere can also access Hadoop's distributed file
+system (HDFS) to read and write data, and Hadoop's next-generation resource
+manager (YARN) to provision cluster resources. Since most Stratosphere users are
+using Hadoop HDFS to store their data, we ship already the required libraries to
+access HDFS.
+
+## Do I have to install Apache Hadoop to use Stratosphere?
+
+No. Stratosphere can run without a Hadoop installation. However, a very common
+setup is to use Stratosphere to analyze data stored in the Hadoop Distributed
+File System (HDFS). To make these setups work out of the box, we bundle the
+Hadoop client libraries with Stratosphere by default.
+
+Additionally, we provide a special YARN Enabled download of Stratosphere for
+users with an existing Hadoop YARN cluster. [Apache Hadoop
+YARN](http://hadoop.apache.org/docs/r2.2.0/hadoop-yarn/hadoop-yarn-
+site/YARN.html) is Hadoop's cluster resource manager that allows to use
+different execution engines next to each other on a cluster.
+
+# Usage
+
+## How do I assess the progress of a Stratosphere program?
+
+There are a multiple of ways to track the progress of a Stratosphere program:
+
+- The JobManager (the master of the distributed system) starts a web interface
+to observe program execution. In runs on port 8081 by default (configured in
+`conf/stratosphere-config.yml`).
+- When you start a program from the command line, it will print the status
+changes of all operators as the program progresses through the operations.
+- All status changes are also logged to the JobManager's log file.
+
+## How can I figure out why a program failed?
+
+- Thw JobManager web frontend (by default on port 8081) displays the exceptions
+of failed tasks.
+- If you run the program from the command-line, task exceptions are printed to
+the standard error stream and shown on the console.
+- Both the command line and the web interface allow you to figure out which
+parallel task first failed and caused the other tasks to cancel the execution.
+- Failing tasks and the corresponding exceptions are reported in the log files
+of the master and the worker where the exception occurred
+(`log/stratosphere-<user>-jobmanager-<host>.log` and
+`log/stratosphere-<user>-taskmanager-<host>.log`).
+
+## How do I debug Stratosphere programs?
+
+- When you start a program locally with the [LocalExecutor](local_execution.html),
+you can place breakpoints in your functions and debug them like normal
+Java/Scala programs.
+- The [Accumulators](java_api_guide.html#accumulators) are very helpful in
+tracking the behavior of the parallel execution. They allow you to gather
+information inside the program's operations and show them after the program
+execution.
+
+# Errors
+
+## I get an error message saying that not enough buffers are available. How do I fix this?
+
+If you run Stratosphere in a massively parallel setting (100+ parallel threads),
+you need to adapt the number of network buffers via the config parameter
+`taskmanager.network.numberOfBuffers`.
+As a rule-of-thumb, the number of buffers should be at least
+`4 * numberOfNodes * numberOfTasksPerNode^2`. See
+[Configuration Reference](config.html) for details.
+
+## My job fails early with a java.io.EOFException. What could be the cause?
+
+Note: In version <em>0.4</em>, the delta iterations limit the solution set to
+records with fixed-length data types. We will  in the next version.
+
+The most common case for these exception is when Stratosphere is set up with the
+wrong HDFS version. Because different HDFS versions are often not compatible
+with each other, the connection between the filesystem master and the client
+breaks.
+
+```bash
+Call to <host:port> failed on local exception: java.io.EOFException
+    at org.apache.hadoop.ipc.Client.wrapException(Client.java:775)
+    at org.apache.hadoop.ipc.Client.call(Client.java:743)
+    at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:220)
+    at $Proxy0.getProtocolVersion(Unknown Source)
+    at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:359)
+    at org.apache.hadoop.hdfs.DFSClient.createRPCNamenode(DFSClient.java:106)
+    at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:207)
+    at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:170)
+    at org.apache.hadoop.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:82)
+    at eu.stratosphere.runtime.fs.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:276
+```
+
+Please refer to the [download page](http://stratosphere.eu/downloads/#maven) and
+the [build instructions](https://github.com/stratosphere/stratosphere/blob/master/README.md)
+for details on how to set up Stratosphere for different Hadoop and HDFS versions.
+
+## My program does not compute the correct result. Why are my custom key types
+are not grouped/joined correctly?
+
+Keys must correctly implement the methods `java.lang.Object#hashCode()`,
+`java.lang.Object#equals(Object o)`, and `java.util.Comparable#compareTo(...)`.
+These methods are always backed with default implementations which are usually
+inadequate. Therefore, all keys must override `hashCode()` and `equals(Object o)`.
+
+## I get a java.lang.InstantiationException for my data type, what is wrong?
+
+All data type classes must be public and have a public nullary constructor
+(constructor with no arguments). Further more, the classes must not be abstract
+or interfaces. If the classes are internal classes, they must be public and
+static.
+
+## I can't stop Stratosphere with the provided stop-scripts. What can I do?
+
+Stopping the processes sometimes takes a few seconds, because the shutdown may
+do some cleanup work.
+
+In some error cases it happens that the JobManager or TaskManager cannot be
+stopped with the provided stop-scripts (`bin/stop-local.sh` or `bin/stop-
+cluster.sh`). You can kill their processes on Linux/Mac as follows:
+
+- Determine the process id (pid) of the JobManager / TaskManager process. You
+can use the `jps` command on Linux(if you have OpenJDK installed) or command
+`ps -ef | grep java` to find all Java processes. 
+- Kill the process with `kill -9 <pid>`, where `pid` is the process id of the
+affected JobManager or TaskManager process.
+    
+On Windows, the TaskManager shows a table of all processes and allows you to
+destroy a process by right its entry.
+
+## I got an OutOfMemoryException. What can I do?
+
+These exceptions occur usually when the functions in the program consume a lot
+of memory by collection large numbers of objects, for example in lists or maps.
+The OutOfMemoryExceptions in Java are kind of tricky. The exception is not
+necessarily thrown by the component that allocated most of the memory but by the
+component that tried to requested the latest bit of memory that could not be
+provided.
+
+There are two ways to go about this:
+
+1. See whether you can use less memory inside the functions. For example, use
+arrays of primitive types instead of object types.
+
+2. Reduce the memory that Stratosphere reserves for its own processing. The
+TaskManager reserves a certain portion of the available memory for sorting,
+hashing, caching, network buffering, etc. That part of the memory is unavailable
+to the user-defined functions. By reserving it, the system can guarantee to not
+run out of memory on large inputs, but to plan with the available memory and
+destage operations to disk, if necessary. By default, the system reserves around
+70% of the memory. If you frequently run applications that need more memory in
+the user-defined functions, you can reduce that value using the configuration
+entries `taskmanager.memory.fraction` or `taskmanager.memory.size`. See the
+[Configuration Reference](http://stratosphere.eu/docs/0.4/setup/config.html
+"Configuration Reference") for details. This will leave more memory to JVM heap,
+but may cause data processing tasks to go to disk more often.
+
+## Why do the TaskManager log files become so huge?
+
+Check the logging behavior of your jobs. Emitting logging per or tuple may be
+helpful to debug jobs in small setups with tiny data sets, it becomes very
+inefficient and disk space consuming if used for large input data.
+
+# YARN Deployment
+
+## The YARN session runs only for a few seconds
+
+The `./bin/yarn-session.sh` script is intended to run while the YARN-session is
+open. In some error cases however, the script immediately stops running. The
+output looks like this:
+
+```
+07:34:27,004 INFO  org.apache.hadoop.yarn.client.api.impl.YarnClientImpl         - Submitted application application_1395604279745_273123 to ResourceManager at jobtracker-host
+Stratosphere JobManager is now running on worker1:6123
+JobManager Web Interface: http://jobtracker-host:54311/proxy/application_1295604279745_273123/
+07:34:51,528 INFO  eu.stratosphere.yarn.Client                                   - Application application_1295604279745_273123 finished with state FINISHED at 1398152089553
+07:34:51,529 INFO  eu.stratosphere.yarn.Client                                   - Killing the Stratosphere-YARN application.
+07:34:51,529 INFO  org.apache.hadoop.yarn.client.api.impl.YarnClientImpl         - Killing application application_1295604279745_273123
+07:34:51,534 INFO  eu.stratosphere.yarn.Client                                   - Deleting files in hdfs://user/marcus/.stratosphere/application_1295604279745_273123
+07:34:51,559 INFO  eu.stratosphere.yarn.Client                                   - YARN Client is shutting down
+```
+
+The problem here is that the Application Master (AM) is stopping and the YARN client assumes that the application has finished.
+
+There are three possible reasons for that behavior:
+
+- The ApplicationMaster exited with an exception. To debug that error, have a
+look in the logfiles of the container. The `yarn-site.xml` file contains the
+configured path. The key for the path is `yarn.nodemanager.log-dirs`, the
+default value is `${yarn.log.dir}/userlogs`.
+
+- YARN has killed the container that runs the ApplicationMaster. This case
+happens when the AM used too much memory or other resources beyond YARN's
+limits. In this case, you'll find error messages in the nodemanager logs on
+the host.
+
+- The operating system has shut down the JVM of the AM. This can happen if the
+YARN configuration is wrong and more memory than physically available is
+configured. Execute `dmesg` on the machine where the AM was running to see if
+this happened. You see messages from Linux' [OOM killer](http://linux-mm.org/OOM_Killer).
+
+## The YARN session crashes with a HDFS permission exception during startup
+
+While starting the YARN session, you are receiving an exception like this:
+
+```
+Exception in thread "main" org.apache.hadoop.security.AccessControlException: Permission denied: user=robert, access=WRITE, inode="/user/robert":hdfs:supergroup:drwxr-xr-x
+  at org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.check(FSPermissionChecker.java:234)
+  at org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.check(FSPermissionChecker.java:214)
+  at org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.checkPermission(FSPermissionChecker.java:158)
+  at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkPermission(FSNamesystem.java:5193)
+  at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkPermission(FSNamesystem.java:5175)
+  at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkAncestorAccess(FSNamesystem.java:5149)
+  at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFileInternal(FSNamesystem.java:2090)
+  at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFileInt(FSNamesystem.java:2043)
+  at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFile(FSNamesystem.java:1996)
+  at org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.create(NameNodeRpcServer.java:491)
+  at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.create(ClientNamenodeProtocolServerSideTranslatorPB.java:301)
+  at org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java:59570)
+  at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:585)
+  at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:928)
+  at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2053)
+  at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2049)
+  at java.security.AccessController.doPrivileged(Native Method)
+  at javax.security.auth.Subject.doAs(Subject.java:396)
+  at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1491)
+  at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2047)
+
+  at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
+  at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:39)
+  at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27)
+  at java.lang.reflect.Constructor.newInstance(Constructor.java:513)
+  at org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteException.java:106)
+  at org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(RemoteException.java:73)
+  at org.apache.hadoop.hdfs.DFSOutputStream.newStreamForCreate(DFSOutputStream.java:1393)
+  at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1382)
+  at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1307)
+  at org.apache.hadoop.hdfs.DistributedFileSystem$6.doCall(DistributedFileSystem.java:384)
+  at org.apache.hadoop.hdfs.DistributedFileSystem$6.doCall(DistributedFileSystem.java:380)
+  at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
+  at org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:380)
+  at org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:324)
+  at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:905)
+  at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:886)
+  at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:783)
+  at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:365)
+  at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:338)
+  at org.apache.hadoop.fs.FileSystem.copyFromLocalFile(FileSystem.java:2021)
+  at org.apache.hadoop.fs.FileSystem.copyFromLocalFile(FileSystem.java:1989)
+  at org.apache.hadoop.fs.FileSystem.copyFromLocalFile(FileSystem.java:1954)
+  at eu.stratosphere.yarn.Utils.setupLocalResource(Utils.java:176)
+  at eu.stratosphere.yarn.Client.run(Client.java:362)
+  at eu.stratosphere.yarn.Client.main(Client.java:568)
+```
+
+The reason for this error is, that the home directory of the user **in HDFS**
+has the wrong permissions. The user (in this case `robert`) can not create
+directories in his own home directory.
+
+Stratosphere creates a `.stratosphere/` directory in the users home directory
+where it stores the Stratosphere jar and configuration file.
+
+# Features
+
+## What kind of fault-tolerance does Stratosphere provide?
+
+Stratospere can restart failed jobs. Mid-query fault tolerance will go into the
+open source project in the next versions.
+
+## Are Hadoop-like utilities, such as Counters and the DistributedCache supported?
+
+[Stratosphere's Accumulators](java_api_guide.html) work very similar like
+[Hadoop's counters, but are more powerful.
+
+Stratosphere has a [Distributed Cache](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-core/src/main/java/eu/stratosphere/api/common/cache/DistributedCache.java) that is deeply integrated with the APIs. Please refer to the [JavaDocs](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-java/src/main/java/eu/stratosphere/api/java/ExecutionEnvironment.java#L561) for details on how to use it.
+
+In order to make data sets available on all tasks, we encourage you to use [Broadcast Variables]({{site.baseurl}}/docs/0.5/programming_guides/java.html#broadcast_variables) instead. They are more efficient and easier to use than the distributed cache.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/hadoop_compatability.md
----------------------------------------------------------------------
diff --git a/docs/hadoop_compatability.md b/docs/hadoop_compatability.md
new file mode 100644
index 0000000..06c0dfa
--- /dev/null
+++ b/docs/hadoop_compatability.md
@@ -0,0 +1,5 @@
+---
+title: "Hadoop Compatability"
+---
+
+To be written.
\ No newline at end of file


[50/53] [abbrv] git commit: Remove debug LOG message of TaskManager heartbeats

Posted by rm...@apache.org.
Remove debug LOG message of TaskManager heartbeats


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

Branch: refs/heads/travis_test
Commit: 9c5183495f71ad834285fd0778a4f8e350948699
Parents: 49cd35a
Author: uce <u....@fu-berlin.de>
Authored: Wed Jun 25 17:53:11 2014 +0200
Committer: uce <u....@fu-berlin.de>
Committed: Wed Jun 25 17:53:11 2014 +0200

----------------------------------------------------------------------
 .../main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java  | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9c518349/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
index cb06d08..0c5e218 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
@@ -566,7 +566,6 @@ public class TaskManager implements TaskOperationProtocol {
 
 			// send heart beat
 			try {
-				LOG.debug("heartbeat");
 				this.jobManager.sendHeartbeat(this.localInstanceConnectionInfo);
 			} catch (IOException e) {
 				if (shutdownStarted.get()) {


[20/53] [abbrv] Merge fix to omit input/output registering on JobManager Rework Invokable Task Hierarchy

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
index f191df3..575454f 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
@@ -173,7 +173,7 @@ public class TaskManager implements TaskOperationProtocol {
 		}
 		
 		
-		LOG.info("TaskManager started as user " + UserGroupInformation.getCurrentUser().getShortUserName());
+//		LOG.info("TaskManager started as user " + UserGroupInformation.getCurrentUser().getShortUserName());
 		LOG.info("User system property: " + System.getProperty("user.name"));
 		LOG.info("Execution mode: " + executionMode);
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractInputTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractInputTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractInputTask.java
deleted file mode 100644
index 88e4fcb..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractInputTask.java
+++ /dev/null
@@ -1,79 +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.template;
-
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-
-import eu.stratosphere.core.io.InputSplit;
-
-/**
- * Abstract base class for tasks submitted as a part of a job input vertex.
- * 
- * @param <T>
- *        the type of input splits generated by this input task
- */
-public abstract class AbstractInputTask<T extends InputSplit> extends AbstractInvokable {
-
-	/**
-	 * Returns an iterator to a (possible empty) list of input splits which is expected to be consumed by this
-	 * instance of the {@link AbstractInputTask}.
-	 * 
-	 * @return an iterator to a (possible empty) list of input splits.
-	 */
-	public Iterator<T> getInputSplits() {
-
-		final InputSplitProvider provider = getEnvironment().getInputSplitProvider();
-
-		return new Iterator<T>() {
-
-			private T nextSplit;
-
-			@Override
-			public boolean hasNext() {
-
-				if (this.nextSplit == null) {
-
-					final InputSplit split = provider.getNextInputSplit();
-					if (split != null) {
-						@SuppressWarnings("unchecked")
-						final T tSplit = (T) split;
-						this.nextSplit = tSplit;
-						return true;
-					} else {
-						return false;
-					}
-				} else {
-					return true;
-				}
-			}
-
-			@Override
-			public T next() {
-				if (this.nextSplit == null && !hasNext()) {
-					throw new NoSuchElementException();
-				}
-
-				final T tmp = this.nextSplit;
-				this.nextSplit = null;
-				return tmp;
-			}
-
-			@Override
-			public void remove() {
-				throw new UnsupportedOperationException();
-			}
-		};
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractInvokable.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractInvokable.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractInvokable.java
index 79390f8..792c1bf 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractInvokable.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractInvokable.java
@@ -14,7 +14,6 @@
 package eu.stratosphere.nephele.template;
 
 import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.configuration.IllegalConfigurationException;
 import eu.stratosphere.nephele.execution.Environment;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractOutputTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractOutputTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractOutputTask.java
deleted file mode 100644
index 13042d4..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractOutputTask.java
+++ /dev/null
@@ -1,22 +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.template;
-
-/**
- * Abstract base class for tasks submitted as a part of a job output vertex.
- * 
- */
-public abstract class AbstractOutputTask extends AbstractInvokable {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractTask.java
deleted file mode 100644
index 6d568ab..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/template/AbstractTask.java
+++ /dev/null
@@ -1,21 +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.template;
-
-/**
- * Abstract base class for tasks submitted as a part of a job task vertex.
- * 
- */
-public abstract class AbstractTask extends AbstractInvokable {
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/io/FakeOutputTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/io/FakeOutputTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/io/FakeOutputTask.java
index ced186b..f2944f4 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/io/FakeOutputTask.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/io/FakeOutputTask.java
@@ -13,14 +13,14 @@
 
 package eu.stratosphere.pact.runtime.iterative.io;
 
+import eu.stratosphere.nephele.template.AbstractInvokable;
 import eu.stratosphere.runtime.io.api.MutableRecordReader;
-import eu.stratosphere.nephele.template.AbstractOutputTask;
 import eu.stratosphere.types.Record;
 
 /**
  * Output task for the iteration tail
  */
-public class FakeOutputTask extends AbstractOutputTask {
+public class FakeOutputTask extends AbstractInvokable {
 
 	private MutableRecordReader<Record> reader;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationSynchronizationSinkTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationSynchronizationSinkTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationSynchronizationSinkTask.java
index 4e7286b..947872f 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationSynchronizationSinkTask.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationSynchronizationSinkTask.java
@@ -29,7 +29,7 @@ import eu.stratosphere.api.common.aggregators.ConvergenceCriterion;
 import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
 import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
 import eu.stratosphere.runtime.io.api.MutableRecordReader;
-import eu.stratosphere.nephele.template.AbstractOutputTask;
+import eu.stratosphere.nephele.template.AbstractInvokable;
 import eu.stratosphere.nephele.types.IntegerRecord;
 import eu.stratosphere.pact.runtime.iterative.event.AllWorkersDoneEvent;
 import eu.stratosphere.pact.runtime.iterative.event.TerminationEvent;
@@ -44,7 +44,7 @@ import eu.stratosphere.types.Value;
  * In each superstep, it simply waits until it has receiced a {@link WorkerDoneEvent} from each head and will send back
  * an {@link AllWorkersDoneEvent} to signal that the next superstep can begin.
  */
-public class IterationSynchronizationSinkTask extends AbstractOutputTask implements Terminable {
+public class IterationSynchronizationSinkTask extends AbstractInvokable implements Terminable {
 
 	private static final Log log = LogFactory.getLog(IterationSynchronizationSinkTask.class);
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationTailPactTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationTailPactTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationTailPactTask.java
index 859a62d..05b58e8 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationTailPactTask.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationTailPactTask.java
@@ -97,7 +97,13 @@ public class IterationTailPactTask<S extends Function, OT> extends AbstractItera
 				log.info(formatLogString("starting iteration [" + currentIteration() + "]"));
 			}
 
-			super.run();
+			try {
+				super.run();
+			}
+			catch (NullPointerException e) {
+				boolean terminationRequested = terminationRequested();
+				System.out.println("Nullpoint exception when termination requested was " + terminationRequested);
+			}
 
 			// check if termination was requested
 			checkForTerminationAndResetEndOfSuperstepState();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSinkTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSinkTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSinkTask.java
index cbe1766..7041679 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSinkTask.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSinkTask.java
@@ -16,26 +16,22 @@ package eu.stratosphere.pact.runtime.task;
 import java.io.IOException;
 
 import eu.stratosphere.pact.runtime.task.chaining.ExceptionInChainedStubException;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
-import eu.stratosphere.api.common.io.FileOutputFormat;
-import eu.stratosphere.api.common.io.FileOutputFormat.OutputDirectoryMode;
 import eu.stratosphere.api.common.io.OutputFormat;
 import eu.stratosphere.api.common.typeutils.TypeComparatorFactory;
 import eu.stratosphere.api.common.typeutils.TypeSerializer;
 import eu.stratosphere.api.common.typeutils.TypeSerializerFactory;
 import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.core.fs.FileSystem;
-import eu.stratosphere.core.fs.FileSystem.WriteMode;
-import eu.stratosphere.core.fs.Path;
 import eu.stratosphere.core.io.IOReadableWritable;
 import eu.stratosphere.nephele.execution.CancelTaskException;
 import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
+import eu.stratosphere.nephele.template.AbstractInvokable;
 import eu.stratosphere.runtime.io.api.MutableReader;
 import eu.stratosphere.runtime.io.api.MutableRecordReader;
 import eu.stratosphere.runtime.io.api.MutableUnionRecordReader;
-import eu.stratosphere.nephele.template.AbstractOutputTask;
 import eu.stratosphere.pact.runtime.plugable.DeserializationDelegate;
 import eu.stratosphere.pact.runtime.sort.UnilateralSortMerger;
 import eu.stratosphere.pact.runtime.task.util.CloseableInputProvider;
@@ -51,7 +47,7 @@ import eu.stratosphere.util.MutableObjectIterator;
  * 
  * @see OutputFormat
  */
-public class DataSinkTask<IT> extends AbstractOutputTask {
+public class DataSinkTask<IT> extends AbstractInvokable {
 	
 	public static final String DEGREE_OF_PARALLELISM_KEY = "sink.dop";
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSourceTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSourceTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSourceTask.java
index f835ace..62226d9 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSourceTask.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSourceTask.java
@@ -18,9 +18,11 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
+import java.util.NoSuchElementException;
 
 import eu.stratosphere.pact.runtime.task.chaining.ExceptionInChainedStubException;
 import eu.stratosphere.runtime.io.api.BufferWriter;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
@@ -32,7 +34,8 @@ import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.core.io.InputSplit;
 import eu.stratosphere.nephele.execution.CancelTaskException;
 import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
-import eu.stratosphere.nephele.template.AbstractInputTask;
+import eu.stratosphere.nephele.template.AbstractInvokable;
+import eu.stratosphere.nephele.template.InputSplitProvider;
 import eu.stratosphere.pact.runtime.shipping.OutputCollector;
 import eu.stratosphere.pact.runtime.shipping.RecordOutputCollector;
 import eu.stratosphere.pact.runtime.task.chaining.ChainedCollectorMapDriver;
@@ -47,11 +50,11 @@ import eu.stratosphere.util.Collector;
  * 
  * @see eu.stratosphere.api.common.io.InputFormat
  */
-public class DataSourceTask<OT> extends AbstractInputTask<InputSplit> {
+public class DataSourceTask<OT> extends AbstractInvokable {
 	
-	// Obtain DataSourceTask Logger
 	private static final Log LOG = LogFactory.getLog(DataSourceTask.class);
 
+	
 	private List<BufferWriter> eventualOutputs;
 
 	// Output collector
@@ -76,11 +79,10 @@ public class DataSourceTask<OT> extends AbstractInputTask<InputSplit> {
 
 
 	@Override
-	public void registerInputOutput()
-	{
+	public void registerInputOutput() {
 		initInputFormat();
 
-		if (LOG.isDebugEnabled())
+		if (LOG.isDebugEnabled()) {
 			LOG.debug(getLogString("Start registering input and output"));
 		}
 
@@ -331,7 +333,7 @@ l	 *
 		}
 		
 		// get the factory for the type serializer
-		this.serializerFactory = this.config.getOutputSerializer(cl);
+		this.serializerFactory = this.config.getOutputSerializer(this.userCodeClassLoader);
 	}
 
 	/**
@@ -343,49 +345,6 @@ l	 *
 		this.eventualOutputs = new ArrayList<BufferWriter>();
 		this.output = RegularPactTask.initOutputs(this, cl, this.config, this.chainedTasks, this.eventualOutputs);
 	}
-	
-	// ------------------------------------------------------------------------
-	//                              Input Split creation
-	// ------------------------------------------------------------------------
-	
-
-	@Override
-	public InputSplit[] computeInputSplits(int requestedMinNumber) throws Exception {
-		// we have to be sure that the format is instantiated at this point
-		if (this.format == null) {
-			throw new IllegalStateException("BUG: Input format hast not been instantiated, yet.");
-		}
-		return this.format.createInputSplits(requestedMinNumber);
-	}
-
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public Class<InputSplit> getInputSplitType() {
-		// we have to be sure that the format is instantiated at this point
-		if (this.format == null) {
-			throw new IllegalStateException("BUG: Input format hast not been instantiated, yet.");
-		}
-		
-		return (Class<InputSplit>) this.format.getInputSplitType();
-	}
-	
-	// ------------------------------------------------------------------------
-	//                       Control of Parallelism
-	// ------------------------------------------------------------------------
-	
-
-	@Override
-	public int getMinimumNumberOfSubtasks() {
-		return 1;
-	}
-
-
-	@Override
-	public int getMaximumNumberOfSubtasks() {
-		// since splits can in theory be arbitrarily small, we report a possible infinite number of subtasks.
-		return -1;
-	}
 
 	// ------------------------------------------------------------------------
 	//                               Utilities
@@ -413,4 +372,54 @@ l	 *
 	private String getLogString(String message, String taskName) {
 		return RegularPactTask.constructLogString(message, taskName, this);
 	}
+	
+	private Iterator<InputSplit> getInputSplits() {
+
+		final InputSplitProvider provider = getEnvironment().getInputSplitProvider();
+
+		return new Iterator<InputSplit>() {
+
+			private InputSplit nextSplit;
+			
+			private boolean exhausted;
+
+			@Override
+			public boolean hasNext() {
+				if (exhausted) {
+					return false;
+				}
+				
+				if (nextSplit != null) {
+					return true;
+				}
+				
+				InputSplit split = provider.getNextInputSplit();
+				
+				if (split != null) {
+					this.nextSplit = split;
+					return true;
+				}
+				else {
+					exhausted = true;
+					return false;
+				}
+			}
+
+			@Override
+			public InputSplit next() {
+				if (this.nextSplit == null && !hasNext()) {
+					throw new NoSuchElementException();
+				}
+
+				final InputSplit tmp = this.nextSplit;
+				this.nextSplit = null;
+				return tmp;
+			}
+
+			@Override
+			public void remove() {
+				throw new UnsupportedOperationException();
+			}
+		};
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/RegularPactTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/RegularPactTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/RegularPactTask.java
index 1d7c931..3140525 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/RegularPactTask.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/RegularPactTask.java
@@ -36,9 +36,7 @@ import eu.stratosphere.runtime.io.api.BufferWriter;
 import eu.stratosphere.nephele.services.accumulators.AccumulatorEvent;
 import eu.stratosphere.nephele.services.iomanager.IOManager;
 import eu.stratosphere.nephele.services.memorymanager.MemoryManager;
-import eu.stratosphere.nephele.template.AbstractInputTask;
 import eu.stratosphere.nephele.template.AbstractInvokable;
-import eu.stratosphere.nephele.template.AbstractTask;
 import eu.stratosphere.pact.runtime.plugable.DeserializationDelegate;
 import eu.stratosphere.pact.runtime.plugable.SerializationDelegate;
 import eu.stratosphere.pact.runtime.resettable.SpillingResettableMutableObjectIterator;
@@ -73,7 +71,7 @@ import java.util.Map;
  * The abstract base class for all tasks. Encapsulated common behavior and implements the main life-cycle
  * of the user code.
  */
-public class RegularPactTask<S extends Function, OT> extends AbstractTask implements PactTaskContext<S, OT> {
+public class RegularPactTask<S extends Function, OT> extends AbstractInvokable implements PactTaskContext<S, OT> {
 
 	protected static final Log LOG = LogFactory.getLog(RegularPactTask.class);
 
@@ -1251,11 +1249,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 					oe = new RecordOutputEmitter(strategy, comparator, distribution);
 				}
 
-				if (task instanceof AbstractTask) {
-					writers.add(new RecordWriter<Record>((AbstractTask) task, oe));
-				} else if (task instanceof AbstractInputTask<?>) {
-					writers.add(new RecordWriter<Record>((AbstractInputTask<?>) task, oe));
-				}
+				writers.add(new RecordWriter<Record>(task, oe));
 			}
 			if (eventualOutputs != null) {
 				eventualOutputs.addAll(writers);
@@ -1288,11 +1282,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 					oe = new OutputEmitter<T>(strategy, comparator, dataDist);
 				}
 
-				if (task instanceof AbstractTask) {
-					writers.add(new RecordWriter<SerializationDelegate<T>>((AbstractTask) task, oe));
-				} else if (task instanceof AbstractInputTask<?>) {
-					writers.add(new RecordWriter<SerializationDelegate<T>>((AbstractInputTask<?>) task, oe));
-				}
+				writers.add(new RecordWriter<SerializationDelegate<T>>(task, oe));
 			}
 			if (eventualOutputs != null) {
 				eventualOutputs.addAll(writers);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/TaskConfig.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/TaskConfig.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/TaskConfig.java
index 2eb003d..b44a489 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/TaskConfig.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/TaskConfig.java
@@ -1102,8 +1102,10 @@ public class TaskConfig {
 	/**
 	 * A configuration that manages a subset of keys with a common prefix from a given configuration.
 	 */
-	public static final class DelegatingConfiguration extends Configuration
-	{
+	public static final class DelegatingConfiguration extends Configuration {
+		
+		private static final long serialVersionUID = 1L;
+
 		private final Configuration backingConfig;		// the configuration actually storing the data
 		
 		private String prefix;							// the prefix key by which keys for this config are marked

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/MutableRecordReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/MutableRecordReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/MutableRecordReader.java
index 9d03c7f..c54b542 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/MutableRecordReader.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/MutableRecordReader.java
@@ -17,8 +17,7 @@ import java.io.IOException;
 
 import eu.stratosphere.core.io.IOReadableWritable;
 import eu.stratosphere.runtime.io.gates.InputChannelResult;
-import eu.stratosphere.nephele.template.AbstractOutputTask;
-import eu.stratosphere.nephele.template.AbstractTask;
+import eu.stratosphere.nephele.template.AbstractInvokable;
 
 public class MutableRecordReader<T extends IOReadableWritable> extends AbstractSingleGateRecordReader<T> implements MutableReader<T> {
 	
@@ -30,42 +29,9 @@ public class MutableRecordReader<T extends IOReadableWritable> extends AbstractS
 	 * 
 	 * @param taskBase The application that instantiated the record reader.
 	 */
-	public MutableRecordReader(final AbstractTask taskBase) {
+	public MutableRecordReader(AbstractInvokable taskBase) {
 		super(taskBase);
 	}
-
-	/**
-	 * Constructs a new record reader and registers a new input gate with the application's environment.
-	 * 
-	 * @param outputBase The application that instantiated the record reader.
-	 */
-	public MutableRecordReader(final AbstractOutputTask outputBase) {
-		super(outputBase);
-	}
-
-	/**
-	 * Constructs a new record reader and registers a new input gate with the application's environment.
-	 * 
-	 * @param taskBase
-	 *        the application that instantiated the record reader
-	 * @param inputGateID
-	 *        The ID of the input gate that the reader reads from.
-	 */
-	public MutableRecordReader(final AbstractTask taskBase, final int inputGateID) {
-		super(taskBase);
-	}
-
-	/**
-	 * Constructs a new record reader and registers a new input gate with the application's environment.
-	 * 
-	 * @param outputBase
-	 *        the application that instantiated the record reader
-	 * @param inputGateID
-	 *        The ID of the input gate that the reader reads from.
-	 */
-	public MutableRecordReader(final AbstractOutputTask outputBase, final int inputGateID) {
-		super(outputBase);
-	}
 	
 	// --------------------------------------------------------------------------------------------
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/RecordReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/RecordReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/RecordReader.java
index bb6a580..5fc436c 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/RecordReader.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/RecordReader.java
@@ -14,8 +14,7 @@
 package eu.stratosphere.runtime.io.api;
 
 import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.template.AbstractOutputTask;
-import eu.stratosphere.nephele.template.AbstractTask;
+import eu.stratosphere.nephele.template.AbstractInvokable;
 import eu.stratosphere.runtime.io.gates.InputChannelResult;
 
 import java.io.IOException;
@@ -50,23 +49,10 @@ public class RecordReader<T extends IOReadableWritable> extends AbstractSingleGa
 	 * @param recordType
 	 *        The class of records that can be read from the record reader.
 	 */
-	public RecordReader(AbstractTask taskBase, Class<T> recordType) {
+	public RecordReader(AbstractInvokable taskBase, Class<T> recordType) {
 		super(taskBase);
 		this.recordType = recordType;
 	}
-
-	/**
-	 * Constructs a new record reader and registers a new input gate with the application's environment.
-	 * 
-	 * @param outputBase
-	 *        The application that instantiated the record reader.
-	 * @param recordType
-	 *        The class of records that can be read from the record reader.
-	 */
-	public RecordReader(AbstractOutputTask outputBase, Class<T> recordType) {
-		super(outputBase);
-		this.recordType = recordType;
-	}
 	
 	// --------------------------------------------------------------------------------------------
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/RecordWriter.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/RecordWriter.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/RecordWriter.java
index 132dc14..a1ff62d 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/RecordWriter.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/RecordWriter.java
@@ -15,9 +15,7 @@ package eu.stratosphere.runtime.io.api;
 
 import eu.stratosphere.core.io.IOReadableWritable;
 import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.template.AbstractInputTask;
 import eu.stratosphere.nephele.template.AbstractInvokable;
-import eu.stratosphere.nephele.template.AbstractTask;
 import eu.stratosphere.runtime.io.Buffer;
 import eu.stratosphere.runtime.io.channels.EndOfSuperstepEvent;
 import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
@@ -47,25 +45,11 @@ public class RecordWriter<T extends IOReadableWritable> extends BufferWriter {
 
 	// -----------------------------------------------------------------------------------------------------------------
 
-	public RecordWriter(AbstractTask task) {
-		this((AbstractInvokable) task, new RoundRobinChannelSelector<T>());
+	public RecordWriter(AbstractInvokable invokable) {
+		this(invokable, new RoundRobinChannelSelector<T>());
 	}
 
-	public RecordWriter(AbstractTask task, ChannelSelector<T> channelSelector) {
-		this((AbstractInvokable) task, channelSelector);
-	}
-
-	public RecordWriter(AbstractInputTask<?> task) {
-		this((AbstractInvokable) task, new RoundRobinChannelSelector<T>());
-	}
-
-	public RecordWriter(AbstractInputTask<?> task, ChannelSelector<T> channelSelector) {
-		this((AbstractInvokable) task, channelSelector);
-	}
-
-	// -----------------------------------------------------------------------------------------------------------------
-
-	private RecordWriter(AbstractInvokable invokable, ChannelSelector<T> channelSelector) {
+	public RecordWriter(AbstractInvokable invokable, ChannelSelector<T> channelSelector) {
 		// initialize the gate
 		super(invokable);
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ExecutionGraphTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ExecutionGraphTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ExecutionGraphTest.java
index fa0653b..2e75305 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ExecutionGraphTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ExecutionGraphTest.java
@@ -28,19 +28,21 @@ import org.apache.log4j.Level;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import eu.stratosphere.api.java.io.DiscardingOuputFormat;
+import eu.stratosphere.api.java.io.TextInputFormat;
 import eu.stratosphere.core.fs.Path;
 import eu.stratosphere.nephele.execution.ExecutionState;
 import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
 import eu.stratosphere.nephele.jobgraph.DistributionPattern;
-import eu.stratosphere.nephele.jobgraph.JobFileInputVertex;
-import eu.stratosphere.nephele.jobgraph.JobFileOutputVertex;
 import eu.stratosphere.nephele.jobgraph.JobGraph;
 import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException;
 import eu.stratosphere.nephele.jobgraph.JobID;
+import eu.stratosphere.nephele.jobgraph.JobInputVertex;
+import eu.stratosphere.nephele.jobgraph.JobOutputVertex;
 import eu.stratosphere.nephele.jobgraph.JobTaskVertex;
-import eu.stratosphere.nephele.util.FileLineReader;
-import eu.stratosphere.nephele.util.FileLineWriter;
 import eu.stratosphere.nephele.util.ServerTestUtils;
+import eu.stratosphere.pact.runtime.task.DataSinkTask;
+import eu.stratosphere.pact.runtime.task.DataSourceTask;
 import eu.stratosphere.runtime.io.channels.ChannelType;
 import eu.stratosphere.util.LogUtils;
 
@@ -49,6 +51,7 @@ import eu.stratosphere.util.LogUtils;
  * 
  */
 public class ExecutionGraphTest {
+	
 	@BeforeClass
 	public static void reduceLogLevel() {
 		LogUtils.initializeDefaultConsoleLogger(Level.WARN);
@@ -76,18 +79,21 @@ public class ExecutionGraphTest {
 			jobID = jg.getJobID();
 
 			// input vertex
-			final JobFileInputVertex i1 = new JobFileInputVertex("Input 1", jg);
-			i1.setFileInputClass(FileLineReader.class);
-			i1.setFilePath(new Path(inputFile.toURI()));
+			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.setTaskClass(ForwardTask1Input1Output.class);
+			t1.setInvokableClass(ForwardTask1Input1Output.class);
 
 			// output vertex
-			final JobFileOutputVertex o1 = new JobFileOutputVertex("Output 1", jg);
-			o1.setFileOutputClass(FileLineWriter.class);
-			o1.setFilePath(new Path(new File(ServerTestUtils.getRandomFilename()).toURI()));
+			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);
@@ -171,7 +177,7 @@ public class ExecutionGraphTest {
 			assertEquals(0, egv0.getNumberOfBackwardLinks());
 			assertEquals(1, egv0.getNumberOfForwardLinks());
 			assertEquals(0, egv0.getStageNumber());
-			assertEquals(-1, egv0.getUserDefinedNumberOfMembers());
+			assertEquals(1, egv0.getUserDefinedNumberOfMembers());
 			assertEquals("Task 1", egv0.getVertexToShareInstancesWith().getName());
 
 			// egv1 (output1)
@@ -189,7 +195,7 @@ public class ExecutionGraphTest {
 			assertEquals(1, egv1.getNumberOfBackwardLinks());
 			assertEquals(0, egv1.getNumberOfForwardLinks());
 			assertEquals(0, egv1.getStageNumber());
-			assertEquals(-1, egv1.getUserDefinedNumberOfMembers());
+			assertEquals(1, egv1.getUserDefinedNumberOfMembers());
 			assertEquals("Input 1", egv1.getVertexToShareInstancesWith().getName());
 
 			// egv2 (task1)
@@ -278,18 +284,20 @@ public class ExecutionGraphTest {
 			jobID = jg.getJobID();
 
 			// input vertex
-			final JobFileInputVertex i1 = new JobFileInputVertex("Input 1", jg);
-			i1.setFileInputClass(FileLineReader.class);
-			i1.setFilePath(new Path(inputFile.toURI()));
+			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.setTaskClass(ForwardTask1Input1Output.class);
+			t1.setInvokableClass(ForwardTask1Input1Output.class);
 
 			// output vertex
-			final JobFileOutputVertex o1 = new JobFileOutputVertex("Output 1", jg);
-			o1.setFileOutputClass(FileLineWriter.class);
-			o1.setFilePath(new Path(new File(ServerTestUtils.getRandomFilename()).toURI()));
+			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);
@@ -381,31 +389,32 @@ public class ExecutionGraphTest {
 			jobID = jg.getJobID();
 
 			// input vertex
-			final JobFileInputVertex i1 = new JobFileInputVertex("Input 1", jg);
-			i1.setFileInputClass(FileLineReader.class);
-			i1.setFilePath(new Path(inputFile1.toURI()));
+			final JobInputVertex i1 = new JobInputVertex("Input 1", jg);
+			i1.setInvokableClass(DataSourceTask.class);
+			i1.setInputFormat(new TextInputFormat(new Path(inputFile1.toURI())));
 			i1.setNumberOfSubtasks(2);
-			final JobFileInputVertex i2 = new JobFileInputVertex("Input 2", jg);
-			i2.setFileInputClass(FileLineReader.class);
-			i2.setFilePath(new Path(inputFile2.toURI()));
+			
+			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.setTaskClass(ForwardTask1Input1Output.class);
+			t1.setInvokableClass(ForwardTask1Input1Output.class);
 			t1.setNumberOfSubtasks(2);
 			final JobTaskVertex t2 = new JobTaskVertex("Task 2", jg);
-			t2.setTaskClass(ForwardTask1Input1Output.class);
+			t2.setInvokableClass(ForwardTask1Input1Output.class);
 			t2.setNumberOfSubtasks(2);
 			final JobTaskVertex t3 = new JobTaskVertex("Task 3", jg);
-			t3.setTaskClass(ForwardTask2Inputs1Output.class);
+			t3.setInvokableClass(ForwardTask2Inputs1Output.class);
 			t3.setNumberOfSubtasks(2);
 
 			
 			// output vertex
-			final JobFileOutputVertex o1 = new JobFileOutputVertex("Output 1", jg);
-			o1.setFileOutputClass(FileLineWriter.class);
-			o1.setFilePath(new Path(outputFile.toURI()));
+			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);
@@ -624,35 +633,35 @@ public class ExecutionGraphTest {
 			jobID = jg.getJobID();
 
 			// input vertex
-			final JobFileInputVertex i1 = new JobFileInputVertex("Input 1", jg);
-			i1.setFileInputClass(FileLineReader.class);
-			i1.setFilePath(new Path(inputFile1.toURI()));
+			final JobInputVertex i1 = new JobInputVertex("Input 1", jg);
+			i1.setInvokableClass(DataSourceTask.class);
+			i1.setInputFormat(new TextInputFormat(new Path(inputFile1.toURI())));
 			i1.setNumberOfSubtasks(4);
-			final JobFileInputVertex i2 = new JobFileInputVertex("Input 2", jg);
-			i2.setFileInputClass(FileLineReader.class);
-			i2.setFilePath(new Path(inputFile2.toURI()));
+			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.setTaskClass(ForwardTask1Input1Output.class);
+			t1.setInvokableClass(ForwardTask1Input1Output.class);
 			t1.setNumberOfSubtasks(4);
 			final JobTaskVertex t2 = new JobTaskVertex("Task 2", jg);
-			t2.setTaskClass(ForwardTask1Input1Output.class);
+			t2.setInvokableClass(ForwardTask1Input1Output.class);
 			t2.setNumberOfSubtasks(4);
 			final JobTaskVertex t3 = new JobTaskVertex("Task 3", jg);
-			t3.setTaskClass(ForwardTask2Inputs1Output.class);
+			t3.setInvokableClass(ForwardTask2Inputs1Output.class);
 			t3.setNumberOfSubtasks(8);
 			final JobTaskVertex t4 = new JobTaskVertex("Task 4", jg);
-			t4.setTaskClass(ForwardTask1Input2Outputs.class);
+			t4.setInvokableClass(ForwardTask1Input2Outputs.class);
 			t4.setNumberOfSubtasks(8);
 			// output vertex
-			final JobFileOutputVertex o1 = new JobFileOutputVertex("Output 1", jg);
-			o1.setFileOutputClass(FileLineWriter.class);
-			o1.setFilePath(new Path(outputFile1.toURI()));
+			final JobOutputVertex o1 = new JobOutputVertex("Output 1", jg);
+			o1.setInvokableClass(DataSinkTask.class);
+			o1.setOutputFormat(new DiscardingOuputFormat<Object>());
 			o1.setNumberOfSubtasks(4);
-			final JobFileOutputVertex o2 = new JobFileOutputVertex("Output 2", jg);
-			o2.setFileOutputClass(FileLineWriter.class);
-			o2.setFilePath(new Path(outputFile2.toURI()));
+			final JobOutputVertex o2 = new JobOutputVertex("Output 2", jg);
+			o2.setInvokableClass(DataSinkTask.class);
+			o2.setOutputFormat(new DiscardingOuputFormat<Object>());
 			o2.setNumberOfSubtasks(4);
 			o1.setVertexToShareInstancesWith(o2);
 
@@ -690,11 +699,8 @@ public class ExecutionGraphTest {
 				ev.updateExecutionState(ExecutionState.FINISHING);
 				ev.updateExecutionState(ExecutionState.FINISHED);
 			}
-		} catch (GraphConversionException e) {
-			fail(e.getMessage());
-		} catch (JobGraphDefinitionException e) {
-			fail(e.getMessage());
-		} catch (IOException e) {
+		} catch (Exception e) {
+			e.printStackTrace();
 			fail(e.getMessage());
 		} finally {
 			if (inputFile1 != null) {
@@ -728,34 +734,33 @@ public class ExecutionGraphTest {
 		final String crossTaskName = "Self Cross Task";
 		final String outputTaskName = "Self Cross Output";
 		final int degreeOfParallelism = 4;
-		File inputFile1 = null;
-		File outputFile1 = null;
+		File inputFile = null;
+		File outputFile = null;
 		JobID jobID = null;
 
 		try {
-
-			inputFile1 = ServerTestUtils.createInputFile(0);
-			outputFile1 = new File(ServerTestUtils.getRandomFilename());
+			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 JobFileInputVertex input = new JobFileInputVertex(inputTaskName, jg);
-			input.setFileInputClass(SelfCrossInputTask.class);
-			input.setFilePath(new Path(inputFile1.toURI()));
+			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.setTaskClass(SelfCrossForwardTask.class);
+			cross.setInvokableClass(SelfCrossForwardTask.class);
 			cross.setNumberOfSubtasks(degreeOfParallelism);
 
 			// output vertex
-			final JobFileOutputVertex output = new JobFileOutputVertex(outputTaskName, jg);
-			output.setFileOutputClass(FileLineWriter.class);
-			output.setFilePath(new Path(outputFile1.toURI()));
+			final JobOutputVertex output = new JobOutputVertex(outputTaskName, jg);
+			output.setInvokableClass(DataSinkTask.class);
+			output.setOutputFormat(new DiscardingOuputFormat<Object>());
 			output.setNumberOfSubtasks(degreeOfParallelism);
 
 			// connect vertices
@@ -835,11 +840,11 @@ public class ExecutionGraphTest {
 		} catch (IOException ioe) {
 			fail(ioe.getMessage());
 		} finally {
-			if (inputFile1 != null) {
-				inputFile1.delete();
+			if (inputFile != null) {
+				inputFile.delete();
 			}
-			if (outputFile1 != null) {
-				outputFile1.delete();
+			if (outputFile != null) {
+				outputFile.delete();
 			}
 			if (jobID != null) {
 				try {
@@ -872,30 +877,32 @@ public class ExecutionGraphTest {
 			jobID = jg.getJobID();
 
 			// input vertex
-			final JobFileInputVertex input1 = new JobFileInputVertex("Input 1", jg);
-			input1.setFileInputClass(FileLineReader.class);
-			input1.setFilePath(new Path(inputFile1.toURI()));
+			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.setTaskClass(ForwardTask1Input1Output.class);
+			forward1.setInvokableClass(ForwardTask1Input1Output.class);
 			forward1.setNumberOfSubtasks(degreeOfParallelism);
 
 			// forward vertex 2
 			final JobTaskVertex forward2 = new JobTaskVertex("Forward 2", jg);
-			forward2.setTaskClass(ForwardTask1Input1Output.class);
+			forward2.setInvokableClass(ForwardTask1Input1Output.class);
 			forward2.setNumberOfSubtasks(degreeOfParallelism);
 
 			// forward vertex 3
 			final JobTaskVertex forward3 = new JobTaskVertex("Forward 3", jg);
-			forward3.setTaskClass(ForwardTask1Input1Output.class);
+			forward3.setInvokableClass(ForwardTask1Input1Output.class);
 			forward3.setNumberOfSubtasks(degreeOfParallelism);
 
 			// output vertex
-			final JobFileOutputVertex output1 = new JobFileOutputVertex("Output 1", jg);
-			output1.setFileOutputClass(FileLineWriter.class);
-			output1.setFilePath(new Path(outputFile1.toURI()));
+			final JobOutputVertex output1 = new JobOutputVertex("Output 1", jg);
+			output1.setInvokableClass(DataSinkTask.class);
+			output1.setOutputFormat(new DiscardingOuputFormat<Object>());
 			output1.setNumberOfSubtasks(degreeOfParallelism);
 
 			// connect vertices

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ForwardTask1Input1Output.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ForwardTask1Input1Output.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ForwardTask1Input1Output.java
index 0a2f52b..24f38b5 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ForwardTask1Input1Output.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ForwardTask1Input1Output.java
@@ -14,11 +14,11 @@
 package eu.stratosphere.nephele.executiongraph;
 
 import eu.stratosphere.core.io.StringRecord;
+import eu.stratosphere.nephele.template.AbstractInvokable;
 import eu.stratosphere.runtime.io.api.RecordReader;
 import eu.stratosphere.runtime.io.api.RecordWriter;
-import eu.stratosphere.nephele.template.AbstractTask;
 
-public class ForwardTask1Input1Output extends AbstractTask {
+public class ForwardTask1Input1Output extends AbstractInvokable {
 
 	private RecordReader<StringRecord> input = null;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ForwardTask1Input2Outputs.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ForwardTask1Input2Outputs.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ForwardTask1Input2Outputs.java
index 5a5c325..370d0e4 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ForwardTask1Input2Outputs.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ForwardTask1Input2Outputs.java
@@ -16,9 +16,9 @@ package eu.stratosphere.nephele.executiongraph;
 import eu.stratosphere.core.io.StringRecord;
 import eu.stratosphere.runtime.io.api.RecordReader;
 import eu.stratosphere.runtime.io.api.RecordWriter;
-import eu.stratosphere.nephele.template.AbstractTask;
+import eu.stratosphere.nephele.template.AbstractInvokable;
 
-public class ForwardTask1Input2Outputs extends AbstractTask {
+public class ForwardTask1Input2Outputs extends AbstractInvokable {
 
 	private RecordReader<StringRecord> input = null;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ForwardTask2Inputs1Output.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ForwardTask2Inputs1Output.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ForwardTask2Inputs1Output.java
index c87d093..b442dc6 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ForwardTask2Inputs1Output.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ForwardTask2Inputs1Output.java
@@ -14,11 +14,11 @@
 package eu.stratosphere.nephele.executiongraph;
 
 import eu.stratosphere.core.io.StringRecord;
+import eu.stratosphere.nephele.template.AbstractInvokable;
 import eu.stratosphere.runtime.io.api.RecordReader;
 import eu.stratosphere.runtime.io.api.RecordWriter;
-import eu.stratosphere.nephele.template.AbstractTask;
 
-public class ForwardTask2Inputs1Output extends AbstractTask {
+public class ForwardTask2Inputs1Output extends AbstractInvokable {
 
 	private RecordReader<StringRecord> input1 = null;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/SelfCrossForwardTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/SelfCrossForwardTask.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/SelfCrossForwardTask.java
index 05f181c..ac6aeb4 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/SelfCrossForwardTask.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/SelfCrossForwardTask.java
@@ -17,27 +17,20 @@ package eu.stratosphere.nephele.executiongraph;
 import eu.stratosphere.core.io.StringRecord;
 import eu.stratosphere.runtime.io.api.RecordReader;
 import eu.stratosphere.runtime.io.api.RecordWriter;
-import eu.stratosphere.nephele.template.AbstractTask;
+import eu.stratosphere.nephele.template.AbstractInvokable;
 
 /**
  * This class represents the cross task in the self cross unit test.
- * 
  */
-public class SelfCrossForwardTask extends AbstractTask {
-
+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() throws Exception {
-		
-		//Nothing to do here
-	}
+	public void invoke() {}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/DoubleSourceTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/DoubleSourceTask.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/DoubleSourceTask.java
new file mode 100644
index 0000000..0f24438
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/DoubleSourceTask.java
@@ -0,0 +1,132 @@
+/***********************************************************************************************************************
+ * 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.jobmanager;
+
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+import eu.stratosphere.core.fs.FSDataInputStream;
+import eu.stratosphere.core.fs.FileInputSplit;
+import eu.stratosphere.core.fs.FileSystem;
+import eu.stratosphere.core.io.StringRecord;
+import eu.stratosphere.nephele.template.AbstractInvokable;
+import eu.stratosphere.nephele.template.InputSplitProvider;
+import eu.stratosphere.runtime.io.api.RecordWriter;
+import eu.stratosphere.runtime.fs.LineReader;
+
+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/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/DoubleTargetTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/DoubleTargetTask.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/DoubleTargetTask.java
index a1ce0b2..5edfe0b 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/DoubleTargetTask.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/DoubleTargetTask.java
@@ -13,18 +13,18 @@
 
 package eu.stratosphere.nephele.jobmanager;
 
+import eu.stratosphere.core.io.StringRecord;
+import eu.stratosphere.nephele.template.AbstractInvokable;
 import eu.stratosphere.runtime.io.api.RecordReader;
 import eu.stratosphere.runtime.io.api.RecordWriter;
-import eu.stratosphere.nephele.template.AbstractTask;
-import eu.stratosphere.types.Record;
 
-public class DoubleTargetTask extends AbstractTask {
+public class DoubleTargetTask extends AbstractInvokable {
 
-	private RecordReader<Record> input1 = null;
+	private RecordReader<StringRecord> input1 = null;
 
-	private RecordReader<Record> input2 = null;
+	private RecordReader<StringRecord> input2 = null;
 
-	private RecordWriter<Record> output = null;
+	private RecordWriter<StringRecord> output = null;
 
 	@Override
 	public void invoke() throws Exception {
@@ -33,13 +33,13 @@ public class DoubleTargetTask extends AbstractTask {
 
 		while (this.input1.hasNext()) {
 
-			Record s = input1.next();
+			StringRecord s = input1.next();
 			this.output.emit(s);
 		}
 
 		while (this.input2.hasNext()) {
 
-			Record s = input2.next();
+			StringRecord s = input2.next();
 			this.output.emit(s);
 		}
 
@@ -49,9 +49,9 @@ public class DoubleTargetTask extends AbstractTask {
 
 	@Override
 	public void registerInputOutput() {
-		this.input1 = new RecordReader<Record>(this, Record.class);
-		this.input2 = new RecordReader<Record>(this, Record.class);
-		this.output = new RecordWriter<Record>(this);
+		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/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ExceptionOutputFormat.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ExceptionOutputFormat.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ExceptionOutputFormat.java
index ffc4b42..e2e09c3 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ExceptionOutputFormat.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ExceptionOutputFormat.java
@@ -13,41 +13,37 @@
 
 package eu.stratosphere.nephele.jobmanager;
 
+import eu.stratosphere.api.common.io.InitializeOnMaster;
 import eu.stratosphere.api.common.io.OutputFormat;
 import eu.stratosphere.configuration.Configuration;
+import eu.stratosphere.core.io.StringRecord;
 
 import java.io.IOException;
 
 
-public class ExceptionOutputFormat implements OutputFormat<Object> {
+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) {
-
-	}
+	public void configure(Configuration parameters) {}
 
 	@Override
-	public void open(int taskNumber, int numTasks) throws IOException {
-
-	}
+	public void open(int taskNumber, int numTasks) {}
 
 	@Override
-	public void writeRecord(Object record) throws IOException {
-
-	}
+	public void writeRecord(StringRecord record) {}
 
 	@Override
-	public void close() throws IOException {
-
-	}
+	public void close() {}
 
 	@Override
-	public void initialize(Configuration configuration) {
+	public void initializeGlobal(int parallelism) throws IOException {
 		throw new RuntimeException(RUNTIME_EXCEPTION_MESSAGE);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ExceptionTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ExceptionTask.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ExceptionTask.java
index 77b4f96..9f4bcdf 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ExceptionTask.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ExceptionTask.java
@@ -14,16 +14,15 @@
 package eu.stratosphere.nephele.jobmanager;
 
 import eu.stratosphere.core.io.StringRecord;
+import eu.stratosphere.nephele.template.AbstractInvokable;
 import eu.stratosphere.runtime.io.api.RecordReader;
 import eu.stratosphere.runtime.io.api.RecordWriter;
-import eu.stratosphere.nephele.template.AbstractTask;
 
 /**
  * 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 AbstractTask {
+public class ExceptionTask extends AbstractInvokable {
 
 	/**
 	 * The test error message included in the thrown exception
@@ -52,20 +51,14 @@ public class ExceptionTask extends AbstractTask {
 		}
 	}
 
-
 	@Override
 	public void registerInputOutput() {
-
 		new RecordReader<StringRecord>(this, StringRecord.class);
 		new RecordWriter<StringRecord>(this);
 	}
 
-
 	@Override
 	public void invoke() throws Exception {
-
-		// Throw the exception immediately
 		throw new TestException(ERROR_MESSAGE);
 	}
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ForwardTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ForwardTask.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ForwardTask.java
index 377e304..e85b5f1 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ForwardTask.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ForwardTask.java
@@ -13,15 +13,15 @@
 
 package eu.stratosphere.nephele.jobmanager;
 
+import eu.stratosphere.core.io.StringRecord;
+import eu.stratosphere.nephele.template.AbstractInvokable;
 import eu.stratosphere.runtime.io.api.RecordReader;
 import eu.stratosphere.runtime.io.api.RecordWriter;
-import eu.stratosphere.nephele.template.AbstractTask;
-import eu.stratosphere.types.Record;
 
-public class ForwardTask extends AbstractTask {
+public class ForwardTask extends AbstractInvokable {
 
-	private RecordReader<Record> input = null;
-	private RecordWriter<Record> output = null;
+	private RecordReader<StringRecord> input = null;
+	private RecordWriter<StringRecord> output = null;
 
 	@Override
 	public void invoke() throws Exception {
@@ -30,7 +30,7 @@ public class ForwardTask extends AbstractTask {
 
 		while (this.input.hasNext()) {
 
-			Record s = input.next();
+			StringRecord s = input.next();
 			this.output.emit(s);
 		}
 
@@ -39,7 +39,7 @@ public class ForwardTask extends AbstractTask {
 
 	@Override
 	public void registerInputOutput() {
-		this.input = new RecordReader<Record>(this, Record.class);
-		this.output = new RecordWriter<Record>(this);
+		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/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java
index db2d9af..2549d4f 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java
@@ -13,6 +13,25 @@
 
 package eu.stratosphere.nephele.jobmanager;
 
+import static org.junit.Assert.assertEquals;
+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 org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import eu.stratosphere.api.common.io.OutputFormat;
+import eu.stratosphere.api.common.operators.util.UserCodeObjectWrapper;
 import eu.stratosphere.configuration.ConfigConstants;
 import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.configuration.GlobalConfiguration;
@@ -22,37 +41,24 @@ import eu.stratosphere.nephele.client.JobClient;
 import eu.stratosphere.nephele.client.JobExecutionException;
 import eu.stratosphere.nephele.execution.RuntimeEnvironment;
 import eu.stratosphere.nephele.jobgraph.DistributionPattern;
-import eu.stratosphere.runtime.io.channels.ChannelType;
-import eu.stratosphere.nephele.jobgraph.JobFileInputVertex;
-import eu.stratosphere.nephele.jobgraph.JobFileOutputVertex;
 import eu.stratosphere.nephele.jobgraph.JobGraph;
 import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException;
+import eu.stratosphere.nephele.jobgraph.JobOutputVertex;
 import eu.stratosphere.nephele.jobgraph.JobTaskVertex;
 import eu.stratosphere.nephele.taskmanager.Task;
 import eu.stratosphere.nephele.taskmanager.TaskManager;
-import eu.stratosphere.nephele.util.FileLineReader;
-import eu.stratosphere.nephele.util.FileLineWriter;
 import eu.stratosphere.nephele.util.JarFileCreator;
 import eu.stratosphere.nephele.util.ServerTestUtils;
+import eu.stratosphere.nephele.util.tasks.DoubleSourceTask;
+import eu.stratosphere.nephele.util.tasks.FileLineReader;
+import eu.stratosphere.nephele.util.tasks.FileLineWriter;
+import eu.stratosphere.nephele.util.tasks.JobFileInputVertex;
+import eu.stratosphere.nephele.util.tasks.JobFileOutputVertex;
+import eu.stratosphere.pact.runtime.task.DataSinkTask;
+import eu.stratosphere.pact.runtime.task.util.TaskConfig;
+import eu.stratosphere.runtime.io.channels.ChannelType;
 import eu.stratosphere.util.LogUtils;
 
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-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 static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
 /**
  * This test is intended to cover the basic functionality of the {@link JobManager}.
  */
@@ -170,23 +176,23 @@ public class JobManagerITCase {
 
 			// input vertex
 			final JobFileInputVertex i1 = new JobFileInputVertex("Input 1", jg);
-			i1.setFileInputClass(FileLineReader.class);
+			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.setTaskClass(ForwardTask.class);
+			t1.setInvokableClass(ForwardTask.class);
 			t1.setNumberOfSubtasks(1);
 
 			// task vertex 2
 			final JobTaskVertex t2 = new JobTaskVertex("Task 2", jg);
-			t2.setTaskClass(ForwardTask.class);
+			t2.setInvokableClass(ForwardTask.class);
 			t2.setNumberOfSubtasks(1);
 
 			// output vertex
 			JobFileOutputVertex o1 = new JobFileOutputVertex("Output 1", jg);
-			o1.setFileOutputClass(FileLineWriter.class);
+			o1.setInvokableClass(FileLineWriter.class);
 			o1.setFilePath(new Path(outputFile.toURI()));
 			o1.setNumberOfSubtasks(1);
 
@@ -282,16 +288,16 @@ public class JobManagerITCase {
 
 			// input vertex
 			final JobFileInputVertex i1 = new JobFileInputVertex("Input 1", jg);
-			i1.setFileInputClass(FileLineReader.class);
+			i1.setInvokableClass(FileLineReader.class);
 			i1.setFilePath(new Path(inputFile.toURI()));
 
 			// task vertex 1
 			final JobTaskVertex t1 = new JobTaskVertex("Task with Exception", jg);
-			t1.setTaskClass(ExceptionTask.class);
+			t1.setInvokableClass(ExceptionTask.class);
 
 			// output vertex
 			JobFileOutputVertex o1 = new JobFileOutputVertex("Output 1", jg);
-			o1.setFileOutputClass(FileLineWriter.class);
+			o1.setInvokableClass(FileLineWriter.class);
 			o1.setFilePath(new Path(outputFile.toURI()));
 
 			t1.setVertexToShareInstancesWith(i1);
@@ -330,10 +336,9 @@ public class JobManagerITCase {
 
 			fail("Expected exception but did not receive it");
 
-		} catch (JobGraphDefinitionException jgde) {
-			fail(jgde.getMessage());
-		} catch (IOException ioe) {
-			fail(ioe.getMessage());
+		} catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
 		} finally {
 
 			// Remove temporary files
@@ -376,16 +381,16 @@ public class JobManagerITCase {
 
 			// input vertex
 			final JobFileInputVertex i1 = new JobFileInputVertex("Input 1", jg);
-			i1.setFileInputClass(FileLineReader.class);
+			i1.setInvokableClass(FileLineReader.class);
 			i1.setFilePath(new Path(inputFile.toURI()));
 
 			// task vertex 1
 			final JobTaskVertex t1 = new JobTaskVertex("Task with Exception", jg);
-			t1.setTaskClass(RuntimeExceptionTask.class);
+			t1.setInvokableClass(RuntimeExceptionTask.class);
 
 			// output vertex
 			JobFileOutputVertex o1 = new JobFileOutputVertex("Output 1", jg);
-			o1.setFileOutputClass(FileLineWriter.class);
+			o1.setInvokableClass(FileLineWriter.class);
 			o1.setFilePath(new Path(outputFile.toURI()));
 
 			t1.setVertexToShareInstancesWith(i1);
@@ -472,32 +477,28 @@ public class JobManagerITCase {
 			final JobGraph jg = new JobGraph("Job Graph for Exception Test");
 
 			// input vertex
-			final JobInputVertex i1 = new JobInputVertex("Input 1", jg);
+			final JobFileInputVertex i1 = new JobFileInputVertex("Input 1", jg);
+			i1.setInvokableClass(FileLineReader.class);
+			i1.setFilePath(new Path(inputFile.toURI()));
 			i1.setNumberOfSubtasks(1);
-			Class<AbstractInputTask<?>> clazz = (Class<AbstractInputTask<?>>)(Class<?>)DataSourceTask
-					.class;
-			i1.setInputClass(clazz);
-			TextInputFormat inputFormat = new TextInputFormat();
-			inputFormat.setFilePath(new Path(inputFile.toURI()));
-			i1.setInputFormat(inputFormat);
-			i1.setInputFormat(inputFormat);
-			i1.setOutputSerializer(RecordSerializerFactory.get());
-			TaskConfig config= new TaskConfig(i1.getConfiguration());
-			config.addOutputShipStrategy(ShipStrategyType.FORWARD);
 
 			// task vertex 1
 			final JobTaskVertex t1 = new JobTaskVertex("Task with Exception", jg);
-			t1.setTaskClass(ForwardTask.class);
+			t1.setInvokableClass(ForwardTask.class);
 
 			// output vertex
 			JobOutputVertex o1 = new JobOutputVertex("Output 1", jg);
 			o1.setNumberOfSubtasks(1);
-			o1.setOutputClass(DataSinkTask.class);
+			o1.setInvokableClass(DataSinkTask.class);
 			ExceptionOutputFormat outputFormat = new ExceptionOutputFormat();
 			o1.setOutputFormat(outputFormat);
 			TaskConfig outputConfig = new TaskConfig(o1.getConfiguration());
-			outputConfig.addInputToGroup(0);
-			outputConfig.setInputSerializer(RecordSerializerFactory.get(), 0);
+			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);
@@ -591,23 +592,23 @@ public class JobManagerITCase {
 
 			// input vertex
 			final JobFileInputVertex i1 = new JobFileInputVertex("Input 1", jg);
-			i1.setFileInputClass(FileLineReader.class);
+			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.setTaskClass(ForwardTask.class);
+			t1.setInvokableClass(ForwardTask.class);
 			t1.setNumberOfSubtasks(1);
 
 			// task vertex 2
 			final JobTaskVertex t2 = new JobTaskVertex("Task 2", jg);
-			t2.setTaskClass(ForwardTask.class);
+			t2.setInvokableClass(ForwardTask.class);
 			t2.setNumberOfSubtasks(1);
 
 			// output vertex
 			JobFileOutputVertex o1 = new JobFileOutputVertex("Output 1", jg);
-			o1.setFileOutputClass(FileLineWriter.class);
+			o1.setInvokableClass(FileLineWriter.class);
 			o1.setFilePath(new Path(outputFile.toURI()));
 			o1.setNumberOfSubtasks(1);
 
@@ -620,8 +621,9 @@ public class JobManagerITCase {
 				i1.connectTo(t1, ChannelType.NETWORK);
 				t1.connectTo(t2, ChannelType.IN_MEMORY);
 				t2.connectTo(o1, ChannelType.IN_MEMORY);
-			} catch (JobGraphDefinitionException e) {
+			} catch (Exception e) {
 				e.printStackTrace();
+				fail(e.getMessage());
 			}
 
 			// add jar
@@ -693,16 +695,16 @@ public class JobManagerITCase {
 
 			// input vertex
 			final JobFileInputVertex i1 = new JobFileInputVertex("Input with two Outputs", jg);
-			i1.setFileInputClass(DoubleSourceTask.class);
+			i1.setInvokableClass(DoubleSourceTask.class);
 			i1.setFilePath(new Path(inputFile.toURI()));
 
 			// task vertex 1
 			final JobTaskVertex t1 = new JobTaskVertex("Task with two Inputs", jg);
-			t1.setTaskClass(DoubleTargetTask.class);
+			t1.setInvokableClass(DoubleTargetTask.class);
 
 			// output vertex
 			JobFileOutputVertex o1 = new JobFileOutputVertex("Output 1", jg);
-			o1.setFileOutputClass(FileLineWriter.class);
+			o1.setInvokableClass(FileLineWriter.class);
 			o1.setFilePath(new Path(outputFile.toURI()));
 
 			t1.setVertexToShareInstancesWith(i1);
@@ -720,12 +722,9 @@ public class JobManagerITCase {
 			jobClient = new JobClient(jg, configuration);
 			jobClient.submitJobAndWait();
 
-		} catch (JobExecutionException e) {
+		} catch (Exception e) {
+			e.printStackTrace();
 			fail(e.getMessage());
-		} catch (JobGraphDefinitionException jgde) {
-			fail(jgde.getMessage());
-		} catch (IOException ioe) {
-			fail(ioe.getMessage());
 		} finally {
 
 			// Remove temporary files
@@ -772,12 +771,12 @@ public class JobManagerITCase {
 
 			// input vertex
 			final JobFileInputVertex i1 = new JobFileInputVertex(jg);
-			i1.setFileInputClass(FileLineReader.class);
+			i1.setInvokableClass(FileLineReader.class);
 			i1.setFilePath(new Path(inputFile.toURI()));
 
 			// output vertex
 			JobFileOutputVertex o1 = new JobFileOutputVertex(jg);
-			o1.setFileOutputClass(FileLineWriter.class);
+			o1.setInvokableClass(FileLineWriter.class);
 			o1.setFilePath(new Path(outputFile.toURI()));
 
 			o1.setVertexToShareInstancesWith(i1);
@@ -791,13 +790,9 @@ public class JobManagerITCase {
 			// Create job client and launch job
 			jobClient = new JobClient(jg, configuration);
 			jobClient.submitJobAndWait();
-
-		} catch (JobExecutionException e) {
+		} catch (Exception e) {
+			e.printStackTrace();
 			fail(e.getMessage());
-		} catch (JobGraphDefinitionException jgde) {
-			fail(jgde.getMessage());
-		} catch (IOException ioe) {
-			fail(ioe.getMessage());
 		} finally {
 
 			// Remove temporary files
@@ -855,21 +850,21 @@ public class JobManagerITCase {
 
 			// input vertex 1
 			final JobFileInputVertex i1 = new JobFileInputVertex("Input 1", jg);
-			i1.setFileInputClass(FileLineReader.class);
+			i1.setInvokableClass(FileLineReader.class);
 			i1.setFilePath(new Path(inputFile1.toURI()));
 
 			// input vertex 2
 			final JobFileInputVertex i2 = new JobFileInputVertex("Input 2", jg);
-			i2.setFileInputClass(FileLineReader.class);
+			i2.setInvokableClass(FileLineReader.class);
 			i2.setFilePath(new Path(inputFile2.toURI()));
 
 			// union task
 			final JobTaskVertex u1 = new JobTaskVertex("Union", jg);
-			u1.setTaskClass(UnionTask.class);
+			u1.setInvokableClass(UnionTask.class);
 
 			// output vertex
 			JobFileOutputVertex o1 = new JobFileOutputVertex("Output", jg);
-			o1.setFileOutputClass(FileLineWriter.class);
+			o1.setInvokableClass(FileLineWriter.class);
 			o1.setFilePath(new Path(outputFile.toURI()));
 			o1.setNumberOfSubtasks(1);
 
@@ -999,24 +994,24 @@ public class JobManagerITCase {
 
 			// input vertex 1
 			final JobFileInputVertex i1 = new JobFileInputVertex("Input 1", jg);
-			i1.setFileInputClass(FileLineReader.class);
+			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.setFileInputClass(FileLineReader.class);
+			i2.setInvokableClass(FileLineReader.class);
 			i2.setFilePath(new Path(inputFile2.toURI()));
 			i2.setNumberOfSubtasks(numberOfSubtasks);
 
 			// union task
 			final JobTaskVertex f1 = new JobTaskVertex("Forward 1", jg);
-			f1.setTaskClass(DoubleTargetTask.class);
+			f1.setInvokableClass(DoubleTargetTask.class);
 			f1.setNumberOfSubtasks(numberOfSubtasks);
 
 			// output vertex
 			JobFileOutputVertex o1 = new JobFileOutputVertex("Output", jg);
-			o1.setFileOutputClass(FileLineWriter.class);
+			o1.setInvokableClass(FileLineWriter.class);
 			o1.setFilePath(new Path(outputFile.toURI()));
 			o1.setNumberOfSubtasks(numberOfSubtasks);
 
@@ -1051,6 +1046,9 @@ public class JobManagerITCase {
 			} catch (JobExecutionException e) {
 				// Job execution should lead to an error due to lack of resources
 				return;
+			} catch (Exception e) {
+				e.printStackTrace();
+				fail(e.getMessage());
 			}
 			finally {
 				tmLogger.setLevel(tmLevel);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/RuntimeExceptionTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/RuntimeExceptionTask.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/RuntimeExceptionTask.java
index 9376099..ce20431 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/RuntimeExceptionTask.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/RuntimeExceptionTask.java
@@ -13,13 +13,12 @@
 
 package eu.stratosphere.nephele.jobmanager;
 
-import eu.stratosphere.nephele.template.AbstractTask;
+import eu.stratosphere.nephele.template.AbstractInvokable;
 
 /**
  * This task throws a {@link RuntimeException} when the method <code>registerInputOutput</code> is called.
- * 
  */
-public class RuntimeExceptionTask extends AbstractTask {
+public class RuntimeExceptionTask extends AbstractInvokable {
 
 	/**
 	 * The message which is used for the test runtime exception.
@@ -29,15 +28,9 @@ public class RuntimeExceptionTask extends AbstractTask {
 
 	@Override
 	public void registerInputOutput() {
-
 		throw new RuntimeException(RUNTIME_EXCEPTION_MESSAGE);
 	}
 
-
 	@Override
-	public void invoke() throws Exception {
-
-		// Nothing to do here
-	}
-
+	public void invoke() {}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/UnionTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/UnionTask.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/UnionTask.java
index 209eff1..f21c60e 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/UnionTask.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/UnionTask.java
@@ -14,36 +14,34 @@
 package eu.stratosphere.nephele.jobmanager;
 
 import eu.stratosphere.core.io.StringRecord;
+import eu.stratosphere.nephele.template.AbstractInvokable;
 import eu.stratosphere.runtime.io.api.MutableRecordReader;
 import eu.stratosphere.runtime.io.api.RecordWriter;
 import eu.stratosphere.runtime.io.api.UnionRecordReader;
-import eu.stratosphere.nephele.template.AbstractTask;
-import eu.stratosphere.types.Record;
 
 /**
  * A simple implementation of a task using a {@link UnionRecordReader}.
  */
-public class UnionTask extends AbstractTask {
+public class UnionTask extends AbstractInvokable {
 
 	/**
 	 * The union record reader to be used during the tests.
 	 */
-	private UnionRecordReader<Record> unionReader;
+	private UnionRecordReader<StringRecord> unionReader;
 
-	private RecordWriter<Record> writer;
+	private RecordWriter<StringRecord> writer;
 	
 	
 	@Override
 	public void registerInputOutput() {
 
 		@SuppressWarnings("unchecked")
-		MutableRecordReader<Record>[] recordReaders = (MutableRecordReader<Record>[]) new
-				MutableRecordReader<?>[2];
-		recordReaders[0] = new MutableRecordReader<Record>(this);
-		recordReaders[1] = new MutableRecordReader<Record>(this);
-		this.unionReader = new UnionRecordReader<Record>(recordReaders, Record.class);
+		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<Record>(this);
+		this.writer = new RecordWriter<StringRecord>(this);
 	}
 
 	@Override
@@ -56,4 +54,4 @@ public class UnionTask extends AbstractTask {
 
 		this.writer.flush();
 	}
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/DefaultSchedulerTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/DefaultSchedulerTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/DefaultSchedulerTest.java
index 6a41fe9..e5cabb8 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/DefaultSchedulerTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/DefaultSchedulerTest.java
@@ -24,6 +24,9 @@ import eu.stratosphere.nephele.jobmanager.scheduler.DefaultScheduler;
 
 import org.junit.Test;
 
+import eu.stratosphere.api.common.io.GenericInputFormat;
+import eu.stratosphere.api.common.io.OutputFormat;
+import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.core.io.StringRecord;
 import eu.stratosphere.nephele.execution.ExecutionState;
 import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
@@ -35,63 +38,41 @@ import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException;
 import eu.stratosphere.nephele.jobgraph.JobInputVertex;
 import eu.stratosphere.nephele.jobgraph.JobOutputVertex;
 import eu.stratosphere.nephele.jobmanager.scheduler.SchedulingException;
-import eu.stratosphere.nephele.template.AbstractGenericInputTask;
-import eu.stratosphere.nephele.template.AbstractOutputTask;
+import eu.stratosphere.nephele.template.AbstractInvokable;
 import eu.stratosphere.runtime.io.api.RecordReader;
 import eu.stratosphere.runtime.io.api.RecordWriter;
 import eu.stratosphere.runtime.io.channels.ChannelType;
+import eu.stratosphere.types.IntValue;
 import eu.stratosphere.util.StringUtils;
 
 /**
- *         This class checks the functionality of the {@link eu.stratosphere.nephele.jobmanager.scheduler.DefaultScheduler} class
+ * This class checks the functionality of the {@link eu.stratosphere.nephele.jobmanager.scheduler.DefaultScheduler} class
  */
+@SuppressWarnings("serial")
 public class DefaultSchedulerTest {
 
-	/**
-	 * Test input task.
-	 * 
-	 */
-	public static final class InputTask extends AbstractGenericInputTask {
 
-		/**
-		 * {@inheritDoc}
-		 */
+	public static final class InputTask extends AbstractInvokable {
+
 		@Override
 		public void registerInputOutput() {
 			new RecordWriter<StringRecord>(this);
 		}
 
-		/**
-		 * {@inheritDoc}
-		 */
 		@Override
-		public void invoke() throws Exception {
-			// Nothing to do here
-		}
+		public void invoke() throws Exception {}
 
 	}
 
-	/**
-	 * Test output task.
-	 * 
-	 */
-	public static final class OutputTask extends AbstractOutputTask {
+	public static final class OutputTask extends AbstractInvokable {
 
-		/**
-		 * {@inheritDoc}
-		 */
 		@Override
 		public void registerInputOutput() {
 			new RecordReader<StringRecord>(this, StringRecord.class);
 		}
 
-		/**
-		 * {@inheritDoc}
-		 */
 		@Override
-		public void invoke() throws Exception {
-			// Nothing to do here
-		}
+		public void invoke() throws Exception {}
 
 	}
 
@@ -111,29 +92,16 @@ public class DefaultSchedulerTest {
 	public static final class DummyOutputFormat implements OutputFormat<IntValue> {
 
 		@Override
-		public void configure(Configuration parameters) {
-
-		}
+		public void configure(Configuration parameters) {}
 
 		@Override
-		public void open(int taskNumber, int numTasks) throws IOException {
-
-		}
-
-		@Override
-		public void writeRecord(IntValue record) throws IOException {
-
-		}
+		public void open(int taskNumber, int numTasks) {}
 
 		@Override
-		public void close() throws IOException {
-
-		}
+		public void writeRecord(IntValue record) {}
 
 		@Override
-		public void initialize(Configuration configuration) {
-
-		}
+		public void close() {}
 	}
 
 	/**
@@ -148,12 +116,12 @@ public class DefaultSchedulerTest {
 		final JobGraph jobGraph = new JobGraph("Job Graph");
 
 		final JobInputVertex inputVertex = new JobInputVertex("Input 1", jobGraph);
-		inputVertex.setInputClass(InputTask.class);
+		inputVertex.setInvokableClass(InputTask.class);
 		inputVertex.setInputFormat(new DummyInputFormat());
 		inputVertex.setNumberOfSubtasks(1);
 
 		final JobOutputVertex outputVertex = new JobOutputVertex("Output 1", jobGraph);
-		outputVertex.setOutputClass(OutputTask.class);
+		outputVertex.setInvokableClass(OutputTask.class);
 		outputVertex.setOutputFormat(new DummyOutputFormat());
 		outputVertex.setNumberOfSubtasks(1);
 


[05/53] [abbrv] Rework the Taskmanager to a slot based model and remove legacy cloud code

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildSecondHashMatchIterator.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildSecondHashMatchIterator.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildSecondHashMatchIterator.java
index f1525d8..732d256 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildSecondHashMatchIterator.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildSecondHashMatchIterator.java
@@ -60,7 +60,7 @@ public final class BuildSecondHashMatchIterator<V1, V2, O> implements JoinTaskIt
 			TypeSerializer<V1> serializer1, TypeComparator<V1> comparator1,
 			TypeSerializer<V2> serializer2, TypeComparator<V2> comparator2,
 			TypePairComparator<V1, V2> pairComparator,
-			MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, long totalMemory)
+			MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, double memoryFraction)
 	throws MemoryAllocationException
 	{		
 		this.memManager = memManager;
@@ -73,7 +73,7 @@ public final class BuildSecondHashMatchIterator<V1, V2, O> implements JoinTaskIt
 		this.probeCopy = serializer1.createInstance();
 		
 		this.hashJoin = getHashJoin(serializer2, comparator2, serializer1, comparator1, pairComparator,
-			memManager, ioManager, ownerTask, totalMemory);
+			memManager, ioManager, ownerTask, memoryFraction);
 	}
 	
 	// --------------------------------------------------------------------------------------------
@@ -149,10 +149,10 @@ public final class BuildSecondHashMatchIterator<V1, V2, O> implements JoinTaskIt
 	public <BT, PT> MutableHashTable<BT, PT> getHashJoin(TypeSerializer<BT> buildSideSerializer, TypeComparator<BT> buildSideComparator,
 			TypeSerializer<PT> probeSideSerializer, TypeComparator<PT> probeSideComparator,
 			TypePairComparator<PT, BT> pairComparator,
-			MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, long totalMemory)
+			MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, double memoryFraction)
 	throws MemoryAllocationException
 	{
-		final int numPages = memManager.computeNumberOfPages(totalMemory);
+		final int numPages = memManager.computeNumberOfPages(memoryFraction);
 		final List<MemorySegment> memorySegments = memManager.allocatePages(ownerTask, numPages);
 		return new MutableHashTable<BT, PT>(buildSideSerializer, probeSideSerializer, buildSideComparator, probeSideComparator, pairComparator, memorySegments, ioManager);
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/InMemoryPartition.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/InMemoryPartition.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/InMemoryPartition.java
index 5587ad2..9cc4f49 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/InMemoryPartition.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/InMemoryPartition.java
@@ -310,6 +310,7 @@ public class InMemoryPartition<T> {
 			return posInArray;
 		}
 		
+		@SuppressWarnings("unused")
 		public void setSegmentNumberOffset(int offset) {
 			this.segmentNumberOffset = offset;
 		}
@@ -364,6 +365,7 @@ public class InMemoryPartition<T> {
 			seekInput(this.segments.get(bufferNum), offset, this.segmentSizeMask + 1);
 		}
 		
+		@SuppressWarnings("unused")
 		public void setSegmentNumberOffset(int offset) {
 			this.segmentNumberOffset = offset;
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationHeadPactTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationHeadPactTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationHeadPactTask.java
index 89571c4..b94e276 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationHeadPactTask.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationHeadPactTask.java
@@ -130,7 +130,8 @@ public class IterationHeadPactTask<X, Y, S extends Function, OT> extends Abstrac
 	private BlockingBackChannel initBackChannel() throws Exception {
 
 		/* get the size of the memory available to the backchannel */
-		int backChannelMemoryPages = getMemoryManager().computeNumberOfPages(this.config.getBackChannelMemory());
+		int backChannelMemoryPages = getMemoryManager().computeNumberOfPages(this.config.getRelativeBackChannelMemory
+				());
 
 		/* allocate the memory available to the backchannel */
 		List<MemorySegment> segments = new ArrayList<MemorySegment>();
@@ -150,7 +151,7 @@ public class IterationHeadPactTask<X, Y, S extends Function, OT> extends Abstrac
 	
 	private <BT> CompactingHashTable<BT> initCompactingHashTable() throws Exception {
 		// get some memory
-		long hashjoinMemorySize = config.getSolutionSetMemory();
+		double hashjoinMemorySize = config.getRelativeSolutionSetMemory();
 
 		TypeSerializerFactory<BT> solutionTypeSerializerFactory = config.getSolutionSetSerializer(userCodeClassLoader);
 		TypeComparatorFactory<BT> solutionTypeComparatorFactory = config.getSolutionSetComparator(userCodeClassLoader);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/ShipStrategyType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/ShipStrategyType.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/ShipStrategyType.java
index a8a1293..ba38821 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/ShipStrategyType.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/ShipStrategyType.java
@@ -22,45 +22,42 @@ public enum ShipStrategyType {
 	/**
 	 * Constant used as an indicator for an unassigned ship strategy.
 	 */
-	NONE(false, false, false),
+	NONE(false, false),
 	
 	/**
-	 * Forwarding the data preserving all global properties.
+	 * Forwarding the data locally in memory.
 	 */
-	FORWARD(false, false, false),
+	FORWARD(false, false),
 	
 	/**
 	 * Repartitioning the data randomly, typically when the degree of parallelism between two nodes changes.
 	 */
-	PARTITION_RANDOM(true, true, false),
+	PARTITION_RANDOM(true, false),
 	
 	/**
 	 * Repartitioning the data deterministically through a hash function.
 	 */
-	PARTITION_HASH(true, true, true),
+	PARTITION_HASH(true, true),
 	
 	/**
 	 * Partitioning the data in ranges according to a total order.
 	 */
-	PARTITION_RANGE(true, true, true),
+	PARTITION_RANGE(true, true),
 	
 	/**
 	 * Replicating the data set to all instances.
 	 */
-	BROADCAST(true, true, false);
+	BROADCAST(true, false);
 	
 	// --------------------------------------------------------------------------------------------
 	
 	private final boolean isNetwork;
 	
-	private final boolean compensatesForLocalDOPChanges;
-	
 	private final boolean requiresComparator;
 	
 	
-	private ShipStrategyType(boolean network, boolean compensatesForLocalDOPChanges, boolean requiresComparator) {
+	private ShipStrategyType(boolean network, boolean requiresComparator) {
 		this.isNetwork = network;
-		this.compensatesForLocalDOPChanges = compensatesForLocalDOPChanges;
 		this.requiresComparator = requiresComparator;
 	}
 	
@@ -68,10 +65,6 @@ public enum ShipStrategyType {
 		return this.isNetwork;
 	}
 	
-	public boolean compensatesForLocalDOPChanges() {
-		return this.compensatesForLocalDOPChanges;
-	}
-	
 	public boolean requiresComparator() {
 		return this.requiresComparator;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/AsynchronousPartialSorter.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/AsynchronousPartialSorter.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/AsynchronousPartialSorter.java
index 0cf6bb0..35377cf 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/AsynchronousPartialSorter.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/AsynchronousPartialSorter.java
@@ -50,7 +50,7 @@ public class AsynchronousPartialSorter<E> extends UnilateralSortMerger<E> {
 	 * @param parentTask The parent task, which owns all resources used by this sorter.
 	 * @param serializerFactory The type serializer.
 	 * @param comparator The type comparator establishing the order relation.
-	 * @param totalMemory The total amount of memory dedicated to sorting.
+	 * @param memoryFraction The fraction of memory dedicated to sorting.
 	 * 
 	 * @throws IOException Thrown, if an error occurs initializing the resources for external sorting.
 	 * @throws MemoryAllocationException Thrown, if not enough memory can be obtained from the memory manager to
@@ -59,12 +59,13 @@ public class AsynchronousPartialSorter<E> extends UnilateralSortMerger<E> {
 	public AsynchronousPartialSorter(MemoryManager memoryManager,
 			MutableObjectIterator<E> input, AbstractInvokable parentTask, 
 			TypeSerializerFactory<E> serializerFactory, TypeComparator<E> comparator,
-			long totalMemory)
+			double memoryFraction)
 	throws IOException, MemoryAllocationException
 	{
-		super(memoryManager, null, input, parentTask, serializerFactory, comparator, totalMemory,
-			totalMemory < 2 * MIN_NUM_SORT_MEM_SEGMENTS * memoryManager.getPageSize() ? 1 : 
-				Math.max((int) Math.ceil(((double) totalMemory) / MAX_MEM_PER_PARTIAL_SORT), 2),
+		super(memoryManager, null, input, parentTask, serializerFactory, comparator, memoryFraction,
+			memoryManager.computeNumberOfPages(memoryFraction) < 2 * MIN_NUM_SORT_MEM_SEGMENTS ? 1 :
+				Math.max((int) Math.ceil(((double) memoryManager.computeMemorySize(memoryFraction)) /
+						MAX_MEM_PER_PARTIAL_SORT),	2),
 			2, 0.0f, true);
 	}
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/AsynchronousPartialSorterCollector.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/AsynchronousPartialSorterCollector.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/AsynchronousPartialSorterCollector.java
index 9064ab9..747f98b 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/AsynchronousPartialSorterCollector.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/AsynchronousPartialSorterCollector.java
@@ -45,7 +45,7 @@ public class AsynchronousPartialSorterCollector<E> extends AsynchronousPartialSo
 	 * @param parentTask The parent task, which owns all resources used by this sorter.
 	 * @param serializerFactory The type serializer.
 	 * @param comparator The type comparator establishing the order relation.
-	 * @param totalMemory The total amount of memory dedicated to sorting.
+	 * @param memoryFraction The fraction of memory dedicated to sorting.
 	 * 
 	 * @throws IOException Thrown, if an error occurs initializing the resources for external sorting.
 	 * @throws MemoryAllocationException Thrown, if not enough memory can be obtained from the memory manager to
@@ -54,10 +54,11 @@ public class AsynchronousPartialSorterCollector<E> extends AsynchronousPartialSo
 	public AsynchronousPartialSorterCollector(MemoryManager memoryManager,
 			AbstractInvokable parentTask, 
 			TypeSerializerFactory<E> serializerFactory, TypeComparator<E> comparator,
-			long totalMemory)
+			double memoryFraction)
 	throws IOException, MemoryAllocationException
 	{
-		super(memoryManager, null, parentTask, serializerFactory, comparator, totalMemory);
+		super(memoryManager, null, parentTask, serializerFactory, comparator,
+				memoryFraction);
 	}
 	
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/CombiningUnilateralSortMerger.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/CombiningUnilateralSortMerger.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/CombiningUnilateralSortMerger.java
index 51d136c..9eb0452 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/CombiningUnilateralSortMerger.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/CombiningUnilateralSortMerger.java
@@ -95,12 +95,11 @@ public class CombiningUnilateralSortMerger<E> extends UnilateralSortMerger<E> {
 	 * @param parentTask The parent task, which owns all resources used by this sorter.
 	 * @param serializerFactory The type serializer.
 	 * @param comparator The type comparator establishing the order relation.
-	 * @param totalMemory The total amount of memory dedicated to sorting, merging and I/O.
+	 * @param memoryFraction The fraction of memory dedicated to sorting, merging and I/O.
 	 * @param maxNumFileHandles The maximum number of files to be merged at once.
 	 * @param startSpillingFraction The faction of the buffers that have to be filled before the spilling thread
 	 *                              actually begins spilling data to disk.
-	 * @param combineLastMerge A flag indicating whether the last merge step applies the combiner as well.
-	 * 
+	 *
 	 * @throws IOException Thrown, if an error occurs initializing the resources for external sorting.
 	 * @throws MemoryAllocationException Thrown, if not enough memory can be obtained from the memory manager to
 	 *                                   perform the sort.
@@ -108,11 +107,11 @@ public class CombiningUnilateralSortMerger<E> extends UnilateralSortMerger<E> {
 	public CombiningUnilateralSortMerger(GenericCombine<E> combineStub, MemoryManager memoryManager, IOManager ioManager,
 			MutableObjectIterator<E> input, AbstractInvokable parentTask, 
 			TypeSerializerFactory<E> serializerFactory, TypeComparator<E> comparator,
-			long totalMemory, int maxNumFileHandles, float startSpillingFraction)
+			double memoryFraction, int maxNumFileHandles, float startSpillingFraction)
 	throws IOException, MemoryAllocationException
 	{
 		this(combineStub, memoryManager, ioManager, input, parentTask, serializerFactory, comparator,
-			totalMemory, -1, maxNumFileHandles, startSpillingFraction);
+			memoryFraction, -1, maxNumFileHandles, startSpillingFraction);
 	}
 	
 	/**
@@ -127,13 +126,12 @@ public class CombiningUnilateralSortMerger<E> extends UnilateralSortMerger<E> {
 	 * @param parentTask The parent task, which owns all resources used by this sorter.
 	 * @param serializerFactory The type serializer.
 	 * @param comparator The type comparator establishing the order relation.
-	 * @param totalMemory The total amount of memory dedicated to sorting, merging and I/O.
+	 * @param memoryFraction The fraction of memory dedicated to sorting, merging and I/O.
 	 * @param numSortBuffers The number of distinct buffers to use creation of the initial runs.
 	 * @param maxNumFileHandles The maximum number of files to be merged at once.
 	 * @param startSpillingFraction The faction of the buffers that have to be filled before the spilling thread
 	 *                              actually begins spilling data to disk.
-	 * @param combineLastMerge A flag indicating whether the last merge step applies the combiner as well.
-	 * 
+	 *
 	 * @throws IOException Thrown, if an error occurs initializing the resources for external sorting.
 	 * @throws MemoryAllocationException Thrown, if not enough memory can be obtained from the memory manager to
 	 *                                   perform the sort.
@@ -141,12 +139,12 @@ public class CombiningUnilateralSortMerger<E> extends UnilateralSortMerger<E> {
 	public CombiningUnilateralSortMerger(GenericCombine<E> combineStub, MemoryManager memoryManager, IOManager ioManager,
 			MutableObjectIterator<E> input, AbstractInvokable parentTask, 
 			TypeSerializerFactory<E> serializerFactory, TypeComparator<E> comparator,
-			long totalMemory, int numSortBuffers, int maxNumFileHandles, 
+			double memoryFraction, int numSortBuffers, int maxNumFileHandles,
 			float startSpillingFraction)
 	throws IOException, MemoryAllocationException
 	{
 		super(memoryManager, ioManager, input, parentTask, serializerFactory, comparator,
-			totalMemory, numSortBuffers, maxNumFileHandles, startSpillingFraction, false);
+			memoryFraction, numSortBuffers, maxNumFileHandles, startSpillingFraction, false);
 		
 		this.combineStub = combineStub;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/UnilateralSortMerger.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/UnilateralSortMerger.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/UnilateralSortMerger.java
index 856ebf8..6905b85 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/UnilateralSortMerger.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/UnilateralSortMerger.java
@@ -174,7 +174,7 @@ public class UnilateralSortMerger<E> implements Sorter<E> {
 	 * @param parentTask The parent task, which owns all resources used by this sorter.
 	 * @param serializerFactory The type serializer.
 	 * @param comparator The type comparator establishing the order relation.
-	 * @param totalMemory The total amount of memory dedicated to sorting, merging and I/O.
+	 * @param memoryFraction The fraction of memory dedicated to sorting, merging and I/O.
 	 * @param maxNumFileHandles The maximum number of files to be merged at once.
 	 * @param startSpillingFraction The faction of the buffers that have to be filled before the spilling thread
 	 *                              actually begins spilling data to disk.
@@ -186,11 +186,11 @@ public class UnilateralSortMerger<E> implements Sorter<E> {
 	public UnilateralSortMerger(MemoryManager memoryManager, IOManager ioManager,
 			MutableObjectIterator<E> input, AbstractInvokable parentTask, 
 			TypeSerializerFactory<E> serializerFactory, TypeComparator<E> comparator,
-			long totalMemory, int maxNumFileHandles, float startSpillingFraction)
+			double memoryFraction, int maxNumFileHandles, float startSpillingFraction)
 	throws IOException, MemoryAllocationException
 	{
 		this(memoryManager, ioManager, input, parentTask, serializerFactory, comparator,
-			totalMemory, -1, maxNumFileHandles, startSpillingFraction);
+			memoryFraction, -1, maxNumFileHandles, startSpillingFraction);
 	}
 	
 	/**
@@ -204,7 +204,7 @@ public class UnilateralSortMerger<E> implements Sorter<E> {
 	 * @param parentTask The parent task, which owns all resources used by this sorter.
 	 * @param serializerFactory The type serializer.
 	 * @param comparator The type comparator establishing the order relation.
-	 * @param totalMemory The total amount of memory dedicated to sorting, merging and I/O.
+	 * @param memoryFraction The fraction of memory dedicated to sorting, merging and I/O.
 	 * @param numSortBuffers The number of distinct buffers to use creation of the initial runs.
 	 * @param maxNumFileHandles The maximum number of files to be merged at once.
 	 * @param startSpillingFraction The faction of the buffers that have to be filled before the spilling thread
@@ -217,12 +217,12 @@ public class UnilateralSortMerger<E> implements Sorter<E> {
 	public UnilateralSortMerger(MemoryManager memoryManager, IOManager ioManager,
 			MutableObjectIterator<E> input, AbstractInvokable parentTask, 
 			TypeSerializerFactory<E> serializerFactory, TypeComparator<E> comparator,
-			long totalMemory, int numSortBuffers, int maxNumFileHandles, 
+			double memoryFraction, int numSortBuffers, int maxNumFileHandles,
 			float startSpillingFraction)
 	throws IOException, MemoryAllocationException
 	{
 		this(memoryManager, ioManager, input, parentTask, serializerFactory, comparator,
-			totalMemory, numSortBuffers, maxNumFileHandles, startSpillingFraction, false);
+			memoryFraction, numSortBuffers, maxNumFileHandles, startSpillingFraction, false);
 	}
 	
 	/**
@@ -234,7 +234,7 @@ public class UnilateralSortMerger<E> implements Sorter<E> {
 	 * @param parentTask The parent task, which owns all resources used by this sorter.
 	 * @param serializerFactory The type serializer.
 	 * @param comparator The type comparator establishing the order relation.
-	 * @param totalMemory The total amount of memory dedicated to sorting, merging and I/O.
+	 * @param memoryFraction The fraction of memory dedicated to sorting, merging and I/O.
 	 * @param numSortBuffers The number of distinct buffers to use creation of the initial runs.
 	 * @param maxNumFileHandles The maximum number of files to be merged at once.
 	 * @param startSpillingFraction The faction of the buffers that have to be filled before the spilling thread
@@ -249,7 +249,7 @@ public class UnilateralSortMerger<E> implements Sorter<E> {
 	protected UnilateralSortMerger(MemoryManager memoryManager, IOManager ioManager,
 			MutableObjectIterator<E> input, AbstractInvokable parentTask, 
 			TypeSerializerFactory<E> serializerFactory, TypeComparator<E> comparator,
-			long totalMemory, int numSortBuffers, int maxNumFileHandles, 
+			double memoryFraction, int numSortBuffers, int maxNumFileHandles,
 			float startSpillingFraction, boolean noSpillingMemory)
 	throws IOException, MemoryAllocationException
 	{
@@ -267,7 +267,7 @@ public class UnilateralSortMerger<E> implements Sorter<E> {
 		this.memoryManager = memoryManager;
 		
 		// adjust the memory quotas to the page size
-		final int numPagesTotal = memoryManager.computeNumberOfPages(totalMemory);
+		final int numPagesTotal = memoryManager.computeNumberOfPages(memoryFraction);
 
 		if (numPagesTotal < MIN_NUM_WRITE_BUFFERS + MIN_NUM_SORT_MEM_SEGMENTS) {
 			throw new IllegalArgumentException("Too little memory provided to sorter to perform task. " +

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/AbstractCachedBuildSideMatchDriver.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/AbstractCachedBuildSideMatchDriver.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/AbstractCachedBuildSideMatchDriver.java
index 66a4986..1d3c55d 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/AbstractCachedBuildSideMatchDriver.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/AbstractCachedBuildSideMatchDriver.java
@@ -67,7 +67,7 @@ public abstract class AbstractCachedBuildSideMatchDriver<IT1, IT2, OT> extends M
 		TypePairComparatorFactory<IT1, IT2> pairComparatorFactory = 
 				this.taskContext.getTaskConfig().getPairComparatorFactory(this.taskContext.getUserCodeClassLoader());
 
-		int numMemoryPages = this.taskContext.getMemoryManager().computeNumberOfPages(config.getMemoryDriver());
+		int numMemoryPages = this.taskContext.getMemoryManager().computeNumberOfPages(config.getRelativeMemoryDriver());
 		List<MemorySegment> memSegments = this.taskContext.getMemoryManager().allocatePages(
 			this.taskContext.getOwningNepheleTask(), numMemoryPages);
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/CrossDriver.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/CrossDriver.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/CrossDriver.java
index 39f563d..181a687 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/CrossDriver.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/CrossDriver.java
@@ -116,8 +116,7 @@ public class CrossDriver<T1, T2, OT> implements PactDriver<GenericCrosser<T1, T2
 		}
 		
 		this.memManager = this.taskContext.getMemoryManager();
-		final long totalAvailableMemory = config.getMemoryDriver();
-		final int numPages = this.memManager.computeNumberOfPages(totalAvailableMemory);
+		final int numPages = this.memManager.computeNumberOfPages(config.getRelativeMemoryDriver());
 		
 		if (numPages < 2) {
 			throw new RuntimeException(	"The Cross task was initialized with too little memory. " +

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSinkTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSinkTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSinkTask.java
index 638a7aa..82359f5 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSinkTask.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSinkTask.java
@@ -138,7 +138,7 @@ public class DataSinkTask<IT> extends AbstractOutputTask {
 							getEnvironment().getMemoryManager(), 
 							getEnvironment().getIOManager(),
 							this.reader, this, this.inputTypeSerializerFactory, compFact.createComparator(),
-							this.config.getMemoryInput(0), this.config.getFilehandlesInput(0),
+							this.config.getRelativeMemoryInput(0), this.config.getFilehandlesInput(0),
 							this.config.getSpillingThresholdInput(0));
 					
 					this.localStrategy = sorter;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/GroupReduceCombineDriver.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/GroupReduceCombineDriver.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/GroupReduceCombineDriver.java
index 0d51363..8cef403 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/GroupReduceCombineDriver.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/GroupReduceCombineDriver.java
@@ -79,8 +79,6 @@ public class GroupReduceCombineDriver<T> implements PactDriver<GenericCombine<T>
 		final TaskConfig config = this.taskContext.getTaskConfig();
 		final DriverStrategy ls = config.getDriverStrategy();
 
-		final long availableMemory = config.getMemoryDriver();
-
 		final MemoryManager memoryManager = this.taskContext.getMemoryManager();
 
 		final MutableObjectIterator<T> in = this.taskContext.getInput(0);
@@ -90,7 +88,7 @@ public class GroupReduceCombineDriver<T> implements PactDriver<GenericCombine<T>
 		switch (ls) {
 		case SORTED_GROUP_COMBINE:
 			this.input = new AsynchronousPartialSorter<T>(memoryManager, in, this.taskContext.getOwningNepheleTask(),
-						this.serializerFactory, this.comparator.duplicate(), availableMemory);
+						this.serializerFactory, this.comparator.duplicate(), config.getRelativeMemoryDriver());
 			break;
 		// obtain and return a grouped iterator from the combining sort-merger
 		default:

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/MatchDriver.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/MatchDriver.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/MatchDriver.java
index b356a58..a651894 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/MatchDriver.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/MatchDriver.java
@@ -83,8 +83,8 @@ public class MatchDriver<IT1, IT2, OT> implements PactDriver<GenericJoiner<IT1,
 		final IOManager ioManager = this.taskContext.getIOManager();
 		
 		// set up memory and I/O parameters
-		final long availableMemory = config.getMemoryDriver();
-		final int numPages = memoryManager.computeNumberOfPages(availableMemory);
+		final double fractionAvailableMemory = config.getRelativeMemoryDriver();
+		final int numPages = memoryManager.computeNumberOfPages(fractionAvailableMemory);
 		
 		// test minimum memory requirements
 		final DriverStrategy ls = config.getDriverStrategy();
@@ -106,23 +106,23 @@ public class MatchDriver<IT1, IT2, OT> implements PactDriver<GenericJoiner<IT1,
 
 		// create and return MatchTaskIterator according to provided local strategy.
 		switch (ls) {
-			case MERGE:
-				this.matchIterator = new MergeMatchIterator<IT1, IT2, OT>(in1, in2, serializer1, comparator1,
-						serializer2, comparator2, pairComparatorFactory.createComparator12(comparator1, comparator2),
-						memoryManager, ioManager, numPages, this.taskContext.getOwningNepheleTask());
-				break;
-			case HYBRIDHASH_BUILD_FIRST:
-				this.matchIterator = new BuildFirstHashMatchIterator<IT1, IT2, OT>(in1, in2, serializer1, comparator1,
-					serializer2, comparator2, pairComparatorFactory.createComparator21(comparator1, comparator2),
-					memoryManager, ioManager, this.taskContext.getOwningNepheleTask(), availableMemory);
-				break;
-			case HYBRIDHASH_BUILD_SECOND:
-				this.matchIterator = new BuildSecondHashMatchIterator<IT1, IT2, OT>(in1, in2, serializer1, comparator1,
-						serializer2, comparator2, pairComparatorFactory.createComparator12(comparator1, comparator2),
-						memoryManager, ioManager, this.taskContext.getOwningNepheleTask(), availableMemory);
-				break;
-			default:
-				throw new Exception("Unsupported driver strategy for Match driver: " + ls.name());
+		case MERGE:
+			this.matchIterator = new MergeMatchIterator<IT1, IT2, OT>(in1, in2, serializer1, comparator1,
+					serializer2, comparator2, pairComparatorFactory.createComparator12(comparator1, comparator2),
+					memoryManager, ioManager, numPages, this.taskContext.getOwningNepheleTask());
+			break;
+		case HYBRIDHASH_BUILD_FIRST:
+			this.matchIterator = new BuildFirstHashMatchIterator<IT1, IT2, OT>(in1, in2, serializer1, comparator1,
+				serializer2, comparator2, pairComparatorFactory.createComparator21(comparator1, comparator2),
+				memoryManager, ioManager, this.taskContext.getOwningNepheleTask(), fractionAvailableMemory);
+			break;
+		case HYBRIDHASH_BUILD_SECOND:
+			this.matchIterator = new BuildSecondHashMatchIterator<IT1, IT2, OT>(in1, in2, serializer1, comparator1,
+					serializer2, comparator2, pairComparatorFactory.createComparator12(comparator1, comparator2),
+					memoryManager, ioManager, this.taskContext.getOwningNepheleTask(), fractionAvailableMemory);
+			break;
+		default:
+			throw new Exception("Unsupported driver strategy for Match driver: " + ls.name());
 		}
 		
 		// open MatchTaskIterator - this triggers the sorting or hash-table building

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/ReduceCombineDriver.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/ReduceCombineDriver.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/ReduceCombineDriver.java
index 14310ca..2eaba54 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/ReduceCombineDriver.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/ReduceCombineDriver.java
@@ -99,7 +99,8 @@ public class ReduceCombineDriver<T> implements PactDriver<GenericReduce<T>, T> {
 		}
 		
 		this.memManager = this.taskContext.getMemoryManager();
-		final int numMemoryPages = memManager.computeNumberOfPages(this.taskContext.getTaskConfig().getMemoryDriver());
+		final int numMemoryPages = memManager.computeNumberOfPages(this.taskContext.getTaskConfig()
+				.getRelativeMemoryDriver());
 		
 		// instantiate the serializer / comparator
 		final TypeSerializerFactory<T> serializerFactory = this.taskContext.getInputSerializer(0);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/RegularPactTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/RegularPactTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/RegularPactTask.java
index 92c4648..1d7c931 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/RegularPactTask.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/RegularPactTask.java
@@ -840,7 +840,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 			this.inputIsCached[i] = cached;
 
 			if (async || cached) {
-				memoryPages = memMan.computeNumberOfPages(this.config.getInputMaterializationMemory(i));
+				memoryPages = memMan.computeNumberOfPages(this.config.getRelativeInputMaterializationMemory(i));
 				if (memoryPages <= 0) {
 					throw new Exception("Input marked as materialized/cached, but no memory for materialization provided.");
 				}
@@ -946,7 +946,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 				@SuppressWarnings({ "rawtypes", "unchecked" })
 				UnilateralSortMerger<?> sorter = new UnilateralSortMerger(getMemoryManager(), getIOManager(),
 					this.inputIterators[inputNum], this, this.inputSerializers[inputNum], getLocalStrategyComparator(inputNum),
-					this.config.getMemoryInput(inputNum), this.config.getFilehandlesInput(inputNum),
+					this.config.getRelativeMemoryInput(inputNum), this.config.getFilehandlesInput(inputNum),
 					this.config.getSpillingThresholdInput(inputNum));
 				// set the input to null such that it will be lazily fetched from the input strategy
 				this.inputs[inputNum] = null;
@@ -982,7 +982,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 				CombiningUnilateralSortMerger<?> cSorter = new CombiningUnilateralSortMerger(
 					(GenericCombine) localStub, getMemoryManager(), getIOManager(), this.inputIterators[inputNum], 
 					this, this.inputSerializers[inputNum], getLocalStrategyComparator(inputNum),
-					this.config.getMemoryInput(inputNum), this.config.getFilehandlesInput(inputNum),
+					this.config.getRelativeMemoryInput(inputNum), this.config.getFilehandlesInput(inputNum),
 					this.config.getSpillingThresholdInput(inputNum));
 				cSorter.setUdfConfiguration(this.config.getStubParameters());
 
@@ -1022,12 +1022,6 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 			final MutableObjectIterator<?> iter = new ReaderIterator(reader, serializerFactory.getSerializer());
 			return iter;
 		}
-//		// generic data type serialization
-//		@SuppressWarnings("unchecked")
-//		MutableReader<DeserializationDelegate<?>> reader = (MutableReader<DeserializationDelegate<?>>) inputReader;
-//		@SuppressWarnings({ "unchecked", "rawtypes" })
-//		final MutableObjectIterator<?> iter = new ReaderIterator(reader, serializer);
-//		return iter;
 	}
 
 	protected int getNumTaskInputs() {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/chaining/SynchronousChainedCombineDriver.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/chaining/SynchronousChainedCombineDriver.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/chaining/SynchronousChainedCombineDriver.java
index 814eb62..98d65f1 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/chaining/SynchronousChainedCombineDriver.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/chaining/SynchronousChainedCombineDriver.java
@@ -81,7 +81,7 @@ public class SynchronousChainedCombineDriver<T> extends ChainedDriver<T, T> {
 		// ----------------- Set up the asynchronous sorter -------------------------
 
 		this.memManager = this.parent.getEnvironment().getMemoryManager();
-		final int numMemoryPages = memManager.computeNumberOfPages(this.config.getMemoryDriver());
+		final int numMemoryPages = memManager.computeNumberOfPages(this.config.getRelativeMemoryDriver());
 
 		// instantiate the serializer / comparator
 		final TypeSerializerFactory<T> serializerFactory = this.config.getInputSerializer(0, this.userCodeClassLoader);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/TaskConfig.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/TaskConfig.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/TaskConfig.java
index 947e22a..a43f8cc 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/TaskConfig.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/TaskConfig.java
@@ -454,12 +454,12 @@ public class TaskConfig {
 		return this.config.getBoolean(INPUT_REPLAYABLE_PREFIX + inputNum, false);
 	}
 	
-	public void setInputMaterializationMemory(int inputNum, long memory) {
-		this.config.setLong(INPUT_DAM_MEMORY_PREFIX + inputNum, memory);
+	public void setRelativeInputMaterializationMemory(int inputNum, double relativeMemory) {
+		this.config.setDouble(INPUT_DAM_MEMORY_PREFIX + inputNum, relativeMemory);
 	}
 	
-	public long getInputMaterializationMemory(int inputNum) {
-		return this.config.getLong(INPUT_DAM_MEMORY_PREFIX + inputNum, -1);
+	public double getRelativeInputMaterializationMemory(int inputNum) {
+		return this.config.getDouble(INPUT_DAM_MEMORY_PREFIX + inputNum, 0);
 	}
 	
 	public void setBroadcastInputName(String name, int groupIndex) {
@@ -577,20 +577,20 @@ public class TaskConfig {
 	//                       Parameters to configure the memory and I/O behavior
 	// --------------------------------------------------------------------------------------------
 
-	public void setMemoryDriver(long memorySize) {
-		this.config.setLong(MEMORY_DRIVER, memorySize);
+	public void setRelativeMemoryDriver(double relativeMemorySize) {
+		this.config.setDouble(MEMORY_DRIVER, relativeMemorySize);
 	}
 
-	public long getMemoryDriver() {
-		return this.config.getLong(MEMORY_DRIVER, -1);
+	public double getRelativeMemoryDriver() {
+		return this.config.getDouble(MEMORY_DRIVER, 0);
 	}
 	
-	public void setMemoryInput(int inputNum, long memorySize) {
-		this.config.setLong(MEMORY_INPUT_PREFIX + inputNum, memorySize);
+	public void setRelativeMemoryInput(int inputNum, double relativeMemorySize) {
+		this.config.setDouble(MEMORY_INPUT_PREFIX + inputNum, relativeMemorySize);
 	}
 
-	public long getMemoryInput(int inputNum) {
-		return this.config.getLong(MEMORY_INPUT_PREFIX + inputNum, -1);
+	public double getRelativeMemoryInput(int inputNum) {
+		return this.config.getDouble(MEMORY_INPUT_PREFIX + inputNum, 0);
 	}
 	
 	// --------------------------------------------------------------------------------------------
@@ -732,30 +732,30 @@ public class TaskConfig {
 		return index;
 	}
 	
-	public void setBackChannelMemory(long memory) {
-		if (memory < 0) {
+	public void setRelativeBackChannelMemory(double relativeMemory) {
+		if (relativeMemory < 0) {
 			throw new IllegalArgumentException();
 		}
-		this.config.setLong(ITERATION_HEAD_BACKCHANNEL_MEMORY, memory);
+		this.config.setDouble(ITERATION_HEAD_BACKCHANNEL_MEMORY, relativeMemory);
 	}
 
-	public long getBackChannelMemory() {
-		long backChannelMemory = this.config.getLong(ITERATION_HEAD_BACKCHANNEL_MEMORY, 0);
-		if (backChannelMemory <= 0) {
+	public double getRelativeBackChannelMemory() {
+		double relativeBackChannelMemory = this.config.getDouble(ITERATION_HEAD_BACKCHANNEL_MEMORY, 0);
+		if (relativeBackChannelMemory <= 0) {
 			throw new IllegalArgumentException();
 		}
-		return backChannelMemory;
+		return relativeBackChannelMemory;
 	}
 	
-	public void setSolutionSetMemory(long memory) {
-		if (memory < 0) {
+	public void setRelativeSolutionSetMemory(double relativeMemory) {
+		if (relativeMemory < 0) {
 			throw new IllegalArgumentException();
 		}
-		this.config.setLong(ITERATION_HEAD_SOLUTION_SET_MEMORY, memory);
+		this.config.setDouble(ITERATION_HEAD_SOLUTION_SET_MEMORY, relativeMemory);
 	}
 
-	public long getSolutionSetMemory() {
-		long backChannelMemory = this.config.getLong(ITERATION_HEAD_SOLUTION_SET_MEMORY, 0);
+	public double getRelativeSolutionSetMemory() {
+		double backChannelMemory = this.config.getDouble(ITERATION_HEAD_SOLUTION_SET_MEMORY, 0);
 		if (backChannelMemory <= 0) {
 			throw new IllegalArgumentException();
 		}
@@ -1198,6 +1198,16 @@ public class TaskConfig {
 		public void setFloat(String key, float value) {
 			this.backingConfig.setFloat(this.prefix + key, value);
 		}
+
+		@Override
+		public double getDouble(String key, double defaultValue) {
+			return this.backingConfig.getDouble(this.prefix + key, defaultValue);
+		}
+
+		@Override
+		public void setDouble(String key, double value) {
+			this.backingConfig.setDouble(this.prefix + key, value);
+		}
 		
 		@Override
 		public byte[] getBytes(final String key, final byte[] defaultValue) {
@@ -1220,16 +1230,6 @@ public class TaskConfig {
 		}
 		
 		@Override
-		public double getDouble(String key, double defaultValue) {
-			return backingConfig.getDouble(this.prefix + key, defaultValue);
-		}
-		
-		@Override
-		public void setDouble(String key, double value) {
-			backingConfig.setDouble(this.prefix + key, value);
-		}
-		
-		@Override
 		public String toString() {
 			return backingConfig.toString();
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/InputChannel.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/InputChannel.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/InputChannel.java
index 6122c36..4109a2b 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/InputChannel.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/InputChannel.java
@@ -82,8 +82,6 @@ public class InputChannel<T extends IOReadableWritable> extends Channel implemen
 
 	private int lastReceivedEnvelope = -1;
 
-	private ChannelID lastSourceID = null;
-
 	private boolean destroyCalled = false;
 
 	// ----------------------
@@ -157,10 +155,6 @@ public class InputChannel<T extends IOReadableWritable> extends Channel implemen
 		return this.inputGate.getJobID();
 	}
 
-//	public abstract AbstractTaskEvent getCurrentEvent();
-
-	private DeserializationResult lastDeserializationResult;
-
 
 	public InputChannelResult readRecord(T target) throws IOException {
 		if (this.dataBuffer == null) {
@@ -207,7 +201,6 @@ public class InputChannel<T extends IOReadableWritable> extends Channel implemen
 		}
 
 		DeserializationResult deserializationResult = this.deserializer.getNextRecord(target);
-		this.lastDeserializationResult = deserializationResult;
 
 		if (deserializationResult.isBufferConsumed()) {
 			releasedConsumedReadBuffer(this.dataBuffer);
@@ -352,7 +345,6 @@ public class InputChannel<T extends IOReadableWritable> extends Channel implemen
 
 				this.queuedEnvelopes.add(envelope);
 				this.lastReceivedEnvelope = sequenceNumber;
-				this.lastSourceID = envelope.getSource();
 
 				// Notify the channel about the new data. notify as much as there is (buffer plus once per event)
 				if (envelope.getBuffer() != null) {
@@ -464,6 +456,7 @@ public class InputChannel<T extends IOReadableWritable> extends Channel implemen
 		}
 
 		// schedule events as pending, because events come always after the buffer!
+		@SuppressWarnings("unchecked")
 		List<AbstractEvent> events = (List<AbstractEvent>) nextEnvelope.deserializeEvents();
 		Iterator<AbstractEvent> eventsIt = events.iterator();
 		if (eventsIt.hasNext()) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/InputGate.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/InputGate.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/InputGate.java
index c623220..dc506ef 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/InputGate.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/InputGate.java
@@ -26,6 +26,7 @@ import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
 import eu.stratosphere.runtime.io.network.bufferprovider.GlobalBufferPool;
 import eu.stratosphere.runtime.io.network.bufferprovider.LocalBufferPool;
 import eu.stratosphere.runtime.io.network.bufferprovider.LocalBufferPoolOwner;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
@@ -95,6 +96,7 @@ public class InputGate<T extends IOReadableWritable> extends Gate<T> implements
 		super(jobID, gateID, index);
 	}
 
+	@SuppressWarnings("unchecked")
 	public void initializeChannels(GateDeploymentDescriptor inputGateDescriptor){
 		channels = new InputChannel[inputGateDescriptor.getNumberOfChannelDescriptors()];
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/RemoteReceiver.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/RemoteReceiver.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/RemoteReceiver.java
index da36ad0..ab65b4c 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/RemoteReceiver.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/RemoteReceiver.java
@@ -18,10 +18,8 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
-import java.net.UnknownHostException;
 
 import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.util.StringUtils;
 
 /**
  * Objects of this class uniquely identify a connection to a remote {@link TaskManager}.
@@ -48,11 +46,9 @@ public final class RemoteReceiver implements IOReadableWritable {
 	 *        the index of the connection to the remote {@link TaskManager}
 	 */
 	public RemoteReceiver(final InetSocketAddress connectionAddress, final int connectionIndex) {
-
 		if (connectionAddress == null) {
 			throw new IllegalArgumentException("Argument connectionAddress must not be null");
 		}
-
 		if (connectionIndex < 0) {
 			throw new IllegalArgumentException("Argument connectionIndex must be a non-negative integer number");
 		}
@@ -75,7 +71,6 @@ public final class RemoteReceiver implements IOReadableWritable {
 	 * @return the address of the connection to the remote {@link TaskManager}
 	 */
 	public InetSocketAddress getConnectionAddress() {
-
 		return this.connectionAddress;
 	}
 
@@ -85,14 +80,12 @@ public final class RemoteReceiver implements IOReadableWritable {
 	 * @return the index of the connection to the remote {@link TaskManager}
 	 */
 	public int getConnectionIndex() {
-
 		return this.connectionIndex;
 	}
 
 
 	@Override
 	public int hashCode() {
-
 		return this.connectionAddress.hashCode() + (31 * this.connectionIndex);
 	}
 
@@ -131,18 +124,12 @@ public final class RemoteReceiver implements IOReadableWritable {
 
 	@Override
 	public void read(final DataInput in) throws IOException {
-
 		final int addr_length = in.readInt();
 		final byte[] address = new byte[addr_length];
 		in.readFully(address);
 
-		InetAddress ia = null;
-		try {
-			ia = InetAddress.getByAddress(address);
-		} catch (UnknownHostException uhe) {
-			throw new IOException(StringUtils.stringifyException(uhe));
-		}
-		final int port = in.readInt();
+		InetAddress ia = InetAddress.getByAddress(address);
+		int port = in.readInt();
 		this.connectionAddress = new InetSocketAddress(ia, port);
 
 		this.connectionIndex = in.readInt();
@@ -151,7 +138,6 @@ public final class RemoteReceiver implements IOReadableWritable {
 
 	@Override
 	public String toString() {
-
 		return this.connectionAddress + " (" + this.connectionIndex + ")";
 	}
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/event/job/ManagementEventTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/event/job/ManagementEventTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/event/job/ManagementEventTest.java
index a6a5875..fd1a672 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/event/job/ManagementEventTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/event/job/ManagementEventTest.java
@@ -85,14 +85,12 @@ public class ManagementEventTest {
 	@Test
 	public void testVertexAssignmentEvent() {
 
-		final VertexAssignmentEvent orig = new VertexAssignmentEvent(TIMESTAMP, new ManagementVertexID(), "test",
-			"standard");
+		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.getInstanceType(), copy.getInstanceType());
 		assertEquals(orig.hashCode(), copy.hashCode());
 		assertTrue(orig.equals(copy));
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ExecutionGraphTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ExecutionGraphTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ExecutionGraphTest.java
index 5ff5f1c..fa0653b 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ExecutionGraphTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ExecutionGraphTest.java
@@ -23,40 +23,25 @@ import static org.junit.Assert.fail;
 import java.io.File;
 import java.io.IOException;
 import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
 
 import org.apache.log4j.Level;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.core.fs.Path;
 import eu.stratosphere.nephele.execution.ExecutionState;
 import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
-import eu.stratosphere.nephele.instance.AbstractInstance;
-import eu.stratosphere.nephele.instance.AllocatedResource;
-import eu.stratosphere.nephele.instance.HardwareDescription;
-import eu.stratosphere.nephele.instance.InstanceConnectionInfo;
-import eu.stratosphere.nephele.instance.InstanceException;
-import eu.stratosphere.nephele.instance.InstanceListener;
-import eu.stratosphere.nephele.instance.InstanceManager;
-import eu.stratosphere.nephele.instance.InstanceRequestMap;
-import eu.stratosphere.nephele.instance.InstanceType;
-import eu.stratosphere.nephele.instance.InstanceTypeDescription;
-import eu.stratosphere.nephele.instance.InstanceTypeFactory;
 import eu.stratosphere.nephele.jobgraph.DistributionPattern;
-import eu.stratosphere.runtime.io.channels.ChannelType;
-import eu.stratosphere.nephele.util.FileLineReader;
-import eu.stratosphere.nephele.util.FileLineWriter;
 import eu.stratosphere.nephele.jobgraph.JobFileInputVertex;
 import eu.stratosphere.nephele.jobgraph.JobFileOutputVertex;
 import eu.stratosphere.nephele.jobgraph.JobGraph;
 import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException;
 import eu.stratosphere.nephele.jobgraph.JobID;
 import eu.stratosphere.nephele.jobgraph.JobTaskVertex;
-import eu.stratosphere.nephele.topology.NetworkTopology;
+import eu.stratosphere.nephele.util.FileLineReader;
+import eu.stratosphere.nephele.util.FileLineWriter;
 import eu.stratosphere.nephele.util.ServerTestUtils;
+import eu.stratosphere.runtime.io.channels.ChannelType;
 import eu.stratosphere.util.LogUtils;
 
 /**
@@ -64,152 +49,6 @@ import eu.stratosphere.util.LogUtils;
  * 
  */
 public class ExecutionGraphTest {
-
-	/**
-	 * The name of the default instance type used during these tests.
-	 */
-	private static final String DEFAULT_INSTANCE_TYPE_NAME = "test";
-
-	/**
-	 * A test implementation of an {@link InstanceManager} which is used as a stub in these tests.
-	 * 
-	 */
-	private static final class TestInstanceManager implements InstanceManager {
-
-		/**
-		 * The default instance type.
-		 */
-		private final InstanceType defaultInstanceType;
-
-		/**
-		 * Constructs a new test instance manager.
-		 */
-		public TestInstanceManager() {
-			this.defaultInstanceType = InstanceTypeFactory.construct(DEFAULT_INSTANCE_TYPE_NAME, 4, 4, 1024, 50, 10);
-		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public void requestInstance(final JobID jobID, final Configuration conf,
-				final InstanceRequestMap instanceRequestMap,
-				final List<String> splitAffinityList) throws InstanceException {
-
-			throw new IllegalStateException("requestInstance called on TestInstanceManager");
-		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public void releaseAllocatedResource(final JobID jobID, final Configuration conf,
-				final AllocatedResource allocatedResource)
-				throws InstanceException {
-
-			throw new IllegalStateException("releaseAllocatedResource called on TestInstanceManager");
-		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public InstanceType getSuitableInstanceType(final int minNumComputeUnits, final int minNumCPUCores,
-				final int minMemorySize, final int minDiskCapacity, final int maxPricePerHour) {
-
-			throw new IllegalStateException("getSuitableInstanceType called on TestInstanceManager");
-		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public void reportHeartBeat(final InstanceConnectionInfo instanceConnectionInfo,
-				final HardwareDescription hardwareDescription) {
-
-			throw new IllegalStateException("reportHeartBeat called on TestInstanceManager");
-		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public InstanceType getInstanceTypeByName(final String instanceTypeName) {
-
-			if (this.defaultInstanceType.getIdentifier().equals(instanceTypeName)) {
-				return this.defaultInstanceType;
-			}
-
-			return null;
-		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public InstanceType getDefaultInstanceType() {
-
-			return this.defaultInstanceType;
-		}
-
-		@Override
-		public NetworkTopology getNetworkTopology(final JobID jobID) {
-
-			throw new IllegalStateException("getNetworkTopology called on TestInstanceManager");
-		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public void setInstanceListener(final InstanceListener instanceListener) {
-
-			throw new IllegalStateException("setInstanceListener called on TestInstanceManager");
-		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public Map<InstanceType, InstanceTypeDescription> getMapOfAvailableInstanceTypes() {
-
-			throw new IllegalStateException("getMapOfAvailableInstanceType called on TestInstanceManager");
-		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public void shutdown() {
-
-			throw new IllegalStateException("shutdown called on TestInstanceManager");
-		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public AbstractInstance getInstanceByName(final String name) {
-			throw new IllegalStateException("getInstanceByName called on TestInstanceManager");
-		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public void cancelPendingRequests(final JobID jobID) {
-			throw new IllegalStateException("cancelPendingRequests called on TestInstanceManager");
-		}
-
-		@Override
-		public int getNumberOfTaskTrackers() {
-			return 0;
-		}
-
-	}
-
-	private static final InstanceManager INSTANCE_MANAGER = new TestInstanceManager();
-
 	@BeforeClass
 	public static void reduceLogLevel() {
 		LogUtils.initializeDefaultConsoleLogger(Level.WARN);
@@ -259,15 +98,11 @@ public class ExecutionGraphTest {
 
 			LibraryCacheManager.register(jobID, new String[0]);
 
-			final ExecutionGraph eg = new ExecutionGraph(jg, INSTANCE_MANAGER);
+			final ExecutionGraph eg = new ExecutionGraph(jg, -1);
 
 			// test all methods of ExecutionGraph
-			final InstanceRequestMap instanceRequestMap = new InstanceRequestMap();
 			final ExecutionStage executionStage = eg.getCurrentExecutionStage();
-			executionStage.collectRequiredInstanceTypes(instanceRequestMap, ExecutionState.CREATED);
-			assertEquals(1, instanceRequestMap.size());
-			assertEquals(1, (int) instanceRequestMap.getMaximumNumberOfInstances(INSTANCE_MANAGER
-					.getInstanceTypeByName(DEFAULT_INSTANCE_TYPE_NAME)));
+			assertEquals(1, executionStage.getMaxNumberSubtasks());
 
 			assertEquals(jobID, eg.getJobID());
 			assertEquals(0, eg.getIndexOfCurrentExecutionStage());
@@ -332,15 +167,11 @@ public class ExecutionGraphTest {
 			assertNotNull(egv0.getGroupMember(0));
 			assertNull(egv0.getGroupMember(1));
 			assertEquals(1, egv0.getInputSplits().length);
-			assertEquals(-1, egv0.getMaximumNumberOfGroupMembers());
-			assertEquals(1, egv0.getMinimumNumberOfGroupMember());
 			assertEquals("Input 1", egv0.getName());
 			assertEquals(0, egv0.getNumberOfBackwardLinks());
 			assertEquals(1, egv0.getNumberOfForwardLinks());
-			assertEquals(1, egv0.getNumberOfSubtasksPerInstance());
 			assertEquals(0, egv0.getStageNumber());
 			assertEquals(-1, egv0.getUserDefinedNumberOfMembers());
-			assertEquals(INSTANCE_MANAGER.getDefaultInstanceType(), egv0.getInstanceType());
 			assertEquals("Task 1", egv0.getVertexToShareInstancesWith().getName());
 
 			// egv1 (output1)
@@ -354,15 +185,11 @@ public class ExecutionGraphTest {
 			assertNull(egv1.getForwardEdge(0));
 			assertNotNull(egv1.getGroupMember(0));
 			assertNull(egv1.getGroupMember(1));
-			assertEquals(1, egv1.getMaximumNumberOfGroupMembers());
-			assertEquals(1, egv1.getMinimumNumberOfGroupMember());
 			assertEquals("Output 1", egv1.getName());
 			assertEquals(1, egv1.getNumberOfBackwardLinks());
 			assertEquals(0, egv1.getNumberOfForwardLinks());
-			assertEquals(1, egv1.getNumberOfSubtasksPerInstance());
 			assertEquals(0, egv1.getStageNumber());
 			assertEquals(-1, egv1.getUserDefinedNumberOfMembers());
-			assertEquals(INSTANCE_MANAGER.getInstanceTypeByName(DEFAULT_INSTANCE_TYPE_NAME), egv1.getInstanceType());
 			assertEquals("Input 1", egv1.getVertexToShareInstancesWith().getName());
 
 			// egv2 (task1)
@@ -378,15 +205,11 @@ public class ExecutionGraphTest {
 			assertNotNull(egv2.getForwardEdges(egv1));
 			assertNotNull(egv2.getGroupMember(0));
 			assertNull(egv2.getGroupMember(1));
-			assertEquals(-1, egv2.getMaximumNumberOfGroupMembers());
-			assertEquals(1, egv2.getMinimumNumberOfGroupMember());
 			assertEquals("Task 1", egv2.getName());
 			assertEquals(1, egv2.getNumberOfBackwardLinks());
 			assertEquals(1, egv2.getNumberOfForwardLinks());
-			assertEquals(1, egv2.getNumberOfSubtasksPerInstance());
 			assertEquals(0, egv2.getStageNumber());
 			assertEquals(-1, egv2.getUserDefinedNumberOfMembers());
-			assertEquals(INSTANCE_MANAGER.getInstanceTypeByName(DEFAULT_INSTANCE_TYPE_NAME), egv2.getInstanceType());
 			assertNull(egv2.getVertexToShareInstancesWith());
 
 			// test all methods of ExecutionVertex
@@ -398,25 +221,16 @@ public class ExecutionGraphTest {
 			assertEquals(egv0, ev0.getGroupVertex());
 			assertNotNull(ev0.getID());
 			assertEquals("Input 1", ev0.getName());
-			assertEquals(INSTANCE_MANAGER.getInstanceTypeByName(DEFAULT_INSTANCE_TYPE_NAME), ev0.getAllocatedResource()
-				.getInstance()
-				.getType());
 
 			// ev1 (output1)
 			assertEquals(egv1, ev1.getGroupVertex());
 			assertNotNull(ev1.getID());
 			assertEquals("Output 1", ev1.getName());
-			assertEquals(INSTANCE_MANAGER.getInstanceTypeByName(DEFAULT_INSTANCE_TYPE_NAME), ev1.getAllocatedResource()
-				.getInstance()
-				.getType());
 
 			// ev2 (task1)
 			assertEquals(egv2, ev2.getGroupVertex());
 			assertNotNull(ev2.getID());
 			assertEquals("Task 1", ev2.getName());
-			assertEquals(INSTANCE_MANAGER.getInstanceTypeByName(DEFAULT_INSTANCE_TYPE_NAME), ev2.getAllocatedResource()
-				.getInstance()
-				.getType());
 
 			assertEquals(ev0.getAllocatedResource(), ev1.getAllocatedResource());
 			assertEquals(ev0.getAllocatedResource(), ev2.getAllocatedResource());
@@ -448,7 +262,7 @@ public class ExecutionGraphTest {
 	 * input1 -> task1 -> output1
 	 * no subtasks defined
 	 * input1 is default, task1 is m1.large, output1 is m1.xlarge
-	 * all channels are IN_MEMORY
+	 * all channels are INMEMORY
 	 */
 	@Test
 	public void testConvertJobGraphToExecutionGraph2() {
@@ -484,15 +298,11 @@ public class ExecutionGraphTest {
 			LibraryCacheManager.register(jobID, new String[0]);
 
 			// now convert job graph to execution graph
-			final ExecutionGraph eg = new ExecutionGraph(jg, INSTANCE_MANAGER);
+			final ExecutionGraph eg = new ExecutionGraph(jg, 1);
 
 			// test instance types in ExecutionGraph
-			final InstanceRequestMap instanceRequestMap = new InstanceRequestMap();
 			final ExecutionStage executionStage = eg.getCurrentExecutionStage();
-			executionStage.collectRequiredInstanceTypes(instanceRequestMap, ExecutionState.CREATED);
-			assertEquals(1, instanceRequestMap.size());
-			assertEquals(1,
-				(int) instanceRequestMap.getMaximumNumberOfInstances(INSTANCE_MANAGER.getDefaultInstanceType()));
+			assertEquals(1, executionStage.getMaxNumberSubtasks());
 
 			// stage0
 			ExecutionStage es = eg.getStage(0);
@@ -523,12 +333,6 @@ public class ExecutionGraphTest {
 			ExecutionVertex ev0 = egv0.getGroupMember(0); // input1
 			ExecutionVertex ev1 = egv1.getGroupMember(0); // output1
 			ExecutionVertex ev2 = egv2.getGroupMember(0); // task1
-			// ev0 (input1)
-			assertEquals(INSTANCE_MANAGER.getDefaultInstanceType(), ev0.getAllocatedResource().getInstance().getType());
-			// ev1 (output1)
-			assertEquals(INSTANCE_MANAGER.getDefaultInstanceType(), ev1.getAllocatedResource().getInstance().getType());
-			// ev2 (task1)
-			assertEquals(INSTANCE_MANAGER.getDefaultInstanceType(), ev2.getAllocatedResource().getInstance().getType());
 			assertEquals(ev0.getAllocatedResource(), ev1.getAllocatedResource());
 			assertEquals(ev0.getAllocatedResource(), ev2.getAllocatedResource());
 		} catch (GraphConversionException e) {
@@ -618,15 +422,11 @@ public class ExecutionGraphTest {
 
 			LibraryCacheManager.register(jobID, new String[0]);
 
-			final ExecutionGraph eg = new ExecutionGraph(jg, INSTANCE_MANAGER);
+			final ExecutionGraph eg = new ExecutionGraph(jg, 1);
 
 			// test instance types in ExecutionGraph
-			final InstanceRequestMap instanceRequestMap = new InstanceRequestMap();
 			final ExecutionStage executionStage = eg.getCurrentExecutionStage();
-			executionStage.collectRequiredInstanceTypes(instanceRequestMap, ExecutionState.CREATED);
-			assertEquals(1, instanceRequestMap.size());
-			assertEquals(2,
-				(int) instanceRequestMap.getMaximumNumberOfInstances(INSTANCE_MANAGER.getDefaultInstanceType()));
+			assertEquals(2, executionStage.getMaxNumberSubtasks());
 
 			// stage0
 			final ExecutionStage es = eg.getStage(0);
@@ -828,40 +628,32 @@ public class ExecutionGraphTest {
 			i1.setFileInputClass(FileLineReader.class);
 			i1.setFilePath(new Path(inputFile1.toURI()));
 			i1.setNumberOfSubtasks(4);
-			i1.setNumberOfSubtasksPerInstance(2);
 			final JobFileInputVertex i2 = new JobFileInputVertex("Input 2", jg);
 			i2.setFileInputClass(FileLineReader.class);
 			i2.setFilePath(new Path(inputFile2.toURI()));
 			i2.setNumberOfSubtasks(4);
-			i2.setNumberOfSubtasksPerInstance(2);
 			// task vertex
 			final JobTaskVertex t1 = new JobTaskVertex("Task 1", jg);
 			t1.setTaskClass(ForwardTask1Input1Output.class);
 			t1.setNumberOfSubtasks(4);
-			t1.setNumberOfSubtasksPerInstance(2);
 			final JobTaskVertex t2 = new JobTaskVertex("Task 2", jg);
 			t2.setTaskClass(ForwardTask1Input1Output.class);
 			t2.setNumberOfSubtasks(4);
-			t2.setNumberOfSubtasksPerInstance(2);
 			final JobTaskVertex t3 = new JobTaskVertex("Task 3", jg);
 			t3.setTaskClass(ForwardTask2Inputs1Output.class);
 			t3.setNumberOfSubtasks(8);
-			t3.setNumberOfSubtasksPerInstance(4);
 			final JobTaskVertex t4 = new JobTaskVertex("Task 4", jg);
 			t4.setTaskClass(ForwardTask1Input2Outputs.class);
 			t4.setNumberOfSubtasks(8);
-			t4.setNumberOfSubtasksPerInstance(4);
 			// output vertex
 			final JobFileOutputVertex o1 = new JobFileOutputVertex("Output 1", jg);
 			o1.setFileOutputClass(FileLineWriter.class);
 			o1.setFilePath(new Path(outputFile1.toURI()));
 			o1.setNumberOfSubtasks(4);
-			o1.setNumberOfSubtasksPerInstance(2);
 			final JobFileOutputVertex o2 = new JobFileOutputVertex("Output 2", jg);
 			o2.setFileOutputClass(FileLineWriter.class);
 			o2.setFilePath(new Path(outputFile2.toURI()));
 			o2.setNumberOfSubtasks(4);
-			o2.setNumberOfSubtasksPerInstance(2);
 			o1.setVertexToShareInstancesWith(o2);
 
 			// connect vertices
@@ -876,19 +668,14 @@ public class ExecutionGraphTest {
 			LibraryCacheManager.register(jobID, new String[0]);
 
 			// now convert job graph to execution graph
-			final ExecutionGraph eg = new ExecutionGraph(jg, INSTANCE_MANAGER);
+			final ExecutionGraph eg = new ExecutionGraph(jg, 1);
 
 			// test instance types in ExecutionGraph
-			final InstanceRequestMap instanceRequestMap = new InstanceRequestMap();
 			ExecutionStage executionStage = eg.getCurrentExecutionStage();
 			assertNotNull(executionStage);
 			assertEquals(0, executionStage.getStageNumber());
 			
-			executionStage.collectRequiredInstanceTypes(instanceRequestMap, ExecutionState.CREATED);
-			assertEquals(1, instanceRequestMap.size());
-			assertEquals(8,
-				(int) instanceRequestMap.getMaximumNumberOfInstances(INSTANCE_MANAGER
-					.getInstanceTypeByName(DEFAULT_INSTANCE_TYPE_NAME)));
+			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);
@@ -903,7 +690,6 @@ public class ExecutionGraphTest {
 				ev.updateExecutionState(ExecutionState.FINISHING);
 				ev.updateExecutionState(ExecutionState.FINISHED);
 			}
-			instanceRequestMap.clear();
 		} catch (GraphConversionException e) {
 			fail(e.getMessage());
 		} catch (JobGraphDefinitionException e) {
@@ -983,7 +769,7 @@ public class ExecutionGraphTest {
 			LibraryCacheManager.register(jobID, new String[0]);
 
 			// now convert job graph to execution graph
-			final ExecutionGraph eg = new ExecutionGraph(jg, INSTANCE_MANAGER);
+			final ExecutionGraph eg = new ExecutionGraph(jg, 1);
 
 			assertEquals(1, eg.getNumberOfStages());
 
@@ -1116,9 +902,9 @@ public class ExecutionGraphTest {
 			input1.connectTo(forward1, ChannelType.IN_MEMORY,
 				DistributionPattern.POINTWISE);
 			forward1.connectTo(forward2, ChannelType.IN_MEMORY,
-				DistributionPattern.POINTWISE);
+					DistributionPattern.POINTWISE);
 			forward2.connectTo(forward3, ChannelType.NETWORK,
-				DistributionPattern.POINTWISE);
+					DistributionPattern.POINTWISE);
 			forward3.connectTo(output1, ChannelType.IN_MEMORY);
 
 			// setup instance sharing
@@ -1130,7 +916,7 @@ public class ExecutionGraphTest {
 			LibraryCacheManager.register(jobID, new String[0]);
 
 			// now convert job graph to execution graph
-			final ExecutionGraph eg = new ExecutionGraph(jg, INSTANCE_MANAGER);
+			final ExecutionGraph eg = new ExecutionGraph(jg, 1);
 
 			// Check number of stages
 			assertEquals(1, eg.getNumberOfStages());
@@ -1139,16 +925,8 @@ public class ExecutionGraphTest {
 			final ExecutionStage stage = eg.getStage(0);
 			assertEquals(5, stage.getNumberOfStageMembers());
 
-			// Check number of required instances
-			final InstanceRequestMap instanceRequestMap = new InstanceRequestMap();
-			stage.collectRequiredInstanceTypes(instanceRequestMap, ExecutionState.CREATED);
-
-			// First, we expect all required instances to be of the same type
-			assertEquals(1, instanceRequestMap.size());
-
-			final int numberOfRequiredInstances = instanceRequestMap.getMinimumNumberOfInstances(INSTANCE_MANAGER
-				.getDefaultInstanceType());
-			assertEquals(degreeOfParallelism, numberOfRequiredInstances);
+			final int numberOfRequiredSlots = stage.getMaxNumberSubtasks();
+			assertEquals(degreeOfParallelism, numberOfRequiredSlots);
 
 		} catch (GraphConversionException e) {
 			fail(e.getMessage());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/cluster/ClusterManagerTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/cluster/ClusterManagerTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/cluster/ClusterManagerTest.java
deleted file mode 100644
index 72d58c9..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/cluster/ClusterManagerTest.java
+++ /dev/null
@@ -1,273 +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.instance.cluster;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import java.net.InetAddress;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import eu.stratosphere.configuration.ConfigConstants;
-import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.configuration.GlobalConfiguration;
-import eu.stratosphere.nephele.instance.AllocatedResource;
-import eu.stratosphere.nephele.instance.AllocationID;
-import eu.stratosphere.nephele.instance.HardwareDescription;
-import eu.stratosphere.nephele.instance.HardwareDescriptionFactory;
-import eu.stratosphere.nephele.instance.InstanceConnectionInfo;
-import eu.stratosphere.nephele.instance.InstanceException;
-import eu.stratosphere.nephele.instance.InstanceRequestMap;
-import eu.stratosphere.nephele.instance.InstanceType;
-import eu.stratosphere.nephele.instance.InstanceTypeDescription;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.util.LogUtils;
-
-/**
- * Tests for {@link ClusterManager}.
- */
-public class ClusterManagerTest {
-
-	@BeforeClass
-	public static void initLogging() {
-		LogUtils.initializeDefaultTestConsoleLogger();
-	}
-	
-	
-	@Test
-	public void testInstanceRegistering() {
-		try {
-			ClusterManager cm = new ClusterManager();
-			TestInstanceListener testInstanceListener = new TestInstanceListener();
-			cm.setInstanceListener(testInstanceListener);
-			
-			
-			int ipcPort = ConfigConstants.DEFAULT_TASK_MANAGER_IPC_PORT;
-			int dataPort = ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT;
-
-			HardwareDescription hardwareDescription = HardwareDescriptionFactory.construct(2, 2L * 1024L * 1024L * 1024L,
-																				2L * 1024L * 1024L * 1024L);
-
-			String hostname = "192.168.198.1";
-			InetAddress address = InetAddress.getByName("192.168.198.1");
-			
-			InstanceConnectionInfo ici1 = new InstanceConnectionInfo(address, hostname, null, ipcPort + 0, dataPort + 0);
-			InstanceConnectionInfo ici2 = new InstanceConnectionInfo(address, hostname, null, ipcPort + 15, dataPort + 15);
-			InstanceConnectionInfo ici3 = new InstanceConnectionInfo(address, hostname, null, ipcPort + 30, dataPort + 30);
-			
-			// register three instances
-			cm.reportHeartBeat(ici1, hardwareDescription);
-			cm.reportHeartBeat(ici2, hardwareDescription);
-			cm.reportHeartBeat(ici3, hardwareDescription);
-			
-			
-			Map<InstanceType, InstanceTypeDescription> instanceTypeDescriptions = cm.getMapOfAvailableInstanceTypes();
-			assertEquals(1, instanceTypeDescriptions.size());
-			
-			InstanceTypeDescription descr = instanceTypeDescriptions.entrySet().iterator().next().getValue();
-			
-			assertEquals(3, descr.getMaximumNumberOfAvailableInstances());
-			
-			cm.shutdown();
-		}
-		catch (Exception e) {
-			System.err.println(e.getMessage());
-			e.printStackTrace();
-			Assert.fail("Test erroneous: " + e.getMessage());
-		}
-	}
-
-	@Test
-	public void testAllocationDeallocation() {
-		try {
-			ClusterManager cm = new ClusterManager();
-			TestInstanceListener testInstanceListener = new TestInstanceListener();
-			cm.setInstanceListener(testInstanceListener);
-			
-			
-			int ipcPort = ConfigConstants.DEFAULT_TASK_MANAGER_IPC_PORT;
-			int dataPort = ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT;
-
-			HardwareDescription hardwareDescription = HardwareDescriptionFactory.construct(2, 2L * 1024L * 1024L * 1024L,
-																				2L * 1024L * 1024L * 1024L);
-
-			String hostname = "192.168.198.1";
-			InetAddress address = InetAddress.getByName("192.168.198.1");
-			
-			InstanceConnectionInfo ici1 = new InstanceConnectionInfo(address, hostname, null, ipcPort + 0, dataPort + 0);
-			InstanceConnectionInfo ici2 = new InstanceConnectionInfo(address, hostname, null, ipcPort + 15, dataPort + 15);
-			
-			// register three instances
-			cm.reportHeartBeat(ici1, hardwareDescription);
-			cm.reportHeartBeat(ici2, hardwareDescription);
-			
-			
-			Map<InstanceType, InstanceTypeDescription> instanceTypeDescriptions = cm.getMapOfAvailableInstanceTypes();
-			assertEquals(1, instanceTypeDescriptions.size());
-			
-			InstanceTypeDescription descr = instanceTypeDescriptions.entrySet().iterator().next().getValue();
-			
-			assertEquals(2, descr.getMaximumNumberOfAvailableInstances());
-			
-			
-			// allocate something
-			JobID jobID = new JobID();
-			Configuration conf = new Configuration();
-			InstanceRequestMap instanceRequestMap = new InstanceRequestMap();
-			instanceRequestMap.setNumberOfInstances(cm.getDefaultInstanceType(), 2);
-			cm.requestInstance(jobID, conf, instanceRequestMap, null);
-			
-			ClusterManagerTestUtils.waitForInstances(jobID, testInstanceListener, 3, 1000);
-			
-			List<AllocatedResource> allocatedResources = testInstanceListener.getAllocatedResourcesForJob(jobID);
-			assertEquals(2, allocatedResources.size());
-			
-			Iterator<AllocatedResource> it = allocatedResources.iterator();
-			Set<AllocationID> allocationIDs = new HashSet<AllocationID>();
-			while (it.hasNext()) {
-				AllocatedResource allocatedResource = it.next();
-				if (ConfigConstants.DEFAULT_INSTANCE_TYPE.equals(allocatedResource.getInstance().getType().getIdentifier())) {
-					fail("Allocated unexpected instance of type "
-						+ allocatedResource.getInstance().getType().getIdentifier());
-				}
-
-				if (allocationIDs.contains(allocatedResource.getAllocationID())) {
-					fail("Discovered allocation ID " + allocatedResource.getAllocationID() + " at least twice");
-				} else {
-					allocationIDs.add(allocatedResource.getAllocationID());
-				}
-			}
-
-			// Try to allocate more resources which must result in an error
-			try {
-				InstanceRequestMap instancem = new InstanceRequestMap();
-				instancem.setNumberOfInstances(cm.getDefaultInstanceType(), 1);
-				cm.requestInstance(jobID, conf, instancem, null);
-
-				fail("ClusterManager allowed to request more instances than actually available");
-
-			} catch (InstanceException ie) {
-				// Exception is expected and correct behavior here
-			}
-
-			// Release all allocated resources
-			it = allocatedResources.iterator();
-			while (it.hasNext()) {
-				final AllocatedResource allocatedResource = it.next();
-				cm.releaseAllocatedResource(jobID, conf, allocatedResource);
-			}
-			
-			// Now further allocations should be possible
-			
-			InstanceRequestMap instancem = new InstanceRequestMap();
-			instancem.setNumberOfInstances(cm.getDefaultInstanceType(), 1);
-			cm.requestInstance(jobID, conf, instancem, null);
-			
-			
-			cm.shutdown();
-		}
-		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 {
-			
-			final int CLEANUP_INTERVAL = 2;
-			
-			// configure a short cleanup interval
-			Configuration config = new Configuration();
-			config.setInteger("instancemanager.cluster.cleanupinterval", CLEANUP_INTERVAL);
-			GlobalConfiguration.includeConfiguration(config);
-			
-			ClusterManager cm = new ClusterManager();
-			TestInstanceListener testInstanceListener = new TestInstanceListener();
-			cm.setInstanceListener(testInstanceListener);
-			
-			
-			int ipcPort = ConfigConstants.DEFAULT_TASK_MANAGER_IPC_PORT;
-			int dataPort = ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT;
-
-			HardwareDescription hardwareDescription = HardwareDescriptionFactory.construct(2, 2L * 1024L * 1024L * 1024L,
-																				2L * 1024L * 1024L * 1024L);
-
-			String hostname = "192.168.198.1";
-			InetAddress address = InetAddress.getByName("192.168.198.1");
-			
-			InstanceConnectionInfo ici1 = new InstanceConnectionInfo(address, hostname, null, ipcPort + 0, dataPort + 0);
-			InstanceConnectionInfo ici2 = new InstanceConnectionInfo(address, hostname, null, ipcPort + 15, dataPort + 15);
-			InstanceConnectionInfo ici3 = new InstanceConnectionInfo(address, hostname, null, ipcPort + 30, dataPort + 30);
-			
-			// register three instances
-			cm.reportHeartBeat(ici1, hardwareDescription);
-			cm.reportHeartBeat(ici2, hardwareDescription);
-			cm.reportHeartBeat(ici3, hardwareDescription);
-			
-			
-			Map<InstanceType, InstanceTypeDescription> instanceTypeDescriptions = cm.getMapOfAvailableInstanceTypes();
-			assertEquals(1, instanceTypeDescriptions.size());
-			
-			InstanceTypeDescription descr = instanceTypeDescriptions.entrySet().iterator().next().getValue();
-			assertEquals(3, descr.getMaximumNumberOfAvailableInstances());
-			
-			// request some instances
-			JobID jobID = new JobID();
-			Configuration conf = new Configuration();
-
-			InstanceRequestMap instancem = new InstanceRequestMap();
-			instancem.setNumberOfInstances(cm.getDefaultInstanceType(), 1);
-			cm.requestInstance(jobID, conf, instancem, null);
-			
-			ClusterManagerTestUtils.waitForInstances(jobID, testInstanceListener, 1, 1000);
-			assertEquals(1, testInstanceListener.getNumberOfAllocatedResourcesForJob(jobID));
-			
-			// wait for the cleanup to kick in
-			Thread.sleep(2000 * CLEANUP_INTERVAL);
-			
-			// check that the instances are gone
-			ClusterManagerTestUtils.waitForInstances(jobID, testInstanceListener, 0, 1000);
-			assertEquals(0, testInstanceListener.getNumberOfAllocatedResourcesForJob(jobID));
-			
-			
-			instanceTypeDescriptions = cm.getMapOfAvailableInstanceTypes();
-			assertEquals(1, instanceTypeDescriptions.size());
-			
-			descr = instanceTypeDescriptions.entrySet().iterator().next().getValue();
-			assertEquals(0, descr.getMaximumNumberOfAvailableInstances());
-			
-			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/86d206c4/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/cluster/ClusterManagerTestUtils.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/cluster/ClusterManagerTestUtils.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/cluster/ClusterManagerTestUtils.java
deleted file mode 100644
index e311533..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/instance/cluster/ClusterManagerTestUtils.java
+++ /dev/null
@@ -1,66 +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.instance.cluster;
-
-import eu.stratosphere.nephele.instance.InstanceListener;
-import eu.stratosphere.nephele.jobgraph.JobID;
-
-/**
- * This class contains utility methods used during the tests of the {@link ClusterManager} implementation.
- * 
- */
-public class ClusterManagerTestUtils {
-
-	/**
-	 * Granularity of the sleep time.
-	 */
-	private static final long SLEEP_TIME = 10; // 10 milliseconds
-
-	/**
-	 * Private constructor so the class cannot be instantiated.
-	 */
-	private ClusterManagerTestUtils() {
-	}
-
-	/**
-	 * Waits until a specific number of instances have registered or deregistrations with the given
-	 * {@link InstanceListener} object for a given job or the maximum wait time has elapsed.
-	 * 
-	 * @param jobID
-	 *        the ID of the job to check the instance registration for
-	 * @param instanceListener
-	 *        the listener which shall be notified when a requested instance is available for the job
-	 * @param numberOfInstances
-	 *        the number of registered instances to wait for
-	 * @param maxWaitTime
-	 *        the maximum wait time before this method returns
-	 */
-	public static void waitForInstances(JobID jobID, TestInstanceListener instanceListener,
-			int numberOfInstances, long maxWaitTime) {
-
-		final long startTime = System.currentTimeMillis();
-
-		while (instanceListener.getNumberOfAllocatedResourcesForJob(jobID) != numberOfInstances) {
-			try {
-				Thread.sleep(SLEEP_TIME);
-			} catch (InterruptedException e) {
-				break;
-			}
-
-			if ((System.currentTimeMillis() - startTime) >= maxWaitTime) {
-				break;
-			}
-		}
-	}
-}


[23/53] [abbrv] [FLINK-962] Initial import of documentation from website into source code (closes #34)

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/spargel_guide.md
----------------------------------------------------------------------
diff --git a/docs/spargel_guide.md b/docs/spargel_guide.md
new file mode 100644
index 0000000..5766f8b
--- /dev/null
+++ b/docs/spargel_guide.md
@@ -0,0 +1,112 @@
+---
+title: "Spargel Graph Processing API"
+---
+
+Spargel
+=======
+
+Spargel is our [Giraph](http://giraph.apache.org) like **graph processing** Java API. It supports basic graph computations, which are run as a sequence of [supersteps]({{ site.baseurl }}/docs/0.4/programming_guides/iterations.html#supersteps). Spargel and Giraph both implement the [Bulk Synchronous Parallel (BSP)](https://en.wikipedia.org/wiki/Bulk_Synchronous_Parallel) programming model, propsed by Google's [Pregel](http://googleresearch.blogspot.de/2009/06/large-scale-graph-computing-at-google.html).
+
+The API provides a **vertex-centric** view on graph processing with two basic operations per superstep:
+
+  1. **Send messages** to other vertices, and
+  2. **Receive messages** from other vertices and **update own vertex state**.
+
+This vertex-centric view makes it easy to express a large class of graph problems efficiently. We will list all *relevant interfaces* of the **Spargel API** to implement and walk through an **example Spargel program**.
+
+Spargel API
+-----------
+
+The Spargel API is part of the *addons* Maven project. All relevant classes are located in the *eu.stratosphere.spargel.java* package.
+
+Add the following dependency to your `pom.xml` to use the Spargel.
+
+```xml
+<dependency>
+	<groupId>eu.stratosphere</groupId>
+	<artifactId>spargel</artifactId>
+	<version>{{site.current_stable}}</version>
+</dependency>
+```
+
+Extend **VertexUpdateFunction&lt;***VertexKeyType*, *VertexValueType*, *MessageType***&gt;** to implement your *custom vertex update logic*.
+
+Extend **MessagingFunction&lt;***VertexKeyType*, *VertexValueType*, *MessageType*, *EdgeValueType***&gt;** to implement your *custom message logic*.
+
+Create a **SpargelIteration** operator to include Spargel in your data flow.
+
+Example: Propagate Minimum Vertex ID in Graph
+---------------------------------------------
+
+The Spargel operator **SpargelIteration** includes Spargel graph processing into your data flow. As usual, it can be combined with other operators like *map*, *reduce*, *join*, etc.
+
+{% highlight java %}
+FileDataSource vertices = new FileDataSource(...);
+FileDataSource edges = new FileDataSource(...);
+
+SpargelIteration iteration = new SpargelIteration(new MinMessager(), new MinNeighborUpdater());
+iteration.setVertexInput(vertices);
+iteration.setEdgesInput(edges);
+iteration.setNumberOfIterations(maxIterations);
+
+FileDataSink result = new FileDataSink(...);
+result.setInput(iteration.getOutput());
+
+new Plan(result);
+{% endhighlight %}
+Besides the **program logic** of vertex updates in *MinNeighborUpdater* and messages in *MinMessager*, you have to specify the **initial vertex** and **edge input**. Every vertex has a **key** and **value**. In each superstep, it **receives messages** from other vertices and updates its state:
+
+  - **Vertex** input: **(id**: *VertexKeyType*, **value**: *VertexValueType***)**
+  - **Edge** input: **(source**: *VertexKeyType*, **target**: *VertexKeyType*[, **value**: *EdgeValueType*])
+
+For our example, we set the vertex ID as both *id and value* (initial minimum) and *leave out the edge values* as we don't need them:
+
+<p class="text-center">
+    <img alt="Spargel Example Input" width="75%" src="{{ site.baseurl }}/docs/0.4/img/spargel_example_input.png" />
+</p>
+
+In order to **propagate the minimum vertex ID**, we iterate over all received messages (which contain the neighboring IDs) and update our value, if we found a new minimum:
+
+{% highlight java %}
+public class MinNeighborUpdater extends VertexUpdateFunction<IntValue, IntValue, IntValue> {
+	
+	@Override
+	public void updateVertex(IntValue id, IntValue currentMin, Iterator<IntValue> messages) {
+		int min = Integer.MAX_VALUE;
+
+		// iterate over all received messages
+		while (messages.hasNext()) {
+			int next = messages.next().getValue();
+			min = next < min ? next : min;
+		}
+
+		// update vertex value, if new minimum
+		if (min < currentMin.getValue()) {
+			setNewVertexValue(new IntValue(min));
+		}
+	}
+}
+{% endhighlight %}
+
+The **messages in each superstep** consist of the **current minimum ID** seen by the vertex:
+
+{% highlight java %}
+public class MinMessager extends MessagingFunction<IntValue, IntValue, IntValue, NullValue> {
+	
+	@Override
+	public void sendMessages(IntValue id, IntValue currentMin) {
+		// send current minimum to neighbors
+		sendMessageToAllNeighbors(currentMin);
+    }
+}
+{% endhighlight %}
+
+The **API-provided method** `sendMessageToAllNeighbors(MessageType)` sends the message to all neighboring vertices. It is also possible to address specific vertices with `sendMessageTo(VertexKeyType, MessageType)`.
+
+If the value of a vertex does not change during a superstep, it will **not send** any messages in the superstep. This allows to do incremental updates to the **hot (changing) parts** of the graph, while leaving **cold (steady) parts** untouched.
+
+The computation **terminates** after a specified *maximum number of supersteps* **-OR-** the *vertex states stop changing*.
+
+<p class="text-center">
+    <img alt="Spargel Example" width="75%" src="{{ site.baseurl }}/docs/0.4/img/spargel_example.png" />
+</p>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/web_client.md
----------------------------------------------------------------------
diff --git a/docs/web_client.md b/docs/web_client.md
new file mode 100644
index 0000000..98cfd69
--- /dev/null
+++ b/docs/web_client.md
@@ -0,0 +1,53 @@
+---
+title:  "Web Client"
+---
+
+Stratosphere provides a web interface to upload jobs, inspect their execution plans, and execute them. The interface is a great tool to showcase programs, debug execution plans, or demonstrate the system as a whole.
+
+# Start, Stop, and Configure the Web Interface
+
+Start the web interface by executing:
+
+    ./bin/start-webclient.sh
+
+and stop it by calling:
+
+    ./bin/stop-webclient.sh
+
+The web interface runs on port 8080 by default. To specify a custom port set the ```webclient.port``` property in the *./conf/stratosphere.yaml* configuration file. Jobs are submitted to the JobManager specified by ```jobmanager.rpc.address``` and ```jobmanager.rpc.port```. Please consult the [configuration](../setup/config.html#web_frontend "Configuration") page for details and further configuration options.
+
+# Use the Web Interface
+
+The web interface provides two views:
+
+1.  The **job view** to upload, preview, and submit Stratosphere programs.
+2.  The **plan view** to analyze the optimized execution plans of Stratosphere programs.
+
+## Job View
+
+The interface starts serving the job view. 
+
+You can **upload** a Stratosphere program as a jar file. To **execute** an uploaded program:
+
+* select it from the job list on the left, 
+* enter the program arguments in the *"Arguments"* field (bottom left), and 
+* click on the *"Run Job"* button (bottom right).
+
+If the *“Show optimizer plan”* option is enabled (default), the *plan view* is display next, otherwise the job is directly submitted to the JobManager for execution.
+
+In case the jar's manifest file does not specify the program class, you can specify it before the argument list as:
+
+```
+assembler <assemblerClass> <programArgs...>
+```
+
+## Plan View
+
+The plan view shows the optimized execution plan of the submitted program in the upper half of the page. The bottom part of the page displays detailed information about the currently selected plan operator including:
+
+* the chosen shipping strategies (local forward, hash partition, range partition, broadcast, ...),
+* the chosen local strategy (sort, hash join, merge join, ...),
+* inferred data properties (partitioning, grouping, sorting), and 
+* used optimizer estimates (data size, I/O and network costs, ...).
+
+To submit the job for execution, click again on the *"Run Job"* button in the bottom right.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/yarn_setup.md
----------------------------------------------------------------------
diff --git a/docs/yarn_setup.md b/docs/yarn_setup.md
new file mode 100644
index 0000000..c317e06
--- /dev/null
+++ b/docs/yarn_setup.md
@@ -0,0 +1,188 @@
+---
+title:  "YARN Setup"
+---
+
+# In a Nutshell
+
+Start YARN session with 4 Taskmanagers (each with 4 GB of Heapspace):
+
+```bash
+wget {{ site.docs_05_yarn_archive }}
+tar xvzf stratosphere-dist-{{ site.docs_05_stable }}-yarn.tar.gz
+cd stratosphere-yarn-{{ site.docs_05_stable }}/
+./bin/yarn-session.sh -n 4 -jm 1024 -tm 4096
+```
+
+# Introducing YARN
+
+Apache [Hadoop YARN](http://hadoop.apache.org/) is a cluster resource management framework. It allows to run various distributed applications on top of a cluster. Stratosphere runs on YARN next to other applications. Users do not have to setup or install anything if there is already a YARN setup.
+
+**Requirements**
+
+- Apache Hadoop 2.2
+- HDFS
+
+If you have troubles using the Stratosphere YARN client, have a look in the [FAQ section]({{site.baseurl}}/docs/0.5/general/faq.html).
+
+## Start Stratosphere Session
+
+Follow these instructions to learn how to launch a Stratosphere Session within your YARN cluster.
+
+A session will start all required Stratosphere services (JobManager and TaskManagers) so that you can submit programs to the cluster. Note that you can run multiple programs per session.
+
+### Download Stratosphere for YARN
+
+Download the YARN tgz package on the [download page]({{site.baseurl}}/downloads/#nightly). It contains the required files.
+
+
+If you want to build the YARN .tgz file from sources, follow the build instructions. Make sure to use the `-Dhadoop.profile=2` profile. You can find the file in `stratosphere-dist/target/stratosphere-dist-{{site.docs_05_stable}}-yarn.tar.gz` (*Note: The version might be different for you* ).
+
+Extract the package using:
+
+```bash
+tar xvzf stratosphere-dist-{{site.docs_05_stable}}-yarn.tar.gz
+cd stratosphere-yarn-{{site.docs_05_stable}}/
+```
+
+### Start a Session
+
+Use the following command to start a session
+
+```bash
+./bin/yarn-session.sh
+```
+
+This command will show you the following overview:
+
+```bash
+Usage:
+   Required
+     -n,--container <arg>   Number of Yarn container to allocate (=Number of TaskTrackers)
+   Optional
+     -jm,--jobManagerMemory <arg>    Memory for JobManager Container [in MB]
+     -q,--query                      Display available YARN resources (memory, cores)
+     -qu,--queue <arg>               Specify YARN queue.
+     -tm,--taskManagerMemory <arg>   Memory per TaskManager Container [in MB]
+     -tmc,--taskManagerCores <arg>   Virtual CPU cores per TaskManager
+     -v,--verbose                    Verbose debug mode
+```
+
+Please note that the Client requires the `HADOOP_HOME` (or `YARN_CONF_DIR` or `HADOOP_CONF_DIR`) environment variable to be set to read the YARN and HDFS configuration.
+
+**Example:** Issue the following command to allocate 10 TaskTrackers, with 8 GB of memory each:
+
+```bash
+./bin/yarn-session.sh -n 10 -tm 8192
+```
+
+The system will use the configuration in `conf/stratosphere-config.yaml`. Please follow our [configuration guide]({{site.baseurl}}/docs/0.5/setup/config.html) if you want to change something. Stratosphere on YARN will overwrite the following configuration parameters `jobmanager.rpc.address` (because the JobManager is always allocated at different machines) and `taskmanager.tmp.dirs` (we are using the tmp directories given by YARN).
+
+The example invocation starts 11 containers, since there is one additional container for the ApplicationMaster and JobTracker.
+
+Once Stratosphere is deployed in your YARN cluster, it will show you the connection details of the JobTracker.
+
+The client has to remain open to keep the deployment running. We suggest to use `screen`, which will start a detachable shell:
+
+1. Open `screen`,
+2. Start Stratosphere on YARN,
+3. Use `CTRL+a`, then press `d` to detach the screen session,
+4. Use `screen -r` to resume again.
+
+# Submit Job to Stratosphere
+
+Use the following command to submit a Stratosphere program to the YARN cluster:
+
+```bash
+./bin/stratosphere
+```
+
+Please refer to the documentation of the [commandline client]({{site.baseurl}}/docs/0.5/program_execution/cli_client.html).
+
+The command will show you a help menu like this:
+
+```bash
+[...]
+Action "run" compiles and submits a Stratosphere program.
+  "run" action arguments:
+     -a,--arguments <programArgs>   Program arguments
+     -c,--class <classname>         Program class
+     -j,--jarfile <jarfile>         Stratosphere program JAR file
+     -m,--jobmanager <host:port>    Jobmanager to which the program is submitted
+     -w,--wait                      Wait for program to finish
+[...]
+```
+
+Use the *run* action to submit a job to YARN. The client is able to determine the address of the JobManager. In the rare event of a problem, you can also pass the JobManager address using the `-m` argument. The JobManager address is visible in the YARN console.
+
+**Example**
+
+```bash
+wget -O apache-license-v2.txt http://www.apache.org/licenses/LICENSE-2.0.txt
+
+./bin/stratosphere run -j ./examples/stratosphere-java-examples-{{site.docs_05_stable}}-WordCount.jar \
+                       -a 1 file://`pwd`/apache-license-v2.txt file://`pwd`/wordcount-result.txt 
+```
+
+If there is the following error, make sure that all TaskManagers started:
+
+```bash
+Exception in thread "main" eu.stratosphere.compiler.CompilerException:
+    Available instances could not be determined from job manager: Connection timed out.
+```
+
+You can check the number of TaskManagers in the JobManager web interface. The address of this interface is printed in the YARN session console.
+
+If the TaskManagers do not show up after a minute, you should investigate the issue using the log files.
+
+# Build Stratosphere for a specific Hadoop Version
+
+This section covers building Stratosphere for a specific Hadoop version. Most users do not need to do this manually.
+The problem is that Stratosphere uses HDFS and YARN which are both from Apache Hadoop. There exist many different builds of Hadoop (from both the upstream project and the different Hadoop distributions). Typically errors arise with the RPC services. An error could look like this:
+
+```
+ERROR: The job was not successfully submitted to the nephele job manager:
+    eu.stratosphere.nephele.executiongraph.GraphConversionException: Cannot compute input splits for TSV:
+    java.io.IOException: Failed on local exception: com.google.protobuf.InvalidProtocolBufferException:
+    Protocol message contained an invalid tag (zero).; Host Details :
+```
+
+**Example**
+
+```
+mvn -Dhadoop.profile=2 -Pcdh-repo -Dhadoop.version=2.2.0-cdh5.0.0-beta-2 -DskipTests package
+```
+
+The commands in detail:
+
+*  `-Dhadoop.profile=2` activates the Hadoop YARN profile of Stratosphere. This will enable all components of Stratosphere that are compatible with Hadoop 2.2
+*  `-Pcdh-repo` activates the Cloudera Hadoop dependencies. If you want other vendor's Hadoop dependencies (not in maven central) add the repository to your local maven configuration in `~/.m2/`.
+* `-Dhadoop.version=2.2.0-cdh5.0.0-beta-2` sets a special version of the Hadoop dependencies. Make sure that the specified Hadoop version is compatible with the profile you activated.
+
+If you want to build HDFS for Hadoop 2 without YARN, use the following parameter:
+
+```
+-P!include-yarn
+```
+
+Some Cloudera versions (such as `2.0.0-cdh4.2.0`) require this, since they have a new HDFS version with the old YARN API.
+
+Please post to the [Stratosphere mailinglist](https://groups.google.com/d/forum/stratosphere-dev) or create an issue on [Github](https://github.com/stratosphere/stratosphere/issues), if you have issues with your YARN setup and Stratosphere.
+
+# Background
+
+This section briefly describes how Stratosphere and YARN interact. 
+
+<img src="{{site.baseurl}}/img/StratosphereOnYarn.svg" class="img-responsive">
+
+The YARN client needs to access the Hadoop configuration to connect to the YARN resource manager and to HDFS. It determines the Hadoop configuration using the following strategy:
+
+* Test if `YARN_CONF_DIR`, `HADOOP_CONF_DIR` or `HADOOP_CONF_PATH` are set (in that order). If one of these variables are set, they are used to read the configuration.
+* If the above strategy fails (this should not be the case in a correct YARN setup), the client is using the `HADOOP_HOME` environment variable. If it is set, the client tries to access `$HADOOP_HOME/etc/hadoop` (Hadoop 2) and `$HADOOP_HOME/conf` (Hadoop 1).
+
+When starting a new Stratosphere YARN session, the client first checks if the requested resources (containers and memory) are available. After that, it uploads a jar that contains Stratosphere and the configuration to HDFS (step 1).
+
+The next step of the client is to request (step 2) a YARN container to start the *ApplicationMaster* (step 3). Since the client registered the configuration and jar-file as a resource for the container, the NodeManager of YARN running on that particular machine will take care of preparing the container (e.g. downloading the files). Once that has finished, the *ApplicationMaster* (AM) is started.
+
+The *JobManager* and AM are running in the same container. Once they successfully started, the AM knows the address of the JobManager (its own host). It is generating a new Stratosphere configuration file for the TaskManagers (so that they can connect to the JobManager). The file is also uploaded to HDFS. Additionally, the *AM* container is also serving Stratosphere's web interface.
+
+After that, the AM starts allocating the containers for Stratosphere's TaskManagers, which will download the jar file and the modified configuration from the HDFS. Once these steps are completed, Stratosphere is set up and ready to accept Jobs.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 96acd1d..4c29132 100644
--- a/pom.xml
+++ b/pom.xml
@@ -368,6 +368,8 @@
 						<exclude>**/*.iml</exclude>
 						<!-- Generated content -->
 						<exclude>**/target/**</exclude>
+						<!-- Documentation -->
+						<exclude>**/docs/**</exclude>
 					</excludes>
 				</configuration>
 			</plugin>


[19/53] [abbrv] Merge fix to omit input/output registering on JobManager Rework Invokable Task Hierarchy

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/tasks/DoubleSourceTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/tasks/DoubleSourceTask.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/tasks/DoubleSourceTask.java
new file mode 100644
index 0000000..aa46af8
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/tasks/DoubleSourceTask.java
@@ -0,0 +1,134 @@
+/***********************************************************************************************************************
+ *
+ * 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.util.tasks;
+
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+import eu.stratosphere.core.fs.FSDataInputStream;
+import eu.stratosphere.core.fs.FileInputSplit;
+import eu.stratosphere.core.fs.FileSystem;
+import eu.stratosphere.core.io.StringRecord;
+import eu.stratosphere.nephele.template.AbstractInvokable;
+import eu.stratosphere.nephele.template.InputSplitProvider;
+import eu.stratosphere.runtime.io.api.RecordWriter;
+import eu.stratosphere.runtime.fs.LineReader;
+
+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/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/tasks/FileLineReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/tasks/FileLineReader.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/tasks/FileLineReader.java
new file mode 100644
index 0000000..c62911a
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/tasks/FileLineReader.java
@@ -0,0 +1,133 @@
+/***********************************************************************************************************************
+ *
+ * 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.util.tasks;
+
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+import eu.stratosphere.core.fs.FSDataInputStream;
+import eu.stratosphere.core.fs.FileInputSplit;
+import eu.stratosphere.core.fs.FileSystem;
+import eu.stratosphere.core.io.StringRecord;
+import eu.stratosphere.runtime.io.api.RecordWriter;
+import eu.stratosphere.nephele.template.AbstractInvokable;
+import eu.stratosphere.nephele.template.InputSplitProvider;
+import eu.stratosphere.runtime.fs.LineReader;
+
+/**
+ * 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/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/tasks/FileLineWriter.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/tasks/FileLineWriter.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/tasks/FileLineWriter.java
new file mode 100644
index 0000000..5f6e2b2
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/tasks/FileLineWriter.java
@@ -0,0 +1,72 @@
+/***********************************************************************************************************************
+ *
+ * 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.util.tasks;
+
+import eu.stratosphere.configuration.Configuration;
+import eu.stratosphere.core.fs.FSDataOutputStream;
+import eu.stratosphere.core.fs.FileStatus;
+import eu.stratosphere.core.fs.FileSystem;
+import eu.stratosphere.core.fs.Path;
+import eu.stratosphere.core.io.StringRecord;
+import eu.stratosphere.nephele.template.AbstractInvokable;
+import eu.stratosphere.runtime.io.api.RecordReader;
+
+/**
+ * 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/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/tasks/JobFileInputVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/tasks/JobFileInputVertex.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/tasks/JobFileInputVertex.java
new file mode 100644
index 0000000..fb0da91
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/tasks/JobFileInputVertex.java
@@ -0,0 +1,255 @@
+/***********************************************************************************************************************
+ *
+ * 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.util.tasks;
+
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import eu.stratosphere.core.fs.BlockLocation;
+import eu.stratosphere.core.fs.FileInputSplit;
+import eu.stratosphere.core.fs.FileStatus;
+import eu.stratosphere.core.fs.FileSystem;
+import eu.stratosphere.core.fs.Path;
+import eu.stratosphere.core.io.InputSplit;
+import eu.stratosphere.nephele.jobgraph.AbstractJobInputVertex;
+import eu.stratosphere.nephele.jobgraph.JobGraph;
+import eu.stratosphere.nephele.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 DataInput 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 DataOutput 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/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/tasks/JobFileOutputVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/tasks/JobFileOutputVertex.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/tasks/JobFileOutputVertex.java
new file mode 100644
index 0000000..593b520
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/tasks/JobFileOutputVertex.java
@@ -0,0 +1,109 @@
+/***********************************************************************************************************************
+ *
+ * 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.util.tasks;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import eu.stratosphere.core.fs.Path;
+import eu.stratosphere.nephele.jobgraph.AbstractJobOutputVertex;
+import eu.stratosphere.nephele.jobgraph.JobGraph;
+import eu.stratosphere.nephele.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 DataInput 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 DataOutput 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/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/hash/HashMatchIteratorITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/hash/HashMatchIteratorITCase.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/hash/HashMatchIteratorITCase.java
index a28ba38..e59f4a6 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/hash/HashMatchIteratorITCase.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/hash/HashMatchIteratorITCase.java
@@ -34,7 +34,7 @@ import eu.stratosphere.api.java.record.functions.JoinFunction;
 import eu.stratosphere.nephele.services.iomanager.IOManager;
 import eu.stratosphere.nephele.services.memorymanager.MemoryManager;
 import eu.stratosphere.nephele.services.memorymanager.spi.DefaultMemoryManager;
-import eu.stratosphere.nephele.template.AbstractTask;
+import eu.stratosphere.nephele.template.AbstractInvokable;
 import eu.stratosphere.api.java.typeutils.runtime.record.RecordComparator;
 import eu.stratosphere.api.java.typeutils.runtime.record.RecordPairComparator;
 import eu.stratosphere.api.java.typeutils.runtime.record.RecordSerializer;
@@ -67,7 +67,7 @@ public class HashMatchIteratorITCase {
 	private static final long SEED1 = 561349061987311L;
 	private static final long SEED2 = 231434613412342L;
 	
-	private final AbstractTask parentTask = new DummyInvokable();
+	private final AbstractInvokable parentTask = new DummyInvokable();
 
 	private IOManager ioManager;
 	private MemoryManager memoryManager;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/hash/ReOpenableHashTableITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/hash/ReOpenableHashTableITCase.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/hash/ReOpenableHashTableITCase.java
index d9c8b08..755d08a 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/hash/ReOpenableHashTableITCase.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/hash/ReOpenableHashTableITCase.java
@@ -38,7 +38,6 @@ import eu.stratosphere.nephele.services.memorymanager.MemoryAllocationException;
 import eu.stratosphere.nephele.services.memorymanager.MemoryManager;
 import eu.stratosphere.nephele.services.memorymanager.spi.DefaultMemoryManager;
 import eu.stratosphere.nephele.template.AbstractInvokable;
-import eu.stratosphere.nephele.template.AbstractTask;
 import eu.stratosphere.pact.runtime.hash.HashMatchIteratorITCase.RecordMatch;
 import eu.stratosphere.pact.runtime.hash.HashMatchIteratorITCase.RecordMatchRemovingJoin;
 import eu.stratosphere.pact.runtime.hash.HashTableITCase.ConstantsKeyValuePairsIterator;
@@ -75,7 +74,7 @@ public class ReOpenableHashTableITCase {
 	
 	private static final int NUM_PROBES = 3; // number of reopenings of hash join
 	
-	private final AbstractTask parentTask = new DummyInvokable();
+	private final AbstractInvokable parentTask = new DummyInvokable();
 
 	private IOManager ioManager;
 	private MemoryManager memoryManager;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/io/ChannelViewsTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/io/ChannelViewsTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/io/ChannelViewsTest.java
index fbe4f5b..c2be01a 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/io/ChannelViewsTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/io/ChannelViewsTest.java
@@ -31,7 +31,7 @@ import eu.stratosphere.nephele.services.iomanager.ChannelWriterOutputView;
 import eu.stratosphere.nephele.services.iomanager.IOManager;
 import eu.stratosphere.nephele.services.memorymanager.MemoryManager;
 import eu.stratosphere.nephele.services.memorymanager.spi.DefaultMemoryManager;
-import eu.stratosphere.nephele.template.AbstractTask;
+import eu.stratosphere.nephele.template.AbstractInvokable;
 import eu.stratosphere.pact.runtime.test.util.DummyInvokable;
 import eu.stratosphere.pact.runtime.test.util.TestData;
 import eu.stratosphere.pact.runtime.test.util.TestData.Generator.KeyMode;
@@ -63,7 +63,7 @@ public class ChannelViewsTest
 	
 	private static final int NUM_MEMORY_SEGMENTS = 3;
 	
-	private final AbstractTask parentTask = new DummyInvokable();
+	private final AbstractInvokable parentTask = new DummyInvokable();
 
 	private IOManager ioManager;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/io/SpillingBufferTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/io/SpillingBufferTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/io/SpillingBufferTest.java
index 1809540..c960280 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/io/SpillingBufferTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/io/SpillingBufferTest.java
@@ -28,7 +28,7 @@ import eu.stratosphere.nephele.services.iomanager.IOManager;
 import eu.stratosphere.nephele.services.memorymanager.ListMemorySegmentSource;
 import eu.stratosphere.nephele.services.memorymanager.MemoryManager;
 import eu.stratosphere.nephele.services.memorymanager.spi.DefaultMemoryManager;
-import eu.stratosphere.nephele.template.AbstractTask;
+import eu.stratosphere.nephele.template.AbstractInvokable;
 import eu.stratosphere.pact.runtime.test.util.DummyInvokable;
 import eu.stratosphere.pact.runtime.test.util.TestData;
 import eu.stratosphere.pact.runtime.test.util.TestData.Generator.KeyMode;
@@ -54,7 +54,7 @@ public class SpillingBufferTest {
 	
 	private static final int NUM_MEMORY_SEGMENTS = 23;
 	
-	private final AbstractTask parentTask = new DummyInvokable();
+	private final AbstractInvokable parentTask = new DummyInvokable();
 
 	private IOManager ioManager;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/AsynchonousPartialSorterITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/AsynchonousPartialSorterITCase.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/AsynchonousPartialSorterITCase.java
index 26ce081..f191075 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/AsynchonousPartialSorterITCase.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/AsynchonousPartialSorterITCase.java
@@ -30,7 +30,6 @@ import eu.stratosphere.nephele.services.memorymanager.MemoryAllocationException;
 import eu.stratosphere.nephele.services.memorymanager.MemoryManager;
 import eu.stratosphere.nephele.services.memorymanager.spi.DefaultMemoryManager;
 import eu.stratosphere.nephele.template.AbstractInvokable;
-import eu.stratosphere.nephele.template.AbstractTask;
 import eu.stratosphere.api.java.typeutils.runtime.record.RecordComparator;
 import eu.stratosphere.api.java.typeutils.runtime.record.RecordSerializerFactory;
 import eu.stratosphere.pact.runtime.test.util.DummyInvokable;
@@ -41,10 +40,9 @@ import eu.stratosphere.pact.runtime.test.util.TestData.Value;
 import eu.stratosphere.types.Record;
 import eu.stratosphere.util.MutableObjectIterator;
 
-/**
- */
-public class AsynchonousPartialSorterITCase
-{
+
+public class AsynchonousPartialSorterITCase {
+	
 	private static final Log LOG = LogFactory.getLog(AsynchonousPartialSorterITCase.class);
 
 	private static final long SEED = 649180756312423613L;
@@ -57,7 +55,7 @@ public class AsynchonousPartialSorterITCase
 
 	public static final int MEMORY_SIZE = 1024 * 1024 * 32;
 	
-	private final AbstractTask parentTask = new DummyInvokable();
+	private final AbstractInvokable parentTask = new DummyInvokable();
 
 	private IOManager ioManager;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/CombiningUnilateralSortMergerITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/CombiningUnilateralSortMergerITCase.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/CombiningUnilateralSortMergerITCase.java
index 1851480..b873f96 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/CombiningUnilateralSortMergerITCase.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/CombiningUnilateralSortMergerITCase.java
@@ -36,7 +36,7 @@ import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.nephele.services.iomanager.IOManager;
 import eu.stratosphere.nephele.services.memorymanager.MemoryManager;
 import eu.stratosphere.nephele.services.memorymanager.spi.DefaultMemoryManager;
-import eu.stratosphere.nephele.template.AbstractTask;
+import eu.stratosphere.nephele.template.AbstractInvokable;
 import eu.stratosphere.api.java.typeutils.runtime.record.RecordComparator;
 import eu.stratosphere.api.java.typeutils.runtime.record.RecordSerializerFactory;
 import eu.stratosphere.pact.runtime.test.util.DummyInvokable;
@@ -66,7 +66,7 @@ public class CombiningUnilateralSortMergerITCase {
 
 	public static final int MEMORY_SIZE = 1024 * 1024 * 256;
 
-	private final AbstractTask parentTask = new DummyInvokable();
+	private final AbstractInvokable parentTask = new DummyInvokable();
 	
 	private IOManager ioManager;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/ExternalSortITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/ExternalSortITCase.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/ExternalSortITCase.java
index 7ba42b9..cdb8421 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/ExternalSortITCase.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/ExternalSortITCase.java
@@ -28,7 +28,7 @@ import eu.stratosphere.api.common.typeutils.TypeSerializerFactory;
 import eu.stratosphere.nephele.services.iomanager.IOManager;
 import eu.stratosphere.nephele.services.memorymanager.MemoryManager;
 import eu.stratosphere.nephele.services.memorymanager.spi.DefaultMemoryManager;
-import eu.stratosphere.nephele.template.AbstractTask;
+import eu.stratosphere.nephele.template.AbstractInvokable;
 import eu.stratosphere.api.java.typeutils.runtime.record.RecordComparator;
 import eu.stratosphere.api.java.typeutils.runtime.record.RecordSerializerFactory;
 import eu.stratosphere.pact.runtime.test.util.DummyInvokable;
@@ -61,7 +61,7 @@ public class ExternalSortITCase {
 
 	private static final int MEMORY_SIZE = 1024 * 1024 * 78;
 	
-	private final AbstractTask parentTask = new DummyInvokable();
+	private final AbstractInvokable parentTask = new DummyInvokable();
 
 	private IOManager ioManager;
 
@@ -238,7 +238,7 @@ public class ExternalSortITCase {
 		merger.close();
 	}
 
-	@Test
+//	@Test
 	public void testSpillingSortWithIntermediateMerge() throws Exception {
 		// amount of pairs
 		final int PAIRS = 10000000;
@@ -292,7 +292,7 @@ public class ExternalSortITCase {
 		merger.close();
 	}
 	
-	@Test
+//	@Test
 	public void testSpillingSortWithIntermediateMergeIntPair() throws Exception {
 		// amount of pairs
 		final int PAIRS = 50000000;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/MassiveStringSortingITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/MassiveStringSortingITCase.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/MassiveStringSortingITCase.java
index f76b802..d9877f4 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/MassiveStringSortingITCase.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/MassiveStringSortingITCase.java
@@ -90,7 +90,7 @@ public class MassiveStringSortingITCase {
 				MutableObjectIterator<String> inputIterator = new StringReaderMutableObjectIterator(reader);
 				
 				sorter = new UnilateralSortMerger<String>(mm, ioMan, inputIterator, new DummyInvokable(),
-						new RuntimeStatelessSerializerFactory<String>(serializer, String.class), comparator, 1024 * 1024, 4, 0.8f);
+						new RuntimeStatelessSerializerFactory<String>(serializer, String.class), comparator, 1.0, 4, 0.8f);
 
 				MutableObjectIterator<String> sortedData = sorter.getIterator();
 				
@@ -182,7 +182,7 @@ public class MassiveStringSortingITCase {
 				MutableObjectIterator<Tuple2<String, String[]>> inputIterator = new StringTupleReaderMutableObjectIterator(reader);
 				
 				sorter = new UnilateralSortMerger<Tuple2<String, String[]>>(mm, ioMan, inputIterator, new DummyInvokable(),
-						new RuntimeStatelessSerializerFactory<Tuple2<String, String[]>>(serializer, (Class<Tuple2<String, String[]>>) (Class<?>) Tuple2.class), comparator, 1024 * 1024, 4, 0.8f);
+						new RuntimeStatelessSerializerFactory<Tuple2<String, String[]>>(serializer, (Class<Tuple2<String, String[]>>) (Class<?>) Tuple2.class), comparator, 1.0, 4, 0.8f);
 
 				
 				
@@ -219,10 +219,6 @@ public class MassiveStringSortingITCase {
 					
 					nextFromStratoSort = sortedData.next(nextFromStratoSort);
 					Assert.assertNotNull(nextFromStratoSort);
-						
-					if (nextFromStratoSort.f0.equals("http://some-uri.com/that/is/a/common/prefix/to/all(()HK;V3__.e*")) {
-						System.out.println("Found at position " + num);
-					}
 					
 					Assert.assertEquals(next.f0, nextFromStratoSort.f0);
 					Assert.assertArrayEquals(next.f1, nextFromStratoSort.f1);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/SortMergeMatchIteratorITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/SortMergeMatchIteratorITCase.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/SortMergeMatchIteratorITCase.java
index 0f3f558..81266d2 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/SortMergeMatchIteratorITCase.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/sort/SortMergeMatchIteratorITCase.java
@@ -32,7 +32,7 @@ import eu.stratosphere.api.java.record.functions.JoinFunction;
 import eu.stratosphere.nephele.services.iomanager.IOManager;
 import eu.stratosphere.nephele.services.memorymanager.MemoryManager;
 import eu.stratosphere.nephele.services.memorymanager.spi.DefaultMemoryManager;
-import eu.stratosphere.nephele.template.AbstractTask;
+import eu.stratosphere.nephele.template.AbstractInvokable;
 import eu.stratosphere.api.java.typeutils.runtime.record.RecordComparator;
 import eu.stratosphere.api.java.typeutils.runtime.record.RecordPairComparator;
 import eu.stratosphere.api.java.typeutils.runtime.record.RecordSerializer;
@@ -47,10 +47,9 @@ import eu.stratosphere.types.Value;
 import eu.stratosphere.util.Collector;
 import eu.stratosphere.util.MutableObjectIterator;
 
-/**
- */
-public class SortMergeMatchIteratorITCase
-{
+
+public class SortMergeMatchIteratorITCase {
+	
 	// total memory
 	private static final int MEMORY_SIZE = 1024 * 1024 * 16;
 	private static final int PAGES_FOR_BNLJN = 2;
@@ -66,7 +65,7 @@ public class SortMergeMatchIteratorITCase
 	private static final long SEED2 = 231434613412342L;
 	
 	// dummy abstract task
-	private final AbstractTask parentTask = new DummyInvokable();
+	private final AbstractInvokable parentTask = new DummyInvokable();
 
 	private IOManager ioManager;
 	private MemoryManager memoryManager;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/HashVsSortMiniBenchmark.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/HashVsSortMiniBenchmark.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/HashVsSortMiniBenchmark.java
index 2999436..b744348 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/HashVsSortMiniBenchmark.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/HashVsSortMiniBenchmark.java
@@ -25,7 +25,7 @@ import eu.stratosphere.api.java.record.functions.JoinFunction;
 import eu.stratosphere.nephele.services.iomanager.IOManager;
 import eu.stratosphere.nephele.services.memorymanager.MemoryManager;
 import eu.stratosphere.nephele.services.memorymanager.spi.DefaultMemoryManager;
-import eu.stratosphere.nephele.template.AbstractTask;
+import eu.stratosphere.nephele.template.AbstractInvokable;
 import eu.stratosphere.pact.runtime.hash.BuildFirstHashMatchIterator;
 import eu.stratosphere.pact.runtime.hash.BuildSecondHashMatchIterator;
 import eu.stratosphere.api.java.typeutils.runtime.record.RecordComparator;
@@ -67,7 +67,7 @@ public class HashVsSortMiniBenchmark {
 
 	
 	// dummy abstract task
-	private final AbstractTask parentTask = new DummyInvokable();
+	private final AbstractInvokable parentTask = new DummyInvokable();
 
 	// memory and io manager
 	private IOManager ioManager;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/DummyInvokable.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/DummyInvokable.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/DummyInvokable.java
index cb0b958..7a4e09e 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/DummyInvokable.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/DummyInvokable.java
@@ -13,14 +13,12 @@
 
 package eu.stratosphere.pact.runtime.test.util;
 
-import eu.stratosphere.nephele.template.AbstractTask;
+import eu.stratosphere.nephele.template.AbstractInvokable;
 
 /**
  * An invokable that does nothing.
- *
  */
-public class DummyInvokable extends AbstractTask
-{
+public class DummyInvokable extends AbstractInvokable {
 
 	@Override
 	public void registerInputOutput() {}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/TaskTestBase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/TaskTestBase.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/TaskTestBase.java
index a60b479..efa69af 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/TaskTestBase.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/TaskTestBase.java
@@ -27,9 +27,7 @@ import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.core.fs.FileSystem.WriteMode;
 import eu.stratosphere.core.fs.Path;
 import eu.stratosphere.nephele.services.memorymanager.MemoryManager;
-import eu.stratosphere.nephele.template.AbstractInputTask;
-import eu.stratosphere.nephele.template.AbstractOutputTask;
-import eu.stratosphere.nephele.template.AbstractTask;
+import eu.stratosphere.nephele.template.AbstractInvokable;
 import eu.stratosphere.api.java.typeutils.runtime.record.RecordSerializerFactory;
 import eu.stratosphere.pact.runtime.shipping.ShipStrategyType;
 import eu.stratosphere.pact.runtime.task.DataSinkTask;
@@ -77,7 +75,7 @@ public abstract class TaskTestBase {
 		return this.mockEnv.getTaskConfiguration();
 	}
 
-	public void registerTask(AbstractTask task, @SuppressWarnings("rawtypes") Class<? extends PactDriver> driver, Class<? extends Function> stubClass) {
+	public void registerTask(AbstractInvokable task, @SuppressWarnings("rawtypes") Class<? extends PactDriver> driver, Class<? extends Function> stubClass) {
 		final TaskConfig config = new TaskConfig(this.mockEnv.getTaskConfiguration());
 		config.setDriver(driver);
 		config.setStubWrapper(new UserCodeClassWrapper<Function>(stubClass));
@@ -91,17 +89,16 @@ public abstract class TaskTestBase {
 		task.registerInputOutput();
 	}
 
-	public void registerTask(AbstractTask task) {
+	public void registerTask(AbstractInvokable task) {
 		task.setEnvironment(this.mockEnv);
 		task.registerInputOutput();
 	}
 
-	public void registerFileOutputTask(AbstractOutputTask outTask, Class<? extends FileOutputFormat> stubClass, String outPath)
-	{
+	public void registerFileOutputTask(AbstractInvokable outTask, Class<? extends FileOutputFormat> stubClass, String outPath) {
 		registerFileOutputTask(outTask, InstantiationUtil.instantiate(stubClass, FileOutputFormat.class), outPath);
 	}
 	
-	public void registerFileOutputTask(AbstractOutputTask outTask, FileOutputFormat outputFormat, String outPath) {
+	public void registerFileOutputTask(AbstractInvokable outTask, FileOutputFormat outputFormat, String outPath) {
 		TaskConfig dsConfig = new TaskConfig(this.mockEnv.getTaskConfiguration());
 		
 		outputFormat.setOutputFilePath(new Path(outPath));
@@ -118,7 +115,7 @@ public abstract class TaskTestBase {
 		outTask.registerInputOutput();
 	}
 
-	public void registerFileInputTask(AbstractInputTask<?> inTask,
+	public void registerFileInputTask(AbstractInvokable inTask,
 			Class<? extends DelimitedInputFormat> stubClass, String inPath, String delimiter)
 	{
 		DelimitedInputFormat format;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPoolTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPoolTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPoolTest.java
index c7d8d41..4f9313f 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPoolTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPoolTest.java
@@ -15,6 +15,8 @@ package eu.stratosphere.runtime.io.network.bufferprovider;
 
 import eu.stratosphere.runtime.io.Buffer;
 import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider.BufferAvailabilityRegistration;
+import eu.stratosphere.util.LogUtils;
+
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -33,6 +35,10 @@ import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 
 public class LocalBufferPoolTest {
+	
+	static {
+		LogUtils.initializeDefaultTestConsoleLogger();
+	}
 
 	private final static int NUM_BUFFERS = 2048;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/TransitiveClosureITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/TransitiveClosureITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/TransitiveClosureITCase.java
index 96761c8..2c5fa9d 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/TransitiveClosureITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/TransitiveClosureITCase.java
@@ -41,7 +41,7 @@ public class TransitiveClosureITCase extends JavaProgramTestBase {
 
 	@Override
 	protected void testProgram() throws Exception {
-		TransitiveClosureNaive.main(edgesPath, resultPath, "100");
+		TransitiveClosureNaive.main(edgesPath, resultPath, "5");
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/JobGraphUtils.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/JobGraphUtils.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/JobGraphUtils.java
index 109c91a..d18160b 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/JobGraphUtils.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/JobGraphUtils.java
@@ -30,7 +30,6 @@ import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException;
 import eu.stratosphere.nephele.jobgraph.JobInputVertex;
 import eu.stratosphere.nephele.jobgraph.JobOutputVertex;
 import eu.stratosphere.nephele.jobgraph.JobTaskVertex;
-import eu.stratosphere.nephele.template.AbstractInputTask;
 import eu.stratosphere.pact.runtime.iterative.io.FakeOutputTask;
 import eu.stratosphere.pact.runtime.iterative.task.IterationSynchronizationSinkTask;
 import eu.stratosphere.pact.runtime.task.DataSinkTask;
@@ -62,9 +61,7 @@ public class JobGraphUtils {
 	{
 		JobInputVertex inputVertex = new JobInputVertex(name, graph);
 		
-		@SuppressWarnings("unchecked")
-		Class<AbstractInputTask<?>> clazz = (Class<AbstractInputTask<?>>) (Class<?>) DataSourceTask.class;
-		inputVertex.setInputClass(clazz);
+		inputVertex.setInvokableClass(DataSourceTask.class);
 		
 		inputVertex.setNumberOfSubtasks(degreeOfParallelism);
 
@@ -91,14 +88,14 @@ public class JobGraphUtils {
 			int degreeOfParallelism)
 	{
 		JobTaskVertex taskVertex = new JobTaskVertex(name, graph);
-		taskVertex.setTaskClass(task);
+		taskVertex.setInvokableClass(task);
 		taskVertex.setNumberOfSubtasks(degreeOfParallelism);
 		return taskVertex;
 	}
 
 	public static JobOutputVertex createSync(JobGraph jobGraph, int degreeOfParallelism) {
 		JobOutputVertex sync = new JobOutputVertex("BulkIterationSync", jobGraph);
-		sync.setOutputClass(IterationSynchronizationSinkTask.class);
+		sync.setInvokableClass(IterationSynchronizationSinkTask.class);
 		sync.setNumberOfSubtasks(1);
 		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
 		syncConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, degreeOfParallelism);
@@ -108,7 +105,7 @@ public class JobGraphUtils {
 	public static JobOutputVertex createFakeOutput(JobGraph jobGraph, String name, int degreeOfParallelism)
 	{
 		JobOutputVertex outputVertex = new JobOutputVertex(name, jobGraph);
-		outputVertex.setOutputClass(FakeOutputTask.class);
+		outputVertex.setInvokableClass(FakeOutputTask.class);
 		outputVertex.setNumberOfSubtasks(degreeOfParallelism);
 		return outputVertex;
 	}
@@ -116,7 +113,7 @@ public class JobGraphUtils {
 	public static JobOutputVertex createFileOutput(JobGraph jobGraph, String name, int degreeOfParallelism)
 	{
 		JobOutputVertex sinkVertex = new JobOutputVertex(name, jobGraph);
-		sinkVertex.setOutputClass(DataSinkTask.class);
+		sinkVertex.setInvokableClass(DataSinkTask.class);
 		sinkVertex.setNumberOfSubtasks(degreeOfParallelism);
 		return sinkVertex;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/DiscardingOutputFormat.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/DiscardingOutputFormat.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/DiscardingOutputFormat.java
index 3de547e..aa498d8 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/DiscardingOutputFormat.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/DiscardingOutputFormat.java
@@ -13,8 +13,6 @@
 
 package eu.stratosphere.test.recordJobs.util;
 
-import java.io.IOException;
-
 import eu.stratosphere.api.common.io.OutputFormat;
 import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.types.Record;
@@ -23,28 +21,20 @@ import eu.stratosphere.types.Record;
  * A simple output format that discards all data by doing nothing.
  */
 public class DiscardingOutputFormat implements OutputFormat<Record> {
-	private static final long serialVersionUID = 1L;
 	
+	private static final long serialVersionUID = 1L;
 
 	@Override
-	public void configure(Configuration parameters)
-	{}
+	public void configure(Configuration parameters) {}
 
 
 	@Override
-	public void open(int taskNumber, int numTasks) throws IOException
-	{}
-
+	public void open(int taskNumber, int numTasks) {}
 
 	@Override
-	public void writeRecord(Record record) throws IOException
-	{}
-
+	public void writeRecord(Record record) {}
 
-	@Override
-	public void close() throws IOException
-	{}
 
 	@Override
-	public void initialize(Configuration configuration){}
+	public void close() {}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8c1d82a8/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackThroughput.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackThroughput.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackThroughput.java
index a8ab311..ed6f608 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackThroughput.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackThroughput.java
@@ -13,38 +13,35 @@
 
 package eu.stratosphere.test.runtime;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.junit.After;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
 import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.core.io.IOReadableWritable;
 import eu.stratosphere.nephele.jobgraph.DistributionPattern;
-import eu.stratosphere.nephele.jobgraph.JobGenericInputVertex;
 import eu.stratosphere.nephele.jobgraph.JobGraph;
 import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException;
 import eu.stratosphere.nephele.jobgraph.JobInputVertex;
 import eu.stratosphere.nephele.jobgraph.JobOutputVertex;
 import eu.stratosphere.nephele.jobgraph.JobTaskVertex;
-import eu.stratosphere.nephele.template.AbstractGenericInputTask;
-import eu.stratosphere.nephele.template.AbstractOutputTask;
-import eu.stratosphere.nephele.template.AbstractTask;
+import eu.stratosphere.nephele.template.AbstractInvokable;
 import eu.stratosphere.runtime.io.api.RecordReader;
 import eu.stratosphere.runtime.io.api.RecordWriter;
 import eu.stratosphere.runtime.io.channels.ChannelType;
 import eu.stratosphere.test.util.RecordAPITestBase;
 import eu.stratosphere.util.LogUtils;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.junit.After;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
 @RunWith(Parameterized.class)
 public class NetworkStackThroughput extends RecordAPITestBase {
 
@@ -153,8 +150,8 @@ public class NetworkStackThroughput extends RecordAPITestBase {
 
 		JobGraph jobGraph = new JobGraph("Speed Test");
 
-		JobInputVertex producer = new JobGenericInputVertex("Speed Test Producer", jobGraph);
-		producer.setInputClass(SpeedTestProducer.class);
+		JobInputVertex producer = new JobInputVertex("Speed Test Producer", jobGraph);
+		producer.setInvokableClass(SpeedTestProducer.class);
 		producer.setNumberOfSubtasks(numSubtasks);
 		producer.getConfiguration().setInteger(DATA_VOLUME_GB_CONFIG_KEY, dataVolumeGb);
 		producer.getConfiguration().setBoolean(IS_SLOW_SENDER_CONFIG_KEY, isSlowSender);
@@ -162,12 +159,12 @@ public class NetworkStackThroughput extends RecordAPITestBase {
 		JobTaskVertex forwarder = null;
 		if (useForwarder) {
 			forwarder = new JobTaskVertex("Speed Test Forwarder", jobGraph);
-			forwarder.setTaskClass(SpeedTestForwarder.class);
+			forwarder.setInvokableClass(SpeedTestForwarder.class);
 			forwarder.setNumberOfSubtasks(numSubtasks);
 		}
 
 		JobOutputVertex consumer = new JobOutputVertex("Speed Test Consumer", jobGraph);
-		consumer.setOutputClass(SpeedTestConsumer.class);
+		consumer.setInvokableClass(SpeedTestConsumer.class);
 		consumer.setNumberOfSubtasks(numSubtasks);
 		consumer.getConfiguration().setBoolean(IS_SLOW_RECEIVER_CONFIG_KEY, isSlowReceiver);
 
@@ -188,7 +185,7 @@ public class NetworkStackThroughput extends RecordAPITestBase {
 
 	// ------------------------------------------------------------------------
 
-	public static class SpeedTestProducer extends AbstractGenericInputTask {
+	public static class SpeedTestProducer extends AbstractInvokable {
 
 		private RecordWriter<SpeedTestRecord> writer;
 
@@ -227,7 +224,7 @@ public class NetworkStackThroughput extends RecordAPITestBase {
 		}
 	}
 
-	public static class SpeedTestForwarder extends AbstractTask {
+	public static class SpeedTestForwarder extends AbstractInvokable {
 
 		private RecordReader<SpeedTestRecord> reader;
 
@@ -252,7 +249,7 @@ public class NetworkStackThroughput extends RecordAPITestBase {
 		}
 	}
 
-	public static class SpeedTestConsumer extends AbstractOutputTask {
+	public static class SpeedTestConsumer extends AbstractInvokable {
 
 		private RecordReader<SpeedTestRecord> reader;
 


[30/53] [abbrv] git commit: Add script to merge pull requests

Posted by rm...@apache.org.
Add script to merge pull requests


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

Branch: refs/heads/travis_test
Commit: 9ff440fcbb8307ac5844b76fdddcee0753a38a18
Parents: 40b94f7
Author: Robert Metzger <rm...@apache.org>
Authored: Mon Jun 23 15:38:09 2014 +0200
Committer: Robert Metzger <rm...@apache.org>
Committed: Mon Jun 23 15:43:15 2014 +0200

----------------------------------------------------------------------
 tools/merge_flink_pr.py              | 336 ++++++++++++++++++++++++++++++
 tools/merge_pull_request.sh.template |  15 ++
 2 files changed, 351 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9ff440fc/tools/merge_flink_pr.py
----------------------------------------------------------------------
diff --git a/tools/merge_flink_pr.py b/tools/merge_flink_pr.py
new file mode 100755
index 0000000..86113b9
--- /dev/null
+++ b/tools/merge_flink_pr.py
@@ -0,0 +1,336 @@
+#!/usr/bin/env python
+
+#
+# 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.
+#
+
+# Utility for creating well-formed pull request merges and pushing them to Apache.
+#   usage: ./merge_flink_pr.py    (see config env vars below)
+#
+# This utility assumes you already have local a Flink git folder and that you
+# have added remotes corresponding to both (i) the github apache FLINK
+# mirror and (ii) the apache git repo.
+
+#
+# Note by Robert Metzger:
+# This script has been written by the Apache Spark team.
+# I found the source here: https://github.com/apache/spark/blob/master/dev/merge_spark_pr.py
+#
+
+import json
+import os
+import re
+import subprocess
+import sys
+import tempfile
+import urllib2
+
+try:
+    import jira.client
+    JIRA_IMPORTED = True
+except ImportError:
+    JIRA_IMPORTED = False
+
+# Location of your FLINK git development area
+FLINK_HOME = os.environ.get("FLINK_HOME", "/home/patrick/Documents/spark")
+# Remote name which points to the Gihub site
+PR_REMOTE_NAME = os.environ.get("PR_REMOTE_NAME", "apache-github")
+# Remote name which points to Apache git
+PUSH_REMOTE_NAME = os.environ.get("PUSH_REMOTE_NAME", "apache")
+# ASF JIRA username
+JIRA_USERNAME = os.environ.get("JIRA_USERNAME", "pwendell")
+# ASF JIRA password
+JIRA_PASSWORD = os.environ.get("JIRA_PASSWORD", "1234")
+
+GITHUB_BASE = "https://github.com/apache/incubator-flink/pull"
+GITHUB_API_BASE = "https://api.github.com/repos/apache/incubator-flink"
+JIRA_BASE = "https://issues.apache.org/jira/browse"
+JIRA_API_BASE = "https://issues.apache.org/jira"
+# Prefix added to temporary branches
+BRANCH_PREFIX = "PR_TOOL"
+
+os.chdir(FLINK_HOME)
+
+
+def get_json(url):
+    try:
+        return json.load(urllib2.urlopen(url))
+    except urllib2.HTTPError as e:
+        print "Unable to fetch URL, exiting: %s" % url
+        sys.exit(-1)
+
+
+def fail(msg):
+    print msg
+    clean_up()
+    sys.exit(-1)
+
+
+def run_cmd(cmd):
+    if isinstance(cmd, list):
+        return subprocess.check_output(cmd)
+    else:
+        return subprocess.check_output(cmd.split(" "))
+
+
+def continue_maybe(prompt):
+    result = raw_input("\n%s (y/n): " % prompt)
+    if result.lower() != "y":
+        fail("Okay, exiting")
+
+
+original_head = run_cmd("git rev-parse HEAD")[:8]
+
+
+def clean_up():
+    print "Restoring head pointer to %s" % original_head
+    run_cmd("git checkout %s" % original_head)
+
+    branches = run_cmd("git branch").replace(" ", "").split("\n")
+
+    for branch in filter(lambda x: x.startswith(BRANCH_PREFIX), branches):
+        print "Deleting local branch %s" % branch
+        run_cmd("git branch -D %s" % branch)
+
+
+# merge the requested PR and return the merge hash
+def merge_pr(pr_num, target_ref):
+    pr_branch_name = "%s_MERGE_PR_%s" % (BRANCH_PREFIX, pr_num)
+    target_branch_name = "%s_MERGE_PR_%s_%s" % (BRANCH_PREFIX, pr_num, target_ref.upper())
+    run_cmd("git fetch %s pull/%s/head:%s" % (PR_REMOTE_NAME, pr_num, pr_branch_name))
+    run_cmd("git fetch %s %s:%s" % (PUSH_REMOTE_NAME, target_ref, target_branch_name))
+    run_cmd("git checkout %s" % target_branch_name)
+
+    had_conflicts = False
+    try:
+        run_cmd(['git', 'merge', pr_branch_name, '--squash'])
+    except Exception as e:
+        msg = "Error merging: %s\nWould you like to manually fix-up this merge?" % e
+        continue_maybe(msg)
+        msg = "Okay, please fix any conflicts and 'git add' conflicting files... Finished?"
+        continue_maybe(msg)
+        had_conflicts = True
+
+    commit_authors = run_cmd(['git', 'log', 'HEAD..%s' % pr_branch_name,
+                             '--pretty=format:%an <%ae>']).split("\n")
+    distinct_authors = sorted(set(commit_authors),
+                              key=lambda x: commit_authors.count(x), reverse=True)
+    primary_author = distinct_authors[0]
+    commits = run_cmd(['git', 'log', 'HEAD..%s' % pr_branch_name,
+                      '--pretty=format:%h [%an] %s']).split("\n\n")
+
+    merge_message_flags = []
+
+    merge_message_flags += ["-m", title]
+    if body != None:
+        merge_message_flags += ["-m", body]
+
+    authors = "\n".join(["Author: %s" % a for a in distinct_authors])
+
+    merge_message_flags += ["-m", authors]
+
+    if had_conflicts:
+        committer_name = run_cmd("git config --get user.name").strip()
+        committer_email = run_cmd("git config --get user.email").strip()
+        message = "This patch had conflicts when merged, resolved by\nCommitter: %s <%s>" % (
+            committer_name, committer_email)
+        merge_message_flags += ["-m", message]
+
+    # The string "Closes #%s" string is required for GitHub to correctly close the PR
+    merge_message_flags += [
+        "-m",
+        "Closes #%s from %s and squashes the following commits:" % (pr_num, pr_repo_desc)]
+    for c in commits:
+        merge_message_flags += ["-m", c]
+
+    run_cmd(['git', 'commit', '--author="%s"' % primary_author] + merge_message_flags)
+
+    continue_maybe("Merge complete (local ref %s). Push to %s?" % (
+        target_branch_name, PUSH_REMOTE_NAME))
+
+    try:
+        run_cmd('git push %s %s:%s' % (PUSH_REMOTE_NAME, target_branch_name, target_ref))
+    except Exception as e:
+        clean_up()
+        fail("Exception while pushing: %s" % e)
+
+    merge_hash = run_cmd("git rev-parse %s" % target_branch_name)[:8]
+    clean_up()
+    print("Pull request #%s merged!" % pr_num)
+    print("Merge hash: %s" % merge_hash)
+    return merge_hash
+
+
+def cherry_pick(pr_num, merge_hash, default_branch):
+    pick_ref = raw_input("Enter a branch name [%s]: " % default_branch)
+    if pick_ref == "":
+        pick_ref = default_branch
+
+    pick_branch_name = "%s_PICK_PR_%s_%s" % (BRANCH_PREFIX, pr_num, pick_ref.upper())
+
+    run_cmd("git fetch %s %s:%s" % (PUSH_REMOTE_NAME, pick_ref, pick_branch_name))
+    run_cmd("git checkout %s" % pick_branch_name)
+    run_cmd("git cherry-pick -sx %s" % merge_hash)
+
+    continue_maybe("Pick complete (local ref %s). Push to %s?" % (
+        pick_branch_name, PUSH_REMOTE_NAME))
+
+    try:
+        run_cmd('git push %s %s:%s' % (PUSH_REMOTE_NAME, pick_branch_name, pick_ref))
+    except Exception as e:
+        clean_up()
+        fail("Exception while pushing: %s" % e)
+
+    pick_hash = run_cmd("git rev-parse %s" % pick_branch_name)[:8]
+    clean_up()
+
+    print("Pull request #%s picked into %s!" % (pr_num, pick_ref))
+    print("Pick hash: %s" % pick_hash)
+    return pick_ref
+
+
+def fix_version_from_branch(branch, versions):
+    # Note: Assumes this is a sorted (newest->oldest) list of un-released versions
+    if branch == "master":
+        return versions[0]
+    else:
+        branch_ver = branch.replace("branch-", "")
+        return filter(lambda x: x.name.startswith(branch_ver), versions)[-1]
+
+
+def resolve_jira(title, merge_branches, comment):
+    asf_jira = jira.client.JIRA({'server': JIRA_API_BASE},
+                                basic_auth=(JIRA_USERNAME, JIRA_PASSWORD))
+
+    default_jira_id = ""
+    search = re.findall("FLINK-[0-9]{4,5}", title)
+    if len(search) > 0:
+        default_jira_id = search[0]
+
+    jira_id = raw_input("Enter a JIRA id [%s]: " % default_jira_id)
+    if jira_id == "":
+        jira_id = default_jira_id
+
+    try:
+        issue = asf_jira.issue(jira_id)
+    except Exception as e:
+        fail("ASF JIRA could not find %s\n%s" % (jira_id, e))
+
+    cur_status = issue.fields.status.name
+    cur_summary = issue.fields.summary
+    cur_assignee = issue.fields.assignee
+    if cur_assignee is None:
+        cur_assignee = "NOT ASSIGNED!!!"
+    else:
+        cur_assignee = cur_assignee.displayName
+
+    if cur_status == "Resolved" or cur_status == "Closed":
+        fail("JIRA issue %s already has status '%s'" % (jira_id, cur_status))
+    print ("=== JIRA %s ===" % jira_id)
+    print ("summary\t\t%s\nassignee\t%s\nstatus\t\t%s\nurl\t\t%s/%s\n" % (
+        cur_summary, cur_assignee, cur_status, JIRA_BASE, jira_id))
+
+    versions = asf_jira.project_versions("FLINK")
+    versions = sorted(versions, key=lambda x: x.name, reverse=True)
+    versions = filter(lambda x: x.raw['released'] is False, versions)
+
+    default_fix_versions = map(lambda x: fix_version_from_branch(x, versions).name, merge_branches)
+    for v in default_fix_versions:
+        # Handles the case where we have forked a release branch but not yet made the release.
+        # In this case, if the PR is committed to the master branch and the release branch, we
+        # only consider the release branch to be the fix version. E.g. it is not valid to have
+        # both 1.1.0 and 1.0.0 as fix versions.
+        (major, minor, patch) = v.split(".")
+        if patch == "0":
+            previous = "%s.%s.%s" % (major, int(minor) - 1, 0)
+            if previous in default_fix_versions:
+                default_fix_versions = filter(lambda x: x != v, default_fix_versions)
+    default_fix_versions = ",".join(default_fix_versions)
+
+    fix_versions = raw_input("Enter comma-separated fix version(s) [%s]: " % default_fix_versions)
+    if fix_versions == "":
+        fix_versions = default_fix_versions
+    fix_versions = fix_versions.replace(" ", "").split(",")
+
+    def get_version_json(version_str):
+        return filter(lambda v: v.name == version_str, versions)[0].raw
+
+    jira_fix_versions = map(lambda v: get_version_json(v), fix_versions)
+
+    resolve = filter(lambda a: a['name'] == "Resolve Issue", asf_jira.transitions(jira_id))[0]
+    asf_jira.transition_issue(
+        jira_id, resolve["id"], fixVersions=jira_fix_versions, comment=comment)
+
+    print "Succesfully resolved %s with fixVersions=%s!" % (jira_id, fix_versions)
+
+
+#branches = get_json("%s/branches" % GITHUB_API_BASE)
+#print "branches %s " % (branches)
+#branch_names = filter(lambda x: x.startswith("release-"), [x['name'] for x in branches])
+# Assumes branch names can be sorted lexicographically
+latest_branch = "master" #sorted(branch_names, reverse=True)[0]
+
+pr_num = raw_input("Which pull request would you like to merge? (e.g. 34): ")
+pr = get_json("%s/pulls/%s" % (GITHUB_API_BASE, pr_num))
+
+url = pr["url"]
+title = pr["title"]
+body = pr["body"]
+target_ref = pr["base"]["ref"]
+user_login = pr["user"]["login"]
+base_ref = pr["head"]["ref"]
+pr_repo_desc = "%s/%s" % (user_login, base_ref)
+
+if pr["merged"] is True:
+    print "Pull request %s has already been merged, assuming you want to backport" % pr_num
+    merge_commit_desc = run_cmd([
+        'git', 'log', '--merges', '--first-parent',
+        '--grep=pull request #%s' % pr_num, '--oneline']).split("\n")[0]
+    if merge_commit_desc == "":
+        fail("Couldn't find any merge commit for #%s, you may need to update HEAD." % pr_num)
+
+    merge_hash = merge_commit_desc[:7]
+    message = merge_commit_desc[8:]
+
+    print "Found: %s" % message
+    maybe_cherry_pick(pr_num, merge_hash, latest_branch)
+    sys.exit(0)
+
+if not bool(pr["mergeable"]):
+    msg = "Pull request %s is not mergeable in its current form.\n" % pr_num + \
+        "Continue? (experts only!)"
+    continue_maybe(msg)
+
+print ("\n=== Pull Request #%s ===" % pr_num)
+print ("title\t%s\nsource\t%s\ntarget\t%s\nurl\t%s" % (
+    title, pr_repo_desc, target_ref, url))
+continue_maybe("Proceed with merging pull request #%s?" % pr_num)
+
+merged_refs = [target_ref]
+
+merge_hash = merge_pr(pr_num, target_ref)
+
+pick_prompt = "Would you like to pick %s into another branch?" % merge_hash
+while raw_input("\n%s (y/n): " % pick_prompt).lower() == "y":
+    merged_refs = merged_refs + [cherry_pick(pr_num, merge_hash, latest_branch)]
+
+if JIRA_IMPORTED:
+    continue_maybe("Would you like to update an associated JIRA?")
+    jira_comment = "Issue resolved by pull request %s\n[%s/%s]" % (pr_num, GITHUB_BASE, pr_num)
+    resolve_jira(title, merged_refs, jira_comment)
+else:
+    print "Could not find jira-python library. Run 'sudo pip install jira-python' to install."
+    print "Exiting without trying to close the associated JIRA."

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9ff440fc/tools/merge_pull_request.sh.template
----------------------------------------------------------------------
diff --git a/tools/merge_pull_request.sh.template b/tools/merge_pull_request.sh.template
new file mode 100755
index 0000000..d506d9d
--- /dev/null
+++ b/tools/merge_pull_request.sh.template
@@ -0,0 +1,15 @@
+#!/bin/sh
+
+# the directory where you have your flink code
+export FLINK_HOME="/home/robert/projects/flink/incubator-flink"
+# Remote name which points to the Gihub site
+export PR_REMOTE_NAME="github_flink"
+# Remote name which points to Apache git
+export PUSH_REMOTE_NAME="asf_flink"
+# ASF JIRA username
+export JIRA_USERNAME"rmetzger"
+# ASF JIRA password
+export JIRA_PASSWORD="Ideally, don't push your password to git."
+
+# Arch Linux users have to call "python2.7" here.
+python merge_flink_pr.py
\ No newline at end of file