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/09 20:30:36 UTC

[01/30] git commit: Fixes bugs where the TypeExtractor throws an NPE instead of the operators

Repository: incubator-flink
Updated Branches:
  refs/heads/master b746f452e -> 24d00598a


Fixes bugs where the TypeExtractor throws an NPE instead of the operators


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

Branch: refs/heads/master
Commit: 2b0baea9b8a6dd99052c2dfa98cae719a39d6bbc
Parents: b746f45
Author: twalthr <in...@twalthr.com>
Authored: Mon Jun 2 14:39:36 2014 +0200
Committer: StephanEwen <st...@tu-berlin.de>
Committed: Fri Jun 6 15:53:24 2014 +0200

----------------------------------------------------------------------
 .../main/java/eu/stratosphere/api/java/DataSet.java  | 15 +++++++++++++++
 .../api/java/operators/CoGroupOperator.java          |  3 +++
 .../api/java/operators/CrossOperator.java            |  3 +++
 .../api/java/operators/FilterOperator.java           |  4 ----
 .../api/java/operators/FlatMapOperator.java          |  4 ----
 .../api/java/operators/JoinOperator.java             |  3 +++
 .../stratosphere/api/java/operators/MapOperator.java |  4 ----
 .../api/java/operators/ReduceGroupOperator.java      |  8 --------
 .../api/java/operators/ReduceOperator.java           |  8 --------
 .../api/java/operators/SortedGrouping.java           |  3 +++
 .../api/java/operators/UnsortedGrouping.java         |  6 ++++++
 11 files changed, 33 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2b0baea9/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 66f8eeb..758cbf2 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
@@ -130,6 +130,9 @@ public abstract class DataSet<T> {
 	 * @see DataSet
 	 */
 	public <R> MapOperator<T, R> map(MapFunction<T, R> mapper) {
+		if (mapper == null) {
+			throw new NullPointerException("Map function must not be null.");
+		}
 		return new MapOperator<T, R>(this, mapper);
 	}
 	
@@ -146,6 +149,9 @@ public abstract class DataSet<T> {
 	 * @see DataSet
 	 */
 	public <R> FlatMapOperator<T, R> flatMap(FlatMapFunction<T, R> flatMapper) {
+		if (flatMapper == null) {
+			throw new NullPointerException("FlatMap function must not be null.");
+		}
 		return new FlatMapOperator<T, R>(this, flatMapper);
 	}
 	
@@ -163,6 +169,9 @@ public abstract class DataSet<T> {
 	 * @see DataSet
 	 */
 	public FilterOperator<T> filter(FilterFunction<T> filter) {
+		if (filter == null) {
+			throw new NullPointerException("Filter function must not be null.");
+		}
 		return new FilterOperator<T>(this, filter);
 	}
 	
@@ -229,6 +238,9 @@ public abstract class DataSet<T> {
 	 * @see DataSet
 	 */
 	public ReduceOperator<T> reduce(ReduceFunction<T> reducer) {
+		if (reducer == null) {
+			throw new NullPointerException("Reduce function must not be null.");
+		}
 		return new ReduceOperator<T>(this, reducer);
 	}
 	
@@ -246,6 +258,9 @@ public abstract class DataSet<T> {
 	 * @see DataSet
 	 */
 	public <R> ReduceGroupOperator<T, R> reduceGroup(GroupReduceFunction<T, R> reducer) {
+		if (reducer == null) {
+			throw new NullPointerException("GroupReduce function must not be null.");
+		}
 		return new ReduceGroupOperator<T, R>(this, reducer);
 	}
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2b0baea9/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CoGroupOperator.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CoGroupOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CoGroupOperator.java
index 67b9e5c..ca4b1db 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CoGroupOperator.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CoGroupOperator.java
@@ -439,6 +439,9 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 				 * @see DataSet
 				 */
 				public <R> CoGroupOperator<I1, I2, R> with(CoGroupFunction<I1, I2, R> function) {
+					if (function == null) {
+						throw new NullPointerException("CoGroup function must not be null.");
+					}
 					TypeInformation<R> returnType = TypeExtractor.getCoGroupReturnTypes(function, input1.getType(), input2.getType());
 					return new CoGroupOperator<I1, I2, R>(input1, input2, keys1, keys2, function, returnType);
 				}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2b0baea9/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 48da306..3566224 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
@@ -113,6 +113,9 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		 * @see DataSet
 		 */
 		public <R> CrossOperator<I1, I2, R> with(CrossFunction<I1, I2, R> function) {
+			if (function == null) {
+				throw new NullPointerException("Cross function must not be null.");
+			}
 			TypeInformation<R> returnType = TypeExtractor.getCrossReturnTypes(function, input1.getType(), input2.getType());
 			return new CrossOperator<I1, I2, R>(input1, input2, function, returnType);
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2b0baea9/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FilterOperator.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FilterOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FilterOperator.java
index d9db757..adbe77f 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FilterOperator.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FilterOperator.java
@@ -34,10 +34,6 @@ public class FilterOperator<T> extends SingleInputUdfOperator<T, T, FilterOperat
 	public FilterOperator(DataSet<T> input, FilterFunction<T> function) {
 		super(input, input.getType());
 		
-		if (function == null) {
-			throw new NullPointerException("Filter function must not be null.");
-		}
-		
 		this.function = function;
 		extractSemanticAnnotationsFromUdf(function.getClass());
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2b0baea9/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FlatMapOperator.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FlatMapOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FlatMapOperator.java
index 27025c7..32f2343 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FlatMapOperator.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FlatMapOperator.java
@@ -37,10 +37,6 @@ public class FlatMapOperator<IN, OUT> extends SingleInputUdfOperator<IN, OUT, Fl
 	public FlatMapOperator(DataSet<IN> input, FlatMapFunction<IN, OUT> function) {
 		super(input, TypeExtractor.getFlatMapReturnTypes(function, input.getType()));
 		
-		if (function == null) {
-			throw new NullPointerException("FlatMap function must not be null.");
-		}
-		
 		this.function = function;
 		extractSemanticAnnotationsFromUdf(function.getClass());
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2b0baea9/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/JoinOperator.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/JoinOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/JoinOperator.java
index 578d68c..992cc0a 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/JoinOperator.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/JoinOperator.java
@@ -421,6 +421,9 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 		 * @see DataSet
 		 */
 		public <R> EquiJoin<I1, I2, R> with(JoinFunction<I1, I2, R> function) {
+			if (function == null) {
+				throw new NullPointerException("Join function must not be null.");
+			}
 			TypeInformation<R> returnType = TypeExtractor.getJoinReturnTypes(function, getInput1Type(), getInput2Type());
 			return new EquiJoin<I1, I2, R>(getInput1(), getInput2(), getKeys1(), getKeys2(), function, returnType, getJoinHint());
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2b0baea9/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/MapOperator.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/MapOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/MapOperator.java
index e2c36bf..00bbb27 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/MapOperator.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/MapOperator.java
@@ -39,10 +39,6 @@ public class MapOperator<IN, OUT> extends SingleInputUdfOperator<IN, OUT, MapOpe
 	public MapOperator(DataSet<IN> input, MapFunction<IN, OUT> function) {
 		super(input, TypeExtractor.getMapReturnTypes(function, input.getType()));
 		
-		if (function == null) {
-			throw new NullPointerException("Map function must not be null.");
-		}
-		
 		this.function = function;
 		extractSemanticAnnotationsFromUdf(function.getClass());
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2b0baea9/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceGroupOperator.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceGroupOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceGroupOperator.java
index 9029b09..e001d91 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceGroupOperator.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceGroupOperator.java
@@ -58,10 +58,6 @@ public class ReduceGroupOperator<IN, OUT> extends SingleInputUdfOperator<IN, OUT
 	public ReduceGroupOperator(DataSet<IN> input, GroupReduceFunction<IN, OUT> function) {
 		super(input, TypeExtractor.getGroupReduceReturnTypes(function, input.getType()));
 		
-		if (function == null) {
-			throw new NullPointerException("GroupReduce function must not be null.");
-		}
-		
 		this.function = function;
 		this.grouper = null;
 		checkCombinability();
@@ -76,10 +72,6 @@ public class ReduceGroupOperator<IN, OUT> extends SingleInputUdfOperator<IN, OUT
 	public ReduceGroupOperator(Grouping<IN> input, GroupReduceFunction<IN, OUT> function) {
 		super(input != null ? input.getDataSet() : null, TypeExtractor.getGroupReduceReturnTypes(function, input.getDataSet().getType()));
 		
-		if (function == null) {
-			throw new NullPointerException("GroupReduce function must not be null.");
-		}
-		
 		this.function = function;
 		this.grouper = input;
 		checkCombinability();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2b0baea9/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceOperator.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceOperator.java
index bdf9436..6056e8b 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceOperator.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceOperator.java
@@ -53,10 +53,6 @@ public class ReduceOperator<IN> extends SingleInputUdfOperator<IN, IN, ReduceOpe
 	public ReduceOperator(DataSet<IN> input, ReduceFunction<IN> function) {
 		super(input, input.getType());
 		
-		if (function == null) {
-			throw new NullPointerException("Reduce function must not be null.");
-		}
-		
 		this.function = function;
 		this.grouper = null;
 		
@@ -67,10 +63,6 @@ public class ReduceOperator<IN> extends SingleInputUdfOperator<IN, IN, ReduceOpe
 	public ReduceOperator(Grouping<IN> input, ReduceFunction<IN> function) {
 		super(input.getDataSet(), input.getDataSet().getType());
 		
-		if (function == null) {
-			throw new NullPointerException("Reduce function must not be null.");
-		}
-		
 		this.function = function;
 		this.grouper = input;
 		

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2b0baea9/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SortedGrouping.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SortedGrouping.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SortedGrouping.java
index 3edb05c..dc26a2b 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SortedGrouping.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SortedGrouping.java
@@ -72,6 +72,9 @@ public class SortedGrouping<T> extends Grouping<T> {
 	 * @see DataSet
 	 */
 	public <R> ReduceGroupOperator<T, R> reduceGroup(GroupReduceFunction<T, R> reducer) {
+		if (reducer == null) {
+			throw new NullPointerException("GroupReduce function must not be null.");
+		}
 		return new ReduceGroupOperator<T, R>(this, reducer);
 	}
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2b0baea9/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/UnsortedGrouping.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/UnsortedGrouping.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/UnsortedGrouping.java
index 711bc24..95e40bc 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/UnsortedGrouping.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/UnsortedGrouping.java
@@ -64,6 +64,9 @@ public class UnsortedGrouping<T> extends Grouping<T> {
 	 * @see DataSet
 	 */
 	public ReduceOperator<T> reduce(ReduceFunction<T> reducer) {
+		if (reducer == null) {
+			throw new NullPointerException("Reduce function must not be null.");
+		}
 		return new ReduceOperator<T>(this, reducer);
 	}
 	
@@ -81,6 +84,9 @@ public class UnsortedGrouping<T> extends Grouping<T> {
 	 * @see DataSet
 	 */
 	public <R> ReduceGroupOperator<T, R> reduceGroup(GroupReduceFunction<T, R> reducer) {
+		if (reducer == null) {
+			throw new NullPointerException("GroupReduce function must not be null.");
+		}
 		return new ReduceGroupOperator<T, R>(this, reducer);
 	}
 


[10/30] Offer buffer-oriented API for I/O (#25)

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 965a5aa..06c857e 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,44 +13,46 @@
 
 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.configuration.ConfigConstants;
 import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.configuration.GlobalConfiguration;
 import eu.stratosphere.core.fs.Path;
 import eu.stratosphere.nephele.client.JobClient;
 import eu.stratosphere.nephele.client.JobExecutionException;
-import eu.stratosphere.nephele.io.DistributionPattern;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-import eu.stratosphere.nephele.io.library.FileLineReader;
-import eu.stratosphere.nephele.io.library.FileLineWriter;
+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.JobTaskVertex;
 import eu.stratosphere.nephele.jobmanager.JobManager.ExecutionMode;
+import eu.stratosphere.nephele.taskmanager.Task;
 import eu.stratosphere.nephele.taskmanager.TaskManager;
-import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTask;
+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.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;
+
+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}.
@@ -185,8 +187,8 @@ public class JobManagerITCase {
 			// connect vertices
 			try {
 				i1.connectTo(t1, ChannelType.NETWORK);
-				t1.connectTo(t2, ChannelType.INMEMORY);
-				t2.connectTo(o1, ChannelType.INMEMORY);
+				t1.connectTo(t2, ChannelType.IN_MEMORY);
+				t2.connectTo(o1, ChannelType.IN_MEMORY);
 			} catch (JobGraphDefinitionException e) {
 				e.printStackTrace();
 			}
@@ -286,8 +288,8 @@ public class JobManagerITCase {
 			o1.setVertexToShareInstancesWith(i1);
 
 			// connect vertices
-			i1.connectTo(t1, ChannelType.INMEMORY);
-			t1.connectTo(o1, ChannelType.INMEMORY);
+			i1.connectTo(t1, ChannelType.IN_MEMORY);
+			t1.connectTo(o1, ChannelType.IN_MEMORY);
 
 			// add jar
 			jg.addJar(new Path(new File(ServerTestUtils.getTempDir() + File.separator + exceptionClassName + ".jar")
@@ -297,7 +299,7 @@ public class JobManagerITCase {
 			jobClient = new JobClient(jg, configuration);
 			
 			// deactivate logging of expected test exceptions
-			Logger rtLogger = Logger.getLogger(RuntimeTask.class);
+			Logger rtLogger = Logger.getLogger(Task.class);
 			Level rtLevel = rtLogger.getEffectiveLevel();
 			rtLogger.setLevel(Level.OFF);
 			
@@ -382,8 +384,8 @@ public class JobManagerITCase {
 			o1.setVertexToShareInstancesWith(i1);
 
 			// connect vertices
-			i1.connectTo(t1, ChannelType.INMEMORY);
-			t1.connectTo(o1, ChannelType.INMEMORY);
+			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
@@ -492,8 +494,8 @@ public class JobManagerITCase {
 			// connect vertices
 			try {
 				i1.connectTo(t1, ChannelType.NETWORK);
-				t1.connectTo(t2, ChannelType.INMEMORY);
-				t2.connectTo(o1, ChannelType.INMEMORY);
+				t1.connectTo(t2, ChannelType.IN_MEMORY);
+				t2.connectTo(o1, ChannelType.IN_MEMORY);
 			} catch (JobGraphDefinitionException e) {
 				e.printStackTrace();
 			}
@@ -583,9 +585,9 @@ public class JobManagerITCase {
 			o1.setVertexToShareInstancesWith(i1);
 
 			// connect vertices
-			i1.connectTo(t1, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+			i1.connectTo(t1, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
 			i1.connectTo(t1, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
-			t1.connectTo(o1, ChannelType.INMEMORY);
+			t1.connectTo(o1, ChannelType.IN_MEMORY);
 
 			// add jar
 			jg.addJar(new Path(jarFile.toURI()));
@@ -657,7 +659,7 @@ public class JobManagerITCase {
 			o1.setVertexToShareInstancesWith(i1);
 
 			// connect vertices
-			i1.connectTo(o1, ChannelType.INMEMORY);
+			i1.connectTo(o1, ChannelType.IN_MEMORY);
 
 			// add jar
 			jg.addJar(new Path(jarFile.toURI()));
@@ -751,9 +753,9 @@ public class JobManagerITCase {
 			u1.setVertexToShareInstancesWith(o1);
 
 			// connect vertices
-			i1.connectTo(u1, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
-			i2.connectTo(u1, ChannelType.INMEMORY);
-			u1.connectTo(o1, ChannelType.INMEMORY);
+			i1.connectTo(u1, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
+			i2.connectTo(u1, ChannelType.IN_MEMORY);
+			u1.connectTo(o1, ChannelType.IN_MEMORY);
 
 			// add jar
 			jg.addJar(new Path(jarFile.toURI()));

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 3b95133..124a24d 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,9 +14,9 @@
 package eu.stratosphere.nephele.jobmanager;
 
 import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.MutableRecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
-import eu.stratosphere.nephele.io.UnionRecordReader;
+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;
 
 /**
@@ -41,13 +41,17 @@ public class UnionTask extends AbstractTask {
 		recordReaders[1] = new MutableRecordReader<StringRecord>(this);
 		this.unionReader = new UnionRecordReader<StringRecord>(recordReaders, StringRecord.class);
 		
-		this.writer = new RecordWriter<StringRecord>(this, StringRecord.class);
+		this.writer = new RecordWriter<StringRecord>(this);
 	}
 
 	@Override
 	public void invoke() throws Exception {
+		this.writer.initializeSerializers();
+
 		while (this.unionReader.hasNext()) {
 			this.writer.emit(this.unionReader.next());
 		}
+
+		this.writer.flush();
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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
index 9fc4256..f1e3191 100644
--- 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
@@ -20,6 +20,7 @@ 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;
@@ -29,9 +30,8 @@ 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.nephele.io.RecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+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;
@@ -57,7 +57,7 @@ public class QueueSchedulerTest {
 		 */
 		@Override
 		public void registerInputOutput() {
-			new RecordWriter<StringRecord>(this, StringRecord.class);
+			new RecordWriter<StringRecord>(this);
 		}
 
 		/**
@@ -145,7 +145,7 @@ public class QueueSchedulerTest {
 		final TestDeploymentManager tdm = new TestDeploymentManager();
 		final QueueScheduler scheduler = new QueueScheduler(tdm, tim);
 
-		final ExecutionGraph executionGraph = createExecutionGraph(ChannelType.INMEMORY, tim);
+		final ExecutionGraph executionGraph = createExecutionGraph(ChannelType.IN_MEMORY, tim);
 
 		try {
 			try {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 4fd1ac1..630f365 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
@@ -19,7 +19,7 @@ import java.util.Iterator;
 
 import org.junit.Test;
 
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.channels.ChannelType;
 import eu.stratosphere.nephele.jobgraph.JobID;
 import eu.stratosphere.nephele.util.ManagementTestUtils;
 
@@ -275,7 +275,7 @@ public class ManagementGraphTest {
 		// Group Edges
 		new ManagementGroupEdge(groupVertex1, 0, groupVertex2, 0, ChannelType.NETWORK);
 		new ManagementGroupEdge(groupVertex2, 0, groupVertex3, 0, ChannelType.NETWORK);
-		new ManagementGroupEdge(groupVertex3, 0, groupVertex4, 0, ChannelType.INMEMORY);
+		new ManagementGroupEdge(groupVertex3, 0, groupVertex4, 0, ChannelType.IN_MEMORY);
 
 		// Edges
 		new ManagementEdge(new ManagementEdgeID(), new ManagementEdgeID(), outputGate1_1, 0, inputGate2_1, 0,
@@ -287,7 +287,7 @@ public class ManagementGraphTest {
 		new ManagementEdge(new ManagementEdgeID(), new ManagementEdgeID(), outputGate2_2, 0, inputGate3_1, 1,
 			ChannelType.NETWORK);
 		new ManagementEdge(new ManagementEdgeID(), new ManagementEdgeID(), outputGate3_1, 0, inputGate4_1, 0,
-			ChannelType.INMEMORY);
+			ChannelType.IN_MEMORY);
 
 		return graph;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/profiling/impl/InstanceProfilerTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/profiling/impl/InstanceProfilerTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/profiling/impl/InstanceProfilerTest.java
index dda9491..f1b83a6 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/profiling/impl/InstanceProfilerTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/profiling/impl/InstanceProfilerTest.java
@@ -74,7 +74,7 @@ public class InstanceProfilerTest {
 	@Before
 	public void setUp() throws Exception {
 		initMocks(this);
-		when(this.infoMock.getAddress()).thenReturn(this.addressMock);
+		when(this.infoMock.address()).thenReturn(this.addressMock);
 		when(this.addressMock.getHostAddress()).thenReturn("192.168.1.1");
 
 		whenNew(FileReader.class).withArguments(InstanceProfiler.PROC_STAT).thenReturn(this.cpuReaderMock);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultDeserializerTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultDeserializerTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultDeserializerTest.java
deleted file mode 100644
index 1c9efd5..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultDeserializerTest.java
+++ /dev/null
@@ -1,358 +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.taskmanager.transferenvelope;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.ReadableByteChannel;
-import java.util.ArrayDeque;
-import java.util.Queue;
-
-import org.junit.Test;
-
-import eu.stratosphere.core.memory.MemorySegment;
-import eu.stratosphere.nephele.io.channels.Buffer;
-import eu.stratosphere.nephele.io.channels.BufferFactory;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.MemoryBuffer;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferAvailabilityListener;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProviderBroker;
-import eu.stratosphere.nephele.util.BufferPoolConnector;
-import eu.stratosphere.nephele.util.InterruptibleByteChannel;
-import eu.stratosphere.util.StringUtils;
-
-/**
- * This class contains tests covering the deserialization of a byte stream to a transfer envelope.
- * 
- */
-public class DefaultDeserializerTest {
-
-	/**
-	 * The size of the test byte buffers in byte.
-	 */
-	private static final int TEST_BUFFER_CAPACITY = 1024;
-
-	/**
-	 * The sequence number to be used during the tests.
-	 */
-	private static final int SEQUENCE_NUMBER = 0;
-
-	/**
-	 * The job ID to be used during the tests.
-	 */
-	private static final JobID JOB_ID = new JobID();
-
-	/**
-	 * The channel ID to be used during the tests.
-	 */
-	private static final ChannelID CHANNEL_ID = new ChannelID();
-
-	/**
-	 * A dummy implementation of a {@link BufferProvider} which is used in this test.
-	 * <p>
-	 * This class is not thread-safe.
-	 * 
-	 */
-	private static final class TestBufferProvider implements BufferProvider {
-
-		/**
-		 * Stores the available byte buffers.
-		 */
-		private final Queue<MemorySegment> bufferPool;
-
-		/**
-		 * Constructs a new test buffer provider.
-		 * 
-		 * @param numberOfBuffers
-		 *        the number of byte buffers this pool has available.
-		 */
-		private TestBufferProvider(final int numberOfBuffers) {
-
-			this.bufferPool = new ArrayDeque<MemorySegment>(numberOfBuffers);
-			for (int i = 0; i < numberOfBuffers; ++i) {
-				this.bufferPool.add(new MemorySegment(new byte[TEST_BUFFER_CAPACITY]));
-			}
-		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public Buffer requestEmptyBuffer(final int minimumSizeOfBuffer) throws IOException {
-
-			if (this.bufferPool.isEmpty()) {
-				return null;
-			}
-
-			return BufferFactory.createFromMemory(minimumSizeOfBuffer, this.bufferPool.poll(),
-				new BufferPoolConnector(this.bufferPool));
-		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public Buffer requestEmptyBufferBlocking(final int minimumSizeOfBuffer) throws IOException,
-				InterruptedException {
-
-			throw new IllegalStateException("requestEmptyBufferBlocking called");
-		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public int getMaximumBufferSize() {
-
-			throw new IllegalStateException("getMaximumBufferSize called");
-		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public boolean isShared() {
-
-			throw new IllegalStateException("isShared called");
-		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public void reportAsynchronousEvent() {
-
-			throw new IllegalStateException("reportAsynchronousEvent called");
-		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public boolean registerBufferAvailabilityListener(final BufferAvailabilityListener bufferAvailabilityListener) {
-
-			throw new IllegalStateException("registerBufferAvailabilityListener called");
-		}
-	}
-
-	/**
-	 * A dummy implementation of a {@link BufferProviderBroker} which is used during this test.
-	 * <p>
-	 * This class is not thread-safe.
-	 * 
-	 */
-	private static final class TestBufferProviderBroker implements BufferProviderBroker {
-
-		private final BufferProvider bufferProvider;
-
-		private TestBufferProviderBroker(final BufferProvider bufferProvider) {
-			this.bufferProvider = bufferProvider;
-		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public BufferProvider getBufferProvider(final JobID jobID, final ChannelID sourceChannelID) throws IOException,
-				InterruptedException {
-
-			return this.bufferProvider;
-		}
-	}
-
-	/**
-	 * Constructs an {@link InterruptibleByteChannel} from which the deserializer to be tested can read its data.
-	 * 
-	 * @param readInterruptPositions
-	 *        the positions after which the byte stream shall be interrupted
-	 * @param testBufferSize
-	 *        the size of the test buffer to create
-	 * @return an {@link InterruptibleByteChannel} holding the serialized data in memory
-	 * @throws IOException
-	 *         thrown if an error occurs while serializing the original data
-	 */
-	private ReadableByteChannel createByteChannel(final int[] readInterruptPositions, final int testBufferSize)
-			throws IOException {
-
-		final TransferEnvelope te = new TransferEnvelope(SEQUENCE_NUMBER, JOB_ID, CHANNEL_ID);
-
-		if (testBufferSize >= 0) {
-
-			if (testBufferSize > 100) {
-				throw new IllegalStateException("Test buffer size can be 100 bytes at most");
-			}
-
-			final Queue<MemorySegment> bufferPool = new ArrayDeque<MemorySegment>();
-			final MemorySegment ms = new MemorySegment(new byte[TEST_BUFFER_CAPACITY]);
-
-			final MemoryBuffer buffer = BufferFactory.createFromMemory(ms.size(), ms, new BufferPoolConnector(bufferPool));
-
-			final ByteBuffer srcBuffer = ByteBuffer.allocate(testBufferSize);
-			for (int i = 0; i < testBufferSize; ++i) {
-				srcBuffer.put((byte) i);
-			}
-			srcBuffer.flip();
-
-			buffer.write(srcBuffer);
-			buffer.flip();
-			te.setBuffer(buffer);
-		}
-
-		final DefaultSerializer ds = new DefaultSerializer();
-		ds.setTransferEnvelope(te);
-
-		final InterruptibleByteChannel ibc = new InterruptibleByteChannel(null, readInterruptPositions);
-
-		while (ds.write(ibc));
-
-		ibc.switchToReadPhase();
-
-		return ibc;
-	}
-
-	/**
-	 * Executes the deserialization method.
-	 * 
-	 * @param rbc
-	 *        the byte channel to read the serialized data from
-	 * @param bpb
-	 *        the buffer provider broker to request empty buffers from
-	 * @return the deserialized transfer envelope
-	 * @throws IOException
-	 *         thrown if an error occurs during the deserialization process
-	 * @throws NoBufferAvailableException
-	 *         thrown if the buffer provider broker could not provide an empty buffer
-	 */
-	private TransferEnvelope executeDeserialization(final ReadableByteChannel rbc, final BufferProviderBroker bpb)
-			throws IOException, NoBufferAvailableException {
-
-		final DefaultDeserializer dd = new DefaultDeserializer(bpb);
-
-		TransferEnvelope te = dd.getFullyDeserializedTransferEnvelope();
-		while (te == null) {
-
-			dd.read(rbc);
-			te = dd.getFullyDeserializedTransferEnvelope();
-		}
-
-		assertEquals(SEQUENCE_NUMBER, te.getSequenceNumber());
-		assertEquals(JOB_ID, te.getJobID());
-		assertEquals(CHANNEL_ID, te.getSource());
-
-		return te;
-	}
-
-	/**
-	 * Tests the deserialization process of a {@link TransferEnvelope} with a buffer when no interruption of the byte
-	 * stream.
-	 */
-	@Test
-	public void testDeserializationWithBufferAndWithoutInterruption() {
-
-		try {
-
-			final ReadableByteChannel rbc = createByteChannel(null, 10);
-
-			final TestBufferProviderBroker tbpb = new TestBufferProviderBroker(new TestBufferProvider(1));
-
-			final TransferEnvelope te = executeDeserialization(rbc, tbpb);
-
-			assertNotNull(te.getBuffer());
-			assertEquals(10, te.getBuffer().size());
-
-		} catch (IOException ioe) {
-			fail(StringUtils.stringifyException(ioe));
-		} catch (NoBufferAvailableException nbae) {
-			fail(StringUtils.stringifyException(nbae));
-		}
-	}
-
-	/**
-	 * Tests the deserialization process of a {@link TransferEnvelope} with a buffer and interruptions of the byte
-	 * stream.
-	 */
-	@Test
-	public void testDeserializationWithBufferAndInterruptions() {
-
-		try {
-
-			final ReadableByteChannel rbc = createByteChannel(new int[] { 3, 7, 24, 52 }, 10);
-
-			final TestBufferProviderBroker tbpb = new TestBufferProviderBroker(new TestBufferProvider(1));
-
-			final TransferEnvelope te = executeDeserialization(rbc, tbpb);
-
-			assertNotNull(te.getBuffer());
-			assertEquals(10, te.getBuffer().size());
-
-		} catch (IOException ioe) {
-			fail(StringUtils.stringifyException(ioe));
-		} catch (NoBufferAvailableException nbae) {
-			fail(StringUtils.stringifyException(nbae));
-		}
-	}
-
-	/**
-	 * Tests the deserialization process of a {@link TransferEnvelope} without a buffer and without interruptions of the
-	 * byte stream.
-	 */
-	@Test
-	public void testDeserializationWithoutBufferAndInterruptions() {
-
-		try {
-
-			final ReadableByteChannel rbc = createByteChannel(null, -1);
-
-			final TestBufferProviderBroker tbpb = new TestBufferProviderBroker(new TestBufferProvider(1));
-
-			final TransferEnvelope te = executeDeserialization(rbc, tbpb);
-
-			assertNull(te.getBuffer());
-
-		} catch (IOException ioe) {
-			fail(StringUtils.stringifyException(ioe));
-		} catch (NoBufferAvailableException nbae) {
-			fail(StringUtils.stringifyException(nbae));
-		}
-	}
-
-	/**
-	 * Tests the deserialization process in case the buffer provide cannot deliver an empty buffer to read the byte
-	 * stream into.
-	 */
-	@Test
-	public void testDeserializationWithNoBufferAvailable() {
-
-		try {
-			final ReadableByteChannel rbc = createByteChannel(null, 10);
-			final TestBufferProviderBroker tbpb = new TestBufferProviderBroker(new TestBufferProvider(0));
-			executeDeserialization(rbc, tbpb);
-
-		} catch (IOException ioe) {
-			fail(StringUtils.stringifyException(ioe));
-		} catch (NoBufferAvailableException nbae) {
-			// Expected exception was successfully caught
-			return;
-		}
-
-		fail("Expected NoBufferAvailableException but has not been thrown");
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultSerializerTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultSerializerTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultSerializerTest.java
deleted file mode 100644
index a50fbe2..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultSerializerTest.java
+++ /dev/null
@@ -1,313 +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.taskmanager.transferenvelope;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.util.ArrayDeque;
-import java.util.Deque;
-
-import org.junit.Test;
-
-import eu.stratosphere.core.memory.MemorySegment;
-import eu.stratosphere.nephele.io.AbstractID;
-import eu.stratosphere.nephele.io.channels.BufferFactory;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.MemoryBuffer;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.util.BufferPoolConnector;
-import eu.stratosphere.nephele.util.ServerTestUtils;
-
-/**
- * This class contains tests covering the serialization of transfer envelopes to a byte stream.
- * 
- */
-public class DefaultSerializerTest {
-
-	/**
-	 * The maximum size of the transfer envelope's buffer.
-	 */
-	private static final int BUFFER_SIZE = 4096; // 4 KB;
-
-	/**
-	 * An arbitrarily chosen byte used to fill the transfer envelope's buffer.
-	 */
-	private static final byte BUFFER_CONTENT = 13;
-
-	/**
-	 * The size of a sequence number.
-	 */
-	private static final int SIZE_OF_SEQ_NR = 4;
-
-	/**
-	 * The size of an ID.
-	 */
-	private static final int SIZE_OF_ID = 16;
-
-	/**
-	 * The size of an integer number.
-	 */
-	private static final int SIZE_OF_INTEGER = 4;
-
-	/**
-	 * The job ID used during the serialization process.
-	 */
-	private final JobID jobID = new JobID();
-
-	/**
-	 * The target channel ID used during the serialization process.
-	 */
-	private final ChannelID sourceChannelID = new ChannelID();
-
-	/**
-	 * Auxiliary class to explicitly access the internal buffer of an ID object.
-	 * 
-	 */
-	private static class SerializationTestID extends AbstractID {
-
-		/**
-		 * Constructs a new ID.
-		 * 
-		 * @param content
-		 *        a byte buffer representing the ID
-		 */
-		private SerializationTestID(byte[] content) {
-			super(content);
-		}
-	}
-
-	/**
-	 * This test checks the correctness of the serialization of {@link TransferEnvelope} objects.
-	 */
-	@Test
-	public void testSerialization() {
-
-		try {
-
-			// Generate test file
-			final File testFile = generateDataStream();
-
-			// Analyze the test file
-			analyzeStream(testFile);
-
-			// Delete the test file
-			testFile.delete();
-
-		} catch (IOException e) {
-			fail(e.getMessage());
-		}
-	}
-
-	/**
-	 * Generates and serializes a series of {@link TransferEnvelope} objects to a random file.
-	 * 
-	 * @return the file containing the serializes envelopes
-	 * @throws IOException
-	 *         thrown if an I/O error occurs while writing the envelopes
-	 */
-	private File generateDataStream() throws IOException {
-
-		final File outputFile = new File(ServerTestUtils.getTempDir() + File.separator
-			+ ServerTestUtils.getRandomFilename());
-		final FileOutputStream outputStream = new FileOutputStream(outputFile);
-		final FileChannel fileChannel = outputStream.getChannel();
-		final Deque<MemorySegment> recycleQueue = new ArrayDeque<MemorySegment>();
-		final DefaultSerializer serializer = new DefaultSerializer();
-		final MemorySegment byteBuffer = new MemorySegment(new byte[BUFFER_SIZE]);
-		final ByteBuffer initBuffer = ByteBuffer.allocate(1);
-		
-		// The byte buffer is initialized from this buffer
-		initBuffer.put(BUFFER_CONTENT);
-		initBuffer.flip();
-		
-		// Put byte buffer to recycled queue
-		recycleQueue.add(byteBuffer);
-
-		for (int i = 0; i < BUFFER_SIZE; i++) {
-
-			final MemoryBuffer buffer = BufferFactory.createFromMemory(i, recycleQueue.poll(), new BufferPoolConnector(
-				recycleQueue));
-
-			// Initialize buffer
-			for (int j = 0; j < i; j++) {
-				buffer.write(initBuffer);
-				initBuffer.position(0);
-			}
-			buffer.flip();
-
-			final TransferEnvelope transferEnvelope = new TransferEnvelope(i, this.jobID, this.sourceChannelID);
-			transferEnvelope.setBuffer(buffer);
-
-			// set envelope to be serialized and write it to file channel
-			serializer.setTransferEnvelope(transferEnvelope);
-			while (serializer.write(fileChannel));
-
-			// Put buffer back to the recycling queue
-			buffer.recycleBuffer();
-		}
-
-		fileChannel.close();
-		
-		return outputFile;
-	}
-
-	/**
-	 * Analyzes the given test file and checks whether its content matches Nephele's serialization pattern.
-	 * 
-	 * @param testFile
-	 *        the test file to analyze
-	 * @throws IOException
-	 *         thrown if an I/O error occurs while reading the test file
-	 */
-	private void analyzeStream(File testFile) throws IOException {
-
-		FileInputStream fileInputStream = new FileInputStream(testFile);
-
-		for (int i = 0; i < BUFFER_SIZE; i++) {
-
-			readAndCheckSequenceNumber(fileInputStream, i);
-			readAndCheckID(fileInputStream, this.jobID);
-			readAndCheckID(fileInputStream, this.sourceChannelID);
-			readAndCheckNotificationList(fileInputStream);
-			readAndCheckBuffer(fileInputStream, i);
-		}
-
-		fileInputStream.close();
-	}
-
-	/**
-	 * Attempts to read a buffer of the given size from the file stream and checks the buffer's content.
-	 * 
-	 * @param fileInputStream
-	 *        the file stream to read from
-	 * @param expectedBufferSize
-	 *        the expected size of the buffer
-	 * @throws IOException
-	 *         thrown if an error occurs while reading from the file stream
-	 */
-	private static void readAndCheckBuffer(FileInputStream fileInputStream, int expectedBufferSize) throws IOException {
-
-		// Check if buffer exists
-		assertEquals(1L, fileInputStream.read());
-
-		byte[] temp = new byte[SIZE_OF_INTEGER];
-		fileInputStream.read(temp);
-		int bufferSize = bufferToInteger(temp);
-
-		assertEquals(expectedBufferSize, bufferSize);
-
-		byte[] buffer = new byte[bufferSize];
-		int r = fileInputStream.read(buffer);
-		for (int i = 0; i < buffer.length; i++) {
-			assertEquals(BUFFER_CONTENT, buffer[i]);
-		}
-	}
-
-	/**
-	 * Attempts to read an empty notification list from the given file input stream.
-	 * 
-	 * @param fileInputStream
-	 *        the file input stream to read from
-	 * @throws IOException
-	 *         thrown if an I/O occurs while reading data from the stream
-	 */
-	private void readAndCheckNotificationList(FileInputStream fileInputStream) throws IOException {
-
-		if (fileInputStream.read() != 0) {
-
-			byte[] temp = new byte[SIZE_OF_INTEGER];
-
-			fileInputStream.read(temp);
-			final int sizeOfDataBlock = bufferToInteger(temp);
-
-			assertEquals(SIZE_OF_INTEGER, sizeOfDataBlock);
-
-			fileInputStream.read(temp);
-			final int sizeOfNotificationList = bufferToInteger(temp);
-
-			assertEquals(0, sizeOfNotificationList);
-		}
-	}
-
-	/**
-	 * Attempts to read an integer number from the given file input stream and compares it to
-	 * <code>expectedSequenceNumber</code>.
-	 * 
-	 * @param fileInputStream
-	 *        the file input stream to read from
-	 * @param expectedSeqNumber
-	 *        the integer number the read number is expected to match
-	 * @throws IOException
-	 *         thrown if an I/O occurs while reading data from the stream
-	 */
-	private void readAndCheckSequenceNumber(FileInputStream fileInputStream, int expectedSeqNumber) throws IOException {
-
-		byte[] temp = new byte[SIZE_OF_SEQ_NR];
-		fileInputStream.read(temp);
-		int seqNumber = bufferToInteger(temp);
-
-		assertEquals(seqNumber, expectedSeqNumber);
-	}
-
-	/**
-	 * Attempts to read a channel ID from the given file input stream and compares it to <code>expectedChannelID</code>.
-	 * 
-	 * @param fileInputStream
-	 *        the file input stream to read from
-	 * @param expectedID
-	 *        the ID which the read ID is expected to match
-	 * @throws IOException
-	 *         thrown if an I/O occurs while reading data from the stream
-	 */
-	private void readAndCheckID(FileInputStream fileInputStream, AbstractID expectedID) throws IOException {
-
-		byte[] temp = new byte[SIZE_OF_INTEGER];
-		fileInputStream.read(temp);
-
-		final int sizeOfID = bufferToInteger(temp); // ID has fixed size and therefore does not announce its size
-
-		assertEquals(sizeOfID, SIZE_OF_ID);
-
-		byte[] id = new byte[sizeOfID];
-		fileInputStream.read(id);
-
-		final AbstractID channelID = new SerializationTestID(id);
-		assertEquals(expectedID, channelID);
-	}
-
-	/**
-	 * Converts the first four bytes of the provided buffer's content to an integer number.
-	 * 
-	 * @param buffer
-	 *        the buffer to convert
-	 * @return the integer number converted from the first four bytes of the buffer's content
-	 */
-	private static int bufferToInteger(byte[] buffer) {
-
-		int integer = 0;
-
-		for (int i = 0; i < SIZE_OF_INTEGER; ++i) {
-			integer |= (buffer[SIZE_OF_INTEGER - 1 - i] & 0xff) << (i << 3);
-		}
-
-		return integer;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/BufferPoolConnector.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/BufferPoolConnector.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/BufferPoolConnector.java
deleted file mode 100644
index 3f2c79e..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/BufferPoolConnector.java
+++ /dev/null
@@ -1,53 +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.Queue;
-
-import eu.stratosphere.core.memory.MemorySegment;
-import eu.stratosphere.nephele.io.channels.MemoryBufferPoolConnector;
-
-/**
- * This is a simple implementation of a {@link MemoryBufferPoolConnector} used for the server unit tests.
- * <p>
- * This class is thread-safe.
- * 
- */
-public final class BufferPoolConnector implements MemoryBufferPoolConnector {
-
-	/**
-	 * Reference to the memory pool the byte buffer was originally taken from.
-	 */
-	private final Queue<MemorySegment> memoryPool;
-
-	/**
-	 * Constructs a new buffer pool connector
-	 * 
-	 * @param bufferPool
-	 *        a reference to the memory pool the byte buffer was originally taken from
-	 */
-	public BufferPoolConnector(final Queue<MemorySegment> bufferPool) {
-		this.memoryPool = bufferPool;
-	}
-
-
-	@Override
-	public void recycle(final MemorySegment memSeg) {
-
-		synchronized (this.memoryPool) {
-			this.memoryPool.add(memSeg);
-			this.memoryPool.notify();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/DiscardingRecycler.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/DiscardingRecycler.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/DiscardingRecycler.java
new file mode 100644
index 0000000..9d4d2a5
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/DiscardingRecycler.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.util;
+
+import eu.stratosphere.core.memory.MemorySegment;
+import eu.stratosphere.runtime.io.BufferRecycler;
+
+public class DiscardingRecycler implements BufferRecycler {
+
+	@Override
+	public void recycle(MemorySegment memSeg) {}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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
new file mode 100644
index 0000000..fcb4fa1
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/FileLineReader.java
@@ -0,0 +1,80 @@
+/***********************************************************************************************************************
+ * 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/2db78a8d/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
new file mode 100644
index 0000000..bc738df
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/FileLineWriter.java
@@ -0,0 +1,75 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES 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/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/TestBufferProvider.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/TestBufferProvider.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/TestBufferProvider.java
new file mode 100644
index 0000000..09b244f
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/TestBufferProvider.java
@@ -0,0 +1,76 @@
+/***********************************************************************************************************************
+ *
+ * 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.memory.MemorySegment;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.BufferRecycler;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferAvailabilityListener;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
+
+import java.io.IOException;
+import java.util.Random;
+
+public class TestBufferProvider implements BufferProvider {
+	
+	private final BufferRecycler recycler = new DiscardingRecycler();
+	
+	private final Random rnd = new Random();
+	
+	private final int sizeOfMemorySegments;
+	
+	private final float probabilityForNoneAvailable;
+	
+	
+	public TestBufferProvider(int sizeOfMemorySegments) {
+		this(sizeOfMemorySegments, -1.0f);
+	}
+	
+	public TestBufferProvider(int sizeOfMemorySegments, float probabilityForNoneAvailable) {
+		this.sizeOfMemorySegments = sizeOfMemorySegments;
+		this.probabilityForNoneAvailable = probabilityForNoneAvailable;
+	}
+
+	@Override
+	public Buffer requestBuffer(int sizeOfBuffer) throws IOException {
+		if (rnd.nextFloat() < this.probabilityForNoneAvailable) {
+			return null;
+		} else {
+			MemorySegment segment = new MemorySegment(new byte[this.sizeOfMemorySegments]);
+			return new Buffer(segment, sizeOfBuffer, this.recycler);
+		}
+	}
+
+	@Override
+	public Buffer requestBufferBlocking(int sizeOfBuffer) throws IOException, InterruptedException {
+		MemorySegment segment = new MemorySegment(new byte[this.sizeOfMemorySegments]);
+		return new Buffer(segment, sizeOfBuffer, this.recycler);
+	}
+
+	@Override
+	public int getBufferSize() {
+		return Integer.MAX_VALUE;
+	}
+	
+	@Override
+	public void reportAsynchronousEvent() {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public boolean registerBufferAvailabilityListener(BufferAvailabilityListener bufferAvailabilityListener) {
+		throw new UnsupportedOperationException();
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 6c66b78..bfd0d42 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,8 +42,12 @@ import eu.stratosphere.types.IntValue;
 import eu.stratosphere.types.Key;
 import eu.stratosphere.types.Record;
 
-public class DataSinkTaskTest extends TaskTestBase
-{
+public class DataSinkTaskTest extends TaskTestBase {
+
+	private static final int MEMORY_MANAGER_SIZE = 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");
@@ -61,8 +65,8 @@ public class DataSinkTaskTest extends TaskTestBase
 
 		int keyCnt = 100;
 		int valCnt = 20;
-		
-		super.initEnvironment(1024 * 1024);
+
+		super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
 		super.addInput(new UniformRecordGenerator(keyCnt, valCnt, false), 0);
 		
 		DataSinkTask<Record> testTask = new DataSinkTask<Record>();
@@ -127,8 +131,8 @@ public class DataSinkTaskTest extends TaskTestBase
 
 		int keyCnt = 100;
 		int valCnt = 20;
-		
-		super.initEnvironment(1024 * 1024);
+
+		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);
 		super.addInput(new UniformRecordGenerator(keyCnt, valCnt, keyCnt*2, 0, false), 0);
@@ -197,8 +201,8 @@ public class DataSinkTaskTest extends TaskTestBase
 
 		int keyCnt = 100;
 		int valCnt = 20;
-		
-		super.initEnvironment(1024 * 1024 * 4);
+
+		super.initEnvironment(MEMORY_MANAGER_SIZE * 4, NETWORK_BUFFER_SIZE);
 		super.addInput(new UniformRecordGenerator(keyCnt, valCnt, true), 0);
 		
 		DataSinkTask<Record> testTask = new DataSinkTask<Record>();
@@ -275,8 +279,8 @@ public class DataSinkTaskTest extends TaskTestBase
 
 		int keyCnt = 100;
 		int valCnt = 20;
-		
-		super.initEnvironment(1024 * 1024);
+
+		super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
 		super.addInput(new UniformRecordGenerator(keyCnt, valCnt, false), 0);
 
 		DataSinkTask<Record> testTask = new DataSinkTask<Record>();
@@ -307,8 +311,8 @@ public class DataSinkTaskTest extends TaskTestBase
 
 		int keyCnt = 100;
 		int valCnt = 20;
-		
-		super.initEnvironment(4 * 1024 * 1024);
+
+		super.initEnvironment(MEMORY_MANAGER_SIZE * 4, NETWORK_BUFFER_SIZE);
 		super.addInput(new UniformRecordGenerator(keyCnt, valCnt, true), 0);
 
 		DataSinkTask<Record> testTask = new DataSinkTask<Record>();
@@ -343,8 +347,8 @@ public class DataSinkTaskTest extends TaskTestBase
 	
 	@Test
 	public void testCancelDataSinkTask() {
-		
-		super.initEnvironment(1024 * 1024);
+
+		super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
 		super.addInput(new InfiniteInputIterator(), 0);
 		
 		final DataSinkTask<Record> testTask = new DataSinkTask<Record>();
@@ -385,8 +389,8 @@ public class DataSinkTaskTest extends TaskTestBase
 	@Test
 	@SuppressWarnings("unchecked")
 	public void testCancelSortingDataSinkTask() {
-		
-		super.initEnvironment(4 * 1024 * 1024);
+
+		super.initEnvironment(MEMORY_MANAGER_SIZE * 4, NETWORK_BUFFER_SIZE);
 		super.addInput(new InfiniteInputIterator(), 0);
 		
 		final DataSinkTask<Record> testTask = new DataSinkTask<Record>();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/DataSourceTaskTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/DataSourceTaskTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/DataSourceTaskTest.java
index 0198db2..732cf8d 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/DataSourceTaskTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/DataSourceTaskTest.java
@@ -37,8 +37,12 @@ import eu.stratosphere.types.IntValue;
 import eu.stratosphere.types.Record;
 import eu.stratosphere.util.MutableObjectIterator;
 
-public class DataSourceTaskTest extends TaskTestBase
-{
+public class DataSourceTaskTest extends TaskTestBase {
+
+	private static final int MEMORY_MANAGER_SIZE = 1024 * 1024;
+
+	private static final int NETWORK_BUFFER_SIZE = 1024;
+
 	private List<Record> outList;
 	
 	private String tempTestPath = Path.constructTestPath("dst_test");
@@ -50,11 +54,9 @@ public class DataSourceTaskTest extends TaskTestBase
 			tempTestFile.delete();
 		}
 	}
-
 	
 	@Test
 	public void testDataSourceTask() {
-
 		int keyCnt = 100;
 		int valCnt = 20;
 		
@@ -67,7 +69,7 @@ public class DataSourceTaskTest extends TaskTestBase
 			Assert.fail("Unable to set-up test input file");
 		}
 		
-		super.initEnvironment(1024 * 1024);
+		super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
 		super.addOutput(this.outList);
 		
 		DataSourceTask<Record> testTask = new DataSourceTask<Record>();
@@ -110,7 +112,6 @@ public class DataSourceTaskTest extends TaskTestBase
 	
 	@Test
 	public void testFailingDataSourceTask() {
-
 		int keyCnt = 20;
 		int valCnt = 10;
 		
@@ -122,8 +123,8 @@ public class DataSourceTaskTest extends TaskTestBase
 		} catch (IOException e1) {
 			Assert.fail("Unable to set-up test input file");
 		}
-		
-		super.initEnvironment(1024 * 1024);
+
+		super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
 		super.addOutput(this.outList);
 		
 		DataSourceTask<Record> testTask = new DataSourceTask<Record>();
@@ -148,11 +149,10 @@ public class DataSourceTaskTest extends TaskTestBase
 	
 	@Test
 	public void testCancelDataSourceTask() {
-		
 		int keyCnt = 20;
 		int valCnt = 4;
-		
-		super.initEnvironment(1024 * 1024);
+
+		super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
 		super.addOutput(new NirvanaOutputList());
 		
 		try {
@@ -184,7 +184,7 @@ public class DataSourceTaskTest extends TaskTestBase
 		
 		try {
 			tct.join();
-			taskRunner.join();		
+			taskRunner.join();
 		} catch(InterruptedException ie) {
 			Assert.fail("Joining threads failed");
 		}
@@ -192,12 +192,10 @@ public class DataSourceTaskTest extends TaskTestBase
 		// assert that temp file was created
 		File tempTestFile = new File(this.tempTestPath);
 		Assert.assertTrue("Temp output file does not exist",tempTestFile.exists());
-				
 	}
 
 	
-	private static class InputFilePreparator
-	{
+	private static class InputFilePreparator {
 		public static void prepareInputFile(MutableObjectIterator<Record> inIt, String inputFilePath, boolean insertInvalidData)
 		throws IOException
 		{

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 100bf7b..dda215e 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,7 +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")
@@ -65,9 +69,8 @@ public class ChainTaskTest extends TaskTestBase {
 		final int valCnt = 20;
 		
 		try {
-		
 			// environment
-			initEnvironment(3*1024*1024);
+			super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
 			addInput(new UniformRecordGenerator(keyCnt, valCnt, false), 0);
 			addOutput(this.outList);
 			
@@ -123,7 +126,7 @@ public class ChainTaskTest extends TaskTestBase {
 		
 		try {
 			// environment
-			initEnvironment(3*1024*1024);
+			super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
 			addInput(new UniformRecordGenerator(keyCnt, valCnt, false), 0);
 			addOutput(this.outList);
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/OutputEmitterTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/OutputEmitterTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/OutputEmitterTest.java
index cc00387..c66d821 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/OutputEmitterTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/OutputEmitterTest.java
@@ -26,6 +26,7 @@ import org.apache.commons.lang.NotImplementedException;
 import org.junit.Test;
 
 import eu.stratosphere.api.common.typeutils.TypeComparator;
+import eu.stratosphere.runtime.io.api.ChannelSelector;
 import eu.stratosphere.api.common.typeutils.base.IntSerializer;
 import eu.stratosphere.api.java.typeutils.runtime.record.RecordComparatorFactory;
 import eu.stratosphere.api.java.typeutils.runtime.record.RecordSerializerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/RecordOutputEmitterTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/RecordOutputEmitterTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/RecordOutputEmitterTest.java
index 8afc78f..0b968d8 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/RecordOutputEmitterTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/RecordOutputEmitterTest.java
@@ -27,6 +27,8 @@ import org.junit.Test;
 
 import eu.stratosphere.api.common.distributions.DataDistribution;
 import eu.stratosphere.api.common.distributions.UniformIntegerDistribution;
+import eu.stratosphere.runtime.io.api.ChannelSelector;
+import eu.stratosphere.pact.runtime.plugable.pactrecord.RecordComparator;
 import eu.stratosphere.nephele.io.ChannelSelector;
 import eu.stratosphere.api.java.typeutils.runtime.record.RecordComparator;
 import eu.stratosphere.pact.runtime.shipping.RecordOutputEmitter;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 8e25082..a397312 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
@@ -13,38 +13,40 @@
 
 package eu.stratosphere.pact.runtime.test.util;
 
-import java.io.IOException;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.FutureTask;
-
 import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.core.fs.Path;
 import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.core.memory.MemorySegment;
 import eu.stratosphere.nephele.execution.Environment;
-import eu.stratosphere.nephele.io.ChannelSelector;
-import eu.stratosphere.nephele.io.GateID;
-import eu.stratosphere.nephele.io.InputChannelResult;
-import eu.stratosphere.nephele.io.InputGate;
-import eu.stratosphere.nephele.io.MutableRecordDeserializerFactory;
-import eu.stratosphere.nephele.io.OutputGate;
-import eu.stratosphere.nephele.io.RecordAvailabilityListener;
-import eu.stratosphere.nephele.io.RecordDeserializerFactory;
-import eu.stratosphere.nephele.io.RuntimeInputGate;
-import eu.stratosphere.nephele.io.RuntimeOutputGate;
-import eu.stratosphere.nephele.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.gates.InputChannelResult;
+import eu.stratosphere.runtime.io.gates.RecordAvailabilityListener;
+import eu.stratosphere.runtime.io.serialization.AdaptiveSpanningRecordDeserializer;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.gates.GateID;
+import eu.stratosphere.runtime.io.gates.InputGate;
+import eu.stratosphere.runtime.io.gates.OutputGate;
 import eu.stratosphere.nephele.jobgraph.JobID;
 import eu.stratosphere.nephele.protocols.AccumulatorProtocol;
 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.runtime.io.network.bufferprovider.BufferAvailabilityListener;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
+import eu.stratosphere.runtime.io.network.bufferprovider.GlobalBufferPool;
+import eu.stratosphere.runtime.io.network.bufferprovider.LocalBufferPoolOwner;
 import eu.stratosphere.nephele.template.InputSplitProvider;
+import eu.stratosphere.runtime.io.serialization.RecordDeserializer;
+import eu.stratosphere.runtime.io.serialization.RecordDeserializer.DeserializationResult;
 import eu.stratosphere.types.Record;
 import eu.stratosphere.util.MutableObjectIterator;
 
-public class MockEnvironment implements Environment {
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+
+public class MockEnvironment implements Environment, BufferProvider, LocalBufferPoolOwner {
 	
 	private final MemoryManager memManager;
 
@@ -56,21 +58,24 @@ public class MockEnvironment implements Environment {
 
 	private final Configuration taskConfiguration;
 
-	private final List<RuntimeInputGate<Record>> inputs;
+	private final List<InputGate<Record>> inputs;
 
-	private final List<RuntimeOutputGate<Record>> outputs;
+	private final List<OutputGate> outputs;
 
 	private final JobID jobID = new JobID();
 
-	public MockEnvironment(long memorySize, MockInputSplitProvider inputSplitProvider) {
+	private final Buffer mockBuffer;
+
+	public MockEnvironment(long memorySize, MockInputSplitProvider inputSplitProvider, int bufferSize) {
 		this.jobConfiguration = new Configuration();
 		this.taskConfiguration = new Configuration();
-		this.inputs = new LinkedList<RuntimeInputGate<Record>>();
-		this.outputs = new LinkedList<RuntimeOutputGate<Record>>();
+		this.inputs = new LinkedList<InputGate<Record>>();
+		this.outputs = new LinkedList<OutputGate>();
 
 		this.memManager = new DefaultMemoryManager(memorySize);
 		this.ioManager = new IOManager(System.getProperty("java.io.tmpdir"));
 		this.inputSplitProvider = inputSplitProvider;
+		this.mockBuffer = new Buffer(new MemorySegment(new byte[bufferSize]), bufferSize, null);
 	}
 
 	public void addInput(MutableObjectIterator<Record> inputIterator) {
@@ -103,13 +108,62 @@ public class MockEnvironment implements Environment {
 		return this.jobID;
 	}
 
+	@Override
+	public Buffer requestBuffer(int minBufferSize) throws IOException {
+		return mockBuffer;
+	}
+
+	@Override
+	public Buffer requestBufferBlocking(int minBufferSize) throws IOException, InterruptedException {
+		return mockBuffer;
+	}
+
+	@Override
+	public int getBufferSize() {
+		return this.mockBuffer.size();
+	}
+
+	@Override
+	public boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
+		return false;
+	}
+
+	@Override
+	public int getNumberOfChannels() {
+		return 1;
+	}
+
+	@Override
+	public void setDesignatedNumberOfBuffers(int numBuffers) {
+
+	}
+
+	@Override
+	public void clearLocalBufferPool() {
+
+	}
+
+	@Override
+	public void registerGlobalBufferPool(GlobalBufferPool globalBufferPool) {
+
+	}
+
+	@Override
+	public void logBufferUtilization() {
+
+	}
+
+	@Override
+	public void reportAsynchronousEvent() {
 
-	private static class MockInputGate extends RuntimeInputGate<Record> {
+	}
+
+	private static class MockInputGate extends InputGate<Record> {
 		
 		private MutableObjectIterator<Record> it;
 
 		public MockInputGate(int id, MutableObjectIterator<Record> it) {
-			super(new JobID(), new GateID(), MutableRecordDeserializerFactory.<Record>get(), id);
+			super(new JobID(), new GateID(), id);
 			this.it = it;
 		}
 
@@ -132,18 +186,43 @@ public class MockEnvironment implements Environment {
 		}
 	}
 
-	private static class MockOutputGate extends RuntimeOutputGate<Record> {
+	private class MockOutputGate extends OutputGate {
 		
 		private List<Record> out;
 
+		private RecordDeserializer<Record> deserializer;
+
+		private Record record;
+
 		public MockOutputGate(int index, List<Record> outList) {
-			super(new JobID(), new GateID(), Record.class, index, null, false);
+			super(new JobID(), new GateID(), index);
 			this.out = outList;
+			this.deserializer = new AdaptiveSpanningRecordDeserializer<Record>();
+			this.record = new Record();
 		}
 
 		@Override
-		public void writeRecord(Record record) throws IOException, InterruptedException {
-			out.add(record.createCopy());
+		public void sendBuffer(Buffer buffer, int targetChannel) throws IOException, InterruptedException {
+
+			this.deserializer.setNextMemorySegment(MockEnvironment.this.mockBuffer.getMemorySegment(), MockEnvironment.this.mockBuffer.size());
+
+			while (this.deserializer.hasUnfinishedData()) {
+				DeserializationResult result = this.deserializer.getNextRecord(this.record);
+
+				if (result.isFullRecord()) {
+					this.out.add(this.record.createCopy());
+				}
+
+				if (result == DeserializationResult.LAST_RECORD_FROM_BUFFER ||
+					result == DeserializationResult.PARTIAL_RECORD) {
+					break;
+				}
+			}
+		}
+
+		@Override
+		public int getNumChannels() {
+			return 1;
 		}
 	}
 
@@ -188,11 +267,6 @@ public class MockEnvironment implements Environment {
 	}
 
 	@Override
-	public GateID getNextUnboundOutputGateID() {
-		return null;
-	}
-
-	@Override
 	public int getNumberOfOutputGates() {
 		return this.outputs.size();
 	}
@@ -203,16 +277,6 @@ public class MockEnvironment implements Environment {
 	}
 
 	@Override
-	public void registerOutputGate(final OutputGate<? extends IOReadableWritable> outputGate) {
-		// Nothing to do here
-	}
-
-	@Override
-	public void registerInputGate(final InputGate<? extends IOReadableWritable> inputGate) {
-		// Nothing to do here
-	}
-
-	@Override
 	public Set<ChannelID> getOutputChannelIDs() {
 		throw new IllegalStateException("getOutputChannelIDs called on MockEnvironment");
 	}
@@ -242,18 +306,14 @@ public class MockEnvironment implements Environment {
 		throw new IllegalStateException("getInputChannelIDsOfGate called on MockEnvironment");
 	}
 
-	@SuppressWarnings("unchecked")
 	@Override
-	public <T extends IOReadableWritable> OutputGate<T> createOutputGate(GateID gateID, Class<T> outputClass,
-			ChannelSelector<T> selector, boolean isBroadcast)
+	public OutputGate createAndRegisterOutputGate()
 	{
-		return (OutputGate<T>) this.outputs.remove(0);
+		return this.outputs.remove(0);
 	}
 
-	@SuppressWarnings("unchecked")
 	@Override
-	public <T extends IOReadableWritable> InputGate<T> createInputGate(GateID gateID,
-			RecordDeserializerFactory<T> deserializerFactory)
+	public <T extends IOReadableWritable> InputGate<T> createAndRegisterInputGate()
 	{
 		return (InputGate<T>) this.inputs.remove(0);
 	}
@@ -275,8 +335,7 @@ public class MockEnvironment implements Environment {
 	}
 
 	@Override
-	public 	Map<String, FutureTask<Path>> getCopyTask() {
-		return null;
+	public BufferProvider getOutputBufferProvider() {
+		return this;
 	}
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 826113c..a60b479 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
@@ -49,10 +49,10 @@ public abstract class TaskTestBase {
 
 	protected MockEnvironment mockEnv;
 
-	public void initEnvironment(long memorySize) {
+	public void initEnvironment(long memorySize, int bufferSize) {
 		this.memorySize = memorySize;
 		this.inputSplitProvider = new MockInputSplitProvider();
-		this.mockEnv = new MockEnvironment(this.memorySize, this.inputSplitProvider);
+		this.mockEnv = new MockEnvironment(this.memorySize, this.inputSplitProvider, bufferSize);
 	}
 
 	public void addInput(MutableObjectIterator<Record> input, int groupId) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/fs/LineReaderTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/fs/LineReaderTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/fs/LineReaderTest.java
new file mode 100644
index 0000000..af46689
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/fs/LineReaderTest.java
@@ -0,0 +1,78 @@
+/***********************************************************************************************************************
+ * 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.fs;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.PrintWriter;
+
+import org.junit.Test;
+
+import eu.stratosphere.core.fs.FSDataInputStream;
+import eu.stratosphere.core.fs.Path;
+import eu.stratosphere.core.fs.local.LocalFileSystem;
+import eu.stratosphere.nephele.util.CommonTestUtils;
+
+/**
+ * This class tests the functionality of the LineReader class using a local filesystem.
+ * 
+ */
+
+public class LineReaderTest {
+
+	/**
+	 * This test tests the LineReader. So far only under usual conditions.
+	 */
+	@Test
+	public void testLineReader() {
+		final File testfile = new File(CommonTestUtils.getTempDir() + File.separator
+			+ CommonTestUtils.getRandomFilename());
+		final Path pathtotestfile = new Path(testfile.toURI().getPath());
+
+		try {
+			PrintWriter pw = new PrintWriter(testfile, "UTF8");
+
+			for (int i = 0; i < 100; i++) {
+				pw.append("line\n");
+			}
+			pw.close();
+
+			LocalFileSystem lfs = new LocalFileSystem();
+			FSDataInputStream fis = lfs.open(pathtotestfile);
+
+			// first, we test under "usual" conditions
+			final LineReader lr = new LineReader(fis, 0, testfile.length(), 256);
+
+			byte[] buffer;
+			int linecount = 0;
+			while ((buffer = lr.readLine()) != null) {
+				assertEquals(new String(buffer, "UTF8"), "line");
+				linecount++;
+			}
+			assertEquals(linecount, 100);
+
+			// the linereader can not handle situations with larger length than the total file...
+
+		} catch (Exception e) {
+			fail(e.toString());
+			e.printStackTrace();
+		} finally {
+			testfile.delete();
+		}
+
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/fs/s3/S3FileSystemTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/fs/s3/S3FileSystemTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/fs/s3/S3FileSystemTest.java
new file mode 100644
index 0000000..30b5219
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/fs/s3/S3FileSystemTest.java
@@ -0,0 +1,460 @@
+/***********************************************************************************************************************
+ * 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.fs.s3;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URI;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import eu.stratosphere.configuration.Configuration;
+import eu.stratosphere.configuration.GlobalConfiguration;
+import eu.stratosphere.core.fs.BlockLocation;
+import eu.stratosphere.core.fs.FSDataInputStream;
+import eu.stratosphere.core.fs.FSDataOutputStream;
+import eu.stratosphere.core.fs.FileStatus;
+import eu.stratosphere.core.fs.FileSystem;
+import eu.stratosphere.core.fs.Path;
+
+/**
+ * This test checks the S3 implementation of the {@link FileSystem} interface.
+ * 
+ */
+public class S3FileSystemTest {
+
+	/**
+	 * The length of the bucket/object names used in this test.
+	 */
+	private static final int NAME_LENGTH = 32;
+
+	/**
+	 * The alphabet to generate the random bucket/object names from.
+	 */
+	private static final char[] ALPHABET = { 'a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k', 'l', 'm', 'n', 'o',
+		'p', 'q', 'r', 's', 't', 'u', 'v', 'w', 'x', 'y', 'z', '0', '1', '2', '3', '4', '5', '6', '7', '8', '9' };
+
+	/**
+	 * The size of the byte buffer used during the tests in bytes.
+	 */
+	private static final int TEST_BUFFER_SIZE = 128;
+
+	/**
+	 * The size of the small test file in bytes.
+	 */
+	private static final int SMALL_FILE_SIZE = 512;
+
+	/**
+	 * The size of the large test file in bytes.
+	 */
+	private static final int LARGE_FILE_SIZE = 1024 * 1024 * 12; // 12 MB
+
+	/**
+	 * The modulus to be used when generating the test data. Must not be larger than 128.
+	 */
+	private static final int MODULUS = 128;
+
+	private static final String S3_BASE_URI = "s3:///";
+
+	/**
+	 * Tries to read the AWS access key and the AWS secret key from the environments variables. If accessing these keys
+	 * fails, all tests will be skipped and marked as successful.
+	 */
+	@Before
+	public void initKeys() {
+		final String accessKey = System.getenv("AK");
+		final String secretKey = System.getenv("SK");
+		
+		if (accessKey != null || secretKey != null) {
+			Configuration conf = new Configuration();
+			if (accessKey != null) {
+				conf.setString(S3FileSystem.S3_ACCESS_KEY_KEY, accessKey);
+			}
+			if (secretKey != null) {
+				conf.setString(S3FileSystem.S3_SECRET_KEY_KEY, secretKey);
+			}
+			GlobalConfiguration.includeConfiguration(conf);
+		}
+	}
+
+	/**
+	 * This test creates and deletes a bucket inside S3 and checks it is correctly displayed inside the directory
+	 * listing.
+	 */
+	@Test
+	public void createAndDeleteBucketTest() {
+
+		if (!testActivated()) {
+			return;
+		}
+
+		final String bucketName = getRandomName();
+		final Path bucketPath = new Path(S3_BASE_URI + bucketName + Path.SEPARATOR);
+
+		try {
+
+			final FileSystem fs = bucketPath.getFileSystem();
+
+			// Create directory
+			fs.mkdirs(bucketPath);
+
+			// Check if directory is correctly displayed in file system hierarchy
+			final FileStatus[] content = fs.listStatus(new Path(S3_BASE_URI));
+			boolean entryFound = false;
+			for (final FileStatus entry : content) {
+				if (bucketPath.equals(entry.getPath())) {
+					entryFound = true;
+					break;
+				}
+			}
+
+			if (!entryFound) {
+				fail("Cannot find entry " + bucketName + " in directory " + S3_BASE_URI);
+			}
+
+			// Check the concrete directory file status
+			try {
+				final FileStatus directoryFileStatus = fs.getFileStatus(bucketPath);
+				assertTrue(directoryFileStatus.isDir());
+				assertEquals(0L, directoryFileStatus.getAccessTime());
+				assertTrue(directoryFileStatus.getModificationTime() > 0L);
+
+			} catch (FileNotFoundException e) {
+				fail(e.getMessage());
+			}
+
+			// Delete the bucket
+			fs.delete(bucketPath, true);
+
+			// Make sure the bucket no longer exists
+			try {
+				fs.getFileStatus(bucketPath);
+				fail("Expected FileNotFoundException for " + bucketPath.toUri());
+			} catch (FileNotFoundException e) {
+				// This is an expected exception
+			}
+
+		} catch (IOException ioe) {
+			fail(ioe.getMessage());
+		}
+	}
+
+	/**
+	 * Creates and reads the a larger test file in S3. The test file is generated according to a specific pattern.
+	 * During the read phase the incoming data stream is also checked against this pattern.
+	 */
+	@Test
+	public void createAndReadLargeFileTest() {
+
+		try {
+			createAndReadFileTest(LARGE_FILE_SIZE);
+		} catch (IOException ioe) {
+			fail(ioe.getMessage());
+		}
+	}
+
+	/**
+	 * Creates and reads the a small test file in S3. The test file is generated according to a specific pattern.
+	 * During the read phase the incoming data stream is also checked against this pattern.
+	 */
+	@Test
+	public void createAndReadSmallFileTest() {
+
+		try {
+			createAndReadFileTest(SMALL_FILE_SIZE);
+		} catch (IOException ioe) {
+			fail(ioe.getMessage());
+		}
+	}
+
+	/**
+	 * The tests checks the mapping of the file system directory structure to the underlying bucket/object model of
+	 * Amazon S3.
+	 */
+	@Test
+	public void multiLevelDirectoryTest() {
+
+		if (!testActivated()) {
+			return;
+		}
+
+		final String dirName = getRandomName();
+		final String subdirName = getRandomName();
+		final String subsubdirName = getRandomName();
+		final String fileName = getRandomName();
+		final Path dir = new Path(S3_BASE_URI + dirName + Path.SEPARATOR);
+		final Path subdir = new Path(S3_BASE_URI + dirName + Path.SEPARATOR + subdirName + Path.SEPARATOR);
+		final Path subsubdir = new Path(S3_BASE_URI + dirName + Path.SEPARATOR + subdirName + Path.SEPARATOR
+			+ subsubdirName + Path.SEPARATOR);
+		final Path file = new Path(S3_BASE_URI + dirName + Path.SEPARATOR + subdirName + Path.SEPARATOR + fileName);
+
+		try {
+
+			final FileSystem fs = dir.getFileSystem();
+
+			fs.mkdirs(subsubdir);
+
+			final OutputStream os = fs.create(file, true);
+			generateTestData(os, SMALL_FILE_SIZE);
+			os.close();
+
+			// On this directory levels there should only be one subdirectory
+			FileStatus[] list = fs.listStatus(dir);
+			int numberOfDirs = 0;
+			int numberOfFiles = 0;
+			for (final FileStatus entry : list) {
+
+				if (entry.isDir()) {
+					++numberOfDirs;
+					assertEquals(subdir, entry.getPath());
+				} else {
+					fail(entry.getPath() + " is a file which must not appear on this directory level");
+				}
+			}
+
+			assertEquals(1, numberOfDirs);
+			assertEquals(0, numberOfFiles);
+
+			list = fs.listStatus(subdir);
+			numberOfDirs = 0;
+
+			for (final FileStatus entry : list) {
+				if (entry.isDir()) {
+					assertEquals(subsubdir, entry.getPath());
+					++numberOfDirs;
+				} else {
+					assertEquals(file, entry.getPath());
+					++numberOfFiles;
+				}
+			}
+
+			assertEquals(1, numberOfDirs);
+			assertEquals(1, numberOfFiles);
+
+			fs.delete(dir, true);
+
+		} catch (IOException ioe) {
+			fail(ioe.getMessage());
+		}
+	}
+
+	/**
+	 * This test checks the S3 implementation of the file system method to retrieve the block locations of a file.
+	 */
+	@Test
+	public void blockLocationTest() {
+
+		if (!testActivated()) {
+			return;
+		}
+
+		final String dirName = getRandomName();
+		final String fileName = getRandomName();
+		final Path dir = new Path(S3_BASE_URI + dirName + Path.SEPARATOR);
+		final Path file = new Path(S3_BASE_URI + dirName + Path.SEPARATOR + fileName);
+
+		try {
+
+			final FileSystem fs = dir.getFileSystem();
+
+			fs.mkdirs(dir);
+
+			final OutputStream os = fs.create(file, true);
+			generateTestData(os, SMALL_FILE_SIZE);
+			os.close();
+
+			final FileStatus fileStatus = fs.getFileStatus(file);
+			assertNotNull(fileStatus);
+
+			BlockLocation[] blockLocations = fs.getFileBlockLocations(fileStatus, 0, SMALL_FILE_SIZE + 1);
+			assertNull(blockLocations);
+
+			blockLocations = fs.getFileBlockLocations(fileStatus, 0, SMALL_FILE_SIZE);
+			assertEquals(1, blockLocations.length);
+
+			final BlockLocation bl = blockLocations[0];
+			assertNotNull(bl.getHosts());
+			assertEquals(1, bl.getHosts().length);
+			assertEquals(SMALL_FILE_SIZE, bl.getLength());
+			assertEquals(0, bl.getOffset());
+			final URI s3Uri = fs.getUri();
+			assertNotNull(s3Uri);
+			assertEquals(s3Uri.getHost(), bl.getHosts()[0]);
+
+			fs.delete(dir, true);
+
+		} catch (IOException ioe) {
+			fail(ioe.getMessage());
+		}
+	}
+
+	/**
+	 * Creates and reads a file with the given size in S3. The test file is generated according to a specific pattern.
+	 * During the read phase the incoming data stream is also checked against this pattern.
+	 * 
+	 * @param fileSize
+	 *        the size of the file to be generated in bytes
+	 * @throws IOException
+	 *         thrown if an I/O error occurs while writing or reading the test file
+	 */
+	private void createAndReadFileTest(final int fileSize) throws IOException {
+
+		if (!testActivated()) {
+			return;
+		}
+
+		final String bucketName = getRandomName();
+		final String objectName = getRandomName();
+		final Path bucketPath = new Path(S3_BASE_URI + bucketName + Path.SEPARATOR);
+		final Path objectPath = new Path(S3_BASE_URI + bucketName + Path.SEPARATOR + objectName);
+
+		FileSystem fs = bucketPath.getFileSystem();
+
+		// Create test bucket
+		fs.mkdirs(bucketPath);
+
+		// Write test file to S3
+		final FSDataOutputStream outputStream = fs.create(objectPath, false);
+		generateTestData(outputStream, fileSize);
+		outputStream.close();
+
+		// Now read the same file back from S3
+		final FSDataInputStream inputStream = fs.open(objectPath);
+		testReceivedData(inputStream, fileSize);
+		inputStream.close();
+
+		// Delete test bucket
+		fs.delete(bucketPath, true);
+	}
+
+	/**
+	 * Receives test data from the given input stream and checks the size of the data as well as the pattern inside the
+	 * received data.
+	 * 
+	 * @param inputStream
+	 *        the input stream to read the test data from
+	 * @param expectedSize
+	 *        the expected size of the data to be read from the input stream in bytes
+	 * @throws IOException
+	 *         thrown if an error occurs while reading the data
+	 */
+	private void testReceivedData(final InputStream inputStream, final int expectedSize) throws IOException {
+
+		final byte[] testBuffer = new byte[TEST_BUFFER_SIZE];
+
+		int totalBytesRead = 0;
+		int nextExpectedNumber = 0;
+		while (true) {
+
+			final int bytesRead = inputStream.read(testBuffer);
+			if (bytesRead < 0) {
+				break;
+			}
+
+			totalBytesRead += bytesRead;
+
+			for (int i = 0; i < bytesRead; ++i) {
+				if (testBuffer[i] != nextExpectedNumber) {
+					throw new IOException("Read number " + testBuffer[i] + " but expected " + nextExpectedNumber);
+				}
+
+				++nextExpectedNumber;
+
+				if (nextExpectedNumber == MODULUS) {
+					nextExpectedNumber = 0;
+				}
+			}
+		}
+
+		if (totalBytesRead != expectedSize) {
+			throw new IOException("Expected to read " + expectedSize + " bytes but only received " + totalBytesRead);
+		}
+	}
+
+	/**
+	 * Generates test data of the given size according to some specific pattern and writes it to the provided output
+	 * stream.
+	 * 
+	 * @param outputStream
+	 *        the output stream to write the data to
+	 * @param size
+	 *        the size of the test data to be generated in bytes
+	 * @throws IOException
+	 *         thrown if an error occurs while writing the data
+	 */
+	private void generateTestData(final OutputStream outputStream, final int size) throws IOException {
+
+		final byte[] testBuffer = new byte[TEST_BUFFER_SIZE];
+		for (int i = 0; i < testBuffer.length; ++i) {
+			testBuffer[i] = (byte) (i % MODULUS);
+		}
+
+		int bytesWritten = 0;
+		while (bytesWritten < size) {
+
+			final int diff = size - bytesWritten;
+			if (diff < testBuffer.length) {
+				outputStream.write(testBuffer, 0, diff);
+				bytesWritten += diff;
+			} else {
+				outputStream.write(testBuffer);
+				bytesWritten += testBuffer.length;
+			}
+		}
+	}
+
+	/**
+	 * Generates a random name.
+	 * 
+	 * @return a random name
+	 */
+	private String getRandomName() {
+
+		final StringBuilder stringBuilder = new StringBuilder();
+		for (int i = 0; i < NAME_LENGTH; ++i) {
+			final char c = ALPHABET[(int) (Math.random() * (double) ALPHABET.length)];
+			stringBuilder.append(c);
+		}
+
+		return stringBuilder.toString();
+	}
+
+	/**
+	 * Checks whether the AWS access key and the AWS secret keys have been successfully loaded from the configuration
+	 * and whether the S3 tests shall be performed.
+	 * 
+	 * @return <code>true</code> if the tests shall be performed, <code>false</code> if the tests shall be skipped
+	 *         because at least one AWS key is missing
+	 */
+	private boolean testActivated() {
+
+		final String accessKey = GlobalConfiguration.getString(S3FileSystem.S3_ACCESS_KEY_KEY, null);
+		final String secretKey = GlobalConfiguration.getString(S3FileSystem.S3_SECRET_KEY_KEY, null);
+
+		if (accessKey != null && secretKey != null) {
+			return true;
+		}
+
+		return false;
+	}
+}


[06/30] Replace custom Java NIO TCP/IP code with Netty 4 library

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/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 405d79e..7b01f4e 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
@@ -31,15 +31,13 @@ import eu.stratosphere.nephele.AbstractID;
 import eu.stratosphere.runtime.io.Buffer;
 import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
 import eu.stratosphere.runtime.io.network.bufferprovider.BufferProviderBroker;
+import eu.stratosphere.runtime.io.network.bufferprovider.DiscardBufferPool;
 import eu.stratosphere.runtime.io.network.bufferprovider.GlobalBufferPool;
 import eu.stratosphere.runtime.io.network.bufferprovider.LocalBufferPoolOwner;
-import eu.stratosphere.runtime.io.network.bufferprovider.SerialSingleBufferPool;
-import eu.stratosphere.runtime.io.network.envelope.Envelope;
-import eu.stratosphere.runtime.io.network.envelope.EnvelopeDispatcher;
-import eu.stratosphere.runtime.io.network.envelope.EnvelopeReceiverList;
 import eu.stratosphere.runtime.io.gates.GateID;
 import eu.stratosphere.runtime.io.gates.InputGate;
 import eu.stratosphere.runtime.io.gates.OutputGate;
+import eu.stratosphere.runtime.io.network.netty.NettyConnectionManager;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
@@ -52,7 +50,7 @@ import java.util.concurrent.ConcurrentHashMap;
 /**
  * The channel manager sets up the network buffers and dispatches data between channels.
  */
-public final class ChannelManager implements EnvelopeDispatcher, BufferProviderBroker {
+public class ChannelManager implements EnvelopeDispatcher, BufferProviderBroker {
 
 	private static final Log LOG = LogFactory.getLog(ChannelManager.class);
 
@@ -68,20 +66,27 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
 
 	private final GlobalBufferPool globalBufferPool;
 
-	private final NetworkConnectionManager networkConnectionManager;
+	private final NettyConnectionManager nettyConnectionManager;
 	
 	private final InetSocketAddress ourAddress;
 	
-	private final SerialSingleBufferPool discardingDataPool;
+	private final DiscardBufferPool discardBufferPool;
 
 	// -----------------------------------------------------------------------------------------------------------------
 
 	public ChannelManager(ChannelLookupProtocol channelLookupService, InstanceConnectionInfo connectionInfo,
-						  int numNetworkBuffers, int networkBufferSize) throws IOException {
+						int numNetworkBuffers, int networkBufferSize,
+						int numInThreads, int numOutThreads,
+						int lowWatermark, int highWaterMark) throws IOException {
+
 		this.channelLookupService = channelLookupService;
 		this.connectionInfo = connectionInfo;
+
 		this.globalBufferPool = new GlobalBufferPool(numNetworkBuffers, networkBufferSize);
-		this.networkConnectionManager = new NetworkConnectionManager(this, connectionInfo.address(), connectionInfo.dataPort());
+
+		this.nettyConnectionManager = new NettyConnectionManager(
+				this, connectionInfo.address(), connectionInfo.dataPort(),
+				networkBufferSize, numInThreads, numOutThreads, lowWatermark, highWaterMark);
 
 		// management data structures
 		this.channels = new ConcurrentHashMap<ChannelID, Channel>();
@@ -91,11 +96,11 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
 		this.ourAddress = new InetSocketAddress(connectionInfo.address(), connectionInfo.dataPort());
 		
 		// a special pool if the data is to be discarded
-		this.discardingDataPool = new SerialSingleBufferPool(networkBufferSize);
+		this.discardBufferPool = new DiscardBufferPool();
 	}
 
 	public void shutdown() {
-		this.networkConnectionManager.shutDown();
+		this.nettyConnectionManager.shutdown();
 		this.globalBufferPool.destroy();
 	}
 
@@ -301,7 +306,7 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
 		}
 	}
 
-	private void generateSenderHint(Envelope envelope, RemoteReceiver receiver) {
+	private void generateSenderHint(Envelope envelope, RemoteReceiver receiver) throws IOException {
 		Channel channel = this.channels.get(envelope.getSource());
 		if (channel == null) {
 			LOG.error("Cannot find channel for channel ID " + envelope.getSource());
@@ -319,7 +324,7 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
 		final RemoteReceiver ourAddress = new RemoteReceiver(this.ourAddress, connectionIndex);
 		final Envelope senderHint = SenderHintEvent.createEnvelopeWithEvent(envelope, targetChannelID, ourAddress);
 
-		this.networkConnectionManager.queueEnvelopeForTransfer(receiver, senderHint);
+		this.nettyConnectionManager.enqueue(senderHint, receiver);
 	}
 
 	/**
@@ -331,7 +336,6 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
 	 *        the source channel ID for which the receiver list shall be retrieved
 	 * @return the list of receivers or <code>null</code> if the receiver could not be determined
 	 * @throws IOException
-	 * @throws InterruptedException
 	 */
 	private EnvelopeReceiverList getReceiverList(JobID jobID, ChannelID sourceChannelID, boolean reportException) throws IOException {
 		EnvelopeReceiverList receiverList = this.receiverCache.get(sourceChannelID);
@@ -383,8 +387,10 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
 		this.receiverCache.put(sourceChannelID, receiverList);
 
 		if (LOG.isDebugEnabled()) {
-			LOG.debug("Receivers for source channel ID " + sourceChannelID + " at task manager " + this.connectionInfo +
-				": " + receiverList);
+			LOG.debug(String.format("Receiver for %s: %s [%s])",
+					sourceChannelID,
+					receiverList.hasLocalReceiver() ? receiverList.getLocalReceiver() : receiverList.getRemoteReceiver(),
+					receiverList.hasLocalReceiver() ? "local" : "remote"));
 		}
 
 		return receiverList;
@@ -436,7 +442,7 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
 					} catch (InterruptedException e) {
 						throw new IOException(e.getMessage());
 					}
-					
+
 					srcBuffer.copyToBuffer(destBuffer);
 					envelope.setBuffer(destBuffer);
 					srcBuffer.recycleBuffer();
@@ -453,7 +459,7 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
 					generateSenderHint(envelope, remoteReceiver);
 				}
 
-				this.networkConnectionManager.queueEnvelopeForTransfer(remoteReceiver, envelope);
+				this.nettyConnectionManager.enqueue(envelope, remoteReceiver);
 				success = true;
 			}
 		} finally {
@@ -501,7 +507,7 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
 				generateSenderHint(envelope, remoteReceiver);
 			}
 
-			this.networkConnectionManager.queueEnvelopeForTransfer(remoteReceiver, envelope);
+			this.nettyConnectionManager.enqueue(envelope, remoteReceiver);
 		}
 	}
 
@@ -597,7 +603,7 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
 		
 		// check if the receiver is already gone
 		if (receiverList == null) {
-			return this.discardingDataPool;
+			return this.discardBufferPool;
 		}
 
 		if (!receiverList.hasLocalReceiver() || receiverList.hasRemoteReceiver()) {
@@ -610,7 +616,7 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
 		
 		if (channel == null) {
 			// receiver is already canceled
-			return this.discardingDataPool;
+			return this.discardBufferPool;
 		}
 
 		if (!channel.isInputChannel()) {
@@ -633,8 +639,6 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
 			bufferPool.logBufferUtilization();
 		}
 
-		this.networkConnectionManager.logBufferUtilization();
-
 		System.out.println("\tIncoming connections:");
 
 		for (Channel channel : this.channels.values()) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/Envelope.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/Envelope.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/Envelope.java
new file mode 100644
index 0000000..0f8bae8
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/Envelope.java
@@ -0,0 +1,178 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.network;
+
+import eu.stratosphere.nephele.event.task.AbstractEvent;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.nephele.jobgraph.JobID;
+import eu.stratosphere.runtime.io.serialization.DataInputDeserializer;
+import eu.stratosphere.runtime.io.serialization.DataOutputSerializer;
+import eu.stratosphere.util.InstantiationUtil;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public final class Envelope {
+
+	private final JobID jobID;
+
+	private final ChannelID source;
+
+	private final int sequenceNumber;
+
+	private ByteBuffer serializedEventList;
+
+	private Buffer buffer;
+
+	public Envelope(int sequenceNumber, JobID jobID, ChannelID source) {
+		this.sequenceNumber = sequenceNumber;
+		this.jobID = jobID;
+		this.source = source;
+	}
+
+	private Envelope(Envelope toDuplicate) {
+		this.jobID = toDuplicate.jobID;
+		this.source = toDuplicate.source;
+		this.sequenceNumber = toDuplicate.sequenceNumber;
+		this.serializedEventList = null;
+		this.buffer = null;
+	}
+
+	public Envelope duplicate() {
+		Envelope duplicate = new Envelope(this);
+		if (hasBuffer()) {
+			duplicate.setBuffer(this.buffer.duplicate());
+		}
+
+		return duplicate;
+	}
+
+	public Envelope duplicateWithoutBuffer() {
+		return new Envelope(this);
+	}
+
+	public JobID getJobID() {
+		return this.jobID;
+	}
+
+	public ChannelID getSource() {
+		return this.source;
+	}
+
+	public int getSequenceNumber() {
+		return this.sequenceNumber;
+	}
+
+	public void setEventsSerialized(ByteBuffer serializedEventList) {
+		if (this.serializedEventList != null) {
+			throw new IllegalStateException("Event list has already been set.");
+		}
+
+		this.serializedEventList = serializedEventList;
+	}
+
+	public void serializeEventList(List<? extends AbstractEvent> eventList) {
+		if (this.serializedEventList != null) {
+			throw new IllegalStateException("Event list has already been set.");
+		}
+
+		this.serializedEventList = serializeEvents(eventList);
+	}
+
+	public ByteBuffer getEventsSerialized() {
+		return this.serializedEventList;
+	}
+
+	public List<? extends AbstractEvent> deserializeEvents() {
+		return deserializeEvents(getClass().getClassLoader());
+	}
+
+	public List<? extends AbstractEvent> deserializeEvents(ClassLoader classloader) {
+		if (this.serializedEventList == null) {
+			return Collections.emptyList();
+		}
+
+		try {
+			DataInputDeserializer deserializer = new DataInputDeserializer(this.serializedEventList);
+
+			int numEvents = deserializer.readInt();
+			ArrayList<AbstractEvent> events = new ArrayList<AbstractEvent>(numEvents);
+
+			for (int i = 0; i < numEvents; i++) {
+				String className = deserializer.readUTF();
+				Class<? extends AbstractEvent> clazz;
+				try {
+					clazz = Class.forName(className).asSubclass(AbstractEvent.class);
+				} catch (ClassNotFoundException e) {
+					throw new RuntimeException("Could not load event class '" + className + "'.", e);
+				} catch (ClassCastException e) {
+					throw new RuntimeException("The class '" + className + "' is no valid subclass of '" + AbstractEvent.class.getName() + "'.", e);
+				}
+
+				AbstractEvent evt = InstantiationUtil.instantiate(clazz, AbstractEvent.class);
+				evt.read(deserializer);
+
+				events.add(evt);
+			}
+
+			return events;
+		}
+		catch (IOException e) {
+			throw new RuntimeException("Error while deserializing the events.", e);
+		}
+	}
+
+	public void setBuffer(Buffer buffer) {
+		this.buffer = buffer;
+	}
+
+	public Buffer getBuffer() {
+		return this.buffer;
+	}
+
+	private ByteBuffer serializeEvents(List<? extends AbstractEvent> events) {
+		try {
+			// create the serialized event list
+			DataOutputSerializer serializer = events.size() == 0
+				? new DataOutputSerializer(4)
+				: new DataOutputSerializer(events.size() * 32);
+			serializer.writeInt(events.size());
+
+			for (AbstractEvent evt : events) {
+				serializer.writeUTF(evt.getClass().getName());
+				evt.write(serializer);
+			}
+
+			return serializer.wrapAsByteBuffer();
+		}
+		catch (IOException e) {
+			throw new RuntimeException("Error while serializing the task events.", e);
+		}
+	}
+
+	public boolean hasBuffer() {
+		return this.buffer != null;
+	}
+
+	@Override
+	public String toString() {
+		return String.format("Envelope %d [source id: %s, buffer size: %d, events size: %d]",
+				this.sequenceNumber, this.getSource(), this.buffer == null ? -1 : this.buffer.size(),
+				this.serializedEventList == null ? -1 : this.serializedEventList.remaining());
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/EnvelopeDispatcher.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/EnvelopeDispatcher.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/EnvelopeDispatcher.java
new file mode 100644
index 0000000..372ff92
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/EnvelopeDispatcher.java
@@ -0,0 +1,46 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.network;
+
+import java.io.IOException;
+
+/**
+ * A envelope dispatcher receives {@link Envelope}s and sends them to all of its destinations.
+ */
+public interface EnvelopeDispatcher {
+
+	/**
+	 * Dispatches an envelope from an output channel to the receiving input channels (forward flow).
+	 *
+	 * @param envelope envelope to be sent
+	 */
+	void dispatchFromOutputChannel(Envelope envelope) throws IOException, InterruptedException;
+
+	/**
+	 * Dispatches an envelope from an input channel to the receiving output channels (backwards flow).
+	 *
+	 * @param envelope envelope to be sent
+	 */
+	void dispatchFromInputChannel(Envelope envelope) throws IOException, InterruptedException;
+
+	/**
+	 * Dispatches an envelope from an incoming TCP connection.
+	 * <p>
+	 * After an envelope has been constructed from a TCP socket, this method is called to send the envelope to the
+	 * receiving input channel.
+	 *
+	 * @param envelope envelope to be sent
+	 */
+	void dispatchFromNetwork(Envelope envelope) throws IOException, InterruptedException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/EnvelopeReceiverList.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/EnvelopeReceiverList.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/EnvelopeReceiverList.java
new file mode 100644
index 0000000..d53d728
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/EnvelopeReceiverList.java
@@ -0,0 +1,75 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES 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.network;
+
+import java.net.InetAddress;
+
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.network.ConnectionInfoLookupResponse;
+import eu.stratosphere.runtime.io.network.RemoteReceiver;
+
+/**
+ * A transfer envelope receiver list contains all recipients of a transfer envelope. Their are three different types of
+ * receivers: Local receivers identified by {@link ChannelID} objects, remote receivers identified by
+ * {@link InetAddress} objects and finally checkpoints which are identified by
+ * <p>
+ * This class is thread-safe.
+ * 
+ */
+public class EnvelopeReceiverList {
+
+	private final ChannelID localReceiver;
+
+	private final RemoteReceiver remoteReceiver;
+
+	public EnvelopeReceiverList(ConnectionInfoLookupResponse cilr) {
+		this.localReceiver = cilr.getLocalTarget();
+		this.remoteReceiver = cilr.getRemoteTarget();
+	}
+
+	public EnvelopeReceiverList(ChannelID localReceiver) {
+		this.localReceiver = localReceiver;
+		this.remoteReceiver = null;
+	}
+
+	public EnvelopeReceiverList(RemoteReceiver remoteReceiver) {
+		this.localReceiver = null;
+		this.remoteReceiver = remoteReceiver;
+	}
+
+	public boolean hasLocalReceiver() {
+		return this.localReceiver != null;
+	}
+
+	public boolean hasRemoteReceiver() {
+		return this.remoteReceiver != null;
+	}
+
+	public int getTotalNumberOfReceivers() {
+		return (this.localReceiver == null ? 0 : 1) + (this.remoteReceiver == null ? 0 : 1);
+	}
+
+	public RemoteReceiver getRemoteReceiver() {
+		return this.remoteReceiver;
+	}
+
+	public ChannelID getLocalReceiver() {
+		return this.localReceiver;
+	}
+	
+	@Override
+	public String toString() {
+		return "local receiver: " + this.localReceiver + ", remote receiver: " + this.remoteReceiver;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/NetworkConnectionManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/NetworkConnectionManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/NetworkConnectionManager.java
deleted file mode 100644
index 44ec642..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/NetworkConnectionManager.java
+++ /dev/null
@@ -1,176 +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.network;
-
-import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.configuration.GlobalConfiguration;
-import eu.stratosphere.runtime.io.network.envelope.Envelope;
-import eu.stratosphere.runtime.io.network.tcp.IncomingConnectionThread;
-import eu.stratosphere.runtime.io.network.tcp.OutgoingConnection;
-import eu.stratosphere.runtime.io.network.tcp.OutgoingConnectionThread;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.CopyOnWriteArrayList;
-
-/**
- * The network connection manager manages incoming and outgoing network connection from and to other hosts.
- * <p>
- * This class is thread-safe.
- * 
- */
-public final class NetworkConnectionManager {
-
-	/**
-	 * The default number of threads dealing with outgoing connections.
-	 */
-	private static final int DEFAULT_NUMBER_OF_OUTGOING_CONNECTION_THREADS = 1;
-
-	/**
-	 * The default number of connection retries before giving up.
-	 */
-	private static final int DEFAULT_NUMBER_OF_CONNECTION_RETRIES = 10;
-
-	/**
-	 * List of active threads dealing with outgoing connections.
-	 */
-	private final List<OutgoingConnectionThread> outgoingConnectionThreads = new CopyOnWriteArrayList<OutgoingConnectionThread>();
-
-	/**
-	 * Thread dealing with incoming connections.
-	 */
-	private final IncomingConnectionThread incomingConnectionThread;
-
-	/**
-	 * Map containing currently active outgoing connections.
-	 */
-	private final ConcurrentMap<RemoteReceiver, OutgoingConnection> outgoingConnections = new ConcurrentHashMap<RemoteReceiver, OutgoingConnection>();
-
-	/**
-	 * The number of connection retries before giving up.
-	 */
-	private final int numberOfConnectionRetries;
-
-	/**
-	 * A buffer provider for read buffers
-	 */
-	private final ChannelManager channelManager;
-
-	public NetworkConnectionManager(final ChannelManager channelManager,
-			final InetAddress bindAddress, final int dataPort) throws IOException {
-
-		final Configuration configuration = GlobalConfiguration.getConfiguration();
-
-		this.channelManager = channelManager;
-
-		// Start the connection threads
-		final int numberOfOutgoingConnectionThreads = configuration.getInteger(
-			"channel.network.numberOfOutgoingConnectionThreads", DEFAULT_NUMBER_OF_OUTGOING_CONNECTION_THREADS);
-
-		for (int i = 0; i < numberOfOutgoingConnectionThreads; i++) {
-			final OutgoingConnectionThread outgoingConnectionThread = new OutgoingConnectionThread();
-			outgoingConnectionThread.start();
-			this.outgoingConnectionThreads.add(outgoingConnectionThread);
-		}
-
-		this.incomingConnectionThread = new IncomingConnectionThread(
-			this.channelManager, true, new InetSocketAddress(bindAddress, dataPort));
-		this.incomingConnectionThread.start();
-
-		this.numberOfConnectionRetries = configuration.getInteger("channel.network.numberOfConnectionRetries",
-			DEFAULT_NUMBER_OF_CONNECTION_RETRIES);
-	}
-
-	/**
-	 * Randomly selects one of the active threads dealing with outgoing connections.
-	 * 
-	 * @return one of the active threads dealing with outgoing connections
-	 */
-	private OutgoingConnectionThread getOutgoingConnectionThread() {
-
-		return this.outgoingConnectionThreads.get((int) (this.outgoingConnectionThreads.size() * Math.random()));
-	}
-
-	/**
-	 * Queues an envelope for transfer to a particular target host.
-	 * 
-	 * @param remoteReceiver
-	 *        the address of the remote receiver
-	 * @param envelope
-	 *        the envelope to be transfered
-	 */
-	public void queueEnvelopeForTransfer(final RemoteReceiver remoteReceiver, final Envelope envelope) {
-
-		getOutgoingConnection(remoteReceiver).queueEnvelope(envelope);
-	}
-
-	/**
-	 * Returns (and possibly creates) the outgoing connection for the given target address.
-	 * 
-	 * @param targetAddress
-	 *        the address of the connection target
-	 * @return the outgoing connection object
-	 */
-	private OutgoingConnection getOutgoingConnection(final RemoteReceiver remoteReceiver) {
-
-		OutgoingConnection outgoingConnection = this.outgoingConnections.get(remoteReceiver);
-
-		if (outgoingConnection == null) {
-
-			outgoingConnection = new OutgoingConnection(remoteReceiver, getOutgoingConnectionThread(),
-				this.numberOfConnectionRetries);
-
-			final OutgoingConnection oldEntry = this.outgoingConnections
-				.putIfAbsent(remoteReceiver, outgoingConnection);
-
-			// We had a race, use the old value
-			if (oldEntry != null) {
-				outgoingConnection = oldEntry;
-			}
-		}
-
-		return outgoingConnection;
-	}
-
-	public void shutDown() {
-
-		// Interrupt the threads we started
-		this.incomingConnectionThread.interrupt();
-
-		final Iterator<OutgoingConnectionThread> it = this.outgoingConnectionThreads.iterator();
-		while (it.hasNext()) {
-			it.next().interrupt();
-		}
-	}
-
-	public void logBufferUtilization() {
-
-		System.out.println("\tOutgoing connections:");
-
-		final Iterator<Map.Entry<RemoteReceiver, OutgoingConnection>> it = this.outgoingConnections.entrySet()
-			.iterator();
-
-		while (it.hasNext()) {
-
-			final Map.Entry<RemoteReceiver, OutgoingConnection> entry = it.next();
-			System.out.println("\t\tOC " + entry.getKey() + ": " + entry.getValue().getNumberOfQueuedWriteBuffers());
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/SenderHintEvent.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/SenderHintEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/SenderHintEvent.java
index 32be058..65287f7 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/SenderHintEvent.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/SenderHintEvent.java
@@ -21,7 +21,6 @@ import java.util.List;
 
 import eu.stratosphere.nephele.event.task.AbstractEvent;
 import eu.stratosphere.runtime.io.channels.ChannelID;
-import eu.stratosphere.runtime.io.network.envelope.Envelope;
 
 public final class SenderHintEvent extends AbstractEvent {
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferAvailabilityListener.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferAvailabilityListener.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferAvailabilityListener.java
index 1d23e93..af08aa8 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferAvailabilityListener.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferAvailabilityListener.java
@@ -1,5 +1,5 @@
 /***********************************************************************************************************************
- * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ * Copyright (C) 2010-2014 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
@@ -13,16 +13,19 @@
 
 package eu.stratosphere.runtime.io.network.bufferprovider;
 
+import eu.stratosphere.runtime.io.Buffer;
+
 /**
- * This interface must be implemented to receive a notification from a {@link BufferProvider} when an empty
- * {@link eu.stratosphere.runtime.io.Buffer} has
- * become available again.
- * 
+ * This interface must be implemented to receive an asynchronous callback from
+ * a {@link BufferProvider} as soon as a buffer has become available again.
  */
 public interface BufferAvailabilityListener {
 
 	/**
-	 * Indicates that at least one {@link eu.stratosphere.runtime.io.Buffer} has become available again.
+	 * Returns a Buffer to the listener.
+	 * <p/>
+	 * Note: the listener has to adjust the size of the returned Buffer to the
+	 * requested size manually via {@link Buffer#limitSize(int)}.
 	 */
-	void bufferAvailable();
+	void bufferAvailable(Buffer buffer) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProvider.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProvider.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProvider.java
index e3085ee..d82b427 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProvider.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProvider.java
@@ -65,5 +65,21 @@ public interface BufferProvider {
 	 * @return <code>true</code> if the registration has been successful; <code>false</code> if the registration
 	 *         failed, because the buffer pool was not empty or has already been destroyed
 	 */
-	boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener);
+	BufferAvailabilityRegistration registerBufferAvailabilityListener(BufferAvailabilityListener listener);
+
+	public enum BufferAvailabilityRegistration {
+		NOT_REGISTERED_BUFFER_AVAILABLE(false),
+		NOT_REGISTERED_BUFFER_POOL_DESTROYED(false),
+		REGISTERED(true);
+
+		private final boolean isSuccessful;
+
+		private BufferAvailabilityRegistration(boolean isSuccessful) {
+			this.isSuccessful = isSuccessful;
+		}
+
+		public boolean isSuccessful() {
+			return isSuccessful;
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/DiscardBufferPool.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/DiscardBufferPool.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/DiscardBufferPool.java
new file mode 100644
index 0000000..5daa509
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/DiscardBufferPool.java
@@ -0,0 +1,51 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.network.bufferprovider;
+
+import eu.stratosphere.core.memory.MemorySegment;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.BufferRecycler;
+
+public final class DiscardBufferPool implements BufferProvider, BufferRecycler {
+	
+	@Override
+	public Buffer requestBuffer(int minBufferSize) {
+		return null;
+	}
+
+	@Override
+	public Buffer requestBufferBlocking(int minBufferSize) {
+		return null;
+	}
+
+	@Override
+	public int getBufferSize() {
+		return 0;
+	}
+
+	@Override
+	public void reportAsynchronousEvent() {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public BufferAvailabilityRegistration registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
+		return BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_POOL_DESTROYED;
+	}
+
+	@Override
+	public void recycle(MemorySegment buffer) {
+		throw new UnsupportedOperationException();
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java
index e8aeb11..3eb10c1 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java
@@ -184,20 +184,20 @@ public final class LocalBufferPool implements BufferProvider {
 	}
 
 	@Override
-	public boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
+	public BufferAvailabilityRegistration registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
 		synchronized (this.buffers) {
 			if (!this.buffers.isEmpty()) {
-				return false;
+				return BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_AVAILABLE;
 			}
 
 			if (this.isDestroyed) {
-				return false;
+				return BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_POOL_DESTROYED;
 			}
 
 			this.listeners.add(listener);
 		}
 
-		return true;
+		return BufferAvailabilityRegistration.REGISTERED;
 	}
 
 	/**
@@ -221,7 +221,7 @@ public final class LocalBufferPool implements BufferProvider {
 				}
 
 				this.globalBufferPool.returnBuffer(this.buffers.poll());
-				this.numRequestedBuffers --;
+				this.numRequestedBuffers--;
 			}
 
 			this.buffers.notify();
@@ -294,11 +294,17 @@ public final class LocalBufferPool implements BufferProvider {
 				this.globalBufferPool.returnBuffer(buffer);
 				this.numRequestedBuffers--;
 			} else {
-				this.buffers.add(buffer);
-				this.buffers.notify();
-
-				while (!this.listeners.isEmpty()) {
-					this.listeners.poll().bufferAvailable();
+				if (!this.listeners.isEmpty()) {
+					Buffer availableBuffer = new Buffer(buffer, buffer.size(), this.recycler);
+					try {
+						this.listeners.poll().bufferAvailable(availableBuffer);
+					} catch (Exception e) {
+						this.buffers.add(buffer);
+						this.buffers.notify();
+					}
+				} else {
+					this.buffers.add(buffer);
+					this.buffers.notify();
 				}
 			}
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/SerialSingleBufferPool.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/SerialSingleBufferPool.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/SerialSingleBufferPool.java
deleted file mode 100644
index 217a5ce..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/SerialSingleBufferPool.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/***********************************************************************************************************************
- * Copyright (C) 2010-2014 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.network.bufferprovider;
-
-import eu.stratosphere.core.memory.MemorySegment;
-import eu.stratosphere.runtime.io.Buffer;
-import eu.stratosphere.runtime.io.BufferRecycler;
-
-/**
- * 
- */
-public final class SerialSingleBufferPool implements BufferProvider, BufferRecycler {
-	
-	private final Buffer buffer;
-
-	/** Size of the buffer in this pool */
-	private final int bufferSize;
-
-
-	// -----------------------------------------------------------------------------------------------------------------
-
-	public SerialSingleBufferPool(int bufferSize) {
-		this.buffer = new Buffer(new MemorySegment(new byte[bufferSize]), bufferSize, this);
-		this.bufferSize = bufferSize;
-	}
-	
-	// -----------------------------------------------------------------------------------------------------------------
-
-	@Override
-	public Buffer requestBuffer(int minBufferSize) {
-		if (minBufferSize <= this.bufferSize) {
-			return this.buffer.duplicate();
-		}
-		else {
-			throw new IllegalArgumentException("Requesting buffer with size " + minBufferSize + ". Pool's buffer size is " + this.bufferSize);
-		}
-	}
-
-	@Override
-	public Buffer requestBufferBlocking(int minBufferSize) {
-		if (minBufferSize <= this.bufferSize) {
-			return this.buffer.duplicate();
-		}
-		else {
-			throw new IllegalArgumentException("Requesting buffer with size " + minBufferSize + ". Pool's buffer size is " + this.bufferSize);
-		}
-	}
-
-	@Override
-	public int getBufferSize() {
-		return this.bufferSize;
-	}
-
-	@Override
-	public void reportAsynchronousEvent() {
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	public boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	public void recycle(MemorySegment buffer) {}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/Envelope.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/Envelope.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/Envelope.java
deleted file mode 100644
index a692aec..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/Envelope.java
+++ /dev/null
@@ -1,169 +0,0 @@
-/***********************************************************************************************************************
- * Copyright (C) 2010-2014 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.network.envelope;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.runtime.io.Buffer;
-import eu.stratosphere.runtime.io.channels.ChannelID;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.runtime.io.serialization.DataInputDeserializer;
-import eu.stratosphere.runtime.io.serialization.DataOutputSerializer;
-import eu.stratosphere.util.InstantiationUtil;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-public final class Envelope {
-
-	private final JobID jobID;
-
-	private final ChannelID source;
-
-	private final int sequenceNumber;
-
-	private ByteBuffer serializedEventList;
-
-	private Buffer buffer;
-
-	public Envelope(int sequenceNumber, JobID jobID, ChannelID source) {
-		this.sequenceNumber = sequenceNumber;
-		this.jobID = jobID;
-		this.source = source;
-	}
-
-	private Envelope(Envelope toDuplicate) {
-		this.jobID = toDuplicate.jobID;
-		this.source = toDuplicate.source;
-		this.sequenceNumber = toDuplicate.sequenceNumber;
-		this.serializedEventList = null;
-		this.buffer = null;
-	}
-
-	public Envelope duplicate() {
-		Envelope duplicate = new Envelope(this);
-		if (hasBuffer()) {
-			duplicate.setBuffer(this.buffer.duplicate());
-		}
-
-		return duplicate;
-	}
-
-	public Envelope duplicateWithoutBuffer() {
-		return new Envelope(this);
-	}
-
-	public JobID getJobID() {
-		return this.jobID;
-	}
-
-	public ChannelID getSource() {
-		return this.source;
-	}
-
-	public int getSequenceNumber() {
-		return this.sequenceNumber;
-	}
-
-	public void setEventsSerialized(ByteBuffer serializedEventList) {
-		if (this.serializedEventList != null)
-			throw new IllegalStateException("Event list has already been set.");
-
-		this.serializedEventList = serializedEventList;
-	}
-
-	public void serializeEventList(List<? extends AbstractEvent> eventList) {
-		if (this.serializedEventList != null)
-			throw new IllegalStateException("Event list has already been set.");
-
-		this.serializedEventList = serializeEvents(eventList);
-	}
-
-	public ByteBuffer getEventsSerialized() {
-		return this.serializedEventList;
-	}
-
-	public List<? extends AbstractEvent> deserializeEvents() {
-		return deserializeEvents(getClass().getClassLoader());
-	}
-
-	public List<? extends AbstractEvent> deserializeEvents(ClassLoader classloader) {
-		if (this.serializedEventList == null) {
-			return Collections.emptyList();
-		}
-
-		try {
-			DataInputDeserializer deserializer = new DataInputDeserializer(this.serializedEventList);
-
-			int numEvents = deserializer.readInt();
-			ArrayList<AbstractEvent> events = new ArrayList<AbstractEvent>(numEvents);
-
-			for (int i = 0; i < numEvents; i++) {
-				String className = deserializer.readUTF();
-				Class<? extends AbstractEvent> clazz;
-				try {
-					clazz = Class.forName(className).asSubclass(AbstractEvent.class);
-				} catch (ClassNotFoundException e) {
-					throw new RuntimeException("Could not load event class '" + className + "'.", e);
-				} catch (ClassCastException e) {
-					throw new RuntimeException("The class '" + className + "' is no valid subclass of '" + AbstractEvent.class.getName() + "'.", e);
-				}
-
-				AbstractEvent evt = InstantiationUtil.instantiate(clazz, AbstractEvent.class);
-				evt.read(deserializer);
-
-				events.add(evt);
-			}
-
-			return events;
-		}
-		catch (IOException e) {
-			throw new RuntimeException("Error while deserializing the events.", e);
-		}
-	}
-
-	public void setBuffer(Buffer buffer) {
-		this.buffer = buffer;
-	}
-
-	public Buffer getBuffer() {
-		return this.buffer;
-	}
-
-	private ByteBuffer serializeEvents(List<? extends AbstractEvent> events) {
-		try {
-			// create the serialized event list
-			DataOutputSerializer serializer = events.size() == 0
-				? new DataOutputSerializer(4)
-				: new DataOutputSerializer(events.size() * 32);
-			serializer.writeInt(events.size());
-
-			for (AbstractEvent evt : events) {
-				serializer.writeUTF(evt.getClass().getName());
-				evt.write(serializer);
-			}
-
-			return serializer.wrapAsByteBuffer();
-		}
-		catch (IOException e) {
-			throw new RuntimeException("Error while serializing the task events.", e);
-		}
-	}
-
-	public boolean hasBuffer() {
-		return this.buffer != null;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeDispatcher.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeDispatcher.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeDispatcher.java
deleted file mode 100644
index 2b69c0d..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeDispatcher.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/***********************************************************************************************************************
- * Copyright (C) 2010-2014 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.network.envelope;
-
-import java.io.IOException;
-
-/**
- * A envelope dispatcher receives {@link Envelope}s and sends them to all of its destinations.
- */
-public interface EnvelopeDispatcher {
-
-	/**
-	 * Dispatches an envelope from an output channel to the receiving input channels (forward flow).
-	 *
-	 * @param envelope envelope to be sent
-	 */
-	void dispatchFromOutputChannel(Envelope envelope) throws IOException, InterruptedException;
-
-	/**
-	 * Dispatches an envelope from an input channel to the receiving output channels (backwards flow).
-	 *
-	 * @param envelope envelope to be sent
-	 */
-	void dispatchFromInputChannel(Envelope envelope) throws IOException, InterruptedException;
-
-	/**
-	 * Dispatches an envelope from an incoming TCP connection.
-	 * <p>
-	 * After an envelope has been constructed from a TCP socket, this method is called to send the envelope to the
-	 * receiving input channel.
-	 *
-	 * @param envelope envelope to be sent
-	 */
-	void dispatchFromNetwork(Envelope envelope) throws IOException, InterruptedException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReader.java
deleted file mode 100644
index 7b7e178..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReader.java
+++ /dev/null
@@ -1,212 +0,0 @@
-/***********************************************************************************************************************
- * Copyright (C) 2010-2014 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.network.envelope;
-
-import eu.stratosphere.runtime.io.Buffer;
-import eu.stratosphere.runtime.io.channels.ChannelID;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
-import eu.stratosphere.runtime.io.network.bufferprovider.BufferProviderBroker;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-import java.nio.channels.ReadableByteChannel;
-
-public class EnvelopeReader {
-
-	public enum DeserializationState {
-		COMPLETE,
-		PENDING,
-		NO_BUFFER_AVAILABLE;
-	}
-
-	private final BufferProviderBroker bufferProviderBroker;
-
-	private final ByteBuffer headerBuffer;
-
-	private ByteBuffer currentHeaderBuffer;
-
-	private ByteBuffer currentEventsList;
-
-	private ByteBuffer currentDataBuffer;
-
-	private int bufferRequestPendingWithSize;
-
-
-	private Envelope pendingEnvelope;
-
-	private Envelope constructedEnvelope;
-
-
-	public BufferProvider bufferProvider;
-
-	private JobID lastDeserializedJobID;
-
-	private ChannelID lastDeserializedSourceID;
-
-
-	public EnvelopeReader(BufferProviderBroker bufferProviderBroker) {
-		this.bufferProviderBroker = bufferProviderBroker;
-
-		this.headerBuffer = ByteBuffer.allocateDirect(EnvelopeWriter.HEADER_SIZE);
-		this.headerBuffer.order(ByteOrder.LITTLE_ENDIAN);
-
-		this.currentHeaderBuffer = this.headerBuffer;
-	}
-
-	public DeserializationState readNextChunk(ReadableByteChannel channel) throws IOException {
-
-		// 1) check if the header is pending
-		if (this.currentHeaderBuffer != null) {
-			ByteBuffer header = this.currentHeaderBuffer;
-
-			channel.read(header);
-			if (header.hasRemaining()) {
-				// not finished with the header
-				return DeserializationState.PENDING;
-			} else {
-				// header done, construct the envelope
-				this.currentHeaderBuffer = null;
-
-				Envelope env = constructEnvelopeFromHeader(header);
-				this.pendingEnvelope = env;
-
-				// check for events and data
-				int eventsSize = getEventListSize(header);
-				int bufferSize = getBufferSize(header);
-
-				// make the events list the next buffer to be read
-				if (eventsSize > 0) {
-					this.currentEventsList = ByteBuffer.allocate(eventsSize);
-				}
-
-				// if we have a data buffer, we need memory segment for it
-				// we may not immediately get the memory segment, though, so we first record
-				// that we need it
-				if (bufferSize > 0) {
-					this.bufferRequestPendingWithSize = bufferSize;
-				}
-			}
-		}
-
-		// 2) read the eventList, if it should have one
-		if (this.currentEventsList != null) {
-			channel.read(this.currentEventsList);
-			if (this.currentEventsList.hasRemaining()) {
-				// events list still incomplete
-				return DeserializationState.PENDING;
-			} else {
-				this.currentEventsList.flip();
-				this.pendingEnvelope.setEventsSerialized(this.currentEventsList);
-				this.currentEventsList = null;
-			}
-		}
-
-		// 3) check if we need to get a buffer
-		if (this.bufferRequestPendingWithSize > 0) {
-			Buffer b = getBufferForTarget(this.pendingEnvelope.getJobID(), this.pendingEnvelope.getSource(), this.bufferRequestPendingWithSize);
-			if (b == null) {
-				// no buffer available at this time. come back later
-				return DeserializationState.NO_BUFFER_AVAILABLE;
-			} else {
-				// buffer is available. set the field so the buffer will be filled
-				this.pendingEnvelope.setBuffer(b);
-				this.currentDataBuffer = b.getMemorySegment().wrap(0, this.bufferRequestPendingWithSize);
-				this.bufferRequestPendingWithSize = 0;
-			}
-		}
-
-		// 4) fill the buffer
-		if (this.currentDataBuffer != null) {
-			channel.read(this.currentDataBuffer);
-			if (this.currentDataBuffer.hasRemaining()) {
-				// data buffer incomplete
-				return DeserializationState.PENDING;
-			} else {
-				this.currentDataBuffer = null;
-			}
-		}
-
-		// if we get here, we completed our job, or did nothing, if the deserializer was not
-		// reset after the previous envelope
-		if (this.pendingEnvelope != null) {
-			this.constructedEnvelope = this.pendingEnvelope;
-			this.pendingEnvelope = null;
-			return DeserializationState.COMPLETE;
-		} else {
-			throw new IllegalStateException("Error: read() was called before reserializer was reset after the last envelope.");
-		}
-	}
-
-	private Envelope constructEnvelopeFromHeader(ByteBuffer header) throws IOException {
-		int magicNumber = header.getInt(EnvelopeWriter.MAGIC_NUMBER_OFFSET);
-
-		if (magicNumber != EnvelopeWriter.MAGIC_NUMBER) {
-			throw new IOException("Network stream corrupted: invalid magic number in envelope header.");
-		}
-
-		int seqNum = header.getInt(EnvelopeWriter.SEQUENCE_NUMBER_OFFSET);
-		JobID jid = JobID.fromByteBuffer(header, EnvelopeWriter.JOB_ID_OFFSET);
-		ChannelID cid = ChannelID.fromByteBuffer(header, EnvelopeWriter.CHANNEL_ID_OFFSET);
-		return new Envelope(seqNum, jid, cid);
-	}
-
-	private int getBufferSize(ByteBuffer header) {
-		return header.getInt(EnvelopeWriter.BUFFER_SIZE_OFFSET);
-	}
-
-	private int getEventListSize(ByteBuffer header) {
-		return header.getInt(EnvelopeWriter.EVENTS_SIZE_OFFSET);
-	}
-
-	private Buffer getBufferForTarget(JobID jid, ChannelID cid, int size) throws IOException {
-		if (!(jid.equals(this.lastDeserializedJobID) && cid.equals(this.lastDeserializedSourceID))) {
-			this.bufferProvider = this.bufferProviderBroker.getBufferProvider(jid, cid);
-			this.lastDeserializedJobID = jid;
-			this.lastDeserializedSourceID = cid;
-		}
-
-		return this.bufferProvider.requestBuffer(size);
-	}
-
-
-	public Envelope getFullyDeserializedTransferEnvelope() {
-		Envelope t = this.constructedEnvelope;
-		if (t == null) {
-			throw new IllegalStateException("Envelope has not yet been fully constructed.");
-		}
-
-		this.constructedEnvelope = null;
-		return t;
-	}
-
-	public void reset() {
-		this.headerBuffer.clear();
-		this.currentHeaderBuffer = this.headerBuffer;
-		this.constructedEnvelope = null;
-	}
-
-	public boolean hasUnfinishedData() {
-		return this.pendingEnvelope != null || this.currentHeaderBuffer != null;
-	}
-
-	public BufferProvider getBufferProvider() {
-		return bufferProvider;
-	}
-
-	public Envelope getPendingEnvelope() {
-		return pendingEnvelope;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReceiverList.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReceiverList.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReceiverList.java
deleted file mode 100644
index f99e1f2..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReceiverList.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.runtime.io.network.envelope;
-
-import java.net.InetAddress;
-
-import eu.stratosphere.runtime.io.channels.ChannelID;
-import eu.stratosphere.runtime.io.network.ConnectionInfoLookupResponse;
-import eu.stratosphere.runtime.io.network.RemoteReceiver;
-
-/**
- * A transfer envelope receiver list contains all recipients of a transfer envelope. Their are three d ifferent types of
- * receivers: Local receivers identified by {@link ChannelID} objects, remote receivers identified by
- * {@link InetAddress} objects and finally checkpoints which are identified by
- * <p>
- * This class is thread-safe.
- * 
- */
-public class EnvelopeReceiverList {
-
-	private final ChannelID localReceiver;
-
-	private final RemoteReceiver remoteReceiver;
-
-	public EnvelopeReceiverList(ConnectionInfoLookupResponse cilr) {
-		this.localReceiver = cilr.getLocalTarget();
-		this.remoteReceiver = cilr.getRemoteTarget();
-	}
-
-	public EnvelopeReceiverList(ChannelID localReceiver) {
-		this.localReceiver = localReceiver;
-		this.remoteReceiver = null;
-	}
-
-	public EnvelopeReceiverList(RemoteReceiver remoteReceiver) {
-		this.localReceiver = null;
-		this.remoteReceiver = remoteReceiver;
-	}
-
-	public boolean hasLocalReceiver() {
-		return this.localReceiver != null;
-	}
-
-	public boolean hasRemoteReceiver() {
-		return this.remoteReceiver != null;
-	}
-
-	public int getTotalNumberOfReceivers() {
-		return (this.localReceiver == null ? 0 : 1) + (this.remoteReceiver == null ? 0 : 1);
-	}
-
-	public RemoteReceiver getRemoteReceiver() {
-		return this.remoteReceiver;
-	}
-
-	public ChannelID getLocalReceiver() {
-		return this.localReceiver;
-	}
-	
-	@Override
-	public String toString() {
-		return "local receiver: " + this.localReceiver + ", remote receiver: " + this.remoteReceiver;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeWriter.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeWriter.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeWriter.java
deleted file mode 100644
index c00e61b..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeWriter.java
+++ /dev/null
@@ -1,134 +0,0 @@
-/***********************************************************************************************************************
- * Copyright (C) 2010-2014 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.network.envelope;
-
-import eu.stratosphere.nephele.AbstractID;
-import eu.stratosphere.runtime.io.Buffer;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-import java.nio.channels.WritableByteChannel;
-
-public class EnvelopeWriter {
-
-	protected static final int MAGIC_NUMBER = 0xBADC0FFE;
-
-	/**
-	 * Size of the envelope header: 48 bytes = 4 bytes magic number, 4 bytes sequence number, 16 bytes job id,
-	 * 16 bytes sender id, 4 bytes bufferSize, 4 bytes event list length
-	 */
-	public static final int HEADER_SIZE = 4 + 4 + 2 * AbstractID.SIZE + 4 + 4;
-
-	public static final int MAGIC_NUMBER_OFFSET = 0;
-
-	public static final int SEQUENCE_NUMBER_OFFSET = 4;
-
-	public static final int JOB_ID_OFFSET = 8;
-
-	public static final int CHANNEL_ID_OFFSET = 24;
-
-	public static final int BUFFER_SIZE_OFFSET = 40;
-
-	public static final int EVENTS_SIZE_OFFSET = 44;
-
-	private ByteBuffer currentHeader;
-
-	private ByteBuffer currentEvents;
-
-	private ByteBuffer currentDataBuffer;
-
-	private final ByteBuffer headerBuffer;
-
-	public EnvelopeWriter() {
-		this.headerBuffer = ByteBuffer.allocateDirect(HEADER_SIZE);
-		this.headerBuffer.order(ByteOrder.LITTLE_ENDIAN);
-	}
-
-	/**
-	 * @param channel
-	 * @return True, if the writer has more pending data for the current envelope, false if not.
-	 *
-	 * @throws java.io.IOException
-	 */
-	public boolean writeNextChunk(WritableByteChannel channel) throws IOException {
-		// 1) check if the the header is still pending
-		if (this.currentHeader != null) {
-			channel.write(this.currentHeader);
-
-			if (this.currentHeader.hasRemaining()) {
-				// header was not fully written, so we can leave this method
-				return true;
-			} else {
-				this.currentHeader = null;
-			}
-		}
-
-		// 2) check if there are events pending
-		if (this.currentEvents != null) {
-			channel.write(this.currentEvents);
-			if (this.currentEvents.hasRemaining()) {
-				// events were not fully written, so leave this method
-				return true;
-			} else {
-				this.currentEvents = null;
-			}
-		}
-
-		// 3) write the data buffer
-		if (this.currentDataBuffer != null) {
-			channel.write(this.currentDataBuffer);
-			if (this.currentDataBuffer.hasRemaining()) {
-				return true;
-			} else {
-				this.currentDataBuffer = null;
-			}
-		}
-
-		return false;
-	}
-
-	public void setEnvelopeForWriting(Envelope env) {
-		// header
-		constructHeader(env);
-		this.currentHeader = this.headerBuffer;
-
-		// events (possibly null)
-		this.currentEvents = env.getEventsSerialized();
-
-		// data buffer (possibly null)
-		Buffer buf = env.getBuffer();
-		if (buf != null && buf.size() > 0) {
-			this.currentDataBuffer = buf.getMemorySegment().wrap(0, buf.size());
-		}
-	}
-
-	private void constructHeader(Envelope env) {
-		final ByteBuffer buf = this.headerBuffer;
-
-		buf.clear();							// reset
-		buf.putInt(MAGIC_NUMBER);
-		buf.putInt(env.getSequenceNumber());	// sequence number (4 bytes)
-		env.getJobID().write(buf);				// job Id (16 bytes)
-		env.getSource().write(buf);				// producerId (16 bytes)
-
-		// buffer size
-		buf.putInt(env.getBuffer() == null ? 0 : env.getBuffer().size());
-
-		// size of event list
-		buf.putInt(env.getEventsSerialized() == null ? 0 : env.getEventsSerialized().remaining());
-
-		buf.flip();
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/NoBufferAvailableException.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/NoBufferAvailableException.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/NoBufferAvailableException.java
deleted file mode 100644
index f7d49bf..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/NoBufferAvailableException.java
+++ /dev/null
@@ -1,53 +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.network.envelope;
-
-import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
-
-/**
- * This exception is thrown to indicate that the deserialization process of a {@link Envelope} could not be
- * continued because a {@link Buffer} to store the envelope's content is currently not available.
- * 
- */
-public final class NoBufferAvailableException extends Exception {
-
-	/**
-	 * Generated serial UID.
-	 */
-	private static final long serialVersionUID = -9164212953646457026L;
-
-	/**
-	 * The buffer provider which could not deliver a buffer.
-	 */
-	private final BufferProvider bufferProvider;
-
-	/**
-	 * Constructs a new exception.
-	 * 
-	 * @param bufferProvider
-	 *        the buffer provider which could not deliver a buffer
-	 */
-	public NoBufferAvailableException(final BufferProvider bufferProvider) {
-		this.bufferProvider = bufferProvider;
-	}
-
-	/**
-	 * Returns the buffer provider which could not deliver a buffer.
-	 * 
-	 * @return the buffer provider which could not deliver a buffer
-	 */
-	public BufferProvider getBufferProvider() {
-		return this.bufferProvider;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoder.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoder.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoder.java
new file mode 100644
index 0000000..1ab1871
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoder.java
@@ -0,0 +1,344 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.network.netty;
+
+import eu.stratosphere.nephele.jobgraph.JobID;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferAvailabilityListener;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferProviderBroker;
+import eu.stratosphere.runtime.io.network.Envelope;
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInboundHandlerAdapter;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+public class InboundEnvelopeDecoder extends ChannelInboundHandlerAdapter implements BufferAvailabilityListener {
+
+	private final BufferProviderBroker bufferProviderBroker;
+
+	private final BufferAvailabilityChangedTask bufferAvailabilityChangedTask = new BufferAvailabilityChangedTask();
+
+	private final ConcurrentLinkedQueue<Buffer> bufferBroker = new ConcurrentLinkedQueue<Buffer>();
+
+	private final ByteBuffer headerBuffer;
+
+	private Envelope currentEnvelope;
+
+	private ByteBuffer currentEventsBuffer;
+
+	private ByteBuffer currentDataBuffer;
+
+	private int currentBufferRequestSize;
+
+	private BufferProvider currentBufferProvider;
+
+	private JobID lastJobId;
+
+	private ChannelID lastSourceId;
+
+	private ByteBuf stagedBuffer;
+
+	private ChannelHandlerContext channelHandlerContext;
+
+	private int bytesToSkip;
+
+	private enum DecoderState {
+		COMPLETE,
+		PENDING,
+		NO_BUFFER_AVAILABLE
+	}
+
+	public InboundEnvelopeDecoder(BufferProviderBroker bufferProviderBroker) {
+		this.bufferProviderBroker = bufferProviderBroker;
+		this.headerBuffer = ByteBuffer.allocateDirect(OutboundEnvelopeEncoder.HEADER_SIZE);
+	}
+
+	@Override
+	public void channelActive(ChannelHandlerContext ctx) throws Exception {
+		if (this.channelHandlerContext == null) {
+			this.channelHandlerContext = ctx;
+		}
+
+		super.channelActive(ctx);
+	}
+
+	@Override
+	public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
+		if (this.stagedBuffer != null) {
+			throw new IllegalStateException("No channel read event should be fired " +
+					"as long as the a buffer is staged.");
+		}
+
+		ByteBuf in = (ByteBuf) msg;
+
+		if (this.bytesToSkip > 0) {
+			this.bytesToSkip = skipBytes(in, this.bytesToSkip);
+
+			// we skipped over the whole buffer
+			if (this.bytesToSkip > 0) {
+				in.release();
+				return;
+			}
+		}
+
+		decodeBuffer(in, ctx);
+	}
+
+	/**
+	 * Decodes all Envelopes contained in a Netty ByteBuf and forwards them in the pipeline.
+	 * Returns true and releases the buffer, if it was fully consumed. Otherwise, returns false and retains the buffer.
+	 * </p>
+	 * In case of no buffer availability (returns false), a buffer availability listener is registered and the input
+	 * buffer is staged for later consumption.
+	 *
+	 * @return <code>true</code>, if buffer fully consumed, <code>false</code> otherwise
+	 * @throws IOException
+	 */
+	private boolean decodeBuffer(ByteBuf in, ChannelHandlerContext ctx) throws IOException {
+
+		DecoderState decoderState;
+		while ((decoderState = decodeEnvelope(in)) != DecoderState.PENDING) {
+			if (decoderState == DecoderState.COMPLETE) {
+				ctx.fireChannelRead(this.currentEnvelope);
+				this.currentEnvelope = null;
+			}
+			else if (decoderState == DecoderState.NO_BUFFER_AVAILABLE) {
+				switch (this.currentBufferProvider.registerBufferAvailabilityListener(this)) {
+					case REGISTERED:
+						if (ctx.channel().config().isAutoRead()) {
+							ctx.channel().config().setAutoRead(false);
+						}
+
+						this.stagedBuffer = in;
+						this.stagedBuffer.retain();
+						return false;
+
+					case NOT_REGISTERED_BUFFER_AVAILABLE:
+						continue;
+
+					case NOT_REGISTERED_BUFFER_POOL_DESTROYED:
+						this.bytesToSkip = skipBytes(in, this.currentBufferRequestSize);
+
+						this.currentBufferRequestSize = 0;
+						this.currentEventsBuffer = null;
+						this.currentEnvelope = null;
+				}
+			}
+		}
+
+		if (in.isReadable()) {
+			throw new IllegalStateException("Every buffer should have been fully" +
+					"consumed after *successfully* decoding it (if it was not successful, " +
+					"the buffer will be staged for later consumption).");
+		}
+
+		in.release();
+		return true;
+	}
+
+	/**
+	 * Notifies the IO thread that a Buffer has become available again.
+	 * <p/>
+	 * This method will be called from outside the Netty IO thread. The caller will be the buffer pool from which the
+	 * available buffer comes (i.e. the InputGate).
+	 * <p/>
+	 * We have to make sure that the available buffer is handed over to the IO thread in a safe manner.
+	 */
+	@Override
+	public void bufferAvailable(Buffer buffer) throws Exception {
+		this.bufferBroker.offer(buffer);
+		this.channelHandlerContext.channel().eventLoop().execute(this.bufferAvailabilityChangedTask);
+	}
+
+	/**
+	 * Continues the decoding of a staged buffer after a buffer has become available again.
+	 * <p/>
+	 * This task should be executed by the IO thread to ensure safe access to the staged buffer.
+	 */
+	private class BufferAvailabilityChangedTask implements Runnable {
+		@Override
+		public void run() {
+			Buffer availableBuffer = bufferBroker.poll();
+			if (availableBuffer == null) {
+				throw new IllegalStateException("The BufferAvailabilityChangedTask" +
+						"should only be executed when a Buffer has been offered" +
+						"to the Buffer broker (after becoming available).");
+			}
+
+			// This alters the state of the last `decodeEnvelope(ByteBuf)`
+			// call to set the buffer, which has become available again
+			availableBuffer.limitSize(currentBufferRequestSize);
+			currentEnvelope.setBuffer(availableBuffer);
+			currentDataBuffer = availableBuffer.getMemorySegment().wrap(0, InboundEnvelopeDecoder.this.currentBufferRequestSize);
+			currentBufferRequestSize = 0;
+
+			stagedBuffer.release();
+
+			try {
+				if (decodeBuffer(stagedBuffer, channelHandlerContext)) {
+					stagedBuffer = null;
+					channelHandlerContext.channel().config().setAutoRead(true);
+				}
+			} catch (IOException e) {
+				availableBuffer.recycleBuffer();
+			}
+		}
+	}
+
+	// --------------------------------------------------------------------
+
+	private DecoderState decodeEnvelope(ByteBuf in) throws IOException {
+		// --------------------------------------------------------------------
+		// (1) header (EnvelopeEncoder.HEADER_SIZE bytes)
+		// --------------------------------------------------------------------
+		if (this.currentEnvelope == null) {
+			copy(in, this.headerBuffer);
+
+			if (this.headerBuffer.hasRemaining()) {
+				return DecoderState.PENDING;
+			}
+			else {
+				this.headerBuffer.flip();
+
+				int magicNum = this.headerBuffer.getInt();
+				if (magicNum != OutboundEnvelopeEncoder.MAGIC_NUMBER) {
+					throw new IOException("Network stream corrupted: invalid magic" +
+							"number in current envelope header.");
+				}
+
+				int seqNum = this.headerBuffer.getInt();
+				JobID jobId = JobID.fromByteBuffer(this.headerBuffer);
+				ChannelID sourceId = ChannelID.fromByteBuffer(this.headerBuffer);
+
+				this.currentEnvelope = new Envelope(seqNum, jobId, sourceId);
+
+				int eventsSize = this.headerBuffer.getInt();
+				int bufferSize = this.headerBuffer.getInt();
+
+				this.currentEventsBuffer = eventsSize > 0 ? ByteBuffer.allocate(eventsSize) : null;
+				this.currentBufferRequestSize = bufferSize > 0 ? bufferSize : 0;
+
+				this.headerBuffer.clear();
+			}
+		}
+
+		// --------------------------------------------------------------------
+		// (2) events (var length)
+		// --------------------------------------------------------------------
+		if (this.currentEventsBuffer != null) {
+			copy(in, this.currentEventsBuffer);
+
+			if (this.currentEventsBuffer.hasRemaining()) {
+				return DecoderState.PENDING;
+			}
+			else {
+				this.currentEventsBuffer.flip();
+				this.currentEnvelope.setEventsSerialized(this.currentEventsBuffer);
+				this.currentEventsBuffer = null;
+			}
+		}
+
+		// --------------------------------------------------------------------
+		// (3) buffer (var length)
+		// --------------------------------------------------------------------
+		// (a) request a buffer from OUR pool
+		if (this.currentBufferRequestSize > 0) {
+			JobID jobId = this.currentEnvelope.getJobID();
+			ChannelID sourceId = this.currentEnvelope.getSource();
+			Buffer buffer = requestBufferForTarget(jobId, sourceId, this.currentBufferRequestSize);
+
+			if (buffer == null) {
+				return DecoderState.NO_BUFFER_AVAILABLE;
+			}
+			else {
+				this.currentEnvelope.setBuffer(buffer);
+				this.currentDataBuffer = buffer.getMemorySegment().wrap(0, this.currentBufferRequestSize);
+				this.currentBufferRequestSize = 0;
+			}
+		}
+
+		// (b) copy data to OUR buffer
+		if (this.currentDataBuffer != null) {
+			copy(in, this.currentDataBuffer);
+
+			if (this.currentDataBuffer.hasRemaining()) {
+				return DecoderState.PENDING;
+			}
+			else {
+				this.currentDataBuffer = null;
+			}
+		}
+
+		// if we made it to this point, we completed the envelope;
+		// in the other cases we return early with PENDING or NO_BUFFER_AVAILABLE
+		return DecoderState.COMPLETE;
+	}
+
+	private Buffer requestBufferForTarget(JobID jobId, ChannelID sourceId, int size) throws IOException {
+		// Request the buffer from the target buffer provider, which is the
+		// InputGate of the receiving InputChannel.
+		if (!(jobId.equals(this.lastJobId) && sourceId.equals(this.lastSourceId))) {
+			this.lastJobId = jobId;
+			this.lastSourceId = sourceId;
+
+			this.currentBufferProvider = this.bufferProviderBroker.getBufferProvider(jobId, sourceId);
+		}
+
+		return this.currentBufferProvider.requestBuffer(size);
+	}
+
+	/**
+	 * Copies min(from.readableBytes(), to.remaining() bytes from Nettys ByteBuf to the Java NIO ByteBuffer.
+	 */
+	private void copy(ByteBuf src, ByteBuffer dst) {
+		// This branch is necessary, because an Exception is thrown if the
+		// destination buffer has more remaining (writable) bytes than
+		// currently readable from the Netty ByteBuf source.
+		if (src.isReadable()) {
+			if (src.readableBytes() < dst.remaining()) {
+				int oldLimit = dst.limit();
+
+				dst.limit(dst.position() + src.readableBytes());
+				src.readBytes(dst);
+				dst.limit(oldLimit);
+			}
+			else {
+				src.readBytes(dst);
+			}
+		}
+	}
+
+	/**
+	 * Skips over min(in.readableBytes(), toSkip) bytes in the Netty ByteBuf and returns how many bytes remain to be
+	 * skipped.
+	 *
+	 * @return remaining bytes to be skipped
+	 */
+	private int skipBytes(ByteBuf in, int toSkip) {
+		if (toSkip <= in.readableBytes()) {
+			in.readBytes(toSkip);
+			return 0;
+		}
+
+		int remainingToSkip = toSkip - in.readableBytes();
+		in.readerIndex(in.readerIndex() + in.readableBytes());
+
+		return remainingToSkip;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDispatcherHandler.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDispatcherHandler.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDispatcherHandler.java
new file mode 100644
index 0000000..d0270b6
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDispatcherHandler.java
@@ -0,0 +1,41 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.network.netty;
+
+import eu.stratosphere.runtime.io.network.Envelope;
+import eu.stratosphere.runtime.io.network.EnvelopeDispatcher;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInboundHandlerAdapter;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+public class InboundEnvelopeDispatcherHandler extends ChannelInboundHandlerAdapter {
+
+	private static final Log LOG = LogFactory.getLog(InboundEnvelopeDispatcherHandler.class);
+
+	private final EnvelopeDispatcher channelManager;
+
+	public InboundEnvelopeDispatcherHandler(EnvelopeDispatcher channelManager) {
+		this.channelManager = channelManager;
+	}
+
+	@Override
+	public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
+		Envelope envelope = (Envelope) msg;
+//		LOG.debug(String.format("Decoded envelope with seq num %d from source channel %s",
+//				envelope.getSequenceNumber(),
+//				envelope.getSource()));
+		this.channelManager.dispatchFromNetwork(envelope);
+	}
+}


[24/30] Offer buffer-oriented API for I/O (#25)

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/UnionJob.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/UnionJob.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/UnionJob.java
deleted file mode 100644
index d9c09d3..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/UnionJob.java
+++ /dev/null
@@ -1,104 +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.example.union;
-
-import java.io.File;
-import java.io.IOException;
-
-import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.core.fs.Path;
-import eu.stratosphere.nephele.client.JobClient;
-import eu.stratosphere.nephele.io.DistributionPattern;
-import eu.stratosphere.nephele.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.JobTaskVertex;
-import eu.stratosphere.nephele.util.JarFileCreator;
-
-public class UnionJob {
-
-	public static void main(final String[] args) {
-
-		// Create graph and define vertices
-		final JobGraph unionGraph = new JobGraph("Union Job");
-
-		final JobFileInputVertex input1 = new JobFileInputVertex("Input 1", unionGraph);
-		input1.setFileInputClass(ProducerTask.class);
-		input1.setFilePath(new Path("file:///tmp/"));
-
-		final JobFileInputVertex input2 = new JobFileInputVertex("Input 2", unionGraph);
-		input2.setFileInputClass(ProducerTask.class);
-		input2.setFilePath(new Path("file:///tmp/"));
-
-		final JobTaskVertex union = new JobTaskVertex("Union", unionGraph);
-		union.setTaskClass(UnionTask.class);
-
-		final JobFileOutputVertex output = new JobFileOutputVertex("Output", unionGraph);
-		output.setFileOutputClass(ConsumerTask.class);
-		output.setFilePath(new Path("file:///tmp/"));
-
-		// Create edges between vertices
-		try {
-			input1.connectTo(union, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
-			input2.connectTo(union, ChannelType.NETWORK,  DistributionPattern.POINTWISE);
-			union.connectTo(output, ChannelType.INMEMORY);
-		} catch (JobGraphDefinitionException e) {
-			e.printStackTrace();
-			return;
-		}
-
-		// Create jar file and attach it
-		final File jarFile = new File("/tmp/unionJob.jar");
-		final JarFileCreator jarFileCreator = new JarFileCreator(jarFile);
-		jarFileCreator.addClass(ProducerTask.class);
-		jarFileCreator.addClass(UnionTask.class);
-		jarFileCreator.addClass(ConsumerTask.class);
-		
-		try {
-			jarFileCreator.createJarFile();
-		} catch (IOException ioe) {
-
-			ioe.printStackTrace();
-
-			if (jarFile.exists()) {
-				jarFile.delete();
-			}
-
-			return;
-		}
-		
-		//Define instance sharing
-		input1.setVertexToShareInstancesWith(output);
-		input2.setVertexToShareInstancesWith(output);
-		union.setVertexToShareInstancesWith(output);
-
-		unionGraph.addJar(new Path("file://" + jarFile.getAbsolutePath()));
-
-		final Configuration conf = new Configuration();
-		conf.setString("jobmanager.rpc.address", "localhost");
-		
-		try {
-			final JobClient jobClient = new JobClient(unionGraph, conf);
-			jobClient.submitJobAndWait();
-		} catch (Exception e) {
-			e.printStackTrace();
-		}
-
-		if (jarFile.exists()) {
-			jarFile.delete();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/UnionTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/UnionTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/UnionTask.java
deleted file mode 100644
index d99eb42..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/UnionTask.java
+++ /dev/null
@@ -1,45 +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.example.union;
-
-import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.MutableRecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
-import eu.stratosphere.nephele.io.UnionRecordReader;
-import eu.stratosphere.nephele.template.AbstractTask;
-
-public class UnionTask extends AbstractTask {
-
-	private UnionRecordReader<StringRecord> input;
-
-	private RecordWriter<StringRecord> output;
-
-	@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.input = new UnionRecordReader<StringRecord>(recordReaders, StringRecord.class);
-		this.output = new RecordWriter<StringRecord>(this, StringRecord.class);
-	}
-
-	@Override
-	public void invoke() throws Exception {
-		while (this.input.hasNext()) {
-			this.output.emit(this.input.next());
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/CancelTaskException.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/CancelTaskException.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/CancelTaskException.java
new file mode 100644
index 0000000..6a09e89
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/CancelTaskException.java
@@ -0,0 +1,25 @@
+/***********************************************************************************************************************
+ *
+ * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ *
+ **********************************************************************************************************************/
+package eu.stratosphere.nephele.execution;
+
+
+import eu.stratosphere.pact.runtime.task.chaining.ExceptionInChainedStubException;
+
+/**
+ * Thrown to trigger a canceling of the executing task. Intended to cause a cancelled status, rather than a failed status.
+ */
+public class CancelTaskException extends RuntimeException {
+	private static final long serialVersionUID = 1L;
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/Environment.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/Environment.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/Environment.java
index a606dcf..e0bcc70 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/Environment.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/Environment.java
@@ -1,5 +1,5 @@
 /***********************************************************************************************************************
- * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ * Copyright (C) 2010-2014 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
@@ -13,30 +13,25 @@
 
 package eu.stratosphere.nephele.execution;
 
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.FutureTask;
-
 import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.core.fs.Path;
 import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.io.ChannelSelector;
-import eu.stratosphere.nephele.io.GateID;
-import eu.stratosphere.nephele.io.InputGate;
-import eu.stratosphere.nephele.io.OutputGate;
-import eu.stratosphere.nephele.io.RecordDeserializerFactory;
-import eu.stratosphere.nephele.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.gates.GateID;
+import eu.stratosphere.runtime.io.gates.InputGate;
+import eu.stratosphere.runtime.io.gates.OutputGate;
+import eu.stratosphere.nephele.jobgraph.JobGraph;
 import eu.stratosphere.nephele.jobgraph.JobID;
 import eu.stratosphere.nephele.protocols.AccumulatorProtocol;
 import eu.stratosphere.nephele.services.iomanager.IOManager;
 import eu.stratosphere.nephele.services.memorymanager.MemoryManager;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
 import eu.stratosphere.nephele.template.InputSplitProvider;
 
 /**
  * The user code of every Nephele task runs inside an <code>Environment</code> object. The environment provides
  * important services to the task. It keeps track of setting up the communication channels and provides access to input
  * splits, memory manager, etc.
- * 
  */
 public interface Environment {
 	/**
@@ -49,9 +44,9 @@ public interface Environment {
 	JobID getJobID();
 
 	/**
-	 * Returns the task configuration object which was attached to the original {@link JobVertex}.
+	 * Returns the task configuration object which was attached to the original JobVertex.
 	 * 
-	 * @return the task configuration object which was attached to the original {@link JobVertex}
+	 * @return the task configuration object which was attached to the original JobVertex.
 	 */
 	Configuration getTaskConfiguration();
 
@@ -128,13 +123,6 @@ public interface Environment {
 	GateID getNextUnboundInputGateID();
 
 	/**
-	 * Returns the next unbound output gate ID or <code>null</code> if no such ID exists
-	 * 
-	 * @return the next unbound output gate ID or <code>null</code> if no such ID exists
-	 */
-	GateID getNextUnboundOutputGateID();
-
-	/**
 	 * Returns the number of output gates registered with this environment.
 	 * 
 	 * @return the number of output gates registered with this environment
@@ -163,46 +151,16 @@ public interface Environment {
 	int getNumberOfInputChannels();
 
 	/**
-	 * Creates an output gate.
-	 * 
-	 * @param gateID
-	 * @param outputClass
-	 * @param selector
-	 * @param isBroadcast
-	 * @param <T>
-	 *        The type of the record consumed by the output gate.
-	 * @return The created output gate.
+	 * Creates a new OutputGate and registers it with the Environment.
+	 *
+	 * @return the newly created output gate
 	 */
-	<T extends IOReadableWritable> OutputGate<T> createOutputGate(GateID gateID, Class<T> outputClass,
-															ChannelSelector<T> selector, boolean isBroadcast);
+	OutputGate createAndRegisterOutputGate();
 
 	/**
-	 * Creates an input gate.
-	 * 
-	 * @param gateID
-	 * @param deserializer
-	 * @param distributionPattern
-	 * @param <T>
-	 *        The type of the record read from the input gate.
-	 * @return The created input gate.
+	 * Creates a new InputGate and registers it with the Environment.
 	 */
-	<T extends IOReadableWritable> InputGate<T> createInputGate(GateID gateID, RecordDeserializerFactory<T> deserializerFactory);
-
-	/**
-	 * Registers an output gate with this environment.
-	 * 
-	 * @param outputGate
-	 *        the output gate to be registered
-	 */
-	void registerOutputGate(OutputGate<? extends IOReadableWritable> outputGate);
-
-	/**
-	 * Registers an input gate with this environment.
-	 * 
-	 * @param inputGate
-	 *        the input gate to be registered
-	 */
-	void registerInputGate(InputGate<? extends IOReadableWritable> inputGate);
+	<T extends IOReadableWritable> InputGate<T> createAndRegisterInputGate();
 
 	/**
 	 * Returns the IDs of all output channels connected to this environment.
@@ -255,5 +213,13 @@ public interface Environment {
 	 */
 	AccumulatorProtocol getAccumulatorProtocolProxy();
 
-	Map<String, FutureTask<Path>> getCopyTask();
+	/**
+	 * Returns the buffer provider for this environment.
+	 * <p>
+	 * The returned buffer provider is used by the output side of the network stack.
+	 *
+	 * @return Buffer provider for the output side of the network stack
+	 * @see eu.stratosphere.runtime.io.api.RecordWriter
+	 */
+	BufferProvider getOutputBufferProvider();
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java
index 7d7cebb..55f036a 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java
@@ -97,29 +97,9 @@ public final class ExecutionStateTransition {
 			unexpectedStateChange = false;
 		}
 
-		// This transition can appear if a task cannot be deployed at the assigned task manager.
-		else if (oldState == ExecutionState.STARTING && newState == ExecutionState.FAILED) {
-			unexpectedStateChange = false;
-		}
-
 		// -------------- error cases --------------
-		else if (newState == FAILED) {
-			// any state may fail
-			unexpectedStateChange = false;
-		}
-		
-		// This is a regular transition as a result of a cancel operation.
-		else if (oldState == ExecutionState.RUNNING && newState == ExecutionState.CANCELING) {
-			unexpectedStateChange = false;
-		}
-
-		// This is a regular transition as a result of a cancel operation.
-		else if (oldState == ExecutionState.FINISHING && newState == ExecutionState.CANCELING) {
-			unexpectedStateChange = false;
-		}
-
-		// This is a regular transition as a result of a cancel operation.
-		else if (oldState == ExecutionState.CANCELING && newState == ExecutionState.CANCELED) {
+		else if (newState == FAILED || newState == CANCELED || newState == CANCELING) {
+			// any state may fail or cancel itself
 			unexpectedStateChange = false;
 		}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 a8ca11c..59787d2 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,21 +36,25 @@ import eu.stratosphere.nephele.deployment.ChannelDeploymentDescriptor;
 import eu.stratosphere.nephele.deployment.GateDeploymentDescriptor;
 import eu.stratosphere.nephele.deployment.TaskDeploymentDescriptor;
 import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
-import eu.stratosphere.nephele.io.ChannelSelector;
-import eu.stratosphere.nephele.io.GateID;
-import eu.stratosphere.nephele.io.InputGate;
-import eu.stratosphere.nephele.io.OutputGate;
-import eu.stratosphere.nephele.io.RecordDeserializerFactory;
-import eu.stratosphere.nephele.io.RuntimeInputGate;
-import eu.stratosphere.nephele.io.RuntimeOutputGate;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.nephele.jobgraph.JobGraph;
 import eu.stratosphere.nephele.jobgraph.JobID;
 import eu.stratosphere.nephele.protocols.AccumulatorProtocol;
 import eu.stratosphere.nephele.services.iomanager.IOManager;
 import eu.stratosphere.nephele.services.memorymanager.MemoryManager;
 import eu.stratosphere.nephele.template.AbstractInvokable;
 import eu.stratosphere.nephele.template.InputSplitProvider;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.channels.OutputChannel;
+import eu.stratosphere.runtime.io.gates.GateID;
+import eu.stratosphere.runtime.io.gates.InputGate;
+import eu.stratosphere.runtime.io.gates.OutputGate;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferAvailabilityListener;
+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 eu.stratosphere.util.StringUtils;
 
 /**
@@ -60,7 +64,7 @@ import eu.stratosphere.util.StringUtils;
  * <p>
  * This class is thread-safe.
  */
-public class RuntimeEnvironment implements Environment, Runnable {
+public class RuntimeEnvironment implements Environment, BufferProvider, LocalBufferPoolOwner, Runnable {
 
 	/**
 	 * The log object used for debugging.
@@ -75,7 +79,7 @@ public class RuntimeEnvironment implements Environment, Runnable {
 	/**
 	 * List of output gates created by the task.
 	 */
-	private final List<OutputGate<? extends IOReadableWritable>> outputGates = new CopyOnWriteArrayList<OutputGate<? extends IOReadableWritable>>();
+	private final List<OutputGate> outputGates = new CopyOnWriteArrayList<OutputGate>();
 
 	/**
 	 * List of input gates created by the task.
@@ -83,12 +87,6 @@ public class RuntimeEnvironment implements Environment, Runnable {
 	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.
 	 */
@@ -145,7 +143,7 @@ public class RuntimeEnvironment implements Environment, Runnable {
 	private volatile ExecutionObserver executionObserver = null;
 	
 	/**
-	 * The RPC procy to report accumulators to JobManager
+	 * The RPC proxy to report accumulators to JobManager
 	 */
 	private AccumulatorProtocol accumulatorProtocolProxy = null;
 
@@ -164,7 +162,7 @@ public class RuntimeEnvironment implements Environment, Runnable {
 	 */
 	private final String taskName;
 
-	private Map<String,FutureTask<Path>> cacheCopyTasks = new HashMap<String, FutureTask<Path>>();
+	private LocalBufferPool outputBufferPool;
 
 	/**
 	 * Creates a new runtime environment object which contains the runtime information for the encapsulated Nephele
@@ -177,7 +175,7 @@ public class RuntimeEnvironment implements Environment, Runnable {
 	 * @param invokableClass
 	 *        invokableClass the class that should be instantiated as a Nephele task
 	 * @param taskConfiguration
-	 *        the configuration object which was attached to the original {@link JobVertex}
+	 *        the configuration object which was attached to the original JobVertex
 	 * @param jobConfiguration
 	 *        the configuration object which was attached to the original {@link JobGraph}
 	 * @throws Exception
@@ -240,60 +238,16 @@ public class RuntimeEnvironment implements Environment, Runnable {
 		this.invokable.setEnvironment(this);
 		this.invokable.registerInputOutput();
 
-		if (!this.unboundOutputGateIDs.isEmpty() && LOG.isErrorEnabled()) {
-			LOG.error("Inconsistency: " + this.unboundOutputGateIDs.size() + " unbound output gate IDs left");
-		}
+		int numOutputGates = tdd.getNumberOfOutputGateDescriptors();
 
-		if (!this.unboundInputGateIDs.isEmpty() && LOG.isErrorEnabled()) {
-			LOG.error("Inconsistency: " + this.unboundInputGateIDs.size() + " unbound output gate IDs left");
+		for (int i = 0; i < numOutputGates; ++i) {
+			this.outputGates.get(i).initializeChannels(tdd.getOutputGateDescriptor(i));
 		}
 
-		final int noogdd = tdd.getNumberOfOutputGateDescriptors();
-		for (int i = 0; i < noogdd; ++i) {
-			final GateDeploymentDescriptor gdd = tdd.getOutputGateDescriptor(i);
-			final OutputGate og = this.outputGates.get(i);
-			final ChannelType channelType = gdd.getChannelType();
-			og.setChannelType(channelType);
+		int numInputGates = tdd.getNumberOfInputGateDescriptors();
 
-			final int nocdd = gdd.getNumberOfChannelDescriptors();
-			for (int j = 0; j < nocdd; ++j) {
-
-				final ChannelDeploymentDescriptor cdd = gdd.getChannelDescriptor(j);
-				switch (channelType) {
-				case NETWORK:
-					og.createNetworkOutputChannel(og, cdd.getOutputChannelID(), cdd.getInputChannelID());
-					break;
-				case INMEMORY:
-					og.createInMemoryOutputChannel(og, cdd.getOutputChannelID(), cdd.getInputChannelID());
-					break;
-				default:
-					throw new IllegalStateException("Unknown channel type");
-				}
-			}
-		}
-
-		final int noigdd = tdd.getNumberOfInputGateDescriptors();
-		for (int i = 0; i < noigdd; ++i) {
-			final GateDeploymentDescriptor gdd = tdd.getInputGateDescriptor(i);
-			final InputGate ig = this.inputGates.get(i);
-			final ChannelType channelType = gdd.getChannelType();
-			ig.setChannelType(channelType);
-
-			final int nicdd = gdd.getNumberOfChannelDescriptors();
-			for (int j = 0; j < nicdd; ++j) {
-
-				final ChannelDeploymentDescriptor cdd = gdd.getChannelDescriptor(j);
-				switch (channelType) {
-				case NETWORK:
-					ig.createNetworkInputChannel(ig, cdd.getInputChannelID(), cdd.getOutputChannelID());
-					break;
-				case INMEMORY:
-					ig.createInMemoryInputChannel(ig, cdd.getInputChannelID(), cdd.getOutputChannelID());
-					break;
-				default:
-					throw new IllegalStateException("Unknown channel type");
-				}
-			}
+		for(int i = 0; i < numInputGates; i++){
+			this.inputGates.get(i).initializeChannels(tdd.getInputGateDescriptor(i));
 		}
 	}
 
@@ -306,29 +260,26 @@ public class RuntimeEnvironment implements Environment, Runnable {
 		return this.invokable;
 	}
 
-
 	@Override
 	public JobID getJobID() {
 		return this.jobID;
 	}
 
-
 	@Override
 	public GateID getNextUnboundInputGateID() {
-
 		return this.unboundInputGateIDs.poll();
 	}
 
+	@Override
+	public OutputGate createAndRegisterOutputGate() {
+		OutputGate gate = new OutputGate(getJobID(), new GateID(),  getNumberOfOutputGates());
+		this.outputGates.add(gate);
 
-	public GateID getNextUnboundOutputGateID() {
-
-		return this.unboundOutputGateIDs.poll();
+		return gate;
 	}
 
-
 	@Override
 	public void run() {
-
 		if (invokable == null) {
 			LOG.fatal("ExecutionEnvironment has no Invokable set");
 		}
@@ -343,9 +294,6 @@ public class RuntimeEnvironment implements Environment, Runnable {
 		}
 
 		try {
-
-			// Activate input channels
-			// activateInputChannels();
 			ClassLoader cl = LibraryCacheManager.getClassLoader(jobID);
 			Thread.currentThread().setContextClassLoader(cl);
 			this.invokable.invoke();
@@ -354,9 +302,7 @@ public class RuntimeEnvironment implements Environment, Runnable {
 			if (this.executionObserver.isCanceled()) {
 				throw new InterruptedException();
 			}
-
 		} catch (Throwable t) {
-
 			if (!this.executionObserver.isCanceled()) {
 
 				// Perform clean up when the task failed and has been not canceled by the user
@@ -370,7 +316,7 @@ public class RuntimeEnvironment implements Environment, Runnable {
 			// Release all resources that may currently be allocated by the individual channels
 			releaseAllChannelResources();
 
-			if (this.executionObserver.isCanceled()) {
+			if (this.executionObserver.isCanceled() || t instanceof CancelTaskException) {
 				changeExecutionState(ExecutionState.CANCELED, null);
 			} else {
 				changeExecutionState(ExecutionState.FAILED, StringUtils.stringifyException(t));
@@ -399,7 +345,7 @@ public class RuntimeEnvironment implements Environment, Runnable {
 			// Release all resources that may currently be allocated by the individual channels
 			releaseAllChannelResources();
 
-			if (this.executionObserver.isCanceled()) {
+			if (this.executionObserver.isCanceled() || t instanceof CancelTaskException) {
 				changeExecutionState(ExecutionState.CANCELED, null);
 			} else {
 				changeExecutionState(ExecutionState.FAILED, StringUtils.stringifyException(t));
@@ -415,62 +361,35 @@ public class RuntimeEnvironment implements Environment, Runnable {
 		changeExecutionState(ExecutionState.FINISHED, null);
 	}
 
-
 	@Override
-	public <T extends IOReadableWritable> OutputGate<T> createOutputGate(final GateID gateID, Class<T> outputClass,
-			final ChannelSelector<T> selector, final boolean isBroadcast) {
-		final RuntimeOutputGate<T> rog = new RuntimeOutputGate<T>(getJobID(), gateID, outputClass,
-															getNumberOfOutputGates(), selector, isBroadcast);
-		return rog;
-	}
+	public <T extends IOReadableWritable> InputGate<T> createAndRegisterInputGate() {
+		InputGate<T> gate = new InputGate<T>(getJobID(), new GateID(), getNumberOfInputGates());
+		this.inputGates.add(gate);
 
-
-	@Override
-	public <T extends IOReadableWritable> InputGate<T> createInputGate(final GateID gateID,
-										final RecordDeserializerFactory<T> deserializerFactory) {
-		final RuntimeInputGate<T> rig = new RuntimeInputGate<T>(getJobID(), gateID, deserializerFactory,
-			getNumberOfInputGates());
-		return rig;
-	}
-
-	@Override
-	public void registerOutputGate(OutputGate<? extends IOReadableWritable> outputGate) {
-
-		this.outputGates.add(outputGate);
+		return gate;
 	}
 
-	@Override
-	public void registerInputGate(InputGate<? extends IOReadableWritable> inputGate) {
-		this.inputGates.add(inputGate);
-	}
-
-
 	public int getNumberOfOutputGates() {
 		return this.outputGates.size();
 	}
 
-
 	@Override
 	public int getNumberOfInputGates() {
 		return this.inputGates.size();
 	}
 
-
 	@Override
 	public int getNumberOfOutputChannels() {
-
 		int numberOfOutputChannels = 0;
 		for (int i = 0; i < this.outputGates.size(); ++i) {
-			numberOfOutputChannels += this.outputGates.get(i).getNumberOfOutputChannels();
+			numberOfOutputChannels += this.outputGates.get(i).getNumChannels();
 		}
 
 		return numberOfOutputChannels;
 	}
 
-
 	@Override
 	public int getNumberOfInputChannels() {
-
 		int numberOfInputChannels = 0;
 		for (int i = 0; i < this.inputGates.size(); ++i) {
 			numberOfInputChannels += this.inputGates.get(i).getNumberOfInputChannels();
@@ -497,13 +416,13 @@ public class RuntimeEnvironment implements Environment, Runnable {
 	/**
 	 * Returns the registered output gate with index <code>pos</code>.
 	 * 
-	 * @param pos
+	 * @param index
 	 *        the index of the output gate to return
 	 * @return the output gate at index <code>pos</code> or <code>null</code> if no such index exists
 	 */
-	public OutputGate<? extends IOReadableWritable> getOutputGate(final int pos) {
-		if (pos < this.outputGates.size()) {
-			return this.outputGates.get(pos);
+	public OutputGate getOutputGate(int index) {
+		if (index < this.outputGates.size()) {
+			return this.outputGates.get(index);
 		}
 
 		return null;
@@ -515,7 +434,6 @@ public class RuntimeEnvironment implements Environment, Runnable {
 	 * @return the thread which is assigned to execute the user code
 	 */
 	public Thread getExecutingThread() {
-
 		synchronized (this) {
 
 			if (this.executingThread == null) {
@@ -538,29 +456,14 @@ public class RuntimeEnvironment implements Environment, Runnable {
 	 * @throws InterruptedException
 	 *         thrown if the thread waiting for the channels to be closed is interrupted
 	 */
-	private void waitForOutputChannelsToBeClosed() throws IOException, InterruptedException {
-
-		// Wait for disconnection of all output gates
-		while (true) {
-
-			// Make sure, we leave this method with an InterruptedException when the task has been canceled
-			if (this.executionObserver.isCanceled()) {
-				throw new InterruptedException();
-			}
-
-			boolean allClosed = true;
-			for (int i = 0; i < getNumberOfOutputGates(); i++) {
-				final OutputGate<? extends IOReadableWritable> og = this.outputGates.get(i);
-				if (!og.isClosed()) {
-					allClosed = false;
-				}
-			}
+	private void waitForOutputChannelsToBeClosed() throws InterruptedException {
+		// Make sure, we leave this method with an InterruptedException when the task has been canceled
+		if (this.executionObserver.isCanceled()) {
+			return;
+		}
 
-			if (allClosed) {
-				break;
-			} else {
-				Thread.sleep(SLEEPINTERVAL);
-			}
+		for (OutputGate og : this.outputGates) {
+			og.waitForGateToBeClosed();
 		}
 	}
 
@@ -573,7 +476,6 @@ public class RuntimeEnvironment implements Environment, Runnable {
 	 *         thrown if the thread waiting for the channels to be closed is interrupted
 	 */
 	private void waitForInputChannelsToBeClosed() throws IOException, InterruptedException {
-
 		// Wait for disconnection of all output gates
 		while (true) {
 
@@ -602,7 +504,6 @@ public class RuntimeEnvironment implements Environment, Runnable {
 	 * Closes all input gates which are not already closed.
 	 */
 	private void closeInputGates() throws IOException, InterruptedException {
-
 		for (int i = 0; i < this.inputGates.size(); i++) {
 			final InputGate<? extends IOReadableWritable> eig = this.inputGates.get(i);
 			// Important: close must be called on each input gate exactly once
@@ -615,61 +516,49 @@ public class RuntimeEnvironment implements Environment, Runnable {
 	 * Requests all output gates to be closed.
 	 */
 	private void requestAllOutputGatesToClose() throws IOException, InterruptedException {
-
 		for (int i = 0; i < this.outputGates.size(); i++) {
 			this.outputGates.get(i).requestClose();
 		}
 	}
 
-
 	@Override
 	public IOManager getIOManager() {
 		return this.ioManager;
 	}
 
-
 	@Override
 	public MemoryManager getMemoryManager() {
 		return this.memoryManager;
 	}
 
-
 	@Override
 	public Configuration getTaskConfiguration() {
 		return this.taskConfiguration;
 	}
 
-
 	@Override
 	public Configuration getJobConfiguration() {
 		return this.jobConfiguration;
 	}
 
-
 	@Override
 	public int getCurrentNumberOfSubtasks() {
-
 		return this.currentNumberOfSubtasks;
 	}
 
-
 	@Override
 	public int getIndexInSubtaskGroup() {
-
 		return this.indexInSubtaskGroup;
 	}
 
 	private void changeExecutionState(final ExecutionState newExecutionState, final String optionalMessage) {
-
 		if (this.executionObserver != null) {
 			this.executionObserver.executionStateChanged(newExecutionState, optionalMessage);
 		}
 	}
 
-
 	@Override
 	public String getTaskName() {
-
 		return this.taskName;
 	}
 
@@ -679,9 +568,7 @@ public class RuntimeEnvironment implements Environment, Runnable {
 	 * @return the name of the task with its index in the subtask group and the total number of subtasks
 	 */
 	public String getTaskNameWithIndex() {
-
-		return this.taskName + " (" + (getIndexInSubtaskGroup() + 1) + "/"
-			+ getCurrentNumberOfSubtasks() + ")";
+		return String.format("%s (%d/%d)", this.taskName, getIndexInSubtaskGroup() + 1, getCurrentNumberOfSubtasks());
 	}
 
 	/**
@@ -694,25 +581,20 @@ public class RuntimeEnvironment implements Environment, Runnable {
 		this.executionObserver = executionObserver;
 	}
 
-
 	@Override
 	public InputSplitProvider getInputSplitProvider() {
 		return this.inputSplitProvider;
 	}
 
-
 	@Override
 	public void userThreadStarted(final Thread userThread) {
-
 		if (this.executionObserver != null) {
 			this.executionObserver.userThreadStarted(userThread);
 		}
 	}
 
-
 	@Override
 	public void userThreadFinished(final Thread userThread) {
-
 		if (this.executionObserver != null) {
 			this.executionObserver.userThreadFinished(userThread);
 		}
@@ -723,7 +605,6 @@ public class RuntimeEnvironment implements Environment, Runnable {
 	 * method should only be called after the respected task has stopped running.
 	 */
 	private void releaseAllChannelResources() {
-
 		for (int i = 0; i < this.inputGates.size(); i++) {
 			this.inputGates.get(i).releaseAllChannelResources();
 		}
@@ -733,28 +614,21 @@ public class RuntimeEnvironment implements Environment, Runnable {
 		}
 	}
 
-
 	@Override
 	public Set<ChannelID> getOutputChannelIDs() {
+		Set<ChannelID> ids= new HashSet<ChannelID>();
 
-		final Set<ChannelID> outputChannelIDs = new HashSet<ChannelID>();
-
-		final Iterator<OutputGate<? extends IOReadableWritable>> gateIterator = this.outputGates.iterator();
-		while (gateIterator.hasNext()) {
-
-			final OutputGate<? extends IOReadableWritable> outputGate = gateIterator.next();
-			for (int i = 0; i < outputGate.getNumberOfOutputChannels(); ++i) {
-				outputChannelIDs.add(outputGate.getOutputChannel(i).getID());
+		for (OutputGate gate : this.outputGates) {
+			for (OutputChannel channel : gate.channels()) {
+				ids.add(channel.getID());
 			}
 		}
 
-		return Collections.unmodifiableSet(outputChannelIDs);
+		return Collections.unmodifiableSet(ids);
 	}
 
-
 	@Override
 	public Set<ChannelID> getInputChannelIDs() {
-
 		final Set<ChannelID> inputChannelIDs = new HashSet<ChannelID>();
 
 		final Iterator<InputGate<? extends IOReadableWritable>> gateIterator = this.inputGates.iterator();
@@ -769,10 +643,8 @@ public class RuntimeEnvironment implements Environment, Runnable {
 		return Collections.unmodifiableSet(inputChannelIDs);
 	}
 
-
 	@Override
 	public Set<GateID> getInputGateIDs() {
-
 		final Set<GateID> inputGateIDs = new HashSet<GateID>();
 
 		final Iterator<InputGate<? extends IOReadableWritable>> gateIterator = this.inputGates.iterator();
@@ -783,13 +655,11 @@ public class RuntimeEnvironment implements Environment, Runnable {
 		return Collections.unmodifiableSet(inputGateIDs);
 	}
 
-
 	@Override
 	public Set<GateID> getOutputGateIDs() {
-
 		final Set<GateID> outputGateIDs = new HashSet<GateID>();
 
-		final Iterator<OutputGate<? extends IOReadableWritable>> gateIterator = this.outputGates.iterator();
+		final Iterator<OutputGate> gateIterator = this.outputGates.iterator();
 		while (gateIterator.hasNext()) {
 			outputGateIDs.add(gateIterator.next().getGateID());
 		}
@@ -800,11 +670,10 @@ public class RuntimeEnvironment implements Environment, Runnable {
 
 	@Override
 	public Set<ChannelID> getOutputChannelIDsOfGate(final GateID gateID) {
-
-		OutputGate<? extends IOReadableWritable> outputGate = null;
-		final Iterator<OutputGate<? extends IOReadableWritable>> gateIterator = this.outputGates.iterator();
+		OutputGate outputGate = null;
+		final Iterator<OutputGate> gateIterator = this.outputGates.iterator();
 		while (gateIterator.hasNext()) {
-			final OutputGate<? extends IOReadableWritable> candidateGate = gateIterator.next();
+			final OutputGate candidateGate = gateIterator.next();
 			if (candidateGate.getGateID().equals(gateID)) {
 				outputGate = candidateGate;
 				break;
@@ -817,8 +686,8 @@ public class RuntimeEnvironment implements Environment, Runnable {
 
 		final Set<ChannelID> outputChannelIDs = new HashSet<ChannelID>();
 
-		for (int i = 0; i < outputGate.getNumberOfOutputChannels(); ++i) {
-			outputChannelIDs.add(outputGate.getOutputChannel(i).getID());
+		for (int i = 0; i < outputGate.getNumChannels(); ++i) {
+			outputChannelIDs.add(outputGate.getChannel(i).getID());
 		}
 
 		return Collections.unmodifiableSet(outputChannelIDs);
@@ -827,7 +696,6 @@ public class RuntimeEnvironment implements Environment, Runnable {
 
 	@Override
 	public Set<ChannelID> getInputChannelIDsOfGate(final GateID gateID) {
-
 		InputGate<? extends IOReadableWritable> inputGate = null;
 		final Iterator<InputGate<? extends IOReadableWritable>> gateIterator = this.inputGates.iterator();
 		while (gateIterator.hasNext()) {
@@ -850,18 +718,86 @@ public class RuntimeEnvironment implements Environment, Runnable {
 
 		return Collections.unmodifiableSet(inputChannelIDs);
 	}
+
+	public List<OutputGate> outputGates() {
+		return this.outputGates;
+	}
+
+	public List<InputGate<? extends IOReadableWritable>> inputGates() {
+		return this.inputGates;
+	}
 	
 	@Override
 	public AccumulatorProtocol getAccumulatorProtocolProxy() {
 		return accumulatorProtocolProxy;
 	}
 
-	public void addCopyTaskForCacheFile(String name, FutureTask<Path> copyTask) {
-		this.cacheCopyTasks.put(name, copyTask);
+	@Override
+	public BufferProvider getOutputBufferProvider() {
+		return this;
 	}
+	
+	// -----------------------------------------------------------------------------------------------------------------
+	//                                            BufferProvider methods
+	// -----------------------------------------------------------------------------------------------------------------
+	
 	@Override
-	public Map<String, FutureTask<Path>> getCopyTask() {
-		return this.cacheCopyTasks;
+	public Buffer requestBuffer(int minBufferSize) throws IOException {
+		return this.outputBufferPool.requestBuffer(minBufferSize);
 	}
+	
+	@Override
+	public Buffer requestBufferBlocking(int minBufferSize) throws IOException, InterruptedException {
+		return this.outputBufferPool.requestBufferBlocking(minBufferSize);
+	}
+	
+	@Override
+	public int getBufferSize() {
+		return this.outputBufferPool.getBufferSize();
+	}
+	
+	@Override
+	public boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
+		return this.outputBufferPool.registerBufferAvailabilityListener(listener);
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+	//                                       LocalBufferPoolOwner methods
+	// -----------------------------------------------------------------------------------------------------------------
 
+	@Override
+	public int getNumberOfChannels() {
+		return getNumberOfOutputChannels();
+	}
+
+	@Override
+	public void setDesignatedNumberOfBuffers(int numBuffers) {
+		this.outputBufferPool.setNumDesignatedBuffers(numBuffers);
+	}
+
+	@Override
+	public void clearLocalBufferPool() {
+		this.outputBufferPool.destroy();
+	}
+
+	@Override
+	public void registerGlobalBufferPool(GlobalBufferPool globalBufferPool) {
+		if (this.outputBufferPool == null) {
+			this.outputBufferPool = new LocalBufferPool(globalBufferPool, 1);
+		}
+	}
+
+	@Override
+	public void logBufferUtilization() {
+		LOG.info(String.format("\t%s: %d available, %d requested, %d designated",
+				getTaskNameWithIndex(),
+				this.outputBufferPool.numAvailableBuffers(),
+				this.outputBufferPool.numRequestedBuffers(),
+				this.outputBufferPool.numDesignatedBuffers()));
+	}
+
+	@Override
+	public void reportAsynchronousEvent() {
+		this.outputBufferPool.reportAsynchronousEvent();
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/DistributionPatternProvider.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/DistributionPatternProvider.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/DistributionPatternProvider.java
index 0c004ad..35350cc 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/DistributionPatternProvider.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/DistributionPatternProvider.java
@@ -13,7 +13,7 @@
 
 package eu.stratosphere.nephele.executiongraph;
 
-import eu.stratosphere.nephele.io.DistributionPattern;
+import eu.stratosphere.nephele.jobgraph.DistributionPattern;
 
 public final class DistributionPatternProvider {
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 cbb957f..0106361 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
@@ -13,11 +13,11 @@
 
 package eu.stratosphere.nephele.executiongraph;
 
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.channels.ChannelType;
 
 /**
- * Objects of this class represent a pair of {@link AbstractInputChannel} and {@link AbstractOutputChannel} objects
+ * 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.
  * 
  */
@@ -85,11 +85,6 @@ public final class ExecutionEdge {
 		return this.groupEdge.getChannelType();
 	}
 	
-	public boolean isBroadcast() {
-		
-		return this.groupEdge.isBroadcast();
-	}
-	
 	public int getConnectionID() {
 		
 		return this.groupEdge.getConnectionID();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGate.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGate.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGate.java
index f1bcb76..20d1a4e 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGate.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGate.java
@@ -16,8 +16,8 @@ package eu.stratosphere.nephele.executiongraph;
 import java.util.Collection;
 import java.util.concurrent.CopyOnWriteArrayList;
 
-import eu.stratosphere.nephele.io.GateID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.gates.GateID;
+import eu.stratosphere.runtime.io.channels.ChannelType;
 
 /**
  * Objects of this class represent either an {@link InputGate} or {@link OutputGate} within an {@link ExecutionGraph},

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 aa08655..ca7eddb 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
@@ -40,17 +40,17 @@ 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.io.DistributionPattern;
-import eu.stratosphere.nephele.io.GateID;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.nephele.jobgraph.DistributionPattern;
+import eu.stratosphere.runtime.io.gates.GateID;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+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.taskmanager.runtime.ExecutorThreadFactory;
+import eu.stratosphere.nephele.taskmanager.ExecutorThreadFactory;
 import eu.stratosphere.nephele.template.AbstractInputTask;
 import eu.stratosphere.nephele.template.AbstractInvokable;
 import eu.stratosphere.util.StringUtils;
@@ -412,9 +412,6 @@ public class ExecutionGraph implements ExecutionListener {
 
 			// First, build the group edges
 			for (int i = 0; i < sjv.getNumberOfForwardConnections(); ++i) {
-
-				final boolean isBroadcast = sgv.getEnvironment().getOutputGate(i).isBroadcast();
-
 				final JobEdge edge = sjv.getForwardConnection(i);
 				final AbstractJobVertex tjv = edge.getConnectedVertex();
 
@@ -427,13 +424,12 @@ public class ExecutionGraph implements ExecutionListener {
 					userDefinedChannelType = false;
 					channelType = ChannelType.NETWORK;
 				}
-				// Use NO_COMPRESSION as default compression level if nothing else is defined by the user
 
 				final DistributionPattern distributionPattern = edge.getDistributionPattern();
 
 				// Connect the corresponding group vertices and copy the user settings from the job edge
 				final ExecutionGroupEdge groupEdge = sgv.wireTo(tgv, edge.getIndexOfInputGate(), i, channelType,
-					userDefinedChannelType,distributionPattern, isBroadcast);
+					userDefinedChannelType,distributionPattern);
 
 				final ExecutionGate outputGate = new ExecutionGate(new GateID(), sev, groupEdge, false);
 				sev.insertOutputGate(i, outputGate);
@@ -970,7 +966,7 @@ public class ExecutionGraph implements ExecutionListener {
 
 				final ExecutionGate outputGate = sourceVertex.getOutputGate(i);
 				final ChannelType channelType = outputGate.getChannelType();
-				if (channelType == ChannelType.INMEMORY) {
+				if (channelType == ChannelType.IN_MEMORY) {
 					final int numberOfOutputChannels = outputGate.getNumberOfEdges();
 					for (int j = 0; j < numberOfOutputChannels; ++j) {
 						final ExecutionEdge outputChannel = outputGate.getEdge(j);
@@ -990,7 +986,7 @@ public class ExecutionGraph implements ExecutionListener {
 
 				final ExecutionGate inputGate = targetVertex.getInputGate(i);
 				final ChannelType channelType = inputGate.getChannelType();
-				if (channelType == ChannelType.INMEMORY) {
+				if (channelType == ChannelType.IN_MEMORY) {
 					final int numberOfInputChannels = inputGate.getNumberOfEdges();
 					for (int j = 0; j < numberOfInputChannels; ++j) {
 						final ExecutionEdge inputChannel = inputGate.getEdge(j);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupEdge.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupEdge.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupEdge.java
index 14970fd..373d120 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupEdge.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupEdge.java
@@ -13,8 +13,8 @@
 
 package eu.stratosphere.nephele.executiongraph;
 
-import eu.stratosphere.nephele.io.DistributionPattern;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.channels.ChannelType;
+import eu.stratosphere.nephele.jobgraph.DistributionPattern;
 
 /**
  * An execution group edge represents an edge between two execution group vertices.
@@ -66,11 +66,6 @@ public class ExecutionGroupEdge {
 	private final DistributionPattern distributionPattern;
 
 	/**
-	 * Stores if the edge is part of a broadcast group.
-	 */
-	private final boolean isBroadcast;
-
-	/**
 	 * Constructs a new group edge.
 	 * 
 	 * @param sourceVertex
@@ -85,19 +80,12 @@ public class ExecutionGroupEdge {
 	 *        the channel type for the edge
 	 * @param userDefinedChannelType
 	 *        <code>true</code> if the channel type has been specified by the user, <code>false</code> otherwise
-	 * @param compressionLevel
-	 *        the compression level for the edge
-	 * @param userDefinedCompressionLevel
-	 *        <code>true</code> if the compression level has been specified by the user, <code>false</code> otherwise
 	 * @param distributionPattern
 	 *        the distribution pattern to create the wiring
-	 * @param isBroadcast
-	 *        indicates that the edge is part of a broadcast group
 	 */
 	public ExecutionGroupEdge(final ExecutionGroupVertex sourceVertex, final int indexOfOutputGate,
 			final ExecutionGroupVertex targetVertex, final int indexOfInputGate, final ChannelType channelType,
-			final boolean userDefinedChannelType, final DistributionPattern distributionPattern,
-			final boolean isBroadcast) {
+			final boolean userDefinedChannelType, final DistributionPattern distributionPattern) {
 		this.sourceVertex = sourceVertex;
 		this.indexOfOutputGate = indexOfOutputGate;
 		this.channelType = channelType;
@@ -105,7 +93,6 @@ public class ExecutionGroupEdge {
 		this.userDefinedChannelType = userDefinedChannelType;
 		this.targetVertex = targetVertex;
 		this.distributionPattern = distributionPattern;
-		this.isBroadcast = isBroadcast;
 	}
 
 	/**
@@ -206,13 +193,4 @@ public class ExecutionGroupEdge {
 	public DistributionPattern getDistributionPattern() {
 		return this.distributionPattern;
 	}
-
-	/**
-	 * Checks if the edge is part of a broadcast group.
-	 * 
-	 * @return <code>true</code> if the edge is part of a broadcast group, <code>false</code> otherwise
-	 */
-	public boolean isBroadcast() {
-		return this.isBroadcast;
-	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 908eb6e..89b4b6d 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
@@ -13,6 +13,17 @@
 
 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.instance.InstanceType;
+import eu.stratosphere.nephele.jobgraph.JobVertexID;
+import eu.stratosphere.nephele.template.AbstractInvokable;
+import eu.stratosphere.runtime.io.channels.ChannelType;
+import eu.stratosphere.nephele.jobgraph.DistributionPattern;
+
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -23,17 +34,6 @@ import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 
-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.instance.InstanceType;
-import eu.stratosphere.nephele.io.DistributionPattern;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-import eu.stratosphere.nephele.jobgraph.JobVertexID;
-import eu.stratosphere.nephele.template.AbstractInvokable;
-
 /**
  * An ExecutionGroupVertex is created for every JobVertex of the initial job graph. It represents a number of execution
  * vertices that originate from the same job vertex.
@@ -388,7 +388,7 @@ public final class ExecutionGroupVertex {
 	 */
 	ExecutionGroupEdge wireTo(final ExecutionGroupVertex groupVertex, final int indexOfInputGate,
 			final int indexOfOutputGate, final ChannelType channelType, final boolean userDefinedChannelType,
-			final DistributionPattern distributionPattern, final boolean isBroadcast) throws GraphConversionException {
+			final DistributionPattern distributionPattern) throws GraphConversionException {
 
 		try {
 			final ExecutionGroupEdge previousEdge = this.forwardLinks.get(indexOfOutputGate);
@@ -401,8 +401,7 @@ public final class ExecutionGroupVertex {
 		}
 
 		final ExecutionGroupEdge edge = new ExecutionGroupEdge(this, indexOfOutputGate, groupVertex, indexOfInputGate,
-			channelType, userDefinedChannelType, distributionPattern,
-			isBroadcast);
+			channelType, userDefinedChannelType, distributionPattern);
 
 		this.forwardLinks.add(edge);
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 a251d3c..eab2375 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
@@ -27,7 +27,7 @@ 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.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.channels.ChannelType;
 
 /**
  * An execution stage contains all execution group vertices (and as a result all execution vertices) which
@@ -401,7 +401,7 @@ public final class ExecutionStage {
 						recurse = true;
 					}
 
-					if (channelType == ChannelType.INMEMORY
+					if (channelType == ChannelType.IN_MEMORY
 						&& !pipeline.equals(connectedVertex.getExecutionPipeline())) {
 
 						connectedVertex.setExecutionPipeline(pipeline);
@@ -432,7 +432,7 @@ public final class ExecutionStage {
 						recurse = true;
 					}
 
-					if (channelType == ChannelType.INMEMORY
+					if (channelType == ChannelType.IN_MEMORY
 						&& !pipeline.equals(connectedVertex.getExecutionPipeline())) {
 
 						connectedVertex.setExecutionPipeline(pipeline);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 7a9c48b..57ff073 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
@@ -35,11 +35,10 @@ import eu.stratosphere.nephele.execution.ExecutionState;
 import eu.stratosphere.nephele.execution.ExecutionStateTransition;
 import eu.stratosphere.nephele.instance.AllocatedResource;
 import eu.stratosphere.nephele.instance.AllocationID;
-import eu.stratosphere.nephele.io.GateID;
+import eu.stratosphere.runtime.io.gates.GateID;
 import eu.stratosphere.nephele.taskmanager.AbstractTaskResult;
 import eu.stratosphere.nephele.taskmanager.AbstractTaskResult.ReturnCode;
 import eu.stratosphere.nephele.taskmanager.TaskCancelResult;
-import eu.stratosphere.nephele.taskmanager.TaskKillResult;
 import eu.stratosphere.nephele.taskmanager.TaskSubmissionResult;
 import eu.stratosphere.nephele.util.AtomicEnum;
 import eu.stratosphere.nephele.util.SerializableArrayList;
@@ -692,41 +691,6 @@ public final class ExecutionVertex {
 	}
 
 	/**
-	 * Kills and removes the task represented by this vertex from the instance it is currently running on. If the
-	 * corresponding task is not in the state <code>RUNNING</code>, this call will be ignored. If the call has been
-	 * executed
-	 * successfully, the task will change the state <code>FAILED</code>.
-	 * 
-	 * @return the result of the task kill attempt
-	 */
-	public TaskKillResult killTask() {
-
-		final ExecutionState state = this.executionState.get();
-
-		if (state != ExecutionState.RUNNING) {
-			final TaskKillResult result = new TaskKillResult(getID(), AbstractTaskResult.ReturnCode.ILLEGAL_STATE);
-			result.setDescription("Vertex " + this.toString() + " is in state " + state);
-			return result;
-		}
-
-		final AllocatedResource ar = this.allocatedResource.get();
-
-		if (ar == null) {
-			final TaskKillResult result = new TaskKillResult(getID(), AbstractTaskResult.ReturnCode.NO_INSTANCE);
-			result.setDescription("Assigned instance of vertex " + this.toString() + " is null!");
-			return result;
-		}
-
-		try {
-			return ar.getInstance().killTask(this.vertexID);
-		} catch (IOException e) {
-			final TaskKillResult result = new TaskKillResult(getID(), AbstractTaskResult.ReturnCode.IPC_ERROR);
-			result.setDescription(StringUtils.stringifyException(e));
-			return result;
-		}
-	}
-
-	/**
 	 * Cancels and removes the task represented by this vertex
 	 * from the instance it is currently running on. If the task
 	 * is not currently running, its execution state is simply

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertexID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertexID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertexID.java
index 266b6ef..6209fde 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertexID.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertexID.java
@@ -13,7 +13,7 @@
 
 package eu.stratosphere.nephele.executiongraph;
 
-import eu.stratosphere.nephele.io.AbstractID;
+import eu.stratosphere.nephele.AbstractID;
 import eu.stratosphere.nephele.managementgraph.ManagementVertexID;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 cd768e2..72e3651 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
@@ -18,7 +18,7 @@ import java.util.Iterator;
 import java.util.Map;
 
 import eu.stratosphere.nephele.instance.AbstractInstance;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.channels.ChannelType;
 import eu.stratosphere.nephele.managementgraph.ManagementEdge;
 import eu.stratosphere.nephele.managementgraph.ManagementEdgeID;
 import eu.stratosphere.nephele.managementgraph.ManagementGate;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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
index 898f8e9..50e0e7f 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java
@@ -24,13 +24,12 @@ 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.io.channels.ChannelID;
+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.TaskKillResult;
 import eu.stratosphere.nephele.taskmanager.TaskSubmissionResult;
 import eu.stratosphere.nephele.topology.NetworkNode;
 import eu.stratosphere.nephele.topology.NetworkTopology;
@@ -96,8 +95,8 @@ public abstract class AbstractInstance extends NetworkNode {
 		if (this.taskManager == null) {
 
 			this.taskManager = RPC.getProxy(TaskOperationProtocol.class,
-				new InetSocketAddress(getInstanceConnectionInfo().getAddress(),
-					getInstanceConnectionInfo().getIPCPort()), NetUtils.getSocketFactory());
+				new InetSocketAddress(getInstanceConnectionInfo().address(),
+					getInstanceConnectionInfo().ipcPort()), NetUtils.getSocketFactory());
 		}
 
 		return this.taskManager;
@@ -206,22 +205,6 @@ public abstract class AbstractInstance extends NetworkNode {
 		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) {
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 7e8339f..3c83b80 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
@@ -13,7 +13,7 @@
 
 package eu.stratosphere.nephele.instance;
 
-import eu.stratosphere.nephele.io.AbstractID;
+import eu.stratosphere.nephele.AbstractID;
 
 /**
  * An allocation ID unambiguously identifies the allocated resources

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceConnectionInfo.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceConnectionInfo.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceConnectionInfo.java
index a0fd608..257421b 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceConnectionInfo.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceConnectionInfo.java
@@ -154,7 +154,8 @@ public class InstanceConnectionInfo implements IOReadableWritable, Comparable<In
 	 * 
 	 * @return the port instance's task manager runs its IPC service on
 	 */
-	public int getIPCPort() {
+	public int ipcPort() {
+
 		return this.ipcPort;
 	}
 
@@ -163,7 +164,8 @@ public class InstanceConnectionInfo implements IOReadableWritable, Comparable<In
 	 * 
 	 * @return the port instance's task manager expects to receive transfer envelopes on
 	 */
-	public int getDataPort() {
+	public int dataPort() {
+
 		return this.dataPort;
 	}
 
@@ -172,7 +174,8 @@ public class InstanceConnectionInfo implements IOReadableWritable, Comparable<In
 	 * 
 	 * @return the network address the instance's task manager binds its sockets to
 	 */
-	public InetAddress getAddress() {
+	public InetAddress address() {
+
 		return this.inetAddress;
 	}
 
@@ -182,7 +185,8 @@ public class InstanceConnectionInfo implements IOReadableWritable, Comparable<In
 	 * 
 	 * @return the host name of the instance
 	 */
-	public String getHostName() {
+	public String hostname() {
+
 		return this.hostName;
 	}
 
@@ -191,7 +195,8 @@ public class InstanceConnectionInfo implements IOReadableWritable, Comparable<In
 	 * 
 	 * @return the domain name of the instance or <code>null</code> if the domain name could not be determined
 	 */
-	public String getDomainName() {
+	public String domainName() {
+
 		return this.domainName;
 	}
 
@@ -251,15 +256,15 @@ public class InstanceConnectionInfo implements IOReadableWritable, Comparable<In
 		if (obj instanceof InstanceConnectionInfo) {
 
 			InstanceConnectionInfo ici = (InstanceConnectionInfo) obj;
-			if (!this.inetAddress.equals(ici.getAddress())) {
+			if (!this.inetAddress.equals(ici.address())) {
 				return false;
 			}
 
-			if (this.ipcPort != ici.getIPCPort()) {
+			if (this.ipcPort != ici.ipcPort()) {
 				return false;
 			}
 
-			if (this.dataPort != ici.getDataPort()) {
+			if (this.dataPort != ici.dataPort()) {
 				return false;
 			}
 
@@ -278,7 +283,9 @@ public class InstanceConnectionInfo implements IOReadableWritable, Comparable<In
 
 	@Override
 	public int compareTo(final InstanceConnectionInfo o) {
-		return this.getAddress().getHostName().compareTo(((InstanceConnectionInfo) o).getAddress().getHostName());
+
+		return this.address().getHostName()
+			.compareTo(((InstanceConnectionInfo) o).address().getHostName());
 	}
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceID.java
index c93b6bd..9b5b707 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceID.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceID.java
@@ -13,7 +13,7 @@
 
 package eu.stratosphere.nephele.instance;
 
-import eu.stratosphere.nephele.io.AbstractID;
+import eu.stratosphere.nephele.AbstractID;
 
 /**
  * Class for statistically unique instance IDs.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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
index b69d23e..480e521 100644
--- 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
@@ -396,14 +396,14 @@ public class ClusterManager implements InstanceManager {
 			final HardwareDescription hardwareDescription) {
 
 		// Check if there is a user-defined instance type for this IP address
-		InstanceType instanceType = this.ipToInstanceTypeMapping.get(instanceConnectionInfo.getAddress());
+		InstanceType instanceType = this.ipToInstanceTypeMapping.get(instanceConnectionInfo.address());
 		if (instanceType != null) {
 			LOG.info("Found user-defined instance type for cluster instance with IP "
-				+ instanceConnectionInfo.getAddress() + ": " + instanceType);
+				+ instanceConnectionInfo.address() + ": " + instanceType);
 		} else {
 			instanceType = matchHardwareDescriptionWithInstanceType(hardwareDescription);
 			if (instanceType != null) {
-				LOG.info("Hardware profile of cluster instance with IP " + instanceConnectionInfo.getAddress()
+				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");
@@ -412,7 +412,7 @@ public class ClusterManager implements InstanceManager {
 		}
 
 		// Try to match new host with a stub host from the existing topology
-		String instanceName = instanceConnectionInfo.getHostName();
+		String instanceName = instanceConnectionInfo.hostname();
 		NetworkNode parentNode = this.networkTopology.getRootNode();
 		NetworkNode currentStubNode = null;
 
@@ -439,7 +439,7 @@ public class ClusterManager implements InstanceManager {
 
 		// Try to match the new host using the IP address
 		if (currentStubNode == null) {
-			instanceName = instanceConnectionInfo.getAddress().toString();
+			instanceName = instanceConnectionInfo.address().toString();
 			instanceName = instanceName.replaceAll("/", ""); // Remove any / characters
 			currentStubNode = this.networkTopology.getNodeByName(instanceName);
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractGate.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractGate.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractGate.java
deleted file mode 100644
index c3e3697..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractGate.java
+++ /dev/null
@@ -1,148 +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.io;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
-import eu.stratosphere.nephele.event.task.EventListener;
-import eu.stratosphere.nephele.event.task.EventNotificationManager;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-import eu.stratosphere.nephele.jobgraph.JobID;
-
-/**
- * In Nephele a gate represents the connection between a user program and the processing framework. A gate
- * must be connected to exactly one record reader/writer and to at least one channel. The <code>Gate</code> class itself
- * is abstract. A gate automatically created for every record reader/writer in the user program. A gate can only be used
- * to transport one specific type of records.
- * <p>
- * This class in general is not thread-safe.
- * 
- * @param <T>
- *        the record type to be transported from this gate
- */
-public abstract class AbstractGate<T extends IOReadableWritable> implements Gate<T> {
-
-	/**
-	 * The ID of the job this gate belongs to.
-	 */
-	private final JobID jobID;
-
-	/**
-	 * The ID of this gate.
-	 */
-	private final GateID gateID;
-
-	/**
-	 * The index of the gate in the list of available input/output gates.
-	 */
-	private final int index;
-
-	/**
-	 * The event notification manager used to dispatch events.
-	 */
-	private final EventNotificationManager eventNotificationManager = new EventNotificationManager();
-
-	/**
-	 * The type of input/output channels connected to this gate.
-	 */
-	private ChannelType channelType = ChannelType.NETWORK;
-
-	/**
-	 * Constructs a new abstract gate
-	 * 
-	 * @param jobID
-	 *        the ID of the job this gate belongs to
-	 * @param gateID
-	 *        the ID of this gate
-	 * @param index
-	 *        the index of the gate in the list of available input/output gates.
-	 */
-	protected AbstractGate(final JobID jobID, final GateID gateID, final int index) {
-		this.jobID = jobID;
-		this.gateID = gateID;
-		this.index = index;
-	}
-
-
-	@Override
-	public final int getIndex() {
-		return this.index;
-	}
-
-	/**
-	 * Returns the event notification manager used to dispatch events.
-	 * 
-	 * @return the event notification manager used to dispatch events
-	 */
-	protected final EventNotificationManager getEventNotificationManager() {
-		return this.eventNotificationManager;
-	}
-
-
-	@Override
-	public String toString() {
-
-		return "Gate " + this.index;
-	}
-
-
-	@Override
-	public final void subscribeToEvent(EventListener eventListener, Class<? extends AbstractTaskEvent> eventType) {
-
-		this.eventNotificationManager.subscribeToEvent(eventListener, eventType);
-	}
-
-
-	@Override
-	public final void unsubscribeFromEvent(final EventListener eventListener,
-			final Class<? extends AbstractTaskEvent> eventType) {
-
-		this.eventNotificationManager.unsubscribeFromEvent(eventListener, eventType);
-	}
-
-
-	@Override
-	public final void deliverEvent(final AbstractTaskEvent event) {
-
-		this.eventNotificationManager.deliverEvent((AbstractTaskEvent) event);
-	}
-
-
-	@Override
-	public final void setChannelType(final ChannelType channelType) {
-
-		this.channelType = channelType;
-	}
-
-
-	@Override
-	public final ChannelType getChannelType() {
-
-		return this.channelType;
-	}
-
-
-	@Override
-	public JobID getJobID() {
-
-		return this.jobID;
-	}
-
-
-	@Override
-	public GateID getGateID() {
-
-		return this.gateID;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractID.java
deleted file mode 100644
index 37e6cbb..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractID.java
+++ /dev/null
@@ -1,214 +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.io;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.util.StringUtils;
-
-/**
- * ID is an abstract base class for providing statistically unique identification numbers in Nephele.
- * Every component that requires these kinds of IDs provides its own concrete type.
- * <p>
- * This class is thread-safe.
- * 
- */
-public abstract class AbstractID implements IOReadableWritable {
-
-	/**
-	 * The size of a long in bytes.
-	 */
-	private static final int SIZE_OF_LONG = 8;
-
-	/**
-	 * The size of the ID in byte.
-	 */
-	protected static final int SIZE = 2 * SIZE_OF_LONG;
-
-	/**
-	 * The upper part of the actual ID.
-	 */
-	private long upperPart;
-
-	/**
-	 * The lower part of the actual ID.
-	 */
-	private long lowerPart;
-
-	/**
-	 * Constructs a new ID with a specific bytes value.
-	 */
-	public AbstractID(final byte[] bytes) {
-
-		if (bytes.length != SIZE) {
-			throw new IllegalArgumentException("Argument bytes must by an array of " + SIZE + " bytes");
-		}
-
-		this.lowerPart = byteArrayToLong(bytes, 0);
-		this.upperPart = byteArrayToLong(bytes, SIZE_OF_LONG);
-	}
-
-	/**
-	 * Constructs a new abstract ID.
-	 * 
-	 * @param lowerPart
-	 *        the lower bytes of the ID
-	 * @param upperPart
-	 *        the higher bytes of the ID
-	 */
-	protected AbstractID(final long lowerPart, final long upperPart) {
-
-		this.lowerPart = lowerPart;
-		this.upperPart = upperPart;
-	}
-
-	/**
-	 * Creates a new abstract ID from the given one. The given and the newly created abtract ID will be identical, i.e.
-	 * a comparison by <code>equals</code> will return <code>true</code> and both objects will have the same hash code.
-	 * 
-	 * @param id
-	 *        the abstract ID to copy
-	 */
-	protected AbstractID(final AbstractID id) {
-
-		this.lowerPart = id.lowerPart;
-		this.upperPart = id.upperPart;
-	}
-
-	/**	
-	 * Constructs a new random ID from a uniform distribution.
-	 */
-	public AbstractID() {
-
-		this.lowerPart = generateRandomBytes();
-		this.upperPart = generateRandomBytes();
-	}
-
-	/**
-	 * Generates a uniformly distributed random positive long.
-	 * 
-	 * @return a uniformly distributed random positive long
-	 */
-	protected static long generateRandomBytes() {
-
-		return (long) (Math.random() * Long.MAX_VALUE);
-	}
-
-	/**
-	 * Converts the given byte array to a long.
-	 * 
-	 * @param ba
-	 *        the byte array to be converted
-	 * @param offset
-	 *        the offset indicating at which byte inside the array the conversion shall begin
-	 * @return the long variable
-	 */
-	private static long byteArrayToLong(final byte[] ba, final int offset) {
-
-		long l = 0;
-
-		for (int i = 0; i < SIZE_OF_LONG; ++i) {
-			l |= (ba[offset + SIZE_OF_LONG - 1 - i] & 0xffL) << (i << 3);
-		}
-
-		return l;
-	}
-
-	/**
-	 * Converts a long to a byte array.
-	 * 
-	 * @param l
-	 *        the long variable to be converted
-	 * @param ba
-	 *        the byte array to store the result the of the conversion
-	 * @param offset
-	 *        the offset indicating at what position inside the byte array the result of the conversion shall be stored
-	 */
-	private static void longToByteArray(final long l, final byte[] ba, final int offset) {
-
-		for (int i = 0; i < SIZE_OF_LONG; ++i) {
-			final int shift = i << 3; // i * 8
-			ba[offset + SIZE_OF_LONG - 1 - i] = (byte) ((l & (0xffL << shift)) >>> shift);
-		}
-	}
-	
-	/**
-	 * Sets an ID from another ID by copying its internal byte representation.
-	 * 
-	 * @param src
-	 *        the source ID
-	 */
-	public void setID(final AbstractID src) {
-		this.lowerPart = src.lowerPart;
-		this.upperPart = src.upperPart;
-	}
-
-
-	@Override
-	public boolean equals(final Object obj) {
-
-		if (!(obj instanceof AbstractID)) {
-			return false;
-		}
-
-		final AbstractID src = (AbstractID) obj;
-
-		if (src.lowerPart != this.lowerPart) {
-			return false;
-		}
-
-		if (src.upperPart != this.upperPart) {
-			return false;
-		}
-
-		return true;
-	}
-
-
-	@Override
-	public int hashCode() {
-
-		return (int) (this.lowerPart ^ (this.upperPart >>> 32));
-	}
-
-
-	@Override
-	public void read(final DataInput in) throws IOException {
-
-		this.lowerPart = in.readLong();
-		this.upperPart = in.readLong();
-	}
-
-
-	@Override
-	public void write(final DataOutput out) throws IOException {
-
-		out.writeLong(this.lowerPart);
-		out.writeLong(this.upperPart);
-	}
-
-
-	@Override
-	public String toString() {
-
-		final byte[] ba = new byte[SIZE];
-		longToByteArray(this.lowerPart, ba, 0);
-		longToByteArray(this.upperPart, ba, SIZE_OF_LONG);
-
-		return StringUtils.byteToHexString(ba);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractRecordReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractRecordReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractRecordReader.java
deleted file mode 100644
index 3a8ed25..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractRecordReader.java
+++ /dev/null
@@ -1,98 +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.io;
-
-import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
-import eu.stratosphere.nephele.event.task.EventListener;
-import eu.stratosphere.nephele.event.task.EventNotificationManager;
-
-/**
- * This is an abstract base class for a record reader, either dealing with mutable or immutable records,
- * and dealing with reads from single gates (single end points) or multiple gates (union).
- */
-public abstract class AbstractRecordReader implements ReaderBase {
-	
-	
-	private final EventNotificationManager eventHandler = new EventNotificationManager();
-	
-	private int numEventsUntilEndOfSuperstep = -1;
-	
-	private int endOfSuperstepEventsCount;
-	
-	// --------------------------------------------------------------------------------------------
-	
-	/**
-	 * Subscribes the listener object to receive events of the given type.
-	 * 
-	 * @param eventListener
-	 *        the listener object to register
-	 * @param eventType
-	 *        the type of event to register the listener for
-	 */
-	@Override
-	public void subscribeToEvent(EventListener eventListener, Class<? extends AbstractTaskEvent> eventType) {
-		this.eventHandler.subscribeToEvent(eventListener, eventType);
-	}
-
-	/**
-	 * Removes the subscription for events of the given type for the listener object.
-	 * 
-	 * @param eventListener The listener object to cancel the subscription for.
-	 * @param eventType The type of the event to cancel the subscription for.
-	 */
-	@Override
-	public void unsubscribeFromEvent(EventListener eventListener, Class<? extends AbstractTaskEvent> eventType) {
-		this.eventHandler.unsubscribeFromEvent(eventListener, eventType);
-	}
-	
-	
-	protected void handleEvent(AbstractTaskEvent evt) {
-		this.eventHandler.deliverEvent(evt);
-	}
-	
-	@Override
-	public void setIterative(int numEventsUntilEndOfSuperstep) {
-		this.numEventsUntilEndOfSuperstep = numEventsUntilEndOfSuperstep;
-	}
-
-	@Override
-	public void startNextSuperstep() {
-		if (this.numEventsUntilEndOfSuperstep == -1) {
-			throw new IllegalStateException("Called 'startNextSuperstep()' in a non-iterative reader.");
-		}
-		else if (endOfSuperstepEventsCount < numEventsUntilEndOfSuperstep) {
-			throw new IllegalStateException("Premature 'startNextSuperstep()'. Not yet reached the end-of-superstep.");
-		}
-		this.endOfSuperstepEventsCount = 0;
-	}
-	
-	@Override
-	public boolean hasReachedEndOfSuperstep() {
-		return endOfSuperstepEventsCount== numEventsUntilEndOfSuperstep;
-	}
-	
-	protected boolean incrementEndOfSuperstepEventAndCheck() {
-		if (numEventsUntilEndOfSuperstep == -1) {
-			throw new IllegalStateException("Received EndOfSuperstep event in a non-iterative reader.");
-		}
-		
-		endOfSuperstepEventsCount++;
-		
-		if (endOfSuperstepEventsCount > numEventsUntilEndOfSuperstep) {
-			throw new IllegalStateException("Received EndOfSuperstep events beyond the number to indicate the end of the superstep");
-		}
-		
-		return endOfSuperstepEventsCount== numEventsUntilEndOfSuperstep;
-	}
-}


[16/30] Offer buffer-oriented API for I/O (#25)

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/NoBufferAvailableException.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/NoBufferAvailableException.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/NoBufferAvailableException.java
deleted file mode 100644
index fe25837..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/NoBufferAvailableException.java
+++ /dev/null
@@ -1,53 +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.taskmanager.transferenvelope;
-
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider;
-
-/**
- * This exception is thrown to indicate that the deserialization process of a {@link TransferEnvelope} could not be
- * continued because a {@link Buffer} to store the envelope's content is currently not available.
- * 
- */
-public final class NoBufferAvailableException extends Exception {
-
-	/**
-	 * Generated serial UID.
-	 */
-	private static final long serialVersionUID = -9164212953646457026L;
-
-	/**
-	 * The buffer provider which could not deliver a buffer.
-	 */
-	private final BufferProvider bufferProvider;
-
-	/**
-	 * Constructs a new exception.
-	 * 
-	 * @param bufferProvider
-	 *        the buffer provider which could not deliver a buffer
-	 */
-	NoBufferAvailableException(final BufferProvider bufferProvider) {
-		this.bufferProvider = bufferProvider;
-	}
-
-	/**
-	 * Returns the buffer provider which could not deliver a buffer.
-	 * 
-	 * @return the buffer provider which could not deliver a buffer
-	 */
-	public BufferProvider getBufferProvider() {
-		return this.bufferProvider;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelope.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelope.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelope.java
deleted file mode 100644
index 32c56c5..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelope.java
+++ /dev/null
@@ -1,165 +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.taskmanager.transferenvelope;
-
-import java.io.IOException;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.event.task.EventList;
-import eu.stratosphere.nephele.io.channels.Buffer;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.jobgraph.JobID;
-
-public final class TransferEnvelope {
-
-	private final JobID jobID;
-
-	private final ChannelID source;
-
-	private final int sequenceNumber;
-
-	private EventList eventList;
-
-	private Buffer buffer = null;
-
-	public TransferEnvelope(int sequenceNumber, JobID jobID, ChannelID source) {
-		this(sequenceNumber, jobID, source, null);
-	}
-
-	public TransferEnvelope(int sequenceNumber, JobID jobID, ChannelID source, EventList eventList) {
-
-		this.sequenceNumber = sequenceNumber;
-		this.jobID = jobID;
-		this.source = source;
-		this.eventList = eventList;
-	}
-
-	public JobID getJobID() {
-		return this.jobID;
-	}
-
-	public ChannelID getSource() {
-		return this.source;
-	}
-
-	public void addEvent(AbstractEvent event) {
-
-		if (this.eventList == null) {
-			this.eventList = new EventList();
-		}
-
-		this.eventList.add(event);
-	}
-
-	public EventList getEventList() {
-
-		return this.eventList;
-	}
-
-	public int getSequenceNumber() {
-		return this.sequenceNumber;
-	}
-
-	public void setBuffer(Buffer buffer) {
-		this.buffer = buffer;
-	}
-
-	public Buffer getBuffer() {
-		return this.buffer;
-	}
-
-	public TransferEnvelope duplicate() throws IOException, InterruptedException {
-
-		final TransferEnvelope duplicatedTransferEnvelope = new TransferEnvelope(this.sequenceNumber, this.jobID,
-			this.source, this.eventList); // No need to duplicate event list
-
-		if (this.buffer != null) {
-			duplicatedTransferEnvelope.buffer = this.buffer.duplicate();
-		} else {
-			duplicatedTransferEnvelope.buffer = null;
-		}
-
-		return duplicatedTransferEnvelope;
-	}
-
-	public TransferEnvelope duplicateWithoutBuffer() {
-
-		final TransferEnvelope duplicatedTransferEnvelope = new TransferEnvelope(this.sequenceNumber, this.jobID,
-			this.source, this.eventList); // No need to duplicate event list
-
-		duplicatedTransferEnvelope.buffer = null;
-
-		return duplicatedTransferEnvelope;
-	}
-
-	@Override
-	public boolean equals(final Object obj) {
-
-		if (!(obj instanceof TransferEnvelope)) {
-			return false;
-		}
-
-		final TransferEnvelope te = (TransferEnvelope) obj;
-
-		if (!this.jobID.equals(te.jobID)) {
-			return false;
-		}
-
-		if (!this.source.equals(te.source)) {
-			return false;
-		}
-
-		if (this.sequenceNumber != te.sequenceNumber) {
-			return false;
-		}
-
-		if (this.buffer == null) {
-			if (te.buffer != null) {
-				return false;
-			}
-			// Both are null
-		} else {
-			if (te.buffer == null) {
-				return false;
-			}
-			// Both are non-null
-			if (!this.buffer.equals(te.buffer)) {
-				return false;
-			}
-		}
-
-		if (this.eventList == null) {
-			if (te.eventList != null) {
-				return false;
-			}
-			// Both are null
-		} else {
-			if (te.eventList == null) {
-				return false;
-			}
-			// Both are non-null
-			if (!this.eventList.equals(te.eventList)) {
-				return false;
-			}
-		}
-
-		return true;
-	}
-
-	@Override
-	public int hashCode() {
-
-		return (31 * this.sequenceNumber * this.jobID.hashCode());
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelopeDispatcher.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelopeDispatcher.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelopeDispatcher.java
deleted file mode 100644
index 7e1fb87..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelopeDispatcher.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.taskmanager.transferenvelope;
-
-import java.io.IOException;
-
-/**
- * A transfer envelope dispatcher receives {@link TransferEnvelopes} and sends them to all of this destinations.
- * 
- */
-public interface TransferEnvelopeDispatcher {
-
-	/**
-	 * Processes a transfer envelope from an output channel. The method may block until the system has allocated enough
-	 * resources to further process the envelope.
-	 * 
-	 * @param transferEnvelope
-	 *        the transfer envelope to be processed
-	 */
-	void processEnvelopeFromOutputChannel(TransferEnvelope transferEnvelope) throws IOException, InterruptedException;
-
-	void processEnvelopeFromInputChannel(TransferEnvelope transferEnvelope) throws IOException, InterruptedException;
-
-	void processEnvelopeFromNetwork(TransferEnvelope transferEnvelope, boolean freeSourceBuffer) throws IOException,
-			InterruptedException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelopeReceiverList.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelopeReceiverList.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelopeReceiverList.java
deleted file mode 100644
index 678897a..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelopeReceiverList.java
+++ /dev/null
@@ -1,86 +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.taskmanager.transferenvelope;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.ConnectionInfoLookupResponse;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.RemoteReceiver;
-
-/**
- * A transfer envelope receiver list contains all recipients of a transfer envelope. Their are three different types of
- * receivers: Local receivers identified by {@link ChannelID} objects, remote receivers identified by
- * {@link InetAddress} objects and finally checkpoints which are identified by
- * <p>
- * This class is thread-safe.
- * 
- */
-public class TransferEnvelopeReceiverList {
-
-	private final List<ChannelID> localReceivers;
-
-	private final List<RemoteReceiver> remoteReceivers;
-
-	public TransferEnvelopeReceiverList(final ConnectionInfoLookupResponse cilr) {
-
-		this.localReceivers = Collections.unmodifiableList(cilr.getLocalTargets());
-		this.remoteReceivers = Collections.unmodifiableList(cilr.getRemoteTargets());
-	}
-
-	public TransferEnvelopeReceiverList(final ChannelID localReceiver) {
-
-		final List<ChannelID> lr = new ArrayList<ChannelID>(1);
-		lr.add(localReceiver);
-
-		this.localReceivers = Collections.unmodifiableList(lr);
-		this.remoteReceivers = Collections.emptyList();
-	}
-
-	public TransferEnvelopeReceiverList(final RemoteReceiver remoteReceiver) {
-
-		final List<RemoteReceiver> rr = new ArrayList<RemoteReceiver>(1);
-		rr.add(remoteReceiver);
-
-		this.localReceivers = Collections.emptyList();
-		this.remoteReceivers = Collections.unmodifiableList(rr);
-	}
-
-	public boolean hasLocalReceivers() {
-
-		return (!this.localReceivers.isEmpty());
-	}
-
-	public boolean hasRemoteReceivers() {
-
-		return (!this.remoteReceivers.isEmpty());
-	}
-
-	public int getTotalNumberOfReceivers() {
-
-		return (this.localReceivers.size() + this.remoteReceivers.size());
-	}
-
-	public List<RemoteReceiver> getRemoteReceivers() {
-
-		return this.remoteReceivers;
-	}
-
-	public List<ChannelID> getLocalReceivers() {
-
-		return this.localReceivers;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/util/BufferPoolConnector.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/util/BufferPoolConnector.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/util/BufferPoolConnector.java
new file mode 100644
index 0000000..ff1d2be
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/util/BufferPoolConnector.java
@@ -0,0 +1,45 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.memory.MemorySegment;
+import eu.stratosphere.runtime.io.BufferRecycler;
+
+import java.util.Queue;
+
+public class BufferPoolConnector implements BufferRecycler {
+
+	/**
+	 * Reference to the memory pool the byte buffer was originally taken from.
+	 */
+	private final Queue<MemorySegment> memoryPool;
+
+	/**
+	 * Constructs a new buffer pool connector
+	 *
+	 * @param bufferPool
+	 *        a reference to the memory pool the byte buffer was originally taken from
+	 */
+	public BufferPoolConnector(final Queue<MemorySegment> bufferPool) {
+		this.memoryPool = bufferPool;
+	}
+
+	@Override
+	public void recycle(final MemorySegment buffer) {
+		synchronized (this.memoryPool) {
+			this.memoryPool.add(buffer);
+			this.memoryPool.notify();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/util/TaskUtils.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/util/TaskUtils.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/util/TaskUtils.java
deleted file mode 100644
index 40f6167..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/util/TaskUtils.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/***********************************************************************************************************************
- * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
- *
- * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- **********************************************************************************************************************/
-
-package eu.stratosphere.nephele.util;
-
-import eu.stratosphere.nephele.annotations.Stateless;
-import eu.stratosphere.nephele.template.AbstractInvokable;
-
-/**
- * This class implements several convenience methods to determine properties of Nephele task classes.
- * 
- */
-public class TaskUtils {
-
-	/**
-	 * Private constructor, so class cannot be instantiated.
-	 */
-	private TaskUtils() {
-	}
-
-	/**
-	 * Checks if a task is declared to be stateless.
-	 * 
-	 * @param taskClass
-	 *        the class of the task to check
-	 * @return <code>true</code> if the given class is declared to be stateless, <code>false</code> otherwise
-	 */
-	public static boolean isStateless(final Class<? extends AbstractInvokable> taskClass) {
-
-		return taskClass.isAnnotationPresent(Stateless.class);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 1795776..ced186b 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,7 +13,7 @@
 
 package eu.stratosphere.pact.runtime.iterative.io;
 
-import eu.stratosphere.nephele.io.MutableRecordReader;
+import eu.stratosphere.runtime.io.api.MutableRecordReader;
 import eu.stratosphere.nephele.template.AbstractOutputTask;
 import eu.stratosphere.types.Record;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 2d725b8..c39e3ef 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
@@ -25,7 +25,6 @@ import eu.stratosphere.api.common.typeutils.TypeComparator;
 import eu.stratosphere.api.common.typeutils.TypeComparatorFactory;
 import eu.stratosphere.api.common.typeutils.TypeSerializer;
 import eu.stratosphere.api.common.typeutils.TypeSerializerFactory;
-import eu.stratosphere.core.io.IOReadableWritable;
 import eu.stratosphere.core.memory.DataInputView;
 import eu.stratosphere.core.memory.MemorySegment;
 import eu.stratosphere.nephele.io.AbstractRecordWriter;
@@ -78,13 +77,13 @@ public class IterationHeadPactTask<X, Y, S extends Function, OT> extends Abstrac
 
 	private Collector<X> finalOutputCollector;
 
-	private List<AbstractRecordWriter<?>> finalOutputWriters;
+	private List<BufferWriter> finalOutputWriters;
 
 	private TypeSerializerFactory<Y> feedbackTypeSerializer;
 
 	private TypeSerializerFactory<X> solutionTypeSerializer;
 
-	private RecordWriter<?> toSync;
+	private BufferWriter toSync;
 
 	private int initialSolutionSetInput; // undefined for bulk iterations
 
@@ -108,7 +107,7 @@ public class IterationHeadPactTask<X, Y, S extends Function, OT> extends Abstrac
 
 		// at this time, the outputs to the step function are created
 		// add the outputs for the final solution
-		this.finalOutputWriters = new ArrayList<AbstractRecordWriter<?>>();
+		this.finalOutputWriters = new ArrayList<BufferWriter>();
 		final TaskConfig finalOutConfig = this.config.getIterationHeadFinalOutputConfig();
 		this.finalOutputCollector = RegularPactTask.getOutputCollector(this, finalOutConfig,
 			this.userCodeClassLoader, this.finalOutputWriters, finalOutConfig.getNumOutputs());
@@ -122,7 +121,7 @@ public class IterationHeadPactTask<X, Y, S extends Function, OT> extends Abstrac
 			throw new Exception("Error: Inconsistent head task setup - wrong mapping of output gates.");
 		}
 		// now, we can instantiate the sync gate
-		this.toSync = new RecordWriter<IOReadableWritable>(this, IOReadableWritable.class);
+		this.toSync = new BufferWriter(this);
 	}
 
 	/**
@@ -203,6 +202,8 @@ public class IterationHeadPactTask<X, Y, S extends Function, OT> extends Abstrac
 
 	@Override
 	public void run() throws Exception {
+		// initialize the serializers (one per channel) of the record writers
+		RegularPactTask.initOutputWriters(this.finalOutputWriters);
 
 		final String brokerKey = brokerKey();
 		final int workerIndex = getEnvironment().getIndexInSubtaskGroup();
@@ -330,6 +331,8 @@ public class IterationHeadPactTask<X, Y, S extends Function, OT> extends Abstrac
 				streamOutFinalOutputBulk(new InputViewIterator<X>(superstepResult, this.solutionTypeSerializer.getSerializer()));
 			}
 
+			this.finalOutputCollector.close();
+
 		} finally {
 			// make sure we unregister everything from the broker:
 			// - backchannel
@@ -388,7 +391,8 @@ public class IterationHeadPactTask<X, Y, S extends Function, OT> extends Abstrac
 		if (log.isInfoEnabled()) {
 			log.info(formatLogString("sending " + WorkerDoneEvent.class.getSimpleName() + " to sync"));
 		}
-		this.toSync.publishEvent(event);
+
+		this.toSync.broadcastEvent(event);
 	}
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationIntermediatePactTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationIntermediatePactTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationIntermediatePactTask.java
index 4ada733..5bdb3aa 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationIntermediatePactTask.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationIntermediatePactTask.java
@@ -19,7 +19,10 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
 import eu.stratosphere.api.common.functions.Function;
-import eu.stratosphere.nephele.io.AbstractRecordWriter;
+import eu.stratosphere.runtime.io.api.BufferWriter;
+import eu.stratosphere.pact.runtime.iterative.concurrent.BlockingBackChannel;
+import eu.stratosphere.runtime.io.channels.EndOfSuperstepEvent;
+import eu.stratosphere.pact.runtime.iterative.event.TerminationEvent;
 import eu.stratosphere.pact.runtime.iterative.io.WorksetUpdateOutputCollector;
 import eu.stratosphere.util.Collector;
 
@@ -107,7 +110,7 @@ public class IterationIntermediatePactTask<S extends Function, OT> extends Abstr
 	}
 
 	private void sendEndOfSuperstep() throws IOException, InterruptedException {
-		for (AbstractRecordWriter<?> eventualOutput : eventualOutputs) {
+		for (BufferWriter eventualOutput : this.eventualOutputs) {
 			eventualOutput.sendEndOfSuperstep();
 		}
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 35786a7..94f9b9f 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
@@ -28,7 +28,7 @@ import eu.stratosphere.api.common.aggregators.AggregatorWithName;
 import eu.stratosphere.api.common.aggregators.ConvergenceCriterion;
 import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
 import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
-import eu.stratosphere.nephele.io.MutableRecordReader;
+import eu.stratosphere.runtime.io.api.MutableRecordReader;
 import eu.stratosphere.nephele.template.AbstractOutputTask;
 import eu.stratosphere.nephele.types.IntegerRecord;
 import eu.stratosphere.pact.runtime.iterative.event.AllWorkersDoneEvent;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 5cfe173..859a62d 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
@@ -106,6 +106,7 @@ public class IterationTailPactTask<S extends Function, OT> extends AbstractItera
 				// aggregate workset update element count
 				long numCollected = worksetUpdateOutputCollector.getElementsCollectedAndReset();
 				worksetAggregator.aggregate(numCollected);
+
 			}
 
 			if (log.isInfoEnabled()) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/OutputCollector.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/OutputCollector.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/OutputCollector.java
index f1c945d..a91a59a 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/OutputCollector.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/OutputCollector.java
@@ -19,7 +19,7 @@ import java.util.Collections;
 import java.util.List;
 
 import eu.stratosphere.api.common.typeutils.TypeSerializer;
-import eu.stratosphere.nephele.io.AbstractRecordWriter;
+import eu.stratosphere.runtime.io.api.RecordWriter;
 import eu.stratosphere.pact.runtime.plugable.SerializationDelegate;
 import eu.stratosphere.util.Collector;
 
@@ -30,7 +30,7 @@ import eu.stratosphere.util.Collector;
 public class OutputCollector<T> implements Collector<T>
 {	
 	// list of writers
-	protected AbstractRecordWriter<SerializationDelegate<T>>[] writers; 
+	protected RecordWriter<SerializationDelegate<T>>[] writers;
 
 	private final SerializationDelegate<T> delegate;
 
@@ -43,10 +43,10 @@ public class OutputCollector<T> implements Collector<T>
 	 * @param writers List of all writers.
 	 */
 	@SuppressWarnings("unchecked")
-	public OutputCollector(List<AbstractRecordWriter<SerializationDelegate<T>>> writers, TypeSerializer<T> serializer)
+	public OutputCollector(List<RecordWriter<SerializationDelegate<T>>> writers, TypeSerializer<T> serializer)
 	{
 		this.delegate = new SerializationDelegate<T>(serializer);
-		this.writers = (AbstractRecordWriter<SerializationDelegate<T>>[]) writers.toArray(new AbstractRecordWriter[writers.size()]);
+		this.writers = (RecordWriter<SerializationDelegate<T>>[]) writers.toArray(new RecordWriter[writers.size()]);
 	}
 	
 	/**
@@ -56,14 +56,14 @@ public class OutputCollector<T> implements Collector<T>
 	 */
 
 	@SuppressWarnings("unchecked")
-	public void addWriter(AbstractRecordWriter<SerializationDelegate<T>> writer)
+	public void addWriter(RecordWriter<SerializationDelegate<T>> writer)
 	{
 		// avoid using the array-list here to reduce one level of object indirection
 		if (this.writers == null) {
-			this.writers = new AbstractRecordWriter[] {writer};
+			this.writers = new RecordWriter[] {writer};
 		}
 		else {
-			AbstractRecordWriter<SerializationDelegate<T>>[] ws = new AbstractRecordWriter[this.writers.length + 1];
+			RecordWriter<SerializationDelegate<T>>[] ws = new RecordWriter[this.writers.length + 1];
 			System.arraycopy(this.writers, 0, ws, 0, this.writers.length);
 			ws[this.writers.length] = writer;
 			this.writers = ws;
@@ -79,7 +79,7 @@ public class OutputCollector<T> implements Collector<T>
 		this.delegate.setInstance(record);
 		try {
 			for (int i = 0; i < writers.length; i++) {
-				this.writers[i].emit(this.delegate);	
+				this.writers[i].emit(this.delegate);
 			}
 		}
 		catch (IOException e) {
@@ -90,19 +90,24 @@ public class OutputCollector<T> implements Collector<T>
 		}
 	}
 
-	/*
-	 * (non-Javadoc)
-	 * @see eu.stratosphere.pact.common.stub.Collector#close()
-	 */
 	@Override
 	public void close() {
+		for (RecordWriter<?> writer : writers) {
+			try {
+				writer.flush();
+			} catch (IOException e) {
+				throw new RuntimeException(e.getMessage(), e);
+			} catch (InterruptedException e) {
+				throw new RuntimeException(e.getMessage(), e);
+			}
+		}
 	}
 
 	/**
 	 * List of writers that are associated with this output collector
 	 * @return list of writers
 	 */
-	public List<AbstractRecordWriter<SerializationDelegate<T>>> getWriters() {
+	public List<RecordWriter<SerializationDelegate<T>>> getWriters() {
 		return Collections.unmodifiableList(Arrays.asList(this.writers));
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/OutputEmitter.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/OutputEmitter.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/OutputEmitter.java
index ebc7ac8..46e3249 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/OutputEmitter.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/OutputEmitter.java
@@ -15,7 +15,7 @@ package eu.stratosphere.pact.runtime.shipping;
 
 import eu.stratosphere.api.common.distributions.DataDistribution;
 import eu.stratosphere.api.common.typeutils.TypeComparator;
-import eu.stratosphere.nephele.io.ChannelSelector;
+import eu.stratosphere.runtime.io.api.ChannelSelector;
 import eu.stratosphere.pact.runtime.plugable.SerializationDelegate;
 
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/RecordOutputCollector.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/RecordOutputCollector.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/RecordOutputCollector.java
index 852fd3a..4394483 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/RecordOutputCollector.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/RecordOutputCollector.java
@@ -18,7 +18,7 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 
-import eu.stratosphere.nephele.io.AbstractRecordWriter;
+import eu.stratosphere.runtime.io.api.RecordWriter;
 import eu.stratosphere.types.Record;
 import eu.stratosphere.util.Collector;
 
@@ -27,38 +27,37 @@ import eu.stratosphere.util.Collector;
  * The OutputCollector tracks to which writers a deep-copy must be given and which not.
  */
 public class RecordOutputCollector implements Collector<Record>
-{	
+{
 	// list of writers
-	protected AbstractRecordWriter<Record>[] writers; 
-	
+	protected RecordWriter<Record>[] writers;
+
 	/**
-	 * Initializes the output collector with a set of writers. 
-	 * To specify for a writer that it must be fed with a deep-copy, set the bit in the copy flag bit mask to 1 that 
+	 * Initializes the output collector with a set of writers.
+	 * To specify for a writer that it must be fed with a deep-copy, set the bit in the copy flag bit mask to 1 that
 	 * corresponds to the position of the writer within the {@link List}.
-	 * 
+	 *
 	 * @param writers List of all writers.
 	 */
 	@SuppressWarnings("unchecked")
-	public RecordOutputCollector(List<AbstractRecordWriter<Record>> writers) {
-		
-		this.writers = (AbstractRecordWriter<Record>[]) writers.toArray(new AbstractRecordWriter[writers.size()]);
+	public RecordOutputCollector(List<RecordWriter<Record>> writers) {
+
+		this.writers = (RecordWriter<Record>[]) writers.toArray(new RecordWriter[writers.size()]);
 	}
-	
+
 	/**
 	 * Adds a writer to the OutputCollector.
-	 * 
+	 *
 	 * @param writer The writer to add.
 	 */
-
 	@SuppressWarnings("unchecked")
-	public void addWriter(AbstractRecordWriter<Record> writer)
+	public void addWriter(RecordWriter<Record> writer)
 	{
 		// avoid using the array-list here to reduce one level of object indirection
 		if (this.writers == null) {
-			this.writers = new AbstractRecordWriter[] {writer};
+			this.writers = new RecordWriter[] {writer};
 		}
 		else {
-			AbstractRecordWriter<Record>[] ws = new AbstractRecordWriter[this.writers.length + 1];
+			RecordWriter<Record>[] ws = new RecordWriter[this.writers.length + 1];
 			System.arraycopy(this.writers, 0, ws, 0, this.writers.length);
 			ws[this.writers.length] = writer;
 			this.writers = ws;
@@ -74,7 +73,7 @@ public class RecordOutputCollector implements Collector<Record>
 	{
 		try {
 			for (int i = 0; i < writers.length; i++) {
-				this.writers[i].emit(record);	
+				this.writers[i].emit(record);
 			}
 		}
 		catch (IOException e) {
@@ -85,19 +84,24 @@ public class RecordOutputCollector implements Collector<Record>
 		}
 	}
 
-	/*
-	 * (non-Javadoc)
-	 * @see eu.stratosphere.pact.common.stub.Collector#close()
-	 */
 	@Override
 	public void close() {
+		for (RecordWriter<?> writer : writers) {
+			try {
+				writer.flush();
+			} catch (IOException e) {
+				throw new RuntimeException(e.getMessage(), e);
+			} catch (InterruptedException e) {
+				throw new RuntimeException(e.getMessage(), e);
+			}
+		}
 	}
 
 	/**
 	 * List of writers that are associated with this output collector
 	 * @return list of writers
 	 */
-	public List<AbstractRecordWriter<Record>> getWriters() {
+	public List<RecordWriter<Record>> getWriters() {
 		return Collections.unmodifiableList(Arrays.asList(writers));
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/RecordOutputEmitter.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/RecordOutputEmitter.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/RecordOutputEmitter.java
index 047c1bf..ba352eb 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/RecordOutputEmitter.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/RecordOutputEmitter.java
@@ -16,6 +16,8 @@ package eu.stratosphere.pact.runtime.shipping;
 import eu.stratosphere.api.common.distributions.DataDistribution;
 import eu.stratosphere.api.common.typeutils.TypeComparator;
 import eu.stratosphere.nephele.io.ChannelSelector;
+import eu.stratosphere.runtime.io.api.ChannelSelector;
+import eu.stratosphere.pact.runtime.plugable.pactrecord.RecordComparator;
 import eu.stratosphere.types.Key;
 import eu.stratosphere.types.Record;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 250efcc..cb3e782 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
@@ -15,6 +15,7 @@ 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;
 
@@ -29,10 +30,11 @@ 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.io.MutableReader;
-import eu.stratosphere.nephele.io.MutableRecordReader;
-import eu.stratosphere.nephele.io.MutableUnionRecordReader;
+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;
@@ -183,11 +185,16 @@ public class DataSinkTask<IT> extends AbstractOutputTask {
 			}
 		}
 		catch (Exception ex) {
+			ex = ExceptionInChainedStubException.exceptionUnwrap(ex);
+
+			if (ex instanceof CancelTaskException) {
+				// forward canceling exception
+				throw ex;
+			}
 			// drop, if the task was canceled
-			if (!this.taskCanceled) {
-				if (LOG.isErrorEnabled()) {
+			else if (!this.taskCanceled) {
+				if (LOG.isErrorEnabled())
 					LOG.error(getLogString("Error in user code: " + ex.getMessage()), ex);
-				}
 				throw ex;
 			}
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 47a8218..af176b9 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
@@ -17,7 +17,10 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
+import java.util.List;
 
+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;
 
@@ -27,6 +30,7 @@ import eu.stratosphere.api.common.typeutils.TypeSerializer;
 import eu.stratosphere.api.common.typeutils.TypeSerializerFactory;
 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.pact.runtime.shipping.OutputCollector;
@@ -41,13 +45,15 @@ import eu.stratosphere.util.Collector;
  * DataSourceTask which is executed by a Nephele task manager. The task reads data and uses an 
  * {@link InputFormat} to create records from the input.
  * 
- * @see eu.stratosphere.api.io.InputFormat
+ * @see eu.stratosphere.api.common.io.InputFormat
  */
 public class DataSourceTask<OT> extends AbstractInputTask<InputSplit> {
 	
 	// Obtain DataSourceTask Logger
 	private static final Log LOG = LogFactory.getLog(DataSourceTask.class);
 
+	private List<BufferWriter> eventualOutputs;
+
 	// Output collector
 	private Collector<OT> output;
 
@@ -116,6 +122,9 @@ public class DataSourceTask<OT> extends AbstractInputTask<InputSplit> {
 		final TypeSerializer<OT> serializer = this.serializerFactory.getSerializer();
 		
 		try {
+			// initialize the serializers (one per channel) of the record writers
+			RegularPactTask.initOutputWriters(this.eventualOutputs);
+
 			// start all chained tasks
 			RegularPactTask.openChainedTasks(this.chainedTasks, this);
 			
@@ -249,8 +258,14 @@ public class DataSourceTask<OT> extends AbstractInputTask<InputSplit> {
 			
 			RegularPactTask.cancelChainedTasks(this.chainedTasks);
 			
-			// drop exception, if the task was canceled
-			if (!this.taskCanceled) {
+			ex = ExceptionInChainedStubException.exceptionUnwrap(ex);
+
+			if (ex instanceof CancelTaskException) {
+				// forward canceling exception
+				throw ex;
+			}
+			else if (!this.taskCanceled) {
+				// drop exception, if the task was canceled
 				RegularPactTask.logAndThrowException(ex, this);
 			}
 		}
@@ -315,7 +330,8 @@ public class DataSourceTask<OT> extends AbstractInputTask<InputSplit> {
 	 */
 	private void initOutputs(ClassLoader cl) throws Exception {
 		this.chainedTasks = new ArrayList<ChainedDriver<?, ?>>();
-		this.output = RegularPactTask.initOutputs(this, cl, this.config, this.chainedTasks, null);
+		this.eventualOutputs = new ArrayList<BufferWriter>();
+		this.output = RegularPactTask.initOutputs(this, cl, this.config, this.chainedTasks, this.eventualOutputs);
 	}
 	
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 7aed8c0..b01799a 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
@@ -1,5 +1,5 @@
 /***********************************************************************************************************************
- * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ * Copyright (C) 2010-2014 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
@@ -13,14 +13,6 @@
 
 package eu.stratosphere.pact.runtime.task;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
 import eu.stratosphere.api.common.accumulators.Accumulator;
 import eu.stratosphere.api.common.accumulators.AccumulatorHelper;
 import eu.stratosphere.api.common.distributions.DataDistribution;
@@ -34,15 +26,15 @@ import eu.stratosphere.configuration.ConfigConstants;
 import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.configuration.GlobalConfiguration;
 import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.nephele.execution.CancelTaskException;
 import eu.stratosphere.nephele.execution.Environment;
 import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
-import eu.stratosphere.nephele.io.AbstractRecordWriter;
-import eu.stratosphere.nephele.io.BroadcastRecordWriter;
-import eu.stratosphere.nephele.io.ChannelSelector;
-import eu.stratosphere.nephele.io.MutableReader;
-import eu.stratosphere.nephele.io.MutableRecordReader;
-import eu.stratosphere.nephele.io.MutableUnionRecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
+import eu.stratosphere.runtime.io.api.ChannelSelector;
+import eu.stratosphere.runtime.io.api.RecordWriter;
+import eu.stratosphere.runtime.io.api.MutableReader;
+import eu.stratosphere.runtime.io.api.MutableRecordReader;
+import eu.stratosphere.runtime.io.api.MutableUnionRecordReader;
+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;
@@ -71,18 +63,22 @@ import eu.stratosphere.types.Record;
 import eu.stratosphere.util.Collector;
 import eu.stratosphere.util.InstantiationUtil;
 import eu.stratosphere.util.MutableObjectIterator;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+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> {
-	
+
 	protected static final Log LOG = LogFactory.getLog(RegularPactTask.class);
-	
-	private static final boolean USE_BROARDCAST_WRITERS = GlobalConfiguration.getBoolean(
-		ConfigConstants.USE_MULTICAST_FOR_BROADCAST, ConfigConstants.DEFAULT_USE_MULTICAST_FOR_BROADCAST);
-	
+
 	// --------------------------------------------------------------------------------------------
 
 	/**
@@ -95,7 +91,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 	 * The instantiated user code of this task's main operator (driver). May be null if the operator has no udf.
 	 */
 	protected S stub;
-	
+
 	/**
 	 * The udf's runtime context.
 	 */
@@ -111,13 +107,13 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 	 * The output writers for the data that this task forwards to the next task. The latest driver (the central, if no chained
 	 * drivers exist, otherwise the last chained driver) produces its output to these writers.
 	 */
-	protected List<AbstractRecordWriter<?>> eventualOutputs;
-	
+	protected List<BufferWriter> eventualOutputs;
+
 	/**
 	 * The input readers to this task.
 	 */
 	protected MutableReader<?>[] inputReaders;
-	
+
 	/**
 	 * The input readers for the configured broadcast variables for this task.
 	 */
@@ -127,7 +123,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 	 * The inputs reader, wrapped in an iterator. Prior to the local strategies, etc...
 	 */
 	protected MutableObjectIterator<?>[] inputIterators;
-	
+
 	/**
 	 * The input readers for the configured broadcast variables, wrapped in an iterator. 
 	 * Prior to the local strategies, etc...
@@ -142,18 +138,18 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 	 * The local strategies that are applied on the inputs.
 	 */
 	protected volatile CloseableInputProvider<?>[] localStrategies;
-	
+
 	/**
 	 * The optional temp barriers on the inputs for dead-lock breaking. Are
 	 * optionally resettable.
 	 */
 	protected volatile TempBarrier<?>[] tempBarriers;
-	
+
 	/**
 	 * The resettable inputs in the case where no temp barrier is needed.
 	 */
 	protected volatile SpillingResettableMutableObjectIterator<?>[] resettableInputs;
-	
+
 	/**
 	 * The inputs to the operator. Return the readers' data after the application of the local strategy
 	 * and the temp-table barrier.
@@ -189,24 +185,24 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 	 * A list of chained drivers, if there are any.
 	 */
 	protected ArrayList<ChainedDriver<?, ?>> chainedTasks;
-	
+
 	/**
 	 * Certain inputs may be excluded from resetting. For example, the initial partial solution
 	 * in an iteration head must not be reseted (it is read through the back channel), when all
 	 * others are reseted.
 	 */
 	private boolean[] excludeFromReset;
-	
+
 	/**
 	 * Flag indicating for each input whether it is cached and can be reseted.
 	 */
 	private boolean[] inputIsCached;
-			
+
 	/**
 	 * flag indicating for each input whether it must be asynchronously materialized.
 	 */
 	private boolean[] inputIsAsyncMaterialized;
-	
+
 	/**
 	 * The amount of memory per input that is dedicated to the materialization.
 	 */
@@ -283,16 +279,18 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 	 */
 	@Override
 	public void invoke() throws Exception {
-		
-		if (LOG.isDebugEnabled()) {
+
+		if (LOG.isDebugEnabled())
 			LOG.debug(formatLogString("Start task code."));
-		}
-		
+
 		// whatever happens in this scope, make sure that the local strategies are cleaned up!
 		// note that the initialization of the local strategies is in the try-finally block as well,
 		// so that the thread that creates them catches its own errors that may happen in that process.
 		// this is especially important, since there may be asynchronous closes (such as through canceling).
 		try {
+			// initialize the serializers (one per channel) of the record writers
+			initOutputWriters(this.eventualOutputs);
+
 			// initialize the remaining data structures on the input and trigger the local processing
 			// the local processing includes building the dams / caches
 			try {
@@ -352,23 +350,23 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 				throw new RuntimeException("Initializing the input processing failed" +
 					e.getMessage() == null ? "." : ": " + e.getMessage(), e);
 			}
-			
+
 			if (!this.running) {
 				if (LOG.isDebugEnabled()) {
 					LOG.debug(formatLogString("Task cancelled before task code was started."));
 				}
 				return;
 			}
-			
+
 			// pre main-function initialization
 			initialize();
-			
+
 			// read the broadcast variables
 			for (int i = 0; i < this.config.getNumBroadcastInputs(); i++) {
 				final String name = this.config.getBroadcastInputName(i);
 				readAndSetBroadcastInput(i, name, this.runtimeUdfContext);
 			}
-	
+
 			// the work goes here
 			run();
 		}
@@ -376,7 +374,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 			// clean up in any case!
 			closeLocalStrategiesAndCaches();
 		}
-		
+
 		if (this.running) {
 			if (LOG.isDebugEnabled()) {
 				LOG.debug(formatLogString("Finished task code."));
@@ -387,15 +385,14 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 			}
 		}
 	}
-	
+
 	@Override
 	public void cancel() throws Exception {
 		this.running = false;
-		
-		if (LOG.isDebugEnabled()) {
+
+		if (LOG.isDebugEnabled())
 			LOG.debug(formatLogString("Cancelling task code"));
-		}
-		
+
 		try {
 			if (this.driver != null) {
 				this.driver.cancel();
@@ -434,7 +431,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 		// instantiate the UDF
 		try {
 			final Class<? super S> userCodeFunctionType = this.driver.getStubType();
-			// if the class is null, the driver has no user code 
+			// if the class is null, the driver has no user code
 			if (userCodeFunctionType != null) {
 				this.stub = initStub(userCodeFunctionType);
 			}
@@ -462,7 +459,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 		}
 		context.setBroadcastVariable(bcVarName, collection);
 	}
-	
+
 	protected void run() throws Exception {
 		// ---------------------------- Now, the actual processing starts ------------------------
 		// check for asynchronous canceling
@@ -483,7 +480,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 				throw new Exception("The data preparation for task '" + this.getEnvironment().getTaskName() +
 					"' , caused an error: " + t.getMessage(), t);
 			}
-			
+
 			// check for canceling
 			if (!this.running) {
 				return;
@@ -517,7 +514,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 
 			// close all chained tasks letting them report failure
 			RegularPactTask.closeChainedTasks(this.chainedTasks, this);
-			
+
 			// Collect the accumulators of all involved UDFs and send them to the
 			// JobManager. close() has been called earlier for all involved UDFs
 			// (using this.stub.close() and closeChainedTasks()), so UDFs can no longer
@@ -539,8 +536,14 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 
 			RegularPactTask.cancelChainedTasks(this.chainedTasks);
 
-			// drop exception, if the task was canceled
-			if (this.running) {
+			ex = ExceptionInChainedStubException.exceptionUnwrap(ex);
+
+			if (ex instanceof CancelTaskException) {
+				// forward canceling exception
+				throw ex;
+			}
+			else if (this.running) {
+				// throw only if task was not cancelled. in the case of canceling, exceptions are expected 
 				RegularPactTask.logAndThrowException(ex, this);
 			}
 		}
@@ -553,9 +556,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 	 * This method is called at the end of a task, receiving the accumulators of
 	 * the task and the chained tasks. It merges them into a single map of
 	 * accumulators and sends them to the JobManager.
-	 * 
-	 * @param env
-	 * @param accumulators
+	 *
 	 * @param chainedTasks
 	 *          Each chained task might have accumulators which will be merged
 	 *          with the accumulators of the stub.
@@ -566,7 +567,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 		// We can merge here the accumulators from the stub and the chained
 		// tasks. Type conflicts can occur here if counters with same name but
 		// different type were used.
-		
+
 		for (ChainedDriver<?, ?> chainedTask : chainedTasks) {
 			Map<String, Accumulator<?, ?>> chainedAccumulators = chainedTask.getStub().getRuntimeContext().getAllAccumulators();
 			AccumulatorHelper.mergeInto(accumulators, chainedAccumulators);
@@ -632,8 +633,6 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 		}
 	}
 
-	
-
 	// --------------------------------------------------------------------------------------------
 	//                                 Task Setup and Teardown
 	// --------------------------------------------------------------------------------------------
@@ -665,12 +664,12 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 
 		chainedTasks.get(numChained - 1).setOutputCollector(newOutputCollector);
 	}
-	
+
 	public TaskConfig getLastTasksConfig() {
 		int numChained = this.chainedTasks.size();
 		return (numChained == 0) ? config : chainedTasks.get(numChained - 1).getTaskConfig();
 	}
-	
+
 	protected S initStub(Class<? super S> stubSuperClass) throws Exception {
 		try {
 			S stub = config.<S>getStubWrapper(this.userCodeClassLoader).getUserCodeObject(stubSuperClass, this.userCodeClassLoader);
@@ -687,7 +686,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 			throw new Exception("The stub class is not a proper subclass of " + stubSuperClass.getName(), ccex);
 		}
 	}
-	
+
 	/**
 	 * Creates the record readers for the number of inputs as defined by {@link #getNumTaskInputs()}.
 	 *
@@ -697,9 +696,9 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 	protected void initInputReaders() throws Exception {
 		final int numInputs = getNumTaskInputs();
 		final MutableReader<?>[] inputReaders = new MutableReader[numInputs];
-		
+
 		int numGates = 0;
-		
+
 		for (int i = 0; i < numInputs; i++) {
 			//  ---------------- create the input readers ---------------------
 			// in case where a logical input unions multiple physical inputs, create a union reader
@@ -720,13 +719,13 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 			}
 		}
 		this.inputReaders = inputReaders;
-		
+
 		// final sanity check
 		if (numGates != this.config.getNumInputs()) {
 			throw new Exception("Illegal configuration: Number of input gates and group sizes are not consistent.");
 		}
 	}
-	
+
 	/**
 	 * Creates the record readers for the extra broadcast inputs as configured by {@link TaskConfig#getNumBroadcastInputs()}.
 	 *
@@ -776,7 +775,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 				final TypeComparatorFactory<?> comparatorFactory = this.config.getDriverComparator(i, this.userCodeClassLoader);
 				this.inputComparators[i] = comparatorFactory.createComparator();
 			}
-			
+
 			this.inputIterators[i] = createInputIterator(this.inputReaders[i], this.inputSerializers[i]);
 		}
 	}
@@ -796,34 +795,34 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 			this.broadcastInputIterators[i] = createInputIterator(this.broadcastInputReaders[i], this.broadcastInputSerializers[i]);
 		}
 	}
-	
+
 	/**
-	 * 
+	 *
 	 * NOTE: This method must be invoked after the invocation of {@code #initInputReaders()} and
 	 * {@code #initInputSerializersAndComparators(int)}!
-	 * 
+	 *
 	 * @param numInputs
 	 */
 	protected void initLocalStrategies(int numInputs) throws Exception {
-		
+
 		final MemoryManager memMan = getMemoryManager();
 		final IOManager ioMan = getIOManager();
-		
+
 		this.localStrategies = new CloseableInputProvider[numInputs];
 		this.inputs = new MutableObjectIterator[numInputs];
 		this.excludeFromReset = new boolean[numInputs];
 		this.inputIsCached = new boolean[numInputs];
 		this.inputIsAsyncMaterialized = new boolean[numInputs];
 		this.materializationMemory = new int[numInputs];
-		
+
 		// set up the local strategies first, such that the can work before any temp barrier is created
 		for (int i = 0; i < numInputs; i++) {
 			initInputLocalStrategy(i);
 		}
-		
+
 		// we do another loop over the inputs, because we want to instantiate all
 		// sorters, etc before requesting the first input (as this call may block)
-		
+
 		// we have two types of materialized inputs, and both are replayable (can act as a cache)
 		// The first variant materializes in a different thread and hence
 		// acts as a pipeline breaker. this one should only be there, if a pipeline breaker is needed.
@@ -831,15 +830,15 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 		// in a pipelined fashion.
 		this.resettableInputs = new SpillingResettableMutableObjectIterator[numInputs];
 		this.tempBarriers = new TempBarrier[numInputs];
-		
+
 		for (int i = 0; i < numInputs; i++) {
 			final int memoryPages;
 			final boolean async = this.config.isInputAsynchronouslyMaterialized(i);
 			final boolean cached =  this.config.isInputCached(i);
-			
+
 			this.inputIsAsyncMaterialized[i] = async;
 			this.inputIsCached[i] = cached;
-			
+
 			if (async || cached) {
 				memoryPages = memMan.computeNumberOfPages(this.config.getInputMaterializationMemory(i));
 				if (memoryPages <= 0) {
@@ -849,7 +848,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 			} else {
 				memoryPages = 0;
 			}
-			
+
 			if (async) {
 				@SuppressWarnings({ "unchecked", "rawtypes" })
 				TempBarrier<?> barrier = new TempBarrier(this, getInput(i), this.inputSerializers[i], memMan, ioMan, memoryPages);
@@ -865,7 +864,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 			}
 		}
 	}
-	
+
 	protected void resetAllInputs() throws Exception {
 		// close all local-strategies. they will either get re-initialized, or we have
 		// read them now and their data is cached
@@ -875,10 +874,10 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 				this.localStrategies[i] = null;
 			}
 		}
-		
+
 		final MemoryManager memMan = getMemoryManager();
 		final IOManager ioMan = getIOManager();
-		
+
 		// reset the caches, or re-run the input local strategy
 		for (int i = 0; i < this.inputs.length; i++) {
 			if (this.excludeFromReset[i]) {
@@ -892,7 +891,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 			} else {
 				// make sure the input is not available directly, but are lazily fetched again
 				this.inputs[i] = null;
-				
+
 				if (this.inputIsCached[i]) {
 					if (this.tempBarriers[i] != null) {
 						this.inputs[i] = this.tempBarriers[i].getIterator();
@@ -908,10 +907,10 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 					if (this.tempBarriers[i] != null) {
 						this.tempBarriers[i].close();
 					}
-					
+
 					// recreate the local strategy
 					initInputLocalStrategy(i);
-					
+
 					if (this.inputIsAsyncMaterialized[i]) {
 						final int pages = this.materializationMemory[i];
 						@SuppressWarnings({ "unchecked", "rawtypes" })
@@ -924,17 +923,17 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 			}
 		}
 	}
-	
+
 	protected void excludeFromReset(int inputNum) {
 		this.excludeFromReset[inputNum] = true;
 	}
-	
+
 	private void initInputLocalStrategy(int inputNum) throws Exception {
 		// check if there is already a strategy
 		if (this.localStrategies[inputNum] != null) {
 			throw new IllegalStateException();
 		}
-		
+
 		// now set up the local strategy
 		final LocalStrategy localStrategy = this.config.getInputLocalStrategy(inputNum);
 		if (localStrategy != null) {
@@ -960,7 +959,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 				if (inputNum != 0) {
 					throw new IllegalStateException("Performing combining sort outside a (group)reduce task!");
 				}
-				
+
 				// instantiate ourselves a combiner. we should not use the stub, because the sort and the
 				// subsequent (group)reduce would otherwise share it multi-threaded
 				final Class<S> userCodeFunctionType = this.driver.getStubType();
@@ -986,7 +985,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 					this.config.getMemoryInput(inputNum), this.config.getFilehandlesInput(inputNum),
 					this.config.getSpillingThresholdInput(inputNum));
 				cSorter.setUdfConfiguration(this.config.getStubParameters());
-				
+
 				// set the input to null such that it will be lazily fetched from the input strategy
 				this.inputs[inputNum] = null;
 				this.localStrategies[inputNum] = cSorter;
@@ -999,7 +998,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 			this.inputs[inputNum] = this.inputIterators[inputNum];
 		}
 	}
-	
+
 	private <T> TypeComparator<T> getLocalStrategyComparator(int inputNum) throws Exception {
 		TypeComparatorFactory<T> compFact = this.config.getInputComparator(inputNum, this.userCodeClassLoader);
 		if (compFact == null) {
@@ -1030,21 +1029,21 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 //		final MutableObjectIterator<?> iter = new ReaderIterator(reader, serializer);
 //		return iter;
 	}
-	
+
 	protected int getNumTaskInputs() {
 		return this.driver.getNumberOfInputs();
 	}
-	
+
 	/**
 	 * Creates a writer for each output. Creates an OutputCollector which forwards its input to all writers.
 	 * The output collector applies the configured shipping strategies for each writer.
 	 */
 	protected void initOutputs() throws Exception {
 		this.chainedTasks = new ArrayList<ChainedDriver<?, ?>>();
-		this.eventualOutputs = new ArrayList<AbstractRecordWriter<?>>();
+		this.eventualOutputs = new ArrayList<BufferWriter>();
 		this.output = initOutputs(this, this.userCodeClassLoader, this.config, this.chainedTasks, this.eventualOutputs);
 	}
-	
+
 	public RuntimeUDFContext createRuntimeContext(String taskName) {
 		Environment env = getEnvironment();
 		return new RuntimeUDFContext(taskName, env.getCurrentNumberOfSubtasks(), env.getIndexInSubtaskGroup(), env.getCopyTask());
@@ -1054,61 +1053,52 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 	//                                   Task Context Signature
 	// -------------------------------------------------------------------------------------------
 
-
 	@Override
 	public TaskConfig getTaskConfig() {
 		return this.config;
 	}
 
-
 	@Override
 	public ClassLoader getUserCodeClassLoader() {
 		return this.userCodeClassLoader;
 	}
 
-
 	@Override
 	public MemoryManager getMemoryManager() {
 		return getEnvironment().getMemoryManager();
 	}
 
-
 	@Override
 	public IOManager getIOManager() {
 		return getEnvironment().getIOManager();
 	}
 
-
 	@Override
 	public S getStub() {
 		return this.stub;
 	}
 
-
 	@Override
 	public Collector<OT> getOutputCollector() {
 		return this.output;
 	}
 
-
 	@Override
 	public AbstractInvokable getOwningNepheleTask() {
 		return this;
 	}
 
-
 	@Override
 	public String formatLogString(String message) {
 		return constructLogString(message, getEnvironment().getTaskName(), this);
 	}
 
-
 	@Override
 	public <X> MutableObjectIterator<X> getInput(int index) {
 		if (index < 0 || index > this.driver.getNumberOfInputs()) {
 			throw new IndexOutOfBoundsException();
 		}
-		
+
 		// check for lazy assignment from input strategies
 		if (this.inputs[index] != null) {
 			@SuppressWarnings("unchecked")
@@ -1235,19 +1225,19 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 	 *
 	 * @return The OutputCollector that data produced in this task is submitted to.
 	 */
-	public static <T> Collector<T> getOutputCollector(AbstractInvokable task, TaskConfig config, ClassLoader cl, List<AbstractRecordWriter<?>> eventualOutputs, int numOutputs)
+	public static <T> Collector<T> getOutputCollector(AbstractInvokable task, TaskConfig config, ClassLoader cl, List<BufferWriter> eventualOutputs, int numOutputs)
 	throws Exception
 	{
 		if (numOutputs <= 0) {
 			throw new Exception("BUG: The task must have at least one output");
 		}
-		
+
 		// get the factory for the serializer
 		final TypeSerializerFactory<T> serializerFactory = config.getOutputSerializer(cl);
 
 		// special case the Record
 		if (serializerFactory.getDataType().equals(Record.class)) {
-			final List<AbstractRecordWriter<Record>> writers = new ArrayList<AbstractRecordWriter<Record>>(numOutputs);
+			final List<RecordWriter<Record>> writers = new ArrayList<RecordWriter<Record>>(numOutputs);
 
 			// create a writer for each output
 			for (int i = 0; i < numOutputs; i++) {
@@ -1267,18 +1257,10 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 					oe = new RecordOutputEmitter(strategy, comparator, distribution);
 				}
 
-				if (strategy == ShipStrategyType.BROADCAST && USE_BROARDCAST_WRITERS) {
-					if (task instanceof AbstractTask) {
-						writers.add(new BroadcastRecordWriter<Record>((AbstractTask) task, Record.class));
-					} else if (task instanceof AbstractInputTask<?>) {
-						writers.add(new BroadcastRecordWriter<Record>((AbstractInputTask<?>) task, Record.class));
-					}
-				} else {
-					if (task instanceof AbstractTask) {
-						writers.add(new RecordWriter<Record>((AbstractTask) task, Record.class, oe));
-					} else if (task instanceof AbstractInputTask<?>) {
-						writers.add(new RecordWriter<Record>((AbstractInputTask<?>) task, Record.class, oe));
-					}
+				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));
 				}
 			}
 			if (eventualOutputs != null) {
@@ -1291,9 +1273,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 		}
 		else {
 			// generic case
-			final List<AbstractRecordWriter<SerializationDelegate<T>>> writers = new ArrayList<AbstractRecordWriter<SerializationDelegate<T>>>(numOutputs);
-			@SuppressWarnings("unchecked") // uncritical, simply due to broken generics
-			final Class<SerializationDelegate<T>> delegateClazz = (Class<SerializationDelegate<T>>) (Class<?>) SerializationDelegate.class;
+			final List<RecordWriter<SerializationDelegate<T>>> writers = new ArrayList<RecordWriter<SerializationDelegate<T>>>(numOutputs);
 
 			// create a writer for each output
 			for (int i = 0; i < numOutputs; i++)
@@ -1314,18 +1294,10 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 					oe = new OutputEmitter<T>(strategy, comparator, dataDist);
 				}
 
-				if (strategy == ShipStrategyType.BROADCAST && USE_BROARDCAST_WRITERS) {
-					if (task instanceof AbstractTask) {
-						writers.add(new BroadcastRecordWriter<SerializationDelegate<T>>((AbstractTask) task, delegateClazz));
-					} else if (task instanceof AbstractInputTask<?>) {
-						writers.add(new BroadcastRecordWriter<SerializationDelegate<T>>((AbstractInputTask<?>) task, delegateClazz));
-					}
-				} else {
-					if (task instanceof AbstractTask) {
-						writers.add(new RecordWriter<SerializationDelegate<T>>((AbstractTask) task, delegateClazz, oe));
-					} else if (task instanceof AbstractInputTask<?>) {
-						writers.add(new RecordWriter<SerializationDelegate<T>>((AbstractInputTask<?>) task, delegateClazz, oe));
-					}
+				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));
 				}
 			}
 			if (eventualOutputs != null) {
@@ -1341,7 +1313,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 	 */
 	@SuppressWarnings("unchecked")
 	public static <T> Collector<T> initOutputs(AbstractInvokable nepheleTask, ClassLoader cl, TaskConfig config,
-					List<ChainedDriver<?, ?>> chainedTasksTarget, List<AbstractRecordWriter<?>> eventualOutputs)
+					List<ChainedDriver<?, ?>> chainedTasksTarget, List<BufferWriter> eventualOutputs)
 	throws Exception
 	{
 		final int numOutputs = config.getNumOutputs();
@@ -1391,6 +1363,12 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 		// instantiate the output collector the default way from this configuration
 		return getOutputCollector(nepheleTask , config, cl, eventualOutputs, numOutputs);
 	}
+
+	public static void initOutputWriters(List<BufferWriter> writers) {
+		for (BufferWriter writer : writers) {
+			((RecordWriter<?>) writer).initializeSerializers();
+		}
+	}
 	
 	// --------------------------------------------------------------------------------------------
 	//                                  User Code LifeCycle

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/chaining/ExceptionInChainedStubException.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/chaining/ExceptionInChainedStubException.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/chaining/ExceptionInChainedStubException.java
index 568bb7d..786928b 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/chaining/ExceptionInChainedStubException.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/chaining/ExceptionInChainedStubException.java
@@ -47,4 +47,12 @@ public class ExceptionInChainedStubException extends RuntimeException
 	public Exception getWrappedException() {
 		return exception;
 	}
+
+	public static Exception exceptionUnwrap(Exception e) {
+		if (e instanceof ExceptionInChainedStubException) {
+			return exceptionUnwrap(((ExceptionInChainedStubException) e).getWrappedException());
+		}
+
+		return e;
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 c5c26eb..814eb62 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
@@ -165,6 +165,8 @@ public class SynchronousChainedCombineDriver<T> extends ChainedDriver<T, T> {
 		} catch (Exception e) {
 			throw new ExceptionInChainedStubException(this.taskName, e);
 		}
+
+		this.outputCollector.close();
 	}
 
 	private void sortAndCombine() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/ReaderIterator.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/ReaderIterator.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/ReaderIterator.java
index c25a106..b02850c 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/ReaderIterator.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/ReaderIterator.java
@@ -16,7 +16,7 @@ package eu.stratosphere.pact.runtime.task.util;
 import java.io.IOException;
 
 import eu.stratosphere.api.common.typeutils.TypeSerializer;
-import eu.stratosphere.nephele.io.MutableReader;
+import eu.stratosphere.runtime.io.api.MutableReader;
 import eu.stratosphere.pact.runtime.plugable.DeserializationDelegate;
 import eu.stratosphere.util.MutableObjectIterator;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/RecordReaderIterator.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/RecordReaderIterator.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/RecordReaderIterator.java
index 7e3ce5f..b087c2c 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/RecordReaderIterator.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/RecordReaderIterator.java
@@ -15,7 +15,7 @@ package eu.stratosphere.pact.runtime.task.util;
 
 import java.io.IOException;
 
-import eu.stratosphere.nephele.io.MutableReader;
+import eu.stratosphere.runtime.io.api.MutableReader;
 import eu.stratosphere.types.Record;
 import eu.stratosphere.util.MutableObjectIterator;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/Buffer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/Buffer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/Buffer.java
new file mode 100644
index 0000000..c192cb9
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/Buffer.java
@@ -0,0 +1,93 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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;
+
+import eu.stratosphere.core.memory.MemorySegment;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class Buffer {
+
+	private final MemorySegment memorySegment;
+
+	private final BufferRecycler recycler;
+
+	// -----------------------------------------------------------------------------------------------------------------
+
+	private final AtomicInteger referenceCounter;
+
+	private int size;
+
+	// -----------------------------------------------------------------------------------------------------------------
+
+	public Buffer(MemorySegment memorySegment, int size, BufferRecycler recycler) {
+		this.memorySegment = memorySegment;
+		this.size = size;
+		this.recycler = recycler;
+
+		// we are the first, so we start with reference count of one
+		this.referenceCounter = new AtomicInteger(1);
+	}
+
+	/**
+	 * NOTE: Requires that the reference counter was increased prior to the constructor call!
+	 *
+	 * @param toDuplicate Buffer instance to duplicate
+	 */
+	private Buffer(Buffer toDuplicate) {
+		if (toDuplicate.referenceCounter.getAndIncrement() == 0) {
+			throw new IllegalStateException("Buffer was released before duplication.");
+		}
+		
+		this.memorySegment = toDuplicate.memorySegment;
+		this.size = toDuplicate.size;
+		this.recycler = toDuplicate.recycler;
+		this.referenceCounter = toDuplicate.referenceCounter;
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+
+	public MemorySegment getMemorySegment() {
+		return this.memorySegment;
+	}
+
+	public int size() {
+		return this.size;
+	}
+
+	public void limitSize(int size) {
+		if (size >= 0 && size <= this.memorySegment.size()) {
+			this.size = size;
+		} else {
+			throw new IllegalArgumentException();
+		}
+	}
+
+	public void recycleBuffer() {
+		if (this.referenceCounter.decrementAndGet() == 0) {
+			this.recycler.recycle(this.memorySegment);
+		}
+	}
+
+	public Buffer duplicate() {
+		return new Buffer(this);
+	}
+
+	public void copyToBuffer(Buffer destinationBuffer) {
+		if (size() > destinationBuffer.size()) {
+			throw new IllegalArgumentException("Destination buffer is too small to store content of source buffer.");
+		}
+
+		this.memorySegment.copyTo(0, destinationBuffer.memorySegment, 0, size);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/BufferRecycler.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/BufferRecycler.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/BufferRecycler.java
new file mode 100644
index 0000000..88f9edb
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/BufferRecycler.java
@@ -0,0 +1,26 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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;
+
+import eu.stratosphere.core.memory.MemorySegment;
+
+public interface BufferRecycler {
+
+	/**
+	 * Called by {@link eu.stratosphere.runtime.io.Buffer} to return a {@link MemorySegment} to its original buffer pool.
+	 *
+	 * @param buffer the segment to be recycled
+	 */
+	void recycle(MemorySegment buffer);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/AbstractRecordReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/AbstractRecordReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/AbstractRecordReader.java
new file mode 100644
index 0000000..d71695c
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/AbstractRecordReader.java
@@ -0,0 +1,98 @@
+/***********************************************************************************************************************
+ * 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.api;
+
+import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
+import eu.stratosphere.nephele.event.task.EventListener;
+import eu.stratosphere.nephele.event.task.EventNotificationManager;
+
+/**
+ * This is an abstract base class for a record reader, either dealing with mutable or immutable records,
+ * and dealing with reads from single gates (single end points) or multiple gates (union).
+ */
+public abstract class AbstractRecordReader implements ReaderBase {
+	
+	
+	private final EventNotificationManager eventHandler = new EventNotificationManager();
+	
+	private int numEventsUntilEndOfSuperstep = -1;
+	
+	private int endOfSuperstepEventsCount;
+	
+	// --------------------------------------------------------------------------------------------
+	
+	/**
+	 * Subscribes the listener object to receive events of the given type.
+	 * 
+	 * @param eventListener
+	 *        the listener object to register
+	 * @param eventType
+	 *        the type of event to register the listener for
+	 */
+	@Override
+	public void subscribeToEvent(EventListener eventListener, Class<? extends AbstractTaskEvent> eventType) {
+		this.eventHandler.subscribeToEvent(eventListener, eventType);
+	}
+
+	/**
+	 * Removes the subscription for events of the given type for the listener object.
+	 * 
+	 * @param eventListener The listener object to cancel the subscription for.
+	 * @param eventType The type of the event to cancel the subscription for.
+	 */
+	@Override
+	public void unsubscribeFromEvent(EventListener eventListener, Class<? extends AbstractTaskEvent> eventType) {
+		this.eventHandler.unsubscribeFromEvent(eventListener, eventType);
+	}
+	
+	
+	protected void handleEvent(AbstractTaskEvent evt) {
+		this.eventHandler.deliverEvent(evt);
+	}
+	
+	@Override
+	public void setIterative(int numEventsUntilEndOfSuperstep) {
+		this.numEventsUntilEndOfSuperstep = numEventsUntilEndOfSuperstep;
+	}
+
+	@Override
+	public void startNextSuperstep() {
+		if (this.numEventsUntilEndOfSuperstep == -1) {
+			throw new IllegalStateException("Called 'startNextSuperstep()' in a non-iterative reader.");
+		}
+		else if (endOfSuperstepEventsCount < numEventsUntilEndOfSuperstep) {
+			throw new IllegalStateException("Premature 'startNextSuperstep()'. Not yet reached the end-of-superstep.");
+		}
+		this.endOfSuperstepEventsCount = 0;
+	}
+	
+	@Override
+	public boolean hasReachedEndOfSuperstep() {
+		return endOfSuperstepEventsCount== numEventsUntilEndOfSuperstep;
+	}
+	
+	protected boolean incrementEndOfSuperstepEventAndCheck() {
+		if (numEventsUntilEndOfSuperstep == -1) {
+			throw new IllegalStateException("Received EndOfSuperstep event in a non-iterative reader.");
+		}
+		
+		endOfSuperstepEventsCount++;
+		
+		if (endOfSuperstepEventsCount > numEventsUntilEndOfSuperstep) {
+			throw new IllegalStateException("Received EndOfSuperstep events beyond the number to indicate the end of the superstep");
+		}
+		
+		return endOfSuperstepEventsCount== numEventsUntilEndOfSuperstep;
+	}
+}


[07/30] git commit: Replace custom Java NIO TCP/IP code with Netty 4 library

Posted by rm...@apache.org.
Replace custom Java NIO TCP/IP code with Netty 4 library


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

Branch: refs/heads/master
Commit: 4cd4a13415d609a2979c8fa3cf4b797c990ee8c2
Parents: 2db78a8
Author: uce <u....@fu-berlin.de>
Authored: Fri May 9 13:39:15 2014 +0200
Committer: StephanEwen <st...@tu-berlin.de>
Committed: Sat Jun 7 09:41:21 2014 +0200

----------------------------------------------------------------------
 .../configuration/ConfigConstants.java          |  58 +-
 stratosphere-runtime/pom.xml                    |   8 +-
 .../eu/stratosphere/nephele/AbstractID.java     |   6 +
 .../nephele/execution/CancelTaskException.java  |   3 -
 .../nephele/execution/Environment.java          |  76 +-
 .../execution/ExecutionStateTransition.java     |   2 +
 .../nephele/execution/RuntimeEnvironment.java   | 170 ++--
 .../nephele/executiongraph/ExecutionVertex.java |  36 +
 .../nephele/instance/AbstractInstance.java      |  16 +
 .../eu/stratosphere/nephele/jobgraph/JobID.java |  11 +-
 .../nephele/jobmanager/JobManager.java          |  34 +
 .../jobmanager/scheduler/RecoveryLogic.java     |   5 +-
 .../protocols/ExtendedManagementProtocol.java   |  13 +
 .../protocols/TaskOperationProtocol.java        |  12 +
 .../stratosphere/nephele/taskmanager/Task.java  |  93 +-
 .../nephele/taskmanager/TaskKillResult.java     |  44 +
 .../nephele/taskmanager/TaskManager.java        |  32 +-
 .../runtime/ExecutorThreadFactory.java          |  35 +
 .../task/AbstractIterativePactTask.java         |   9 +-
 .../iterative/task/IterationHeadPactTask.java   |   3 +-
 .../pact/runtime/shipping/OutputEmitter.java    |   1 -
 .../runtime/shipping/RecordOutputEmitter.java   |   3 -
 .../pact/runtime/task/DataSinkTask.java         |   3 +-
 .../pact/runtime/task/RegularPactTask.java      |   8 +-
 .../java/eu/stratosphere/runtime/io/Buffer.java |   5 +-
 .../runtime/io/channels/Channel.java            |   4 +-
 .../runtime/io/channels/ChannelID.java          |  10 +-
 .../runtime/io/channels/ChannelType.java        |   5 +
 .../runtime/io/channels/InputChannel.java       |  17 +-
 .../runtime/io/channels/OutputChannel.java      |   5 +-
 .../runtime/io/gates/InputGate.java             |   7 +-
 .../runtime/io/gates/OutputGate.java            |   1 -
 .../runtime/io/network/ChannelManager.java      |  50 +-
 .../runtime/io/network/Envelope.java            | 178 ++++
 .../runtime/io/network/EnvelopeDispatcher.java  |  46 +
 .../io/network/EnvelopeReceiverList.java        |  75 ++
 .../io/network/NetworkConnectionManager.java    | 176 ----
 .../runtime/io/network/SenderHintEvent.java     |   1 -
 .../BufferAvailabilityListener.java             |  17 +-
 .../network/bufferprovider/BufferProvider.java  |  18 +-
 .../bufferprovider/DiscardBufferPool.java       |  51 ++
 .../network/bufferprovider/LocalBufferPool.java |  26 +-
 .../bufferprovider/SerialSingleBufferPool.java  |  77 --
 .../runtime/io/network/envelope/Envelope.java   | 169 ----
 .../io/network/envelope/EnvelopeDispatcher.java |  46 -
 .../io/network/envelope/EnvelopeReader.java     | 212 -----
 .../network/envelope/EnvelopeReceiverList.java  |  75 --
 .../io/network/envelope/EnvelopeWriter.java     | 134 ---
 .../envelope/NoBufferAvailableException.java    |  53 --
 .../network/netty/InboundEnvelopeDecoder.java   | 344 ++++++++
 .../netty/InboundEnvelopeDispatcherHandler.java |  41 +
 .../network/netty/NettyConnectionManager.java   | 251 ++++++
 .../network/netty/OutboundConnectionQueue.java  |  94 ++
 .../network/netty/OutboundEnvelopeEncoder.java  |  65 ++
 .../io/network/tcp/IncomingConnection.java      | 115 ---
 .../network/tcp/IncomingConnectionThread.java   | 226 -----
 .../io/network/tcp/OutgoingConnection.java      | 529 ------------
 .../network/tcp/OutgoingConnectionThread.java   | 276 ------
 .../AdaptiveSpanningRecordDeserializer.java     |  32 +-
 .../io/serialization/DataInputDeserializer.java |  25 +-
 .../io/serialization/DataOutputSerializer.java  |   7 +-
 .../serialization/SpanningRecordSerializer.java |   7 +-
 .../nephele/util/TestBufferProvider.java        |   2 +-
 .../runtime/task/util/OutputEmitterTest.java    |   1 -
 .../task/util/RecordOutputEmitterTest.java      |   2 -
 .../pact/runtime/test/util/MockEnvironment.java |  11 +-
 .../envelope/EnvelopeReaderWriterTest.java      | 394 ---------
 .../netty/InboundEnvelopeDecoderTest.java       | 857 +++++++++++++++++++
 .../netty/NettyConnectionManagerTest.java       | 196 +++++
 .../netty/OutboundEnvelopeEncoderTest.java      |  97 +++
 .../KMeansIterativeNepheleITCase.java           |   3 +-
 .../clients/examples/LocalExecutorITCase.java   |   3 +-
 .../exampleJavaPrograms/WordCountITCase.java    |   5 +-
 .../ConnectedComponentsNepheleITCase.java       |  32 +-
 .../IterationWithChainingNepheleITCase.java     |  12 +-
 .../WordCountUnionReduceITCase.java             |   4 -
 .../test/runtime/NetworkStackNepheleITCase.java | 286 +++++++
 77 files changed, 3236 insertions(+), 2820 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/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 3b9ba3d..96c8965 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/configuration/ConfigConstants.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/configuration/ConfigConstants.java
@@ -34,7 +34,6 @@ public final class ConfigConstants {
 	 * 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 -------------------------------
 
@@ -98,6 +97,26 @@ public final class ConfigConstants {
 	 * The config parameter defining the size of the buffers used in the network stack.
 	 */
 	public static final String TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY = "taskmanager.network.bufferSizeInBytes";
+
+	/**
+	 * The number of incoming connection threads used in NettyConnectionManager for the ServerBootstrap.
+	 */
+	public static final String TASK_MANAGER_NETTY_NUM_IN_THREADS_KEY = "taskmanager.netty.numInThreads";
+
+	/**
+	 * The number of outgoing connection threads used in NettyConnectionManager for the Bootstrap.
+	 */
+	public static final String TASK_MANAGER_NETTY_NUM_OUT_THREADS_KEY = "taskmanager.netty.numOutThreads";
+
+	/**
+	 * The low water mark used in NettyConnectionManager for the Bootstrap.
+	 */
+	public static final String TASK_MANAGER_NETTY_LOW_WATER_MARK = "taskmanager.netty.lowWaterMark";
+
+	/**
+	 * The high water mark used in NettyConnectionManager for the Bootstrap.
+	 */
+	public static final String TASK_MANAGER_NETTY_HIGH_WATER_MARK = "taskmanager.netty.highWaterMark";
 	
 	/**
 	 * Parameter for the interval in which the RaskManager sends the periodic heart beat messages
@@ -134,10 +153,9 @@ public final class ConfigConstants {
 	 * The parameter defining the polling interval (in seconds) for the JobClient.
 	 */
 	public static final String JOBCLIENT_POLLING_INTERVAL_KEY = "jobclient.polling.interval";
-	
-	
+
 	// ------------------------ Hadoop Configuration ------------------------
-	
+
 	/**
 	 * Path to hdfs-defaul.xml file
 	 */
@@ -153,7 +171,6 @@ public final class ConfigConstants {
 	 */
 	public static final String PATH_HADOOP_CONFIG = "fs.hdfs.hadoopconf";
 	
-	
 	// ------------------------ File System Bahavior ------------------------
 
 	/**
@@ -251,11 +268,7 @@ public final class ConfigConstants {
 	public static final String STRATOSPHERE_BASE_DIR_PATH_KEY = "stratosphere.base.dir.path";
 	
 	public static final String STRATOSPHERE_JVM_OPTIONS = "env.java.opts";
-	
-	
 
-	
-	
 	// ------------------------------------------------------------------------
 	//                            Default Values
 	// ------------------------------------------------------------------------
@@ -318,7 +331,31 @@ public final class ConfigConstants {
 	 * Default size of network stack buffers.
 	 */
 	public static final int DEFAULT_TASK_MANAGER_NETWORK_BUFFER_SIZE = 32768;
-	
+
+	/**
+	 * Default number of incoming connection threads used in NettyConnectionManager for the ServerBootstrap. If set
+	 * to -1, NettyConnectionManager will pick a reasonable default depending on the number of cores of the machine.
+	 */
+	public static final int DEFAULT_TASK_MANAGER_NETTY_NUM_IN_THREADS = -1;
+
+	/**
+	 * Default number of outgoing connection threads used in NettyConnectionManager for the Bootstrap. If set
+	 * to -1, NettyConnectionManager will pick a reasonable default depending on the number of cores of the machine.
+	 */
+	public static final int DEFAULT_TASK_MANAGER_NETTY_NUM_OUT_THREADS = -1;
+
+	/**
+	 * Default low water mark used in NettyConnectionManager for the Bootstrap. If set to -1, NettyConnectionManager
+	 * will use half of the network buffer size as the low water mark.
+	 */
+	public static final int DEFAULT_TASK_MANAGER_NETTY_LOW_WATER_MARK = -1;
+
+	/**
+	 * Default high water mark used in NettyConnectionManager for the Bootstrap. If set to -1, NettyConnectionManager
+	 * will use the network buffer size as the high water mark.
+	 */
+	public static final int DEFAULT_TASK_MANAGER_NETTY_HIGH_WATER_MARK = -1;
+
 	/**
 	 * The default interval for TaskManager heart beats (2000 msecs).
 	 */
@@ -452,7 +489,6 @@ public final class ConfigConstants {
 	 */
 	public static final int DEFAULT_DEFAULT_INSTANCE_TYPE_INDEX = 1;
 
-	
 	// ------------------------------------------------------------------------
 	
 	/**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/pom.xml
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/pom.xml b/stratosphere-runtime/pom.xml
index bb5ca2a..074725b 100644
--- a/stratosphere-runtime/pom.xml
+++ b/stratosphere-runtime/pom.xml
@@ -52,7 +52,13 @@
 			<artifactId>aws-java-sdk</artifactId>
 			<version>1.2.1</version>
 		</dependency>
-		
+
+		<dependency>
+			<groupId>io.netty</groupId>
+			<artifactId>netty-all</artifactId>
+			<version>4.0.19.Final</version>
+		</dependency>
+
 		<dependency>
 			<groupId>eu.stratosphere</groupId>
 			<artifactId>stratosphere-java</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/AbstractID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/AbstractID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/AbstractID.java
index 476e22a..648c8dc 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/AbstractID.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/AbstractID.java
@@ -20,6 +20,7 @@ import java.nio.ByteBuffer;
 
 import eu.stratosphere.core.io.IOReadableWritable;
 import eu.stratosphere.util.StringUtils;
+import io.netty.buffer.ByteBuf;
 
 /**
  * A statistically unique identification number.
@@ -167,6 +168,11 @@ public class AbstractID implements IOReadableWritable {
 		buffer.putLong(this.upperPart);
 	}
 
+	public void writeTo(ByteBuf buf) {
+		buf.writeLong(this.lowerPart);
+		buf.writeLong(this.upperPart);
+	}
+
 	@Override
 	public String toString() {
 		final byte[] ba = new byte[SIZE];

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/CancelTaskException.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/CancelTaskException.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/CancelTaskException.java
index 6a09e89..aa196da 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/CancelTaskException.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/CancelTaskException.java
@@ -14,9 +14,6 @@
  **********************************************************************************************************************/
 package eu.stratosphere.nephele.execution;
 
-
-import eu.stratosphere.pact.runtime.task.chaining.ExceptionInChainedStubException;
-
 /**
  * Thrown to trigger a canceling of the executing task. Intended to cause a cancelled status, rather than a failed status.
  */

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/Environment.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/Environment.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/Environment.java
index e0bcc70..05a420c 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/Environment.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/Environment.java
@@ -16,17 +16,21 @@ package eu.stratosphere.nephele.execution;
 import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.core.fs.Path;
 import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.runtime.io.channels.ChannelID;
-import eu.stratosphere.runtime.io.gates.GateID;
-import eu.stratosphere.runtime.io.gates.InputGate;
-import eu.stratosphere.runtime.io.gates.OutputGate;
 import eu.stratosphere.nephele.jobgraph.JobGraph;
 import eu.stratosphere.nephele.jobgraph.JobID;
 import eu.stratosphere.nephele.protocols.AccumulatorProtocol;
 import eu.stratosphere.nephele.services.iomanager.IOManager;
 import eu.stratosphere.nephele.services.memorymanager.MemoryManager;
-import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
 import eu.stratosphere.nephele.template.InputSplitProvider;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.gates.GateID;
+import eu.stratosphere.runtime.io.gates.InputGate;
+import eu.stratosphere.runtime.io.gates.OutputGate;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.FutureTask;
 
 /**
  * The user code of every Nephele task runs inside an <code>Environment</code> object. The environment provides
@@ -38,114 +42,112 @@ public interface Environment {
 	 * Returns the ID of the job from the original job graph. It is used by the library cache manager to find the
 	 * required
 	 * libraries for executing the assigned Nephele task.
-	 * 
+	 *
 	 * @return the ID of the job from the original job graph
 	 */
 	JobID getJobID();
 
 	/**
 	 * Returns the task configuration object which was attached to the original JobVertex.
-	 * 
+	 *
 	 * @return the task configuration object which was attached to the original JobVertex.
 	 */
 	Configuration getTaskConfiguration();
 
 	/**
 	 * Returns the job configuration object which was attached to the original {@link JobGraph}.
-	 * 
+	 *
 	 * @return the job configuration object which was attached to the original {@link JobGraph}
 	 */
 	Configuration getJobConfiguration();
 
 	/**
 	 * Returns the current number of subtasks the respective task is split into.
-	 * 
+	 *
 	 * @return the current number of subtasks the respective task is split into
 	 */
 	int getCurrentNumberOfSubtasks();
 
 	/**
 	 * Returns the index of this subtask in the subtask group.
-	 * 
+	 *
 	 * @return the index of this subtask in the subtask group
 	 */
 	int getIndexInSubtaskGroup();
 
 	/**
 	 * Sends a notification that objects that a new user thread has been started to the execution observer.
-	 * 
-	 * @param userThread
-	 *        the user thread which has been started
+	 *
+	 * @param userThread the user thread which has been started
 	 */
 	void userThreadStarted(Thread userThread);
 
 	/**
 	 * Sends a notification that a user thread has finished to the execution observer.
-	 * 
-	 * @param userThread
-	 *        the user thread which has finished
+	 *
+	 * @param userThread the user thread which has finished
 	 */
 	void userThreadFinished(Thread userThread);
 
 	/**
 	 * Returns the input split provider assigned to this environment.
-	 * 
+	 *
 	 * @return the input split provider or <code>null</code> if no such provider has been assigned to this environment.
 	 */
 	InputSplitProvider getInputSplitProvider();
 
 	/**
 	 * Returns the current {@link IOManager}.
-	 * 
+	 *
 	 * @return the current {@link IOManager}.
 	 */
 	IOManager getIOManager();
 
 	/**
 	 * Returns the current {@link MemoryManager}.
-	 * 
+	 *
 	 * @return the current {@link MemoryManager}.
 	 */
 	MemoryManager getMemoryManager();
 
 	/**
 	 * Returns the name of the task running in this environment.
-	 * 
+	 *
 	 * @return the name of the task running in this environment
 	 */
 	String getTaskName();
 
 	/**
 	 * Returns the next unbound input gate ID or <code>null</code> if no such ID exists
-	 * 
+	 *
 	 * @return the next unbound input gate ID or <code>null</code> if no such ID exists
 	 */
 	GateID getNextUnboundInputGateID();
 
 	/**
 	 * Returns the number of output gates registered with this environment.
-	 * 
+	 *
 	 * @return the number of output gates registered with this environment
 	 */
 	int getNumberOfOutputGates();
 
 	/**
 	 * Returns the number of input gates registered with this environment.
-	 * 
+	 *
 	 * @return the number of input gates registered with this environment
 	 */
 	int getNumberOfInputGates();
 
 	/**
 	 * Returns the number of output channels attached to this environment.
-	 * 
+	 *
 	 * @return the number of output channels attached to this environment
 	 */
 	int getNumberOfOutputChannels();
 
 	/**
 	 * Returns the number of input channels attached to this environment.
-	 * 
+	 *
 	 * @return the number of input channels attached to this environment
 	 */
 	int getNumberOfInputChannels();
@@ -164,50 +166,48 @@ public interface Environment {
 
 	/**
 	 * Returns the IDs of all output channels connected to this environment.
-	 * 
+	 *
 	 * @return the IDs of all output channels connected to this environment
 	 */
 	Set<ChannelID> getOutputChannelIDs();
 
 	/**
 	 * Returns the IDs of all input channels connected to this environment.
-	 * 
+	 *
 	 * @return the IDs of all input channels connected to this environment
 	 */
 	Set<ChannelID> getInputChannelIDs();
 
 	/**
 	 * Returns the IDs of all output gates connected to this environment.
-	 * 
+	 *
 	 * @return the IDs of all output gates connected to this environment
 	 */
 	Set<GateID> getOutputGateIDs();
 
 	/**
 	 * Returns the IDs of all input gates connected to this environment.
-	 * 
+	 *
 	 * @return the IDs of all input gates connected to this environment
 	 */
 	Set<GateID> getInputGateIDs();
 
 	/**
 	 * Returns the IDs of all the output channels connected to the gate with the given ID.
-	 * 
-	 * @param gateID
-	 *        the gate ID
+	 *
+	 * @param gateID the gate ID
 	 * @return the IDs of all the output channels connected to the gate with the given ID
 	 */
 	Set<ChannelID> getOutputChannelIDsOfGate(GateID gateID);
 
 	/**
 	 * Returns the IDs of all the input channels connected to the gate with the given ID.
-	 * 
-	 * @param gateID
-	 *        the gate ID
+	 *
+	 * @param gateID the gate ID
 	 * @return the IDs of all the input channels connected to the gate with the given ID
 	 */
 	Set<ChannelID> getInputChannelIDsOfGate(GateID gateID);
-	
+
 	/**
 	 * Returns the proxy object for the accumulator protocol.
 	 */
@@ -215,11 +215,13 @@ public interface Environment {
 
 	/**
 	 * Returns the buffer provider for this environment.
-	 * <p>
+	 * <p/>
 	 * The returned buffer provider is used by the output side of the network stack.
 	 *
 	 * @return Buffer provider for the output side of the network stack
 	 * @see eu.stratosphere.runtime.io.api.RecordWriter
 	 */
 	BufferProvider getOutputBufferProvider();
+
+	Map<String, FutureTask<Path>> getCopyTask();
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java
index 55f036a..87674c8 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java
@@ -14,6 +14,8 @@
 package eu.stratosphere.nephele.execution;
 
 import static eu.stratosphere.nephele.execution.ExecutionState.FAILED;
+import static eu.stratosphere.nephele.execution.ExecutionState.CANCELED;
+import static eu.stratosphere.nephele.execution.ExecutionState.CANCELING;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/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 59787d2..29d6853 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
@@ -13,30 +13,11 @@
 
 package eu.stratosphere.nephele.execution;
 
-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;
-import java.util.Map;
-import java.util.Queue;
-import java.util.Set;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.FutureTask;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
 import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.core.fs.Path;
 import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.deployment.ChannelDeploymentDescriptor;
-import eu.stratosphere.nephele.deployment.GateDeploymentDescriptor;
 import eu.stratosphere.nephele.deployment.TaskDeploymentDescriptor;
 import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
-import eu.stratosphere.nephele.jobgraph.JobGraph;
 import eu.stratosphere.nephele.jobgraph.JobID;
 import eu.stratosphere.nephele.protocols.AccumulatorProtocol;
 import eu.stratosphere.nephele.services.iomanager.IOManager;
@@ -45,7 +26,6 @@ import eu.stratosphere.nephele.template.AbstractInvokable;
 import eu.stratosphere.nephele.template.InputSplitProvider;
 import eu.stratosphere.runtime.io.Buffer;
 import eu.stratosphere.runtime.io.channels.ChannelID;
-import eu.stratosphere.runtime.io.channels.ChannelType;
 import eu.stratosphere.runtime.io.channels.OutputChannel;
 import eu.stratosphere.runtime.io.gates.GateID;
 import eu.stratosphere.runtime.io.gates.InputGate;
@@ -56,12 +36,27 @@ 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;
+
+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;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.FutureTask;
 
 /**
  * The user code of every Nephele task runs inside a <code>RuntimeEnvironment</code> object. The environment provides
  * important services to the task. It keeps track of setting up the communication channels and provides access to input
  * splits, memory manager, etc.
- * <p>
+ * <p/>
  * This class is thread-safe.
  */
 public class RuntimeEnvironment implements Environment, BufferProvider, LocalBufferPoolOwner, Runnable {
@@ -141,7 +136,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 	 * The observer object for the task's execution.
 	 */
 	private volatile ExecutionObserver executionObserver = null;
-	
+
 	/**
 	 * The RPC proxy to report accumulators to JobManager
 	 */
@@ -164,26 +159,22 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 
 	private LocalBufferPool outputBufferPool;
 
+	private Map<String,FutureTask<Path>> cacheCopyTasks = new HashMap<String, FutureTask<Path>>();
+
 	/**
 	 * 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 {@link JobGraph}
-	 * @throws Exception
-	 *         thrown if an error occurs while instantiating the invokable class
+	 *
+	 * @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 {
+							final Class<? extends AbstractInvokable> invokableClass, final Configuration taskConfiguration,
+							final Configuration jobConfiguration) throws Exception {
 
 		this.jobID = jobID;
 		this.taskName = taskName;
@@ -203,23 +194,18 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 
 	/**
 	 * 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
-	 */
-	@SuppressWarnings({ "unchecked", "rawtypes" })
+	 *
+	 * @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
+	 */
+	@SuppressWarnings({"unchecked", "rawtypes"})
 	public RuntimeEnvironment(final TaskDeploymentDescriptor tdd,
-			final MemoryManager memoryManager, final IOManager ioManager,
-			final InputSplitProvider inputSplitProvider,
-			AccumulatorProtocol accumulatorProtocolProxy, Map<String, FutureTask<Path>> cpTasks) throws Exception {
+							final MemoryManager memoryManager, final IOManager ioManager,
+							final InputSplitProvider inputSplitProvider,
+							AccumulatorProtocol accumulatorProtocolProxy, Map<String, FutureTask<Path>> cpTasks) throws Exception {
 
 		this.jobID = tdd.getJobID();
 		this.taskName = tdd.getTaskName();
@@ -246,14 +232,14 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 
 		int numInputGates = tdd.getNumberOfInputGateDescriptors();
 
-		for(int i = 0; i < numInputGates; i++){
+		for (int i = 0; i < numInputGates; i++) {
 			this.inputGates.get(i).initializeChannels(tdd.getInputGateDescriptor(i));
 		}
 	}
 
 	/**
 	 * Returns the invokable object that represents the Nephele task.
-	 * 
+	 *
 	 * @return the invokable object that represents the Nephele task
 	 */
 	public AbstractInvokable getInvokable() {
@@ -272,7 +258,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 
 	@Override
 	public OutputGate createAndRegisterOutputGate() {
-		OutputGate gate = new OutputGate(getJobID(), new GateID(),  getNumberOfOutputGates());
+		OutputGate gate = new OutputGate(getJobID(), new GateID(), getNumberOfOutputGates());
 		this.outputGates.add(gate);
 
 		return gate;
@@ -318,7 +304,8 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 
 			if (this.executionObserver.isCanceled() || t instanceof CancelTaskException) {
 				changeExecutionState(ExecutionState.CANCELED, null);
-			} else {
+			}
+			else {
 				changeExecutionState(ExecutionState.FAILED, StringUtils.stringifyException(t));
 			}
 
@@ -347,7 +334,8 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 
 			if (this.executionObserver.isCanceled() || t instanceof CancelTaskException) {
 				changeExecutionState(ExecutionState.CANCELED, null);
-			} else {
+			}
+			else {
 				changeExecutionState(ExecutionState.FAILED, StringUtils.stringifyException(t));
 			}
 
@@ -400,9 +388,8 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 
 	/**
 	 * Returns the registered input gate with index <code>pos</code>.
-	 * 
-	 * @param pos
-	 *        the index of the input gate to return
+	 *
+	 * @param pos the index of the input gate to return
 	 * @return the input gate at index <code>pos</code> or <code>null</code> if no such index exists
 	 */
 	public InputGate<? extends IOReadableWritable> getInputGate(final int pos) {
@@ -415,9 +402,8 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 
 	/**
 	 * Returns the registered output gate with index <code>pos</code>.
-	 * 
-	 * @param index
-	 *        the index of the output gate to return
+	 *
+	 * @param index the index of the output gate to return
 	 * @return the output gate at index <code>pos</code> or <code>null</code> if no such index exists
 	 */
 	public OutputGate getOutputGate(int index) {
@@ -430,7 +416,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 
 	/**
 	 * Returns the thread which is assigned to execute the user code.
-	 * 
+	 *
 	 * @return the thread which is assigned to execute the user code
 	 */
 	public Thread getExecutingThread() {
@@ -439,7 +425,8 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 			if (this.executingThread == null) {
 				if (this.taskName == null) {
 					this.executingThread = new Thread(this);
-				} else {
+				}
+				else {
 					this.executingThread = new Thread(this, getTaskNameWithIndex());
 				}
 			}
@@ -450,11 +437,9 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 
 	/**
 	 * Blocks until all output channels are closed.
-	 * 
-	 * @throws IOException
-	 *         thrown if an error occurred while closing the output channels
-	 * @throws InterruptedException
-	 *         thrown if the thread waiting for the channels to be closed is interrupted
+	 *
+	 * @throws IOException          thrown if an error occurred while closing the output channels
+	 * @throws InterruptedException thrown if the thread waiting for the channels to be closed is interrupted
 	 */
 	private void waitForOutputChannelsToBeClosed() throws InterruptedException {
 		// Make sure, we leave this method with an InterruptedException when the task has been canceled
@@ -469,11 +454,9 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 
 	/**
 	 * Blocks until all input channels are closed.
-	 * 
-	 * @throws IOException
-	 *         thrown if an error occurred while closing the input channels
-	 * @throws InterruptedException
-	 *         thrown if the thread waiting for the channels to be closed is interrupted
+	 *
+	 * @throws IOException          thrown if an error occurred while closing the input channels
+	 * @throws InterruptedException thrown if the thread waiting for the channels to be closed is interrupted
 	 */
 	private void waitForInputChannelsToBeClosed() throws IOException, InterruptedException {
 		// Wait for disconnection of all output gates
@@ -494,7 +477,8 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 
 			if (allClosed) {
 				break;
-			} else {
+			}
+			else {
 				Thread.sleep(SLEEPINTERVAL);
 			}
 		}
@@ -564,7 +548,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 
 	/**
 	 * Returns the name of the task with its index in the subtask group and the total number of subtasks.
-	 * 
+	 *
 	 * @return the name of the task with its index in the subtask group and the total number of subtasks
 	 */
 	public String getTaskNameWithIndex() {
@@ -573,9 +557,8 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 
 	/**
 	 * Sets the execution observer for this environment.
-	 * 
-	 * @param executionObserver
-	 *        the execution observer for this environment
+	 *
+	 * @param executionObserver the execution observer for this environment
 	 */
 	public void setExecutionObserver(final ExecutionObserver executionObserver) {
 		this.executionObserver = executionObserver;
@@ -616,7 +599,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 
 	@Override
 	public Set<ChannelID> getOutputChannelIDs() {
-		Set<ChannelID> ids= new HashSet<ChannelID>();
+		Set<ChannelID> ids = new HashSet<ChannelID>();
 
 		for (OutputGate gate : this.outputGates) {
 			for (OutputChannel channel : gate.channels()) {
@@ -726,38 +709,47 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 	public List<InputGate<? extends IOReadableWritable>> inputGates() {
 		return this.inputGates;
 	}
-	
+
 	@Override
 	public AccumulatorProtocol getAccumulatorProtocolProxy() {
 		return accumulatorProtocolProxy;
 	}
 
+	public void addCopyTaskForCacheFile(String name, FutureTask<Path> copyTask) {
+		this.cacheCopyTasks.put(name, copyTask);
+	}
+
+	@Override
+	public Map<String, FutureTask<Path>> getCopyTask() {
+		return this.cacheCopyTasks;
+	}
+
 	@Override
 	public BufferProvider getOutputBufferProvider() {
 		return this;
 	}
-	
+
 	// -----------------------------------------------------------------------------------------------------------------
 	//                                            BufferProvider methods
 	// -----------------------------------------------------------------------------------------------------------------
-	
+
 	@Override
 	public Buffer requestBuffer(int minBufferSize) throws IOException {
 		return this.outputBufferPool.requestBuffer(minBufferSize);
 	}
-	
+
 	@Override
 	public Buffer requestBufferBlocking(int minBufferSize) throws IOException, InterruptedException {
 		return this.outputBufferPool.requestBufferBlocking(minBufferSize);
 	}
-	
+
 	@Override
 	public int getBufferSize() {
 		return this.outputBufferPool.getBufferSize();
 	}
-	
+
 	@Override
-	public boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
+	public BufferAvailabilityRegistration registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
 		return this.outputBufferPool.registerBufferAvailabilityListener(listener);
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/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 57ff073..8e9395a 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
@@ -24,6 +24,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 
+import eu.stratosphere.nephele.taskmanager.TaskKillResult;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
@@ -691,6 +692,41 @@ public final class ExecutionVertex {
 	}
 
 	/**
+	 * Kills and removes the task represented by this vertex from the instance it is currently running on. If the
+	 * corresponding task is not in the state <code>RUNNING</code>, this call will be ignored. If the call has been
+	 * executed
+	 * successfully, the task will change the state <code>FAILED</code>.
+	 *
+	 * @return the result of the task kill attempt
+	 */
+	public TaskKillResult killTask() {
+
+		final ExecutionState state = this.executionState.get();
+
+		if (state != ExecutionState.RUNNING) {
+			final TaskKillResult result = new TaskKillResult(getID(), AbstractTaskResult.ReturnCode.ILLEGAL_STATE);
+			result.setDescription("Vertex " + this.toString() + " is in state " + state);
+			return result;
+		}
+
+		final AllocatedResource ar = this.allocatedResource.get();
+
+		if (ar == null) {
+			final TaskKillResult result = new TaskKillResult(getID(), AbstractTaskResult.ReturnCode.NO_INSTANCE);
+			result.setDescription("Assigned instance of vertex " + this.toString() + " is null!");
+			return result;
+		}
+
+		try {
+			return ar.getInstance().killTask(this.vertexID);
+		} catch (IOException e) {
+			final TaskKillResult result = new TaskKillResult(getID(), AbstractTaskResult.ReturnCode.IPC_ERROR);
+			result.setDescription(StringUtils.stringifyException(e));
+			return result;
+		}
+	}
+
+	/**
 	 * Cancels and removes the task represented by this vertex
 	 * from the instance it is currently running on. If the task
 	 * is not currently running, its execution state is simply

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/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
index 50e0e7f..56b4eae 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java
@@ -24,6 +24,7 @@ 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;
@@ -205,6 +206,21 @@ public abstract class AbstractInstance extends NetworkNode {
 		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) {
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobID.java
index 2ec2ed6..e32df61 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobID.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobID.java
@@ -13,11 +13,10 @@
 
 package eu.stratosphere.nephele.jobgraph;
 
-import java.nio.ByteBuffer;
+import eu.stratosphere.nephele.AbstractID;
 
 import javax.xml.bind.DatatypeConverter;
-
-import eu.stratosphere.nephele.AbstractID;
+import java.nio.ByteBuffer;
 
 public final class JobID extends AbstractID {
 
@@ -44,6 +43,12 @@ public final class JobID extends AbstractID {
 		return new JobID(bytes);
 	}
 
+	public static JobID fromByteBuffer(ByteBuffer buf) {
+		long lower = buf.getLong();
+		long upper = buf.getLong();
+		return new JobID(lower, upper);
+	}
+
 	public static JobID fromByteBuffer(ByteBuffer buf, int offset) {
 		long lower = buf.getLong(offset);
 		long upper = buf.getLong(offset + 8);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/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 846ca2e..3ae9f3b 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
@@ -31,6 +31,8 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+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;
@@ -893,6 +895,38 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 	}
 
 	@Override
+	public void killTask(final JobID jobID, final ManagementVertexID id) throws IOException {
+
+		final ExecutionGraph eg = this.scheduler.getExecutionGraphByID(jobID);
+		if (eg == null) {
+			LOG.error("Cannot find execution graph for job " + jobID);
+			return;
+		}
+
+		final ExecutionVertex vertex = eg.getVertexByID(ExecutionVertexID.fromManagementVertexID(id));
+		if (vertex == null) {
+			LOG.error("Cannot find execution vertex with ID " + id);
+			return;
+		}
+
+		LOG.info("Killing task " + vertex + " of job " + jobID);
+
+		final Runnable runnable = new Runnable() {
+
+			@Override
+			public void run() {
+
+				final TaskKillResult result = vertex.killTask();
+				if (result.getReturnCode() != AbstractTaskResult.ReturnCode.SUCCESS) {
+					LOG.error(result.getDescription());
+				}
+			}
+		};
+
+		eg.executeCommand(runnable);
+	}
+
+	@Override
 	public void killInstance(final StringRecord instanceName) throws IOException {
 
 		final AbstractInstance instance = this.instanceManager.getInstanceByName(instanceName.toString());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/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
index 762b494..e369613 100644
--- 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
@@ -21,6 +21,7 @@ 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;
 
@@ -87,8 +88,8 @@ public final class RecoveryLogic {
 				verticesToBeRestarted.put(vertex.getID(), vertex);
 				final TaskCancelResult cancelResult = vertex.cancelTask();
 
-				if (cancelResult.getReturnCode() != ReturnCode.SUCCESS
-						&& cancelResult.getReturnCode() != ReturnCode.TASK_NOT_FOUND) {
+				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());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/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 461c797..59ec15d 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
@@ -24,6 +24,7 @@ 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;
 import eu.stratosphere.nephele.topology.NetworkTopology;
 
 /**
@@ -81,6 +82,18 @@ public interface ExtendedManagementProtocol extends JobManagementProtocol {
 	List<AbstractEvent> getEvents(JobID jobID) throws IOException;
 
 	/**
+	 * Kills the task with the given vertex ID.
+	 *
+	 * @param jobID
+	 *        the ID of the job the vertex to be killed belongs to
+	 * @param id
+	 *        the vertex ID which identified the task be killed
+	 * @throws IOException
+	 *         thrown if an error occurs while transmitting the kill request
+	 */
+	void killTask(JobID jobID, ManagementVertexID id) throws IOException;
+
+	/**
 	 * Kills the instance with the given name (i.e. shuts down its task manager).
 	 * 
 	 * @param instanceName

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java
index 19522db..93d8cdf 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java
@@ -23,6 +23,7 @@ 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.taskmanager.TaskCancelResult;
 import eu.stratosphere.nephele.taskmanager.TaskSubmissionResult;
@@ -58,6 +59,17 @@ public interface TaskOperationProtocol extends VersionedProtocol {
 	TaskCancelResult cancelTask(ExecutionVertexID id) throws IOException;
 
 	/**
+	 * Advises the task manager to kill the task with the given ID.
+	 *
+	 * @param id
+	 *        the ID of the task to kill
+	 * @return the result of the task kill attempt
+	 * @throws IOException
+	 *         thrown if an error occurs during this remote procedure call
+	 */
+	TaskKillResult killTask(ExecutionVertexID id) throws IOException;
+
+	/**
 	 * Queries the task manager about the cache status of the libraries stated in the {@link LibraryCacheProfileRequest}
 	 * object.
 	 * 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/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 06eec0c..825eae1 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
@@ -25,6 +25,7 @@ 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;
 
@@ -57,9 +58,7 @@ public final class Task implements ExecutionObserver {
 
 	private Queue<ExecutionListener> registeredListeners = new ConcurrentLinkedQueue<ExecutionListener>();
 
-	public Task(final ExecutionVertexID vertexID, final RuntimeEnvironment environment,
-					   final TaskManager taskManager) {
-
+	public Task(ExecutionVertexID vertexID, final RuntimeEnvironment environment, TaskManager taskManager) {
 		this.vertexID = vertexID;
 		this.environment = environment;
 		this.taskManager = taskManager;
@@ -102,58 +101,51 @@ public final class Task implements ExecutionObserver {
 		executionStateChanged(ExecutionState.FAILED, "Execution thread died unexpectedly");
 	}
 
-	/**
-	 * Checks if the state of the thread which is associated with this task is <code>TERMINATED</code>.
-	 * 
-	 * @return <code>true</code> if the state of this thread which is associated with this task is
-	 *         <code>TERMINATED</code>, <code>false</code> otherwise
-	 */
-	public boolean isTerminated() {
-		final Thread executingThread = this.environment.getExecutingThread();
-		if (executingThread.getState() == Thread.State.TERMINATED) {
-			return true;
-		}
-
-		return false;
+	public void cancelExecution() {
+		cancelOrKillExecution(true);
 	}
 
-	/**
-	 * Starts the execution of this task.
-	 */
-	public void startExecution() {
-
-		final Thread thread = this.environment.getExecutingThread();
-		thread.start();
+	public void killExecution() {
+		cancelOrKillExecution(false);
 	}
 
 	/**
-	 * Cancels the execution of the task (i.e. interrupts the execution thread).
+	 * Cancels or kills the task.
+	 *
+	 * @param cancel <code>true/code> if the task shall be canceled, <code>false</code> if it shall be killed
 	 */
-	public void cancelExecution() {
+	private void cancelOrKillExecution(boolean cancel) {
 		final Thread executingThread = this.environment.getExecutingThread();
 
 		if (executingThread == null) {
 			return;
 		}
 
-		LOG.info("Canceling " + this.environment.getTaskNameWithIndex());
+		if (this.executionState != ExecutionState.RUNNING && this.executionState != ExecutionState.FINISHING) {
+			return;
+		}
+
+		LOG.info((cancel ? "Canceling " : "Killing ") + this.environment.getTaskNameWithIndex());
 
-		this.isCanceled = true;
-		// Change state
-		executionStateChanged(ExecutionState.CANCELING, null);
+		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();
+			// 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));
 			}
-		} catch (Throwable e) {
-			LOG.error("Error while canceling task", e);
 		}
 
 		// Continuously interrupt the user thread until it changed to state CANCELED
 		while (true) {
+
 			executingThread.interrupt();
 
 			if (!executingThread.isAlive()) {
@@ -168,12 +160,36 @@ public final class Task implements ExecutionObserver {
 				break;
 			}
 
-			if (LOG.isDebugEnabled())
-				LOG.debug("Sending repeated canceling  signal to " +
+			if (LOG.isDebugEnabled()) {
+				LOG.debug("Sending repeated " + (cancel == true ? "canceling" : "killing") + " signal to " +
 						this.environment.getTaskName() + " with state " + this.executionState);
+			}
 		}
 	}
 
+	/**
+	 * Checks if the state of the thread which is associated with this task is <code>TERMINATED</code>.
+	 * 
+	 * @return <code>true</code> if the state of this thread which is associated with this task is
+	 *         <code>TERMINATED</code>, <code>false</code> otherwise
+	 */
+	public boolean isTerminated() {
+		final Thread executingThread = this.environment.getExecutingThread();
+		if (executingThread.getState() == Thread.State.TERMINATED) {
+			return true;
+		}
+
+		return false;
+	}
+
+	/**
+	 * Starts the execution of this task.
+	 */
+	public void startExecution() {
+
+		final Thread thread = this.environment.getExecutingThread();
+		thread.start();
+	}
 
 	/**
 	 * Registers the task manager profiler with the task.
@@ -324,5 +340,4 @@ public final class Task implements ExecutionObserver {
 
 		return this.environment;
 	}
-
 }

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

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/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 5240fc8..3b478cf 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
@@ -126,7 +126,7 @@ public class TaskManager implements TaskOperationProtocol {
 	private final InstanceConnectionInfo localInstanceConnectionInfo;
 
 	/**
-	 * The instance of the {@link eu.stratosphere.nephele.taskmanager.io.bytebuffered.ChannelManager} which is responsible for
+	 * The instance of the {@link ChannelManager} which is responsible for
 	 * setting up and cleaning up the byte buffered channels of the tasks.
 	 */
 	private final ChannelManager channelManager;
@@ -286,16 +286,32 @@ public class TaskManager implements TaskOperationProtocol {
 				ConfigConstants.TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY,
 				ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_BUFFER_SIZE);
 
-		// Initialize the byte buffered channel manager
-		ChannelManager channelManager = null;
+		int numInThreads = GlobalConfiguration.getInteger(
+				ConfigConstants.TASK_MANAGER_NETTY_NUM_IN_THREADS_KEY,
+				ConfigConstants.DEFAULT_TASK_MANAGER_NETTY_NUM_IN_THREADS);
+
+		int numOutThreads = GlobalConfiguration.getInteger(
+				ConfigConstants.TASK_MANAGER_NETTY_NUM_OUT_THREADS_KEY,
+				ConfigConstants.DEFAULT_TASK_MANAGER_NETTY_NUM_OUT_THREADS);
+
+		int lowWaterMark = GlobalConfiguration.getInteger(
+				ConfigConstants.TASK_MANAGER_NETTY_LOW_WATER_MARK,
+				ConfigConstants.DEFAULT_TASK_MANAGER_NETTY_LOW_WATER_MARK);
+
+		int highWaterMark = GlobalConfiguration.getInteger(
+				ConfigConstants.TASK_MANAGER_NETTY_HIGH_WATER_MARK,
+				ConfigConstants.DEFAULT_TASK_MANAGER_NETTY_HIGH_WATER_MARK);
+
+		// Initialize the channel manager
 		try {
-			channelManager = new ChannelManager(this.lookupService, this.localInstanceConnectionInfo, numBuffers, bufferSize);
+			this.channelManager = new ChannelManager(
+					this.lookupService, this.localInstanceConnectionInfo,
+					numBuffers, bufferSize, numInThreads, numOutThreads, lowWaterMark, highWaterMark);
 		} catch (IOException ioe) {
 			LOG.error(StringUtils.stringifyException(ioe));
 			throw new Exception("Failed to instantiate Byte-buffered channel manager. " + ioe.getMessage(), ioe);
 		}
-		this.channelManager = channelManager;
-		
+
 		{
 			HardwareDescription resources = HardwareDescriptionFactory.extractFromSystem();
 
@@ -933,7 +949,7 @@ public class TaskManager implements TaskOperationProtocol {
 	}
 
 	@Override
-	public void logBufferUtilization() throws IOException {
+	public void logBufferUtilization() {
 
 		this.channelManager.logBufferUtilization();
 	}
@@ -956,7 +972,7 @@ public class TaskManager implements TaskOperationProtocol {
 
 	@Override
 	public void invalidateLookupCacheEntries(final Set<ChannelID> channelIDs) throws IOException {
-		this.byteBufferedChannelManager.invalidateLookupCacheEntries(channelIDs);
+		this.channelManager.invalidateLookupCacheEntries(channelIDs);
 	}
 
 	/**

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

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/AbstractIterativePactTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/AbstractIterativePactTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/AbstractIterativePactTask.java
index 674f270..79b9c83 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/AbstractIterativePactTask.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/AbstractIterativePactTask.java
@@ -21,7 +21,6 @@ import eu.stratosphere.api.common.typeutils.TypeSerializer;
 import eu.stratosphere.api.common.typeutils.TypeSerializerFactory;
 import eu.stratosphere.core.memory.DataOutputView;
 import eu.stratosphere.nephele.execution.Environment;
-import eu.stratosphere.nephele.io.MutableReader;
 import eu.stratosphere.pact.runtime.hash.CompactingHashTable;
 import eu.stratosphere.pact.runtime.iterative.concurrent.BlockingBackChannel;
 import eu.stratosphere.pact.runtime.iterative.concurrent.BlockingBackChannelBroker;
@@ -36,6 +35,7 @@ import eu.stratosphere.pact.runtime.task.RegularPactTask;
 import eu.stratosphere.pact.runtime.task.ResettablePactDriver;
 import eu.stratosphere.pact.runtime.task.util.TaskConfig;
 import eu.stratosphere.pact.runtime.udf.RuntimeUDFContext;
+import eu.stratosphere.runtime.io.api.MutableReader;
 import eu.stratosphere.types.Value;
 import eu.stratosphere.util.Collector;
 import eu.stratosphere.util.InstantiationUtil;
@@ -322,14 +322,15 @@ public abstract class AbstractIterativePactTask<S extends Function, OT> extends
 	 * <p/>
 	 * This collector is used by {@link IterationIntermediatePactTask} or {@link IterationTailPactTask} to update the
 	 * solution set of workset iterations. Depending on the task configuration, either a fast (non-probing)
-	 * {@link SolutionSetFastUpdateOutputCollector} or normal (re-probing) {@link SolutionSetUpdateOutputCollector}
-	 * is created.
+	 * {@link eu.stratosphere.pact.runtime.iterative.io.SolutionSetFastUpdateOutputCollector} or normal (re-probing)
+	 * {@link SolutionSetUpdateOutputCollector} is created.
 	 * <p/>
 	 * If a non-null delegate is given, the new {@link Collector} will write back to the solution set and also call
 	 * collect(T) of the delegate.
 	 *
 	 * @param delegate null -OR- a delegate collector to be called by the newly created collector
-	 * @return a new {@link SolutionSetFastUpdateOutputCollector} or {@link SolutionSetUpdateOutputCollector}
+	 * @return a new {@link eu.stratosphere.pact.runtime.iterative.io.SolutionSetFastUpdateOutputCollector} or
+	 * {@link SolutionSetUpdateOutputCollector}
 	 */
 	protected Collector<OT> createSolutionSetUpdateOutputCollector(Collector<OT> delegate) {
 		Broker<CompactingHashTable<?>> solutionSetBroker = SolutionSetBroker.instance();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/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 c39e3ef..89571c4 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
@@ -17,6 +17,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
+import eu.stratosphere.runtime.io.api.BufferWriter;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
@@ -27,8 +28,6 @@ import eu.stratosphere.api.common.typeutils.TypeSerializer;
 import eu.stratosphere.api.common.typeutils.TypeSerializerFactory;
 import eu.stratosphere.core.memory.DataInputView;
 import eu.stratosphere.core.memory.MemorySegment;
-import eu.stratosphere.nephele.io.AbstractRecordWriter;
-import eu.stratosphere.nephele.io.RecordWriter;
 import eu.stratosphere.pact.runtime.hash.CompactingHashTable;
 import eu.stratosphere.pact.runtime.io.InputViewIterator;
 import eu.stratosphere.pact.runtime.iterative.concurrent.BlockingBackChannel;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/OutputEmitter.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/OutputEmitter.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/OutputEmitter.java
index 46e3249..6491749 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/OutputEmitter.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/OutputEmitter.java
@@ -18,7 +18,6 @@ import eu.stratosphere.api.common.typeutils.TypeComparator;
 import eu.stratosphere.runtime.io.api.ChannelSelector;
 import eu.stratosphere.pact.runtime.plugable.SerializationDelegate;
 
-
 public class OutputEmitter<T> implements ChannelSelector<SerializationDelegate<T>> {
 	
 	private final ShipStrategyType strategy;		// the shipping strategy used by this output emitter

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/RecordOutputEmitter.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/RecordOutputEmitter.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/RecordOutputEmitter.java
index ba352eb..7fe35b4 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/RecordOutputEmitter.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/RecordOutputEmitter.java
@@ -15,13 +15,10 @@ package eu.stratosphere.pact.runtime.shipping;
 
 import eu.stratosphere.api.common.distributions.DataDistribution;
 import eu.stratosphere.api.common.typeutils.TypeComparator;
-import eu.stratosphere.nephele.io.ChannelSelector;
 import eu.stratosphere.runtime.io.api.ChannelSelector;
-import eu.stratosphere.pact.runtime.plugable.pactrecord.RecordComparator;
 import eu.stratosphere.types.Key;
 import eu.stratosphere.types.Record;
 
-
 public class RecordOutputEmitter implements ChannelSelector<Record> {
 	
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/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 cb3e782..638a7aa 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
@@ -193,8 +193,9 @@ public class DataSinkTask<IT> extends AbstractOutputTask {
 			}
 			// drop, if the task was canceled
 			else if (!this.taskCanceled) {
-				if (LOG.isErrorEnabled())
+				if (LOG.isErrorEnabled()) {
 					LOG.error(getLogString("Error in user code: " + ex.getMessage()), ex);
+				}
 				throw ex;
 			}
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/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 b01799a..92c4648 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
@@ -22,9 +22,7 @@ import eu.stratosphere.api.common.typeutils.TypeComparator;
 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.ConfigConstants;
 import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.configuration.GlobalConfiguration;
 import eu.stratosphere.core.io.IOReadableWritable;
 import eu.stratosphere.nephele.execution.CancelTaskException;
 import eu.stratosphere.nephele.execution.Environment;
@@ -280,8 +278,9 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 	@Override
 	public void invoke() throws Exception {
 
-		if (LOG.isDebugEnabled())
+		if (LOG.isDebugEnabled()) {
 			LOG.debug(formatLogString("Start task code."));
+		}
 
 		// whatever happens in this scope, make sure that the local strategies are cleaned up!
 		// note that the initialization of the local strategies is in the try-finally block as well,
@@ -390,8 +389,9 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
 	public void cancel() throws Exception {
 		this.running = false;
 
-		if (LOG.isDebugEnabled())
+		if (LOG.isDebugEnabled()) {
 			LOG.debug(formatLogString("Cancelling task code"));
+		}
 
 		try {
 			if (this.driver != null) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/Buffer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/Buffer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/Buffer.java
index c192cb9..f3f51f1 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/Buffer.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/Buffer.java
@@ -40,8 +40,6 @@ public class Buffer {
 	}
 
 	/**
-	 * NOTE: Requires that the reference counter was increased prior to the constructor call!
-	 *
 	 * @param toDuplicate Buffer instance to duplicate
 	 */
 	private Buffer(Buffer toDuplicate) {
@@ -74,7 +72,8 @@ public class Buffer {
 	}
 
 	public void recycleBuffer() {
-		if (this.referenceCounter.decrementAndGet() == 0) {
+		int refCount = this.referenceCounter.decrementAndGet();
+		if (refCount == 0) {
 			this.recycler.recycle(this.memorySegment);
 		}
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/Channel.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/Channel.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/Channel.java
index dfa5d5e..17fff02 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/Channel.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/Channel.java
@@ -17,8 +17,8 @@ import java.io.IOException;
 
 import eu.stratosphere.nephele.event.task.AbstractEvent;
 import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.runtime.io.network.envelope.Envelope;
-import eu.stratosphere.runtime.io.network.envelope.EnvelopeDispatcher;
+import eu.stratosphere.runtime.io.network.Envelope;
+import eu.stratosphere.runtime.io.network.EnvelopeDispatcher;
 
 /**
  * The base class for channel objects.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelID.java
index 66be7de..ba78c01 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelID.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelID.java
@@ -13,10 +13,10 @@
 
 package eu.stratosphere.runtime.io.channels;
 
-import java.nio.ByteBuffer;
-
 import eu.stratosphere.nephele.AbstractID;
 
+import java.nio.ByteBuffer;
+
 public class ChannelID extends AbstractID {
 
 	public ChannelID() {
@@ -31,6 +31,12 @@ public class ChannelID extends AbstractID {
 		super(bytes);
 	}
 
+	public static ChannelID fromByteBuffer(ByteBuffer buf) {
+		long lower = buf.getLong();
+		long upper = buf.getLong();
+		return new ChannelID(lower, upper);
+	}
+
 	public static ChannelID fromByteBuffer(ByteBuffer buf, int offset) {
 		long lower = buf.getLong(offset);
 		long upper = buf.getLong(offset + 8);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelType.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelType.java
index 5d5b53d..3007489 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelType.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelType.java
@@ -24,3 +24,8 @@ public enum ChannelType {
 	/** In-memory channels */
 	IN_MEMORY
 }
+
+
+
+
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/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 860141d..6122c36 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
@@ -21,7 +21,7 @@ import eu.stratosphere.runtime.io.Buffer;
 import eu.stratosphere.runtime.io.gates.InputChannelResult;
 import eu.stratosphere.runtime.io.network.bufferprovider.BufferAvailabilityListener;
 import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
-import eu.stratosphere.runtime.io.network.envelope.Envelope;
+import eu.stratosphere.runtime.io.network.Envelope;
 import eu.stratosphere.runtime.io.gates.InputGate;
 import eu.stratosphere.runtime.io.serialization.AdaptiveSpanningRecordDeserializer;
 import eu.stratosphere.runtime.io.serialization.RecordDeserializer;
@@ -30,7 +30,6 @@ import eu.stratosphere.runtime.io.serialization.RecordDeserializer.Deserializati
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
-import java.io.EOFException;
 import java.io.IOException;
 import java.util.ArrayDeque;
 import java.util.Arrays;
@@ -83,6 +82,8 @@ public class InputChannel<T extends IOReadableWritable> extends Channel implemen
 
 	private int lastReceivedEnvelope = -1;
 
+	private ChannelID lastSourceID = null;
+
 	private boolean destroyCalled = false;
 
 	// ----------------------
@@ -104,7 +105,7 @@ public class InputChannel<T extends IOReadableWritable> extends Channel implemen
 	 *        the ID of the channel this channel is connected to
 	 */
 	public InputChannel(final InputGate<T> inputGate, final int channelIndex, final ChannelID channelID,
-						   final ChannelID connectedChannelID, ChannelType type) {
+						final ChannelID connectedChannelID, ChannelType type) {
 		super(channelIndex, channelID, connectedChannelID, type);
 		this.inputGate = inputGate;
 		this.deserializer = new AdaptiveSpanningRecordDeserializer<T>();
@@ -158,6 +159,9 @@ public class InputChannel<T extends IOReadableWritable> extends Channel implemen
 
 //	public abstract AbstractTaskEvent getCurrentEvent();
 
+	private DeserializationResult lastDeserializationResult;
+
+
 	public InputChannelResult readRecord(T target) throws IOException {
 		if (this.dataBuffer == null) {
 			if (isClosed()) {
@@ -176,7 +180,7 @@ public class InputChannel<T extends IOReadableWritable> extends Channel implemen
 			{
 				// sanity check: an event may only come after a complete record.
 				if (this.deserializer.hasUnfinishedData()) {
-					throw new IOException("Channel received an event before completing the current partial record.");
+					throw new IllegalStateException("Channel received an event before completing the current partial record.");
 				}
 
 				AbstractEvent evt = boe.getEvent();
@@ -202,8 +206,8 @@ public class InputChannel<T extends IOReadableWritable> extends Channel implemen
 			}
 		}
 
-
 		DeserializationResult deserializationResult = this.deserializer.getNextRecord(target);
+		this.lastDeserializationResult = deserializationResult;
 
 		if (deserializationResult.isBufferConsumed()) {
 			releasedConsumedReadBuffer(this.dataBuffer);
@@ -348,6 +352,7 @@ 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) {
@@ -432,7 +437,7 @@ public class InputChannel<T extends IOReadableWritable> extends Channel implemen
 	}
 
 	@Override
-	public boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
+	public BufferAvailabilityRegistration registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
 		return this.inputGate.registerBufferAvailabilityListener(listener);
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/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 7ca916c..f4fed65 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
@@ -17,7 +17,7 @@ import eu.stratosphere.nephele.event.task.AbstractEvent;
 import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
 import eu.stratosphere.nephele.jobgraph.JobID;
 import eu.stratosphere.runtime.io.Buffer;
-import eu.stratosphere.runtime.io.network.envelope.Envelope;
+import eu.stratosphere.runtime.io.network.Envelope;
 import eu.stratosphere.runtime.io.gates.OutputGate;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -107,7 +107,6 @@ public class OutputChannel extends Channel {
 					this.receiverCloseRequested = true;
 					this.closeLock.notifyAll();
 				}
-				LOG.debug("OutputChannel received close event from target.");
 			} 
 			else if (event instanceof AbstractTaskEvent) {
 				if (LOG.isDebugEnabled()) {
@@ -165,7 +164,7 @@ public class OutputChannel extends Channel {
 	
 	private void checkStatus() throws IOException {
 		if (this.senderCloseRequested) {
-			throw new IllegalStateException(String.format("Channel %s already requested to be closed.", getID()));
+			throw new IllegalStateException(String.format("Channel %s already requested to be closed", getID()));
 		}
 		if (this.receiverCloseRequested) {
 			throw new ReceiverAlreadyClosedException();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/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 bdac7a2..c623220 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
@@ -14,9 +14,6 @@
 package eu.stratosphere.runtime.io.gates;
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.atomic.AtomicReference;
@@ -24,7 +21,6 @@ import java.util.concurrent.atomic.AtomicReference;
 import eu.stratosphere.nephele.deployment.ChannelDeploymentDescriptor;
 import eu.stratosphere.nephele.deployment.GateDeploymentDescriptor;
 import eu.stratosphere.runtime.io.Buffer;
-import eu.stratosphere.runtime.io.channels.ChannelType;
 import eu.stratosphere.runtime.io.network.bufferprovider.BufferAvailabilityListener;
 import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
 import eu.stratosphere.runtime.io.network.bufferprovider.GlobalBufferPool;
@@ -38,7 +34,6 @@ import eu.stratosphere.nephele.event.task.AbstractEvent;
 import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
 import eu.stratosphere.nephele.execution.Environment;
 import eu.stratosphere.runtime.io.channels.InputChannel;
-import eu.stratosphere.runtime.io.channels.ChannelID;
 import eu.stratosphere.nephele.jobgraph.JobID;
 
 /**
@@ -378,7 +373,7 @@ public class InputGate<T extends IOReadableWritable> extends Gate<T> implements
 	}
 
 	@Override
-	public boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
+	public BufferAvailabilityRegistration registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
 		return this.bufferPool.registerBufferAvailabilityListener(listener);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/OutputGate.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/OutputGate.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/OutputGate.java
index d3eaea1..bff9180 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/OutputGate.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/OutputGate.java
@@ -19,7 +19,6 @@ import eu.stratosphere.nephele.deployment.GateDeploymentDescriptor;
 import eu.stratosphere.nephele.event.task.AbstractEvent;
 import eu.stratosphere.runtime.io.Buffer;
 import eu.stratosphere.runtime.io.channels.ChannelID;
-import eu.stratosphere.runtime.io.channels.ChannelType;
 import eu.stratosphere.runtime.io.channels.OutputChannel;
 import eu.stratosphere.nephele.jobgraph.JobID;
 


[15/30] Offer buffer-oriented API for I/O (#25)

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/AbstractSingleGateRecordReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/AbstractSingleGateRecordReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/AbstractSingleGateRecordReader.java
new file mode 100644
index 0000000..71d5628
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/AbstractSingleGateRecordReader.java
@@ -0,0 +1,69 @@
+/***********************************************************************************************************************
+ * 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.api;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
+import eu.stratosphere.runtime.io.gates.InputGate;
+import eu.stratosphere.nephele.template.AbstractInvokable;
+
+import java.io.IOException;
+
+/**
+ * This is an abstract base class for a record reader, either dealing with mutable or immutable records.
+ * 
+ * @param <T> The type of the record that can be read from this record reader.
+ */
+public abstract class AbstractSingleGateRecordReader<T extends IOReadableWritable> extends AbstractRecordReader {
+	
+	/**
+	 * The input gate associated with the record reader.
+	 */
+	protected final InputGate<T> inputGate;
+	
+	// --------------------------------------------------------------------------------------------
+
+	protected AbstractSingleGateRecordReader(AbstractInvokable invokable) {
+		this.inputGate = invokable.getEnvironment().createAndRegisterInputGate();
+	}
+
+	/**
+	 * Returns the number of input channels wired to this reader's input gate.
+	 * 
+	 * @return the number of input channels wired to this reader's input gate
+	 */
+	public int getNumberOfInputChannels() {
+		return this.inputGate.getNumberOfInputChannels();
+	}
+
+	/**
+	 * Publishes an event.
+	 * 
+	 * @param event
+	 *        the event to be published
+	 * @throws IOException
+	 *         thrown if an error occurs while transmitting the event
+	 * @throws InterruptedException
+	 *         thrown if the thread is interrupted while waiting for the event to be published
+	 */
+	@Override
+	public void publishEvent(AbstractTaskEvent event) throws IOException, InterruptedException {
+		// Delegate call to input gate to send events
+		this.inputGate.publishEvent(event);
+	}
+
+	InputGate<T> getInputGate() {
+		return this.inputGate;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/AbstractUnionRecordReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/AbstractUnionRecordReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/AbstractUnionRecordReader.java
new file mode 100644
index 0000000..69ac327
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/AbstractUnionRecordReader.java
@@ -0,0 +1,155 @@
+/***********************************************************************************************************************
+ * 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.api;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
+import eu.stratosphere.runtime.io.gates.InputChannelResult;
+import eu.stratosphere.runtime.io.gates.RecordAvailabilityListener;
+import eu.stratosphere.runtime.io.gates.InputGate;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.HashSet;
+import java.util.Set;
+
+public abstract class AbstractUnionRecordReader<T extends IOReadableWritable> extends AbstractRecordReader implements RecordAvailabilityListener<T> {
+
+	/**
+	 * The set of all input gates.
+	 */
+	private final InputGate<T>[] allInputGates;
+	
+	/**
+	 * The set of unclosed input gates.
+	 */
+	private final Set<InputGate<T>> remainingInputGates;
+
+	/**
+	 * Queue with indices of channels that store at least one available record.
+	 */
+	private final ArrayDeque<InputGate<T>> availableInputGates = new ArrayDeque<InputGate<T>>();
+	
+	/**
+	 * The next input gate to read a record from.
+	 */
+	private InputGate<T> nextInputGateToReadFrom;
+
+	
+	@Override
+	public boolean isInputClosed() {
+		return this.remainingInputGates.isEmpty();
+	}
+	
+	/**
+	 * Constructs a new mutable union record reader.
+	 * 
+	 * @param recordReaders
+	 *        the individual mutable record readers whose input is used to construct the union
+	 */
+	@SuppressWarnings("unchecked")
+	protected AbstractUnionRecordReader(MutableRecordReader<T>[] recordReaders) {
+
+		if (recordReaders == null) {
+			throw new IllegalArgumentException("Provided argument recordReaders is null");
+		}
+
+		if (recordReaders.length < 2) {
+			throw new IllegalArgumentException(
+				"The mutable union record reader must at least be initialized with two individual mutable record readers");
+		}
+		
+		this.allInputGates = new InputGate[recordReaders.length];
+		this.remainingInputGates = new HashSet<InputGate<T>>((int) (recordReaders.length * 1.6f));
+		
+		for (int i = 0; i < recordReaders.length; i++) {
+			InputGate<T> inputGate = recordReaders[i].getInputGate();
+			inputGate.registerRecordAvailabilityListener(this);
+			this.allInputGates[i] = inputGate;
+			this.remainingInputGates.add(inputGate);
+		}
+	}
+	
+	
+	@Override
+	public void publishEvent(AbstractTaskEvent event) throws IOException, InterruptedException {
+		for (InputGate<T> gate : this.allInputGates) {
+			gate.publishEvent(event);
+		}
+	}
+	
+	@Override
+	public void reportRecordAvailability(InputGate<T> inputGate) {
+		synchronized (this.availableInputGates) {
+			this.availableInputGates.add(inputGate);
+			this.availableInputGates.notifyAll();
+		}
+	}
+	
+	protected boolean getNextRecord(T target) throws IOException, InterruptedException {
+
+		while (true) {
+			// has the current input gate more data?
+			if (this.nextInputGateToReadFrom == null) {
+				if (this.remainingInputGates.isEmpty()) {
+					return false;
+				}
+				
+				this.nextInputGateToReadFrom = getNextAvailableInputGate();
+			}
+
+			InputChannelResult result = this.nextInputGateToReadFrom.readRecord(target);
+			switch (result) {
+				case INTERMEDIATE_RECORD_FROM_BUFFER: // record is available and we can stay on the same channel
+					return true;
+					
+				case LAST_RECORD_FROM_BUFFER: // record is available, but we need to re-check the channels
+					this.nextInputGateToReadFrom = null;
+					return true;
+					
+				case END_OF_SUPERSTEP:
+					this.nextInputGateToReadFrom = null;
+					if (incrementEndOfSuperstepEventAndCheck()) {
+						return false; // end of the superstep
+					}
+					else {
+						break; // fall through and wait for next record/event
+					}
+					
+				case TASK_EVENT:	// event for the subscribers is available
+					handleEvent(this.nextInputGateToReadFrom.getCurrentEvent());
+					this.nextInputGateToReadFrom = null;
+					break;
+					
+				case END_OF_STREAM: // one gate is empty
+					this.remainingInputGates.remove(this.nextInputGateToReadFrom);
+					this.nextInputGateToReadFrom = null;
+					break;
+					
+				case NONE: // gate processed an internal event and could not return a record on this call
+					this.nextInputGateToReadFrom = null;
+					break;
+			}
+		}
+	}
+	
+	private InputGate<T> getNextAvailableInputGate() throws InterruptedException {
+		synchronized (this.availableInputGates) {
+			while (this.availableInputGates.isEmpty()) {
+				this.availableInputGates.wait();
+			}
+			return this.availableInputGates.pop();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/BufferWriter.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/BufferWriter.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/BufferWriter.java
new file mode 100644
index 0000000..a7b62e0
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/BufferWriter.java
@@ -0,0 +1,67 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.api;
+
+import eu.stratosphere.nephele.event.task.AbstractEvent;
+import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
+import eu.stratosphere.nephele.event.task.EventListener;
+import eu.stratosphere.runtime.io.channels.EndOfSuperstepEvent;
+import eu.stratosphere.nephele.template.AbstractInvokable;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.gates.OutputGate;
+
+import java.io.IOException;
+
+public class BufferWriter {
+
+	protected final OutputGate outputGate;
+
+	public BufferWriter(AbstractInvokable invokable) {
+		this.outputGate = invokable.getEnvironment().createAndRegisterOutputGate();
+	}
+
+	public void sendBuffer(Buffer buffer, int targetChannel) throws IOException, InterruptedException {
+		this.outputGate.sendBuffer(buffer, targetChannel);
+	}
+
+	public void sendEvent(AbstractEvent event, int targetChannel) throws IOException, InterruptedException {
+		this.outputGate.sendEvent(event, targetChannel);
+	}
+
+	public void sendBufferAndEvent(Buffer buffer, AbstractEvent event, int targetChannel) throws IOException, InterruptedException {
+		this.outputGate.sendBufferAndEvent(buffer, event, targetChannel);
+	}
+
+	public void broadcastBuffer(Buffer buffer) throws IOException, InterruptedException {
+		this.outputGate.broadcastBuffer(buffer);
+	}
+
+	public void broadcastEvent(AbstractEvent event) throws IOException, InterruptedException {
+		this.outputGate.broadcastEvent(event);
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+
+	public void subscribeToEvent(EventListener eventListener, Class<? extends AbstractTaskEvent> eventType) {
+		this.outputGate.subscribeToEvent(eventListener, eventType);
+	}
+
+	public void unsubscribeFromEvent(EventListener eventListener, Class<? extends AbstractTaskEvent> eventType) {
+		this.outputGate.unsubscribeFromEvent(eventListener, eventType);
+	}
+
+	public void sendEndOfSuperstep() throws IOException, InterruptedException {
+		this.outputGate.broadcastEvent(EndOfSuperstepEvent.INSTANCE);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/ChannelSelector.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/ChannelSelector.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/ChannelSelector.java
new file mode 100644
index 0000000..b9638ea
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/ChannelSelector.java
@@ -0,0 +1,39 @@
+/***********************************************************************************************************************
+ * 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.api;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+
+/**
+ * Objects implementing this interface are passed to an {@link eu.stratosphere.runtime.io.gates.OutputGate}. When a record is sent through the output
+ * gate, the channel selector object is called to determine to which {@link eu.stratosphere.runtime.io.channels.OutputChannel} objects the record
+ * shall be passed on.
+ * 
+ * @param <T>
+ *        the type of record which is sent through the attached output gate
+ */
+public interface ChannelSelector<T extends IOReadableWritable> {
+
+	/**
+	 * Called to determine to which attached {@link eu.stratosphere.runtime.io.channels.OutputChannel} objects the given record shall be forwarded.
+	 * 
+	 * @param record
+	 *        the record to the determine the output channels for
+	 * @param numberOfOutputChannels
+	 *        the total number of output channels which are attached to respective output gate
+	 * @return a (possibly empty) array of integer numbers which indicate the indices of the output channels through
+	 *         which the record shall be forwarded
+	 */
+	int[] selectChannels(T record, int numberOfOutputChannels);
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/MutableReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/MutableReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/MutableReader.java
new file mode 100644
index 0000000..ddc080f
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/MutableReader.java
@@ -0,0 +1,32 @@
+/***********************************************************************************************************************
+ * 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.api;
+
+import java.io.IOException;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+
+/**
+ * 
+ */
+public interface MutableReader<T extends IOReadableWritable> extends ReaderBase {
+	
+	/**
+	 * @param target
+	 * @return
+	 * @throws IOException
+	 * @throws InterruptedException
+	 */
+	boolean next(T target) throws IOException, InterruptedException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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
new file mode 100644
index 0000000..9d03c7f
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/MutableRecordReader.java
@@ -0,0 +1,120 @@
+/***********************************************************************************************************************
+ * 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.api;
+
+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;
+
+public class MutableRecordReader<T extends IOReadableWritable> extends AbstractSingleGateRecordReader<T> implements MutableReader<T> {
+	
+	private boolean endOfStream;
+	
+	
+	/**
+	 * Constructs a new mutable record reader and registers a new input gate with the application's environment.
+	 * 
+	 * @param taskBase The application that instantiated the record reader.
+	 */
+	public MutableRecordReader(final AbstractTask 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);
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public boolean next(final T target) throws IOException, InterruptedException {
+		if (this.endOfStream) {
+			return false;
+			
+		}
+		while (true) {
+			InputChannelResult result = this.inputGate.readRecord(target);
+			switch (result) {
+				case INTERMEDIATE_RECORD_FROM_BUFFER:
+				case LAST_RECORD_FROM_BUFFER:
+					return true;
+					
+				case END_OF_SUPERSTEP:
+					if (incrementEndOfSuperstepEventAndCheck()) {
+						return false; // end of the superstep
+					}
+					else {
+						break; // fall through and wait for next record/event
+					}
+					
+				case TASK_EVENT:
+					handleEvent(this.inputGate.getCurrentEvent());
+					break;	// fall through to get next record
+				
+				case END_OF_STREAM:
+					this.endOfStream = true;
+					return false;
+					
+				default:
+					; // fall through to get next record
+			}
+		}
+	}
+	
+	@Override
+	public boolean isInputClosed() {
+		return this.endOfStream;
+	}
+
+	@Override
+	public void setIterative(int numEventsUntilEndOfSuperstep) {
+		// sanity check for debug purposes
+		if (numEventsUntilEndOfSuperstep != getNumberOfInputChannels()) {
+			throw new IllegalArgumentException("Number of events till end of superstep is different from the number of input channels.");
+		}
+		super.setIterative(numEventsUntilEndOfSuperstep);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/MutableUnionRecordReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/MutableUnionRecordReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/MutableUnionRecordReader.java
new file mode 100644
index 0000000..e79c7bb
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/MutableUnionRecordReader.java
@@ -0,0 +1,37 @@
+/***********************************************************************************************************************
+ * 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.api;
+
+import java.io.IOException;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+
+public class MutableUnionRecordReader<T extends IOReadableWritable> extends AbstractUnionRecordReader<T> implements MutableReader<T> {
+
+	
+	/**
+	 * Constructs a new mutable union record reader.
+	 * 
+	 * @param recordReaders
+	 *        the individual mutable record readers whose input is used to construct the union
+	 */
+	public MutableUnionRecordReader(MutableRecordReader<T>[] recordReaders) {
+		super(recordReaders);
+	}
+
+	@Override
+	public boolean next(T target) throws IOException, InterruptedException {
+		return getNextRecord(target);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/Reader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/Reader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/Reader.java
new file mode 100644
index 0000000..ba0d3b6
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/Reader.java
@@ -0,0 +1,30 @@
+/***********************************************************************************************************************
+ * 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.api;
+
+import java.io.IOException;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+
+/**
+ * A reader interface to read records from an input.
+ * 
+ * @param <T> The type of the record that can be emitted with this record writer
+ */
+public interface Reader<T extends IOReadableWritable> extends ReaderBase {
+
+	boolean hasNext() throws IOException, InterruptedException;
+
+	T next() throws IOException, InterruptedException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/ReaderBase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/ReaderBase.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/ReaderBase.java
new file mode 100644
index 0000000..17f6b7d
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/ReaderBase.java
@@ -0,0 +1,67 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ **********************************************************************************************************************/
+package eu.stratosphere.runtime.io.api;
+
+import java.io.IOException;
+
+import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
+import eu.stratosphere.nephele.event.task.EventListener;
+
+
+/**
+ *
+ */
+public interface ReaderBase {
+
+	boolean isInputClosed();
+	
+	/**
+	 * Subscribes the listener object to receive events of the given type.
+	 * 
+	 * @param eventListener
+	 *        the listener object to register
+	 * @param eventType
+	 *        the type of event to register the listener for
+	 */
+	void subscribeToEvent(EventListener eventListener, Class<? extends AbstractTaskEvent> eventType);
+	
+	/**
+	 * Removes the subscription for events of the given type for the listener object.
+	 * 
+	 * @param eventListener
+	 *        the listener object to cancel the subscription for
+	 * @param eventType
+	 *        the type of the event to cancel the subscription for
+	 */
+	void unsubscribeFromEvent(final EventListener eventListener, final Class<? extends AbstractTaskEvent> eventType);
+
+	/**
+	 * Publishes an event.
+	 * 
+	 * @param event
+	 *        the event to be published
+	 * @throws IOException
+	 *         thrown if an error occurs while transmitting the event
+	 * @throws InterruptedException
+	 *         thrown if the thread is interrupted while waiting for the event to be published
+	 */
+	void publishEvent(AbstractTaskEvent event) throws IOException, InterruptedException;
+	
+	
+	void setIterative(int numEventsUntilEndOfSuperstep);
+
+	
+	void startNextSuperstep();
+	
+	boolean hasReachedEndOfSuperstep();
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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
new file mode 100644
index 0000000..bb6a580
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/RecordReader.java
@@ -0,0 +1,154 @@
+/***********************************************************************************************************************
+ * 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.api;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.nephele.template.AbstractOutputTask;
+import eu.stratosphere.nephele.template.AbstractTask;
+import eu.stratosphere.runtime.io.gates.InputChannelResult;
+
+import java.io.IOException;
+
+/**
+ * A record writer connects an input gate to an application. It allows the application
+ * query for incoming records and read them from input gate.
+ * 
+ * @param <T> The type of the record that can be read from this record reader.
+ */
+public class RecordReader<T extends IOReadableWritable> extends AbstractSingleGateRecordReader<T> implements Reader<T> {
+	
+	private final Class<T> recordType;
+	
+	/**
+	 * Stores the last read record.
+	 */
+	private T lookahead;
+
+	/**
+	 * Stores if more no more records will be received from the assigned input gate.
+	 */
+	private boolean noMoreRecordsWillFollow;
+
+	// --------------------------------------------------------------------------------------------
+	
+	/**
+	 * 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 recordType
+	 *        The class of records that can be read from the record reader.
+	 */
+	public RecordReader(AbstractTask 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;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	/**
+	 * Checks if at least one more record can be read from the associated input gate. This method may block
+	 * until the associated input gate is able to read the record from one of its input channels.
+	 * 
+	 * @return <code>true</code>it at least one more record can be read from the associated input gate, otherwise
+	 *         <code>false</code>
+	 */
+	@Override
+	public boolean hasNext() throws IOException, InterruptedException{
+		if (this.lookahead != null) {
+			return true;
+		} else {
+			if (this.noMoreRecordsWillFollow) {
+				return false;
+			}
+			
+			T record = instantiateRecordType();
+			
+			while (true) {
+				InputChannelResult result = this.inputGate.readRecord(record);
+				switch (result) {
+					case INTERMEDIATE_RECORD_FROM_BUFFER:
+					case LAST_RECORD_FROM_BUFFER:
+						this.lookahead = record;
+						return true;
+						
+					case END_OF_SUPERSTEP:
+						if (incrementEndOfSuperstepEventAndCheck()) {
+							return false;
+						}
+						else {
+							break; // fall through and wait for next record/event
+						}
+						
+					case TASK_EVENT:
+						handleEvent(this.inputGate.getCurrentEvent());
+						break;
+						
+					case END_OF_STREAM:
+						this.noMoreRecordsWillFollow = true;
+						return false;
+				
+					default:
+						; // fall through the loop
+				}
+			}
+		}
+	}
+
+	/**
+	 * Reads the current record from the associated input gate.
+	 * 
+	 * @return the current record from the associated input gate.
+	 * @throws IOException
+	 *         thrown if any error occurs while reading the record from the input gate
+	 */
+	@Override
+	public T next() throws IOException, InterruptedException {
+		if (hasNext()) {
+			T tmp = this.lookahead;
+			this.lookahead = null;
+			return tmp;
+		} else {
+			return null;
+		}
+	}
+	
+	@Override
+	public boolean isInputClosed() {
+		return this.noMoreRecordsWillFollow;
+	}
+	
+	private T instantiateRecordType() {
+		try {
+			return this.recordType.newInstance();
+		} catch (InstantiationException e) {
+			throw new RuntimeException("Cannot instantiate class '" + this.recordType.getName() + "'.", e);
+		} catch (IllegalAccessException e) {
+			throw new RuntimeException("Cannot instantiate class '" + this.recordType.getName() + "'.", e);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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
new file mode 100644
index 0000000..132dc14
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/RecordWriter.java
@@ -0,0 +1,151 @@
+/***********************************************************************************************************************
+ * 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.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;
+import eu.stratosphere.runtime.io.serialization.RecordSerializer;
+import eu.stratosphere.runtime.io.serialization.SpanningRecordSerializer;
+
+import java.io.IOException;
+
+/**
+ * A record writer connects the application to an output gate. It allows the application
+ * of emit (send out) to the output gate. The output gate will then take care of distributing
+ * the emitted records among the output channels.
+ * 
+ * @param <T>
+ *        the type of the record that can be emitted with this record writer
+ */
+public class RecordWriter<T extends IOReadableWritable> extends BufferWriter {
+
+	private final BufferProvider bufferPool;
+
+	private final ChannelSelector<T> channelSelector;
+
+	private int numChannels;
+
+	/** RecordSerializer per outgoing channel */
+	private RecordSerializer<T>[] serializers;
+
+	// -----------------------------------------------------------------------------------------------------------------
+
+	public RecordWriter(AbstractTask task) {
+		this((AbstractInvokable) task, 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) {
+		// initialize the gate
+		super(invokable);
+
+		this.bufferPool = invokable.getEnvironment().getOutputBufferProvider();
+		this.channelSelector = channelSelector;
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+
+	@SuppressWarnings("unchecked")
+	public void initializeSerializers() {
+		this.numChannels = this.outputGate.getNumChannels();
+		this.serializers = new RecordSerializer[numChannels];
+		for (int i = 0; i < this.numChannels; i++) {
+			this.serializers[i] = new SpanningRecordSerializer<T>();
+		}
+	}
+
+	public void emit(final T record) throws IOException, InterruptedException {
+		for (int targetChannel : this.channelSelector.selectChannels(record, this.numChannels)) {
+			// serialize with corresponding serializer and send full buffer
+			RecordSerializer<T> serializer = this.serializers[targetChannel];
+
+			RecordSerializer.SerializationResult result = serializer.addRecord(record);
+			while (result.isFullBuffer()) {
+				Buffer buffer = serializer.getCurrentBuffer();
+				if (buffer != null) {
+					sendBuffer(buffer, targetChannel);
+				}
+
+				buffer = this.bufferPool.requestBufferBlocking(this.bufferPool.getBufferSize());
+				result = serializer.setNextBuffer(buffer);
+			}
+		}
+	}
+
+	public void flush() throws IOException, InterruptedException {
+		for (int targetChannel = 0; targetChannel < this.numChannels; targetChannel++) {
+			RecordSerializer<T> serializer = this.serializers[targetChannel];
+
+			Buffer buffer = serializer.getCurrentBuffer();
+			if (buffer != null) {
+				sendBuffer(buffer, targetChannel);
+			}
+
+			serializer.clear();
+		}
+	}
+
+	@Override
+	public void broadcastEvent(AbstractEvent event) throws IOException, InterruptedException {
+		for (int targetChannel = 0; targetChannel < this.numChannels; targetChannel++) {
+			RecordSerializer<T> serializer = this.serializers[targetChannel];
+
+			Buffer buffer = serializer.getCurrentBuffer();
+			if (buffer == null) {
+				super.sendEvent(event, targetChannel);
+			} else {
+				super.sendBufferAndEvent(buffer, event, targetChannel);
+
+				buffer = this.bufferPool.requestBufferBlocking(this.bufferPool.getBufferSize());
+				serializer.setNextBuffer(buffer);
+			}
+		}
+	}
+
+	@Override
+	public void sendEndOfSuperstep() throws IOException, InterruptedException {
+		for (int targetChannel = 0; targetChannel < this.numChannels; targetChannel++) {
+			RecordSerializer<T> serializer = this.serializers[targetChannel];
+
+			Buffer buffer = serializer.getCurrentBuffer();
+			if (buffer == null) {
+				super.sendEvent(EndOfSuperstepEvent.INSTANCE, targetChannel);
+			} else {
+				super.sendBufferAndEvent(buffer, EndOfSuperstepEvent.INSTANCE, targetChannel);
+
+				buffer = this.bufferPool.requestBufferBlocking(this.bufferPool.getBufferSize());
+				serializer.setNextBuffer(buffer);
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/RoundRobinChannelSelector.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/RoundRobinChannelSelector.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/RoundRobinChannelSelector.java
new file mode 100644
index 0000000..aaa1506
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/RoundRobinChannelSelector.java
@@ -0,0 +1,47 @@
+/***********************************************************************************************************************
+ * 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.api;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+
+/**
+ * This is the default implementation of the {@link ChannelSelector} interface. It represents a simple round-robin
+ * strategy, i.e. regardless of the record every attached exactly one output channel is selected at a time.
+
+ * @param <T>
+ *        the type of record which is sent through the attached output gate
+ */
+public class RoundRobinChannelSelector<T extends IOReadableWritable> implements ChannelSelector<T> {
+
+	/**
+	 * Stores the index of the channel to send the next record to.
+	 */
+	private final int[] nextChannelToSendTo = new int[1];
+
+	/**
+	 * Constructs a new default channel selector.
+	 */
+	public RoundRobinChannelSelector() {
+		this.nextChannelToSendTo[0] = 0;
+	}
+
+
+	@Override
+	public int[] selectChannels(final T record, final int numberOfOutputChannels) {
+
+		this.nextChannelToSendTo[0] = (this.nextChannelToSendTo[0] + 1) % numberOfOutputChannels;
+
+		return this.nextChannelToSendTo;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/UnionRecordReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/UnionRecordReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/UnionRecordReader.java
new file mode 100644
index 0000000..85ce389
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/UnionRecordReader.java
@@ -0,0 +1,67 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ **********************************************************************************************************************/
+
+package eu.stratosphere.runtime.io.api;
+
+import java.io.IOException;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+
+public final class UnionRecordReader<T extends IOReadableWritable> extends AbstractUnionRecordReader<T> implements Reader<T> {
+	
+	private final Class<T> recordType;
+	
+	private T lookahead;
+	
+
+	public UnionRecordReader(MutableRecordReader<T>[] recordReaders, Class<T> recordType) {
+		super(recordReaders);
+		this.recordType = recordType;
+	}
+
+	@Override
+	public boolean hasNext() throws IOException, InterruptedException {
+		if (this.lookahead != null) {
+			return true;
+		} else {
+			T record = instantiateRecordType();
+			if (getNextRecord(record)) {
+				this.lookahead = record;
+				return true;
+			} else {
+				return false;
+			}
+		}
+	}
+
+	@Override
+	public T next() throws IOException, InterruptedException {
+		if (hasNext()) {
+			T tmp = this.lookahead;
+			this.lookahead = null;
+			return tmp;
+		} else {
+			return null;
+		}
+	}
+	
+	private T instantiateRecordType() {
+		try {
+			return this.recordType.newInstance();
+		} catch (InstantiationException e) {
+			throw new RuntimeException("Cannot instantiate class '" + this.recordType.getName() + "'.", e);
+		} catch (IllegalAccessException e) {
+			throw new RuntimeException("Cannot instantiate class '" + this.recordType.getName() + "'.", e);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/BufferOrEvent.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/BufferOrEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/BufferOrEvent.java
new file mode 100644
index 0000000..2d51d13
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/BufferOrEvent.java
@@ -0,0 +1,52 @@
+/***********************************************************************************************************************
+ * 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.channels;
+
+import eu.stratosphere.nephele.event.task.AbstractEvent;
+import eu.stratosphere.runtime.io.Buffer;
+
+/**
+ * Either type for {@link eu.stratosphere.runtime.io.Buffer} and {@link AbstractEvent}.
+ */
+public class BufferOrEvent {
+	
+	private final Buffer buffer;
+	
+	private final AbstractEvent event;
+	
+	public BufferOrEvent(Buffer buffer) {
+		this.buffer = buffer;
+		this.event = null;
+	}
+	
+	public BufferOrEvent(AbstractEvent event) {
+		this.buffer = null;
+		this.event = event;
+	}
+	
+	public boolean isBuffer() {
+		return this.buffer != null;
+	}
+	
+	public boolean isEvent() {
+		return this.event != null;
+	}
+	
+	public Buffer getBuffer() {
+		return this.buffer;
+	}
+	
+	public AbstractEvent getEvent() {
+		return this.event;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/Channel.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/Channel.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/Channel.java
new file mode 100644
index 0000000..dfa5d5e
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/Channel.java
@@ -0,0 +1,97 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.channels;
+
+import java.io.IOException;
+
+import eu.stratosphere.nephele.event.task.AbstractEvent;
+import eu.stratosphere.nephele.jobgraph.JobID;
+import eu.stratosphere.runtime.io.network.envelope.Envelope;
+import eu.stratosphere.runtime.io.network.envelope.EnvelopeDispatcher;
+
+/**
+ * The base class for channel objects.
+ * <p>
+ * Every channel has an index (at the corresponding gate), ID, and type. The connected channel is given by the ID of
+ * destination channel.
+ */
+public abstract class Channel {
+
+	private final ChannelID id;
+
+	private final ChannelID connectedId;
+
+	private final int index;
+
+	private final ChannelType type;
+
+	protected EnvelopeDispatcher envelopeDispatcher;
+
+	/**
+	 * Auxiliary constructor for channels
+	 * 
+	 * @param index the index of the channel in either the output or input gate
+	 * @param id the ID of the channel
+	 * @param connectedId the ID of the channel this channel is connected to
+	 */
+	protected Channel(int index, ChannelID id, ChannelID connectedId, ChannelType type) {
+		this.index = index;
+		this.id = id;
+		this.connectedId = connectedId;
+		this.type = type;
+	}
+
+	public int getIndex() {
+		return this.index;
+	}
+
+	public ChannelID getID() {
+		return this.id;
+	}
+
+	public ChannelID getConnectedId() {
+		return this.connectedId;
+	}
+
+	public ChannelType getChannelType() {
+		return this.type;
+	}
+
+	/**
+	 * Registers an EnvelopeDispatcher with this channel at runtime.
+	 *
+	 * @param envelopeDispatcher the envelope dispatcher to use for data transfers
+	 */
+	public void registerEnvelopeDispatcher(EnvelopeDispatcher envelopeDispatcher) {
+		this.envelopeDispatcher = envelopeDispatcher;
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+
+	public abstract JobID getJobID();
+
+	public abstract boolean isInputChannel();
+
+	public abstract boolean isClosed() throws IOException, InterruptedException;
+
+	public abstract void transferEvent(AbstractEvent event) throws IOException, InterruptedException;
+
+	public abstract void queueEnvelope(Envelope envelope);
+
+	// nothing to do for buffer oriented runtime => TODO remove with pending changes for input side
+	public abstract void releaseAllResources();
+
+	// nothing to do for buffer oriented runtime => TODO remove with pending changes for input side
+	public abstract void destroy();
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelCloseEvent.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelCloseEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelCloseEvent.java
new file mode 100644
index 0000000..dcdcbb2
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelCloseEvent.java
@@ -0,0 +1,33 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.channels;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import eu.stratosphere.nephele.event.task.AbstractEvent;
+
+public final class ChannelCloseEvent extends AbstractEvent {
+
+	@Override
+	public void read(DataInput in) throws IOException {
+		// Nothing to do here
+	}
+
+	@Override
+	public void write(DataOutput out) throws IOException {
+		// Nothing to do here
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelID.java
new file mode 100644
index 0000000..66be7de
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelID.java
@@ -0,0 +1,39 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.channels;
+
+import java.nio.ByteBuffer;
+
+import eu.stratosphere.nephele.AbstractID;
+
+public class ChannelID extends AbstractID {
+
+	public ChannelID() {
+		super();
+	}
+
+	public ChannelID(long lowerPart, long upperPart) {
+		super(lowerPart, upperPart);
+	}
+
+	public ChannelID(byte[] bytes) {
+		super(bytes);
+	}
+
+	public static ChannelID fromByteBuffer(ByteBuffer buf, int offset) {
+		long lower = buf.getLong(offset);
+		long upper = buf.getLong(offset + 8);
+		return new ChannelID(lower, upper);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelType.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelType.java
new file mode 100644
index 0000000..5d5b53d
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelType.java
@@ -0,0 +1,26 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.channels;
+
+/**
+ * An enumeration for the channel types.
+ */
+public enum ChannelType {
+	
+	/** Network channels */
+	NETWORK,
+
+	/** In-memory channels */
+	IN_MEMORY
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/EndOfSuperstepEvent.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/EndOfSuperstepEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/EndOfSuperstepEvent.java
new file mode 100644
index 0000000..e5f9589
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/EndOfSuperstepEvent.java
@@ -0,0 +1,34 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ **********************************************************************************************************************/
+
+package eu.stratosphere.runtime.io.channels;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import eu.stratosphere.nephele.event.task.AbstractEvent;
+
+/**
+ * Marks the end of a superstep of one particular iteration head
+ */
+public class EndOfSuperstepEvent extends AbstractEvent {
+	
+	public static final EndOfSuperstepEvent INSTANCE = new EndOfSuperstepEvent();
+
+	@Override
+	public void write(DataOutput out) throws IOException {}
+
+	@Override
+	public void read(DataInput in) throws IOException {}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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
new file mode 100644
index 0000000..860141d
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/InputChannel.java
@@ -0,0 +1,493 @@
+/***********************************************************************************************************************
+ * 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.channels;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.nephele.event.task.AbstractEvent;
+import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
+import eu.stratosphere.nephele.jobgraph.JobID;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.gates.InputChannelResult;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferAvailabilityListener;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
+import eu.stratosphere.runtime.io.network.envelope.Envelope;
+import eu.stratosphere.runtime.io.gates.InputGate;
+import eu.stratosphere.runtime.io.serialization.AdaptiveSpanningRecordDeserializer;
+import eu.stratosphere.runtime.io.serialization.RecordDeserializer;
+import eu.stratosphere.runtime.io.serialization.RecordDeserializer.DeserializationResult;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Queue;
+
+/**
+ * InputChannel is an abstract base class to all different kinds of concrete
+ * input channels that can be used. Input channels are always parameterized to
+ * a specific type that can be transported through the channel.
+
+ * @param <T> The Type of the record that can be transported through the channel.
+ */
+public class InputChannel<T extends IOReadableWritable> extends Channel implements BufferProvider {
+
+	private final InputGate<T> inputGate;
+
+	/**
+	 * The log object used to report warnings and errors.
+	 */
+	private static final Log LOG = LogFactory.getLog(InputChannel.class);
+
+	/**
+	 * The deserializer used to deserialize records.
+	 */
+	private final RecordDeserializer<T> deserializer;
+
+	/**
+	 * Buffer for the uncompressed (raw) data.
+	 */
+	private Buffer dataBuffer;
+
+	private AbstractTaskEvent currentEvent;
+
+	/**
+	 * The exception observed in this channel while processing the buffers. Checked and thrown
+	 * per-buffer.
+	 */
+	private volatile IOException ioException;
+
+	/**
+	 * Stores the number of bytes read through this input channel since its instantiation.
+	 */
+	private long amountOfDataTransmitted;
+
+	private volatile boolean brokerAggreedToCloseChannel;
+
+	// -------------------------------------------------------------------------------------------
+
+	private int lastReceivedEnvelope = -1;
+
+	private boolean destroyCalled = false;
+
+	// ----------------------
+
+	private Queue<Envelope> queuedEnvelopes = new ArrayDeque<Envelope>();
+
+	private Iterator<AbstractEvent> pendingEvents;
+
+	/**
+	 * Constructs an input channel with a given input gate associated.
+	 * 
+	 * @param inputGate
+	 *        the input gate this channel is connected to
+	 * @param channelIndex
+	 *        the index of the channel in the input gate
+	 * @param channelID
+	 *        the ID of the channel
+	 * @param connectedChannelID
+	 *        the ID of the channel this channel is connected to
+	 */
+	public InputChannel(final InputGate<T> inputGate, final int channelIndex, final ChannelID channelID,
+						   final ChannelID connectedChannelID, ChannelType type) {
+		super(channelIndex, channelID, connectedChannelID, type);
+		this.inputGate = inputGate;
+		this.deserializer = new AdaptiveSpanningRecordDeserializer<T>();
+	}
+
+	/**
+	 * Returns the input gate associated with the input channel.
+	 * 
+	 * @return the input gate associated with the input channel.
+	 */
+	public InputGate<T> getInputGate() {
+		return this.inputGate;
+	}
+
+	/**
+	 * Reads a record from the input channel. If currently no record is available the method
+	 * returns <code>null</code>. If the channel is closed (i.e. no more records will be received), the method
+	 * throws an {@link EOFException}.
+	 * 
+	 * @return a record that has been transported through the channel or <code>null</code> if currently no record is
+	 *         available
+	 * @throws IOException
+	 *         thrown if the input channel is already closed {@link EOFException} or a transmission error has occurred
+	 */
+//	public abstract InputChannelResult readRecord(T target) throws IOException;
+
+	/**
+	 * Immediately closes the input channel. The corresponding output channels are
+	 * notified if necessary. Any remaining records in any buffers or queue is considered
+	 * irrelevant and is discarded.
+	 * 
+	 * @throws InterruptedException
+	 *         thrown if the thread is interrupted while waiting for the channel to close
+	 * @throws IOException
+	 *         thrown if an I/O error occurs while closing the channel
+	 */
+//	public abstract void close() throws IOException, InterruptedException;
+
+
+
+	@Override
+	public boolean isInputChannel() {
+		return true;
+	}
+
+
+	@Override
+	public JobID getJobID() {
+		return this.inputGate.getJobID();
+	}
+
+//	public abstract AbstractTaskEvent getCurrentEvent();
+
+	public InputChannelResult readRecord(T target) throws IOException {
+		if (this.dataBuffer == null) {
+			if (isClosed()) {
+				return InputChannelResult.END_OF_STREAM;
+			}
+
+			// get the next element we need to handle (buffer or event)
+			BufferOrEvent boe = getNextBufferOrEvent();
+
+			if (boe == null) {
+				throw new IllegalStateException("Input channel was queries for data even though none was announced available.");
+			}
+
+			// handle events
+			if (boe.isEvent())
+			{
+				// sanity check: an event may only come after a complete record.
+				if (this.deserializer.hasUnfinishedData()) {
+					throw new IOException("Channel received an event before completing the current partial record.");
+				}
+
+				AbstractEvent evt = boe.getEvent();
+				if (evt.getClass() == ChannelCloseEvent.class) {
+					this.brokerAggreedToCloseChannel = true;
+					return InputChannelResult.END_OF_STREAM;
+				}
+				else if (evt.getClass() == EndOfSuperstepEvent.class) {
+					return InputChannelResult.END_OF_SUPERSTEP;
+				}
+				else if (evt instanceof AbstractTaskEvent) {
+					this.currentEvent = (AbstractTaskEvent) evt;
+					return InputChannelResult.TASK_EVENT;
+				}
+				else {
+					LOG.error("Received unknown event: " + evt);
+					return InputChannelResult.NONE;
+				}
+			} else {
+				// buffer case
+				this.dataBuffer = boe.getBuffer();
+				this.deserializer.setNextMemorySegment(this.dataBuffer.getMemorySegment(), this.dataBuffer.size());
+			}
+		}
+
+
+		DeserializationResult deserializationResult = this.deserializer.getNextRecord(target);
+
+		if (deserializationResult.isBufferConsumed()) {
+			releasedConsumedReadBuffer(this.dataBuffer);
+			this.dataBuffer = null;
+		}
+
+		if (deserializationResult == DeserializationResult.INTERMEDIATE_RECORD_FROM_BUFFER) {
+			return InputChannelResult.INTERMEDIATE_RECORD_FROM_BUFFER;
+		} else if (deserializationResult == DeserializationResult.LAST_RECORD_FROM_BUFFER) {
+			return InputChannelResult.LAST_RECORD_FROM_BUFFER;
+		} else if (deserializationResult == DeserializationResult.PARTIAL_RECORD) {
+			return InputChannelResult.NONE;
+		} else {
+			throw new IllegalStateException();
+		}
+	}
+
+	@Override
+	public ChannelType getChannelType() {
+		return null;
+	}
+
+	@Override
+	public boolean isClosed() throws IOException{
+		if (this.ioException != null) {
+			throw new IOException("An error occurred in the channel: " + this.ioException.getMessage(), this.ioException);
+		} else {
+			return this.brokerAggreedToCloseChannel;
+		}
+	}
+
+	public void close() throws IOException, InterruptedException {
+
+		this.deserializer.clear();
+		if (this.dataBuffer != null) {
+			releasedConsumedReadBuffer(this.dataBuffer);
+			this.dataBuffer = null;
+		}
+
+		// This code fragment makes sure the isClosed method works in case the channel input has not been fully consumed
+		while (!this.brokerAggreedToCloseChannel)
+		{
+			BufferOrEvent next = getNextBufferOrEvent();
+			if (next != null) {
+				if (next.isEvent()) {
+					if (next.getEvent() instanceof ChannelCloseEvent) {
+						this.brokerAggreedToCloseChannel = true;
+					}
+				} else {
+					releasedConsumedReadBuffer(next.getBuffer());
+				}
+			} else {
+				Thread.sleep(200);
+			}
+		}
+
+		// Send close event to indicate the input channel has successfully
+		// processed all data it is interested in.
+		transferEventToOutputChannel(new ChannelCloseEvent());
+	}
+
+
+	private void releasedConsumedReadBuffer(Buffer buffer) {
+		this.amountOfDataTransmitted += buffer.size();
+		buffer.recycleBuffer();
+	}
+
+
+	public void notifyGateThatInputIsAvailable() {
+		this.getInputGate().notifyRecordIsAvailable(getIndex());
+	}
+
+
+	@Override
+	public void transferEvent(AbstractEvent event) throws IOException, InterruptedException {
+		transferEventToOutputChannel(event);
+	}
+
+
+	public void reportIOException(IOException ioe) {
+		this.ioException = ioe;
+	}
+
+
+	@Override
+	public void releaseAllResources() {
+		this.brokerAggreedToCloseChannel = true;
+		this.deserializer.clear();
+
+		// The buffers are recycled by the input channel wrapper
+	}
+
+	/**
+	 * Notify the channel that a data unit has been consumed.
+	 */
+	public void notifyDataUnitConsumed() {
+		this.getInputGate().notifyDataUnitConsumed(getIndex());
+	}
+
+	public AbstractTaskEvent getCurrentEvent() {
+		AbstractTaskEvent e = this.currentEvent;
+		this.currentEvent = null;
+		return e;
+	}
+
+	// InputChannelContext
+
+	@Override
+	public void queueEnvelope(Envelope envelope) {
+		// The sequence number of the envelope to be queued
+		final int sequenceNumber = envelope.getSequenceNumber();
+
+		synchronized (this.queuedEnvelopes) {
+
+			if (this.destroyCalled) {
+				final Buffer buffer = envelope.getBuffer();
+				if (buffer != null) {
+					buffer.recycleBuffer();
+				}
+				return;
+			}
+
+			final int expectedSequenceNumber = this.lastReceivedEnvelope + 1;
+			if (sequenceNumber != expectedSequenceNumber) {
+				// This is a problem, now we are actually missing some data
+				reportIOException(new IOException("Expected data packet " + expectedSequenceNumber + " but received " + sequenceNumber));
+
+				// notify that something (an exception) is available
+				notifyGateThatInputIsAvailable();
+
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("Input channel " + this.toString() + " expected envelope " + expectedSequenceNumber
+							+ " but received " + sequenceNumber);
+				}
+
+				// rescue the buffer
+				final Buffer buffer = envelope.getBuffer();
+				if (buffer != null) {
+					buffer.recycleBuffer();
+				}
+			} else {
+
+				this.queuedEnvelopes.add(envelope);
+				this.lastReceivedEnvelope = sequenceNumber;
+
+				// Notify the channel about the new data. notify as much as there is (buffer plus once per event)
+				if (envelope.getBuffer() != null) {
+					notifyGateThatInputIsAvailable();
+				}
+
+				List<? extends AbstractEvent> events = envelope.deserializeEvents();
+
+				if (events != null) {
+					for (int i = 0; i < events.size(); i++) {
+						notifyGateThatInputIsAvailable();
+					}
+				}
+			}
+		}
+	}
+
+	@Override
+	public void destroy() {
+		final Queue<Buffer> buffersToRecycle = new ArrayDeque<Buffer>();
+
+		synchronized (this.queuedEnvelopes) {
+			this.destroyCalled = true;
+
+			while (!this.queuedEnvelopes.isEmpty()) {
+				final Envelope envelope = this.queuedEnvelopes.poll();
+				if (envelope.getBuffer() != null) {
+					buffersToRecycle.add(envelope.getBuffer());
+				}
+			}
+		}
+
+		while (!buffersToRecycle.isEmpty()) {
+			buffersToRecycle.poll().recycleBuffer();
+		}
+	}
+
+	public void logQueuedEnvelopes() {
+		int numberOfQueuedEnvelopes = 0;
+		int numberOfQueuedEnvelopesWithMemoryBuffers = 0;
+		int numberOfQueuedEnvelopesWithFileBuffers = 0;
+
+		synchronized (this.queuedEnvelopes) {
+
+			final Iterator<Envelope> it = this.queuedEnvelopes.iterator();
+			while (it.hasNext()) {
+
+				final Envelope envelope = it.next();
+				++numberOfQueuedEnvelopes;
+				final Buffer buffer = envelope.getBuffer();
+				if (buffer == null) {
+					continue;
+				}
+
+				++numberOfQueuedEnvelopesWithMemoryBuffers;
+			}
+		}
+
+		System.out.println("\t\t" + this.toString() + ": " + numberOfQueuedEnvelopes + " ("
+				+ numberOfQueuedEnvelopesWithMemoryBuffers + ", " + numberOfQueuedEnvelopesWithFileBuffers + ")");
+
+	}
+
+	@Override
+	public Buffer requestBuffer(int minBufferSize) throws IOException {
+		return this.inputGate.requestBuffer(minBufferSize);
+	}
+
+	@Override
+	public Buffer requestBufferBlocking(int minBufferSize) throws IOException, InterruptedException {
+		return this.inputGate.requestBufferBlocking(minBufferSize);
+	}
+
+	@Override
+	public int getBufferSize() {
+		return this.inputGate.getBufferSize();
+	}
+
+	@Override
+	public void reportAsynchronousEvent() {
+		this.inputGate.reportAsynchronousEvent();
+	}
+
+	@Override
+	public boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
+		return this.inputGate.registerBufferAvailabilityListener(listener);
+	}
+
+	// ChannelBroker
+
+	public BufferOrEvent getNextBufferOrEvent() throws IOException {
+		// return pending events first
+		if (this.pendingEvents != null) {
+			// if the field is not null, it must always have a next value!
+			BufferOrEvent next = new BufferOrEvent(this.pendingEvents.next());
+			if (!this.pendingEvents.hasNext()) {
+				this.pendingEvents = null;
+			}
+			return next;
+		}
+
+		// if no events are pending, get the next buffer
+		Envelope nextEnvelope;
+		synchronized (this.queuedEnvelopes) {
+			if (this.queuedEnvelopes.isEmpty()) {
+				return null;
+			}
+			nextEnvelope = this.queuedEnvelopes.poll();
+		}
+
+		// schedule events as pending, because events come always after the buffer!
+		List<AbstractEvent> events = (List<AbstractEvent>) nextEnvelope.deserializeEvents();
+		Iterator<AbstractEvent> eventsIt = events.iterator();
+		if (eventsIt.hasNext()) {
+			this.pendingEvents = eventsIt;
+		}
+
+		// get the buffer, if there is one
+		if (nextEnvelope.getBuffer() != null) {
+			return new BufferOrEvent(nextEnvelope.getBuffer());
+		}
+		else if (this.pendingEvents != null) {
+			// if the field is not null, it must always have a next value!
+			BufferOrEvent next = new BufferOrEvent(this.pendingEvents.next());
+			if (!this.pendingEvents.hasNext()) {
+				this.pendingEvents = null;
+			}
+
+			return next;
+		}
+		else {
+			// no buffer and no events, this should be an error
+			throw new IOException("Received an envelope with neither data nor events.");
+		}
+	}
+
+	public void transferEventToOutputChannel(AbstractEvent event) throws IOException, InterruptedException {
+		Envelope ephemeralEnvelope = new Envelope(0, getJobID(), getID());
+		ephemeralEnvelope.serializeEventList(Arrays.asList(event));
+
+		this.envelopeDispatcher.dispatchFromInputChannel(ephemeralEnvelope);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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
new file mode 100644
index 0000000..7ca916c
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/OutputChannel.java
@@ -0,0 +1,193 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.channels;
+
+import eu.stratosphere.nephele.event.task.AbstractEvent;
+import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
+import eu.stratosphere.nephele.jobgraph.JobID;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.network.envelope.Envelope;
+import eu.stratosphere.runtime.io.gates.OutputGate;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+
+public class OutputChannel extends Channel {
+
+	private static final Log LOG = LogFactory.getLog(OutputChannel.class);
+
+	private final Object closeLock = new Object();
+	
+	private final OutputGate outputGate;
+
+	private boolean senderCloseRequested;
+
+	private boolean receiverCloseRequested;
+
+	private int currentSeqNum;
+
+	// -----------------------------------------------------------------------------------------------------------------
+
+	/**
+	 * Creates a new output channel object.
+	 *
+	 * @param outputGate the output gate this channel is connected to
+	 * @param index the index of the channel in the output gate
+	 * @param id the ID of the channel
+	 * @param connectedId the ID of the channel this channel is connected to
+	 * @param type the type of this channel
+	 */
+	public OutputChannel(OutputGate outputGate, int index, ChannelID id, ChannelID connectedId, ChannelType type) {
+		super(index, id, connectedId, type);
+
+		this.outputGate = outputGate;
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+	//                                           Data processing
+	// -----------------------------------------------------------------------------------------------------------------
+
+	public void sendBuffer(Buffer buffer) throws IOException, InterruptedException {
+		checkStatus();
+
+		// discard empty buffers
+		if (buffer.size() == 0) {
+			return;
+		}
+
+		Envelope envelope = createNextEnvelope();
+		envelope.setBuffer(buffer);
+		this.envelopeDispatcher.dispatchFromOutputChannel(envelope);
+	}
+
+	public void sendEvent(AbstractEvent event) throws IOException, InterruptedException {
+		checkStatus();
+
+		Envelope envelope = createNextEnvelope();
+		envelope.serializeEventList(Arrays.asList(event));
+		this.envelopeDispatcher.dispatchFromOutputChannel(envelope);
+	}
+
+	public void sendBufferAndEvent(Buffer buffer, AbstractEvent event) throws IOException, InterruptedException {
+		checkStatus();
+
+		Envelope envelope = createNextEnvelope();
+		envelope.setBuffer(buffer);
+		envelope.serializeEventList(Arrays.asList(event));
+		this.envelopeDispatcher.dispatchFromOutputChannel(envelope);
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+	//                                          Event processing
+	// -----------------------------------------------------------------------------------------------------------------
+
+	@Override
+	public void queueEnvelope(Envelope envelope) {
+		if (envelope.hasBuffer()) {
+			throw new IllegalStateException("Envelope for OutputChannel has Buffer attached.");
+		}
+
+		for (AbstractEvent event : envelope.deserializeEvents()) {
+			if (event.getClass() == ChannelCloseEvent.class) {
+				synchronized (this.closeLock) {
+					this.receiverCloseRequested = true;
+					this.closeLock.notifyAll();
+				}
+				LOG.debug("OutputChannel received close event from target.");
+			} 
+			else if (event instanceof AbstractTaskEvent) {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("OutputChannel received task event: " + event);
+				}
+				
+				this.outputGate.deliverEvent((AbstractTaskEvent) event);
+			}
+			else {
+				throw new RuntimeException("OutputChannel received an event that is neither close nor task event.");
+			}
+		}
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+	//                                              Shutdown
+	// -----------------------------------------------------------------------------------------------------------------
+
+	public void requestClose() throws IOException, InterruptedException {
+		if (this.senderCloseRequested) {
+			return;
+		}
+
+		this.senderCloseRequested = true;
+
+		Envelope envelope = createNextEnvelope();
+		envelope.serializeEventList(Arrays.asList(new ChannelCloseEvent()));
+		this.envelopeDispatcher.dispatchFromOutputChannel(envelope);
+	}
+
+	@Override
+	public boolean isClosed() {
+		return this.senderCloseRequested && this.receiverCloseRequested;
+	}
+	
+	public void waitForChannelToBeClosed() throws InterruptedException {
+		synchronized (this.closeLock) {
+			while (!this.receiverCloseRequested) {
+				this.closeLock.wait(1000);
+			}
+		}
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+
+	@Override
+	public boolean isInputChannel() {
+		return false;
+	}
+
+	@Override
+	public JobID getJobID() {
+		return this.outputGate.getJobID();
+	}
+	
+	private void checkStatus() throws IOException {
+		if (this.senderCloseRequested) {
+			throw new IllegalStateException(String.format("Channel %s already requested to be closed.", getID()));
+		}
+		if (this.receiverCloseRequested) {
+			throw new ReceiverAlreadyClosedException();
+		}
+	}
+
+	private Envelope createNextEnvelope() {
+		return new Envelope(this.currentSeqNum++, getJobID(), getID());
+	}
+
+	@Override
+	public void transferEvent(AbstractEvent event) throws IOException, InterruptedException {
+		// TODO remove with pending changes for input side
+	}
+
+	@Override
+	public void releaseAllResources() {
+		// nothing to do for buffer oriented runtime => TODO remove with pending changes for input side
+	}
+
+	@Override
+	public void destroy() {
+		// nothing to do for buffer oriented runtime => TODO remove with pending changes for input side
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ReceiverAlreadyClosedException.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ReceiverAlreadyClosedException.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ReceiverAlreadyClosedException.java
new file mode 100644
index 0000000..d03179f
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ReceiverAlreadyClosedException.java
@@ -0,0 +1,22 @@
+/***********************************************************************************************************************
+ *
+ * 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.channels;
+
+import java.io.IOException;
+
+
+public class ReceiverAlreadyClosedException extends IOException {
+	private static final long serialVersionUID = 1L;
+}


[26/30] git commit: Add debug log messages to Netty stack

Posted by rm...@apache.org.
Add debug log messages to Netty 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/6d8fea96
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/6d8fea96
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/6d8fea96

Branch: refs/heads/master
Commit: 6d8fea9602c7cd248ac29d9d5c2c2f1ade42c73f
Parents: 4cd4a13
Author: uce <u....@fu-berlin.de>
Authored: Mon Jun 2 15:46:07 2014 +0200
Committer: StephanEwen <st...@tu-berlin.de>
Committed: Sat Jun 7 09:41:21 2014 +0200

----------------------------------------------------------------------
 .../network/netty/InboundEnvelopeDecoder.java   | 11 +++++
 .../network/netty/NettyConnectionManager.java   | 52 ++++++++++++++++++++
 .../network/netty/OutboundConnectionQueue.java  | 17 ++++++-
 .../test/runtime/NetworkStackNepheleITCase.java |  6 +--
 4 files changed, 81 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6d8fea96/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoder.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoder.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoder.java
index 1ab1871..54f4617 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoder.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoder.java
@@ -23,6 +23,8 @@ import eu.stratosphere.runtime.io.network.Envelope;
 import io.netty.buffer.ByteBuf;
 import io.netty.channel.ChannelHandlerContext;
 import io.netty.channel.ChannelInboundHandlerAdapter;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -30,6 +32,8 @@ import java.util.concurrent.ConcurrentLinkedQueue;
 
 public class InboundEnvelopeDecoder extends ChannelInboundHandlerAdapter implements BufferAvailabilityListener {
 
+	private static final Log LOG = LogFactory.getLog(InboundEnvelopeDecoder.class);
+
 	private final BufferProviderBroker bufferProviderBroker;
 
 	private final BufferAvailabilityChangedTask bufferAvailabilityChangedTask = new BufferAvailabilityChangedTask();
@@ -123,6 +127,10 @@ public class InboundEnvelopeDecoder extends ChannelInboundHandlerAdapter impleme
 					case REGISTERED:
 						if (ctx.channel().config().isAutoRead()) {
 							ctx.channel().config().setAutoRead(false);
+
+							if (LOG.isDebugEnabled()) {
+								LOG.debug(String.format("Set channel %s auto read to false.", ctx.channel()));
+							}
 						}
 
 						this.stagedBuffer = in;
@@ -194,6 +202,9 @@ public class InboundEnvelopeDecoder extends ChannelInboundHandlerAdapter impleme
 				if (decodeBuffer(stagedBuffer, channelHandlerContext)) {
 					stagedBuffer = null;
 					channelHandlerContext.channel().config().setAutoRead(true);
+					if (LOG.isDebugEnabled()) {
+						LOG.debug(String.format("Set channel %s auto read to true.", channelHandlerContext.channel()));
+					}
 				}
 			} catch (IOException e) {
 				availableBuffer.recycleBuffer();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6d8fea96/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 ae67f42..4e3f043 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
@@ -34,6 +34,8 @@ import org.apache.commons.logging.LogFactory;
 
 import java.io.IOException;
 import java.net.InetAddress;
+import java.util.Date;
+import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
@@ -41,6 +43,8 @@ public class NettyConnectionManager {
 
 	private static final Log LOG = LogFactory.getLog(NettyConnectionManager.class);
 
+	private static final int DEBUG_PRINT_QUEUED_ENVELOPES_EVERY_MS = 10000;
+
 	private final ChannelManager channelManager;
 
 	private final ServerBootstrap in;
@@ -108,6 +112,30 @@ public class NettyConnectionManager {
 		} catch (InterruptedException e) {
 			throw new RuntimeException("Could not bind server socket for incoming connections.");
 		}
+
+		if (LOG.isDebugEnabled()) {
+			new Thread(new Runnable() {
+				@Override
+				public void run() {
+					Date date = new Date();
+
+
+					while (true) {
+						try {
+							Thread.sleep(DEBUG_PRINT_QUEUED_ENVELOPES_EVERY_MS);
+
+							date.setTime(System.currentTimeMillis());
+							System.out.println(date);
+
+							System.out.println(getNonZeroNumQueuedEnvelopes());
+
+						} catch (InterruptedException e) {
+							e.printStackTrace();
+						}
+					}
+				}
+			}).start();
+		}
 	}
 
 	public void shutdown() {
@@ -170,6 +198,30 @@ public class NettyConnectionManager {
 		channel.enqueue(envelope);
 	}
 
+	private String getNonZeroNumQueuedEnvelopes() {
+		StringBuilder str = new StringBuilder();
+
+		str.append(String.format("==== %d outgoing connections ===\n", this.outConnections.size()));
+
+		for (Map.Entry<RemoteReceiver, Object> entry : this.outConnections.entrySet()) {
+			RemoteReceiver receiver = entry.getKey();
+
+			Object value = entry.getValue();
+			if (value instanceof OutboundConnectionQueue) {
+				OutboundConnectionQueue queue = (OutboundConnectionQueue) value;
+				if (queue.getNumQueuedEnvelopes() > 0) {
+					str.append(String.format("%s> Number of queued envelopes for %s with channel %s: %d\n",
+							Thread.currentThread().getId(), receiver, queue.getChannel(), queue.getNumQueuedEnvelopes()));
+				}
+			} else if (value instanceof ChannelInBuildup) {
+				str.append(String.format("%s> Connection to %s is still in buildup\n",
+						Thread.currentThread().getId(), receiver));
+			}
+		}
+
+		return str.toString();
+	}
+
 	// ------------------------------------------------------------------------
 
 	private static final class ChannelInBuildup implements ChannelFutureListener {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6d8fea96/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 c687408..b6ec915 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
@@ -23,6 +23,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
 import java.util.ArrayDeque;
+import java.util.concurrent.atomic.AtomicInteger;
 
 public class OutboundConnectionQueue extends ChannelInboundHandlerAdapter implements ChannelFutureListener {
 
@@ -30,11 +31,12 @@ public class OutboundConnectionQueue extends ChannelInboundHandlerAdapter implem
 
 	private final Channel channel;
 
-	private final ArrayDeque<Envelope> queuedEnvelopes;
+	private final ArrayDeque<Envelope> queuedEnvelopes = new ArrayDeque<Envelope>();
+
+	private final AtomicInteger numQueued = new AtomicInteger(0);
 
 	public OutboundConnectionQueue(Channel channel) {
 		this.channel = channel;
-		this.queuedEnvelopes = new ArrayDeque<Envelope>();
 
 		channel.pipeline().addFirst(this);
 	}
@@ -56,6 +58,7 @@ public class OutboundConnectionQueue extends ChannelInboundHandlerAdapter implem
 		boolean triggerWrite = this.queuedEnvelopes.isEmpty();
 
 		this.queuedEnvelopes.addLast((Envelope) envelopeToEnqueue);
+		this.numQueued.incrementAndGet();
 
 		if (triggerWrite) {
 			writeAndFlushNextEnvelopeIfPossible();
@@ -80,9 +83,19 @@ public class OutboundConnectionQueue extends ChannelInboundHandlerAdapter implem
 		}
 	}
 
+	public int getNumQueuedEnvelopes() {
+		return this.numQueued.intValue();
+	}
+
+	public Channel getChannel() {
+		return this.channel;
+	}
+
 	private void writeAndFlushNextEnvelopeIfPossible() {
 		if (this.channel.isWritable() && !this.queuedEnvelopes.isEmpty()) {
 			Envelope nextEnvelope = this.queuedEnvelopes.pollFirst();
+			this.numQueued.decrementAndGet();
+
 			this.channel.writeAndFlush(nextEnvelope).addListener(this);
 		}
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6d8fea96/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackNepheleITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackNepheleITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackNepheleITCase.java
index f5beda4..64026a2 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackNepheleITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackNepheleITCase.java
@@ -162,8 +162,8 @@ public class NetworkStackNepheleITCase extends RecordAPITestBase {
 			producer.connectTo(forwarder, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
 			forwarder.connectTo(consumer, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
 
-			producer.setVertexToShareInstancesWith(forwarder);
-			forwarder.setVertexToShareInstancesWith(consumer);
+			forwarder.setVertexToShareInstancesWith(producer);
+			consumer.setVertexToShareInstancesWith(producer);
 		}
 		else {
 			producer.connectTo(consumer, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
@@ -196,7 +196,7 @@ public class NetworkStackNepheleITCase extends RecordAPITestBase {
 
 			LOG.info(String.format("%d/%d: Producing %d records (each record: %d bytes, total: %.2f GB)",
 					getIndexInSubtaskGroup() + 1, getCurrentNumberOfSubtasks(), numRecordsToEmit,
-					SpeedTestRecord.RECORD_SIZE, dataMbPerSubtask/1024.0));
+					SpeedTestRecord.RECORD_SIZE, dataMbPerSubtask / 1024.0));
 
 			boolean isSlow = getTaskConfiguration().getBoolean(IS_SLOW_SENDER_CONFIG_KEY, false);
 


[30/30] git commit: Check number of designated buffers before returning buffer to BufferAvailabilityListener

Posted by rm...@apache.org.
Check number of designated buffers before returning buffer to BufferAvailabilityListener


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

Branch: refs/heads/master
Commit: 3beca629b8f11073415e8107784b2ffe4368f613
Parents: 6d8fea9
Author: uce <u....@fu-berlin.de>
Authored: Mon Jun 2 16:23:40 2014 +0200
Committer: StephanEwen <st...@tu-berlin.de>
Committed: Sat Jun 7 09:41:22 2014 +0200

----------------------------------------------------------------------
 .../runtime/io/network/bufferprovider/LocalBufferPool.java  | 9 +++++++++
 1 file changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/3beca629/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java
index 3eb10c1..6285726 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java
@@ -294,6 +294,15 @@ public final class LocalBufferPool implements BufferProvider {
 				this.globalBufferPool.returnBuffer(buffer);
 				this.numRequestedBuffers--;
 			} else {
+				// if the number of designated buffers changed in the meantime, make sure
+				// to return the buffer to the global buffer pool
+				if (this.numRequestedBuffers > this.numDesignatedBuffers) {
+					this.globalBufferPool.returnBuffer(buffer);
+					this.numRequestedBuffers--;
+
+					return;
+				}
+
 				if (!this.listeners.isEmpty()) {
 					Buffer availableBuffer = new Buffer(buffer, buffer.size(), this.recycler);
 					try {


[05/30] Replace custom Java NIO TCP/IP code with Netty 4 library

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/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
new file mode 100644
index 0000000..ae67f42
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManager.java
@@ -0,0 +1,251 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.network.netty;
+
+import eu.stratosphere.runtime.io.network.ChannelManager;
+import eu.stratosphere.runtime.io.network.Envelope;
+import eu.stratosphere.runtime.io.network.RemoteReceiver;
+import io.netty.bootstrap.Bootstrap;
+import io.netty.bootstrap.ServerBootstrap;
+import io.netty.buffer.PooledByteBufAllocator;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.ChannelOption;
+import io.netty.channel.FixedRecvByteBufAllocator;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.SocketChannel;
+import io.netty.channel.socket.nio.NioServerSocketChannel;
+import io.netty.channel.socket.nio.NioSocketChannel;
+import io.netty.util.concurrent.Future;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+public class NettyConnectionManager {
+
+	private static final Log LOG = LogFactory.getLog(NettyConnectionManager.class);
+
+	private final ChannelManager channelManager;
+
+	private final ServerBootstrap in;
+
+	private final Bootstrap out;
+
+	private final ConcurrentMap<RemoteReceiver, Object> outConnections;
+
+	public NettyConnectionManager(ChannelManager channelManager, InetAddress bindAddress, int bindPort,
+								int bufferSize, int numInThreads, int numOutThreads,
+								int lowWaterMark, int highWaterMark) {
+		this.outConnections = new ConcurrentHashMap<RemoteReceiver, Object>();
+		this.channelManager = channelManager;
+
+		// --------------------------------------------------------------------
+
+		int defaultNumThreads = Math.max(Runtime.getRuntime().availableProcessors() / 4, 1);
+		numInThreads = (numInThreads == -1) ? defaultNumThreads : numInThreads;
+		numOutThreads = (numOutThreads == -1) ? defaultNumThreads : numOutThreads;
+		LOG.info(String.format("Starting with %d incoming and %d outgoing connection threads.", numInThreads, numOutThreads));
+
+		lowWaterMark = (lowWaterMark == -1) ? bufferSize / 2 : lowWaterMark;
+		highWaterMark = (highWaterMark == -1) ? bufferSize : highWaterMark;
+		LOG.info(String.format("Setting low water mark to %d and high water mark to %d bytes.", lowWaterMark, highWaterMark));
+
+		// --------------------------------------------------------------------
+		// server bootstrap (incoming connections)
+		// --------------------------------------------------------------------
+		this.in = new ServerBootstrap();
+		this.in.group(new NioEventLoopGroup(numInThreads))
+				.channel(NioServerSocketChannel.class)
+				.localAddress(bindAddress, bindPort)
+				.childHandler(new ChannelInitializer<SocketChannel>() {
+					@Override
+					public void initChannel(SocketChannel channel) throws Exception {
+						channel.pipeline()
+								.addLast(new InboundEnvelopeDecoder(NettyConnectionManager.this.channelManager))
+								.addLast(new InboundEnvelopeDispatcherHandler(NettyConnectionManager.this.channelManager));
+					}
+				})
+				.option(ChannelOption.RCVBUF_ALLOCATOR, new FixedRecvByteBufAllocator(bufferSize))
+				.option(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT);
+
+		// --------------------------------------------------------------------
+		// client bootstrap (outgoing connections)
+		// --------------------------------------------------------------------
+		this.out = new Bootstrap();
+		this.out.group(new NioEventLoopGroup(numOutThreads))
+				.channel(NioSocketChannel.class)
+				.handler(new ChannelInitializer<SocketChannel>() {
+					@Override
+					public void initChannel(SocketChannel channel) throws Exception {
+						channel.pipeline()
+								.addLast(new OutboundEnvelopeEncoder());
+					}
+				})
+				.option(ChannelOption.WRITE_BUFFER_LOW_WATER_MARK, lowWaterMark)
+				.option(ChannelOption.WRITE_BUFFER_HIGH_WATER_MARK, highWaterMark)
+				.option(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT)
+				.option(ChannelOption.TCP_NODELAY, false)
+				.option(ChannelOption.SO_KEEPALIVE, true);
+
+		try {
+			this.in.bind().sync();
+		} catch (InterruptedException e) {
+			throw new RuntimeException("Could not bind server socket for incoming connections.");
+		}
+	}
+
+	public void shutdown() {
+		Future<?> inShutdownFuture = this.in.group().shutdownGracefully();
+		Future<?> outShutdownFuture = this.out.group().shutdownGracefully();
+
+		try {
+			inShutdownFuture.sync();
+			outShutdownFuture.sync();
+		} catch (InterruptedException e) {
+			throw new RuntimeException("Could not properly shutdown connections.");
+		}
+	}
+
+	public void enqueue(Envelope envelope, RemoteReceiver receiver) throws IOException {
+		// Get the channel. The channel may be
+		// 1) a channel that already exists (usual case) -> just send the data
+		// 2) a channel that is in buildup (sometimes) -> attach to the future and wait for the actual channel
+		// 3) not yet existing -> establish the channel
+
+		final Object entry = this.outConnections.get(receiver);
+		final OutboundConnectionQueue channel;
+
+		if (entry != null) {
+			// existing channel or channel in buildup
+			if (entry instanceof OutboundConnectionQueue) {
+				channel = (OutboundConnectionQueue) entry;
+			}
+			else {
+				ChannelInBuildup future = (ChannelInBuildup) entry;
+				channel = future.waitForChannel();
+			}
+		}
+		else {
+			// No channel yet. Create one, but watch out for a race.
+			// We create a "buildup future" and atomically add it to the map.
+			// Only the thread that really added it establishes the channel.
+			// The others need to wait on that original establisher's future.
+			ChannelInBuildup inBuildup = new ChannelInBuildup(this.out, receiver);
+			Object old = this.outConnections.putIfAbsent(receiver, inBuildup);
+
+			if (old == null) {
+				this.out.connect(receiver.getConnectionAddress()).addListener(inBuildup);
+				channel = inBuildup.waitForChannel();
+
+				Object previous = this.outConnections.put(receiver, channel);
+
+				if (inBuildup != previous) {
+					throw new IOException("Race condition during channel build up.");
+				}
+			}
+			else if (old instanceof ChannelInBuildup) {
+				channel = ((ChannelInBuildup) old).waitForChannel();
+			}
+			else {
+				channel = (OutboundConnectionQueue) old;
+			}
+		}
+
+		channel.enqueue(envelope);
+	}
+
+	// ------------------------------------------------------------------------
+
+	private static final class ChannelInBuildup implements ChannelFutureListener {
+
+		private Object lock = new Object();
+
+		private volatile OutboundConnectionQueue channel;
+
+		private volatile Throwable error;
+
+		private int numRetries = 2;
+
+		private final Bootstrap out;
+
+		private final RemoteReceiver receiver;
+
+		private ChannelInBuildup(Bootstrap out, RemoteReceiver receiver) {
+			this.out = out;
+			this.receiver = receiver;
+		}
+
+		private void handInChannel(OutboundConnectionQueue c) {
+			synchronized (this.lock) {
+				this.channel = c;
+				this.lock.notifyAll();
+			}
+		}
+
+		private void notifyOfError(Throwable error) {
+			synchronized (this.lock) {
+				this.error = error;
+				this.lock.notifyAll();
+			}
+		}
+
+		private OutboundConnectionQueue waitForChannel() throws IOException {
+			synchronized (this.lock) {
+				while (this.error == null && this.channel == null) {
+					try {
+						this.lock.wait(2000);
+					} catch (InterruptedException e) {
+						throw new RuntimeException("Channel buildup interrupted.");
+					}
+				}
+			}
+
+			if (this.error != null) {
+				throw new IOException("Connecting the channel failed: " + error.getMessage(), error);
+			}
+
+			return this.channel;
+		}
+
+		@Override
+		public void operationComplete(ChannelFuture future) throws Exception {
+			if (future.isSuccess()) {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(String.format("Channel %s connected", future.channel()));
+				}
+
+				handInChannel(new OutboundConnectionQueue(future.channel()));
+			}
+			else if (this.numRetries > 0) {
+				LOG.debug(String.format("Connection request did not succeed, retrying (%d attempts left)", this.numRetries));
+
+				this.out.connect(this.receiver.getConnectionAddress()).addListener(this);
+				this.numRetries--;
+			}
+			else {
+				if (future.getClass() != null) {
+					notifyOfError(future.cause());
+				}
+				else {
+					notifyOfError(new Exception("Connection could not be established."));
+				}
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/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
new file mode 100644
index 0000000..c687408
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/OutboundConnectionQueue.java
@@ -0,0 +1,94 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.network.netty;
+
+import eu.stratosphere.runtime.io.network.Envelope;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInboundHandlerAdapter;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.util.ArrayDeque;
+
+public class OutboundConnectionQueue extends ChannelInboundHandlerAdapter implements ChannelFutureListener {
+
+	private static final Log LOG = LogFactory.getLog(OutboundConnectionQueue.class);
+
+	private final Channel channel;
+
+	private final ArrayDeque<Envelope> queuedEnvelopes;
+
+	public OutboundConnectionQueue(Channel channel) {
+		this.channel = channel;
+		this.queuedEnvelopes = new ArrayDeque<Envelope>();
+
+		channel.pipeline().addFirst(this);
+	}
+
+	/**
+	 * Enqueues an envelope so be sent later.
+	 * <p/>
+	 * This method is always invoked by the task thread that wants the envelope sent.
+	 *
+	 * @param env The envelope to be sent.
+	 */
+	public void enqueue(Envelope env) {
+		// the user event trigger ensure thread-safe hand-over of the envelope
+		this.channel.pipeline().fireUserEventTriggered(env);
+	}
+
+	@Override
+	public void userEventTriggered(ChannelHandlerContext ctx, Object envelopeToEnqueue) throws Exception {
+		boolean triggerWrite = this.queuedEnvelopes.isEmpty();
+
+		this.queuedEnvelopes.addLast((Envelope) envelopeToEnqueue);
+
+		if (triggerWrite) {
+			writeAndFlushNextEnvelopeIfPossible();
+		}
+	}
+
+	@Override
+	public void channelWritabilityChanged(ChannelHandlerContext ctx) throws Exception {
+		writeAndFlushNextEnvelopeIfPossible();
+	}
+
+	@Override
+	public void operationComplete(ChannelFuture future) throws Exception {
+		if (future.isSuccess()) {
+			writeAndFlushNextEnvelopeIfPossible();
+		}
+		else if (future.cause() != null) {
+			exceptionOccurred(future.cause());
+		}
+		else {
+			exceptionOccurred(new Exception("Envelope send aborted."));
+		}
+	}
+
+	private void writeAndFlushNextEnvelopeIfPossible() {
+		if (this.channel.isWritable() && !this.queuedEnvelopes.isEmpty()) {
+			Envelope nextEnvelope = this.queuedEnvelopes.pollFirst();
+			this.channel.writeAndFlush(nextEnvelope).addListener(this);
+		}
+	}
+
+	private void exceptionOccurred(Throwable t) throws Exception {
+		LOG.error(String.format("An exception occurred in Channel %s: %s", this.channel, t.getMessage()));
+		throw new Exception(t);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/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
new file mode 100644
index 0000000..424f2c0
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/OutboundEnvelopeEncoder.java
@@ -0,0 +1,65 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.network.netty;
+
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.network.Envelope;
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelHandler;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.MessageToByteEncoder;
+
+@ChannelHandler.Sharable
+public class OutboundEnvelopeEncoder extends MessageToByteEncoder<Envelope> {
+
+	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 {
+		// --------------------------------------------------------------------
+		// (1) header (48 bytes)
+		// --------------------------------------------------------------------
+		out.writeInt(MAGIC_NUMBER); // 4 bytes
+
+		if (out.getInt(out.writerIndex()-4) != MAGIC_NUMBER) {
+			throw new RuntimeException();
+		}
+
+		out.writeInt(env.getSequenceNumber()); // 4 bytes
+		env.getJobID().writeTo(out); // 16 bytes
+		env.getSource().writeTo(out); // 16 bytes
+		out.writeInt(env.getEventsSerialized() != null ? env.getEventsSerialized().remaining() : 0); // 4 bytes
+		out.writeInt(env.getBuffer() != null ? env.getBuffer().size() : 0); // 4 bytes
+		// --------------------------------------------------------------------
+		// (2) events (var length)
+		// --------------------------------------------------------------------
+		if (env.getEventsSerialized() != null) {
+			out.writeBytes(env.getEventsSerialized());
+		}
+
+		// --------------------------------------------------------------------
+		// (3) buffer (var length)
+		// --------------------------------------------------------------------
+		if (env.getBuffer() != null) {
+			Buffer buffer = env.getBuffer();
+			out.writeBytes(buffer.getMemorySegment().wrap(0, buffer.size()));
+
+			// Recycle the buffer from OUR buffer pool after everything has been
+			// copied to Nettys buffer space.
+			buffer.recycleBuffer();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnection.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnection.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnection.java
deleted file mode 100644
index f22e6f7..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnection.java
+++ /dev/null
@@ -1,115 +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.network.tcp;
-
-import eu.stratosphere.runtime.io.Buffer;
-import eu.stratosphere.runtime.io.network.ChannelManager;
-import eu.stratosphere.runtime.io.network.envelope.Envelope;
-import eu.stratosphere.runtime.io.network.envelope.EnvelopeReader;
-import eu.stratosphere.runtime.io.network.envelope.EnvelopeReader.DeserializationState;
-import eu.stratosphere.runtime.io.network.envelope.NoBufferAvailableException;
-import eu.stratosphere.util.StringUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import java.io.IOException;
-import java.nio.channels.ReadableByteChannel;
-import java.nio.channels.SelectionKey;
-
-/**
- * An incoming TCP connection through which data is read and transformed into {@link Envelope} objects.
- */
-public class IncomingConnection {
-
-	private static final Log LOG = LogFactory.getLog(IncomingConnection.class);
-
-	/** Readable byte channel (TCP socket) to read data from */
-	private final ReadableByteChannel channel;
-
-	/** Channel manager to dispatch complete envelopes */
-	private final ChannelManager channelManager;
-
-	/** Envelope reader to turn the channel data into envelopes */
-	private final EnvelopeReader reader;
-
-	// -----------------------------------------------------------------------------------------------------------------
-
-	public IncomingConnection(ReadableByteChannel channel, ChannelManager channelManager) {
-		this.channel = channel;
-		this.channelManager = channelManager;
-		this.reader = new EnvelopeReader(channelManager);
-	}
-
-	// -----------------------------------------------------------------------------------------------------------------
-
-	public void read() throws IOException, InterruptedException, NoBufferAvailableException {
-		DeserializationState deserializationState = this.reader.readNextChunk(this.channel);
-
-		switch (deserializationState) {
-			case COMPLETE:
-				Envelope envelope = this.reader.getFullyDeserializedTransferEnvelope();
-				this.channelManager.dispatchFromNetwork(envelope);
-				this.reader.reset();
-				break;
-
-			case NO_BUFFER_AVAILABLE:
-				throw new NoBufferAvailableException(this.reader.getBufferProvider());
-
-			case PENDING:
-				break;
-		}
-	}
-
-	public void reportTransmissionProblem(SelectionKey key, IOException ioe) {
-		LOG.error(StringUtils.stringifyException(ioe));
-
-		try {
-			this.channel.close();
-		} catch (IOException e) {
-			LOG.debug("An error occurred while closing the byte channel");
-		}
-
-		if (key != null) {
-			key.cancel();
-		}
-
-		Envelope pendingEnvelope = this.reader.getPendingEnvelope();
-		if (pendingEnvelope != null) {
-			if (pendingEnvelope.hasBuffer()) {
-				Buffer buffer = pendingEnvelope.getBuffer();
-				if (buffer != null) {
-					buffer.recycleBuffer();
-				}
-			}
-		}
-
-		this.reader.reset();
-	}
-
-	public boolean isCloseUnexpected() {
-		return this.reader.hasUnfinishedData();
-	}
-
-	public void closeConnection(SelectionKey key) {
-		try {
-			this.channel.close();
-		} catch (IOException ioe) {
-			LOG.error("An IOException occurred while closing the socket: + " + StringUtils.stringifyException(ioe));
-		}
-
-		if (key != null) {
-			key.cancel();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnectionThread.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnectionThread.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnectionThread.java
deleted file mode 100644
index 774ad4e..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnectionThread.java
+++ /dev/null
@@ -1,226 +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.network.tcp;
-
-import eu.stratosphere.runtime.io.network.ChannelManager;
-import eu.stratosphere.runtime.io.network.bufferprovider.BufferAvailabilityListener;
-import eu.stratosphere.runtime.io.network.envelope.NoBufferAvailableException;
-import eu.stratosphere.util.StringUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.nio.channels.ClosedChannelException;
-import java.nio.channels.SelectionKey;
-import java.nio.channels.Selector;
-import java.nio.channels.ServerSocketChannel;
-import java.nio.channels.SocketChannel;
-import java.util.ArrayDeque;
-import java.util.Iterator;
-import java.util.Queue;
-
-public class IncomingConnectionThread extends Thread {
-
-	private static final Log LOG = LogFactory.getLog(IncomingConnectionThread.class);
-
-	private final ChannelManager channelManager;
-
-	private final Selector selector;
-
-	private final Queue<SelectionKey> pendingReadEventSubscribeRequests = new ArrayDeque<SelectionKey>();
-
-	private final ServerSocketChannel listeningSocket;
-
-	private static final class IncomingConnectionBufferAvailListener implements BufferAvailabilityListener {
-
-		private final Queue<SelectionKey> pendingReadEventSubscribeRequests;
-
-		private final SelectionKey key;
-
-		private IncomingConnectionBufferAvailListener(final Queue<SelectionKey> pendingReadEventSubscribeRequests,
-				final SelectionKey key) {
-
-			this.pendingReadEventSubscribeRequests = pendingReadEventSubscribeRequests;
-			this.key = key;
-		}
-
-		@Override
-		public void bufferAvailable() {
-
-			synchronized (this.pendingReadEventSubscribeRequests) {
-				this.pendingReadEventSubscribeRequests.add(this.key);
-			}
-		}
-	}
-
-	public IncomingConnectionThread(ChannelManager channelManager,
-			boolean isListeningThread, InetSocketAddress listeningAddress) throws IOException {
-		super("Incoming Connection Thread");
-
-		this.selector = Selector.open();
-		this.channelManager = channelManager;
-
-		if (isListeningThread) {
-			this.listeningSocket = ServerSocketChannel.open();
-			this.listeningSocket.configureBlocking(false);
-			listeningSocket.register(this.selector, SelectionKey.OP_ACCEPT);
-			this.listeningSocket.socket().bind(listeningAddress);
-			LOG.debug("Listening on " + this.listeningSocket.socket().getLocalSocketAddress());
-		} else {
-			this.listeningSocket = null;
-		}
-	}
-
-	@Override
-	public void run() {
-		try {
-			while (!this.isInterrupted()) {
-	
-				synchronized (this.pendingReadEventSubscribeRequests) {
-					while (!this.pendingReadEventSubscribeRequests.isEmpty()) {
-						final SelectionKey key = this.pendingReadEventSubscribeRequests.poll();
-						final IncomingConnection incomingConnection = (IncomingConnection) key.attachment();
-						final SocketChannel socketChannel = (SocketChannel) key.channel();
-	
-						try {
-							final SelectionKey newKey = socketChannel.register(this.selector, SelectionKey.OP_READ);
-							newKey.attach(incomingConnection);
-						} catch (ClosedChannelException e) {
-							incomingConnection.reportTransmissionProblem(key, e);
-						}
-					}
-				}
-	
-				try {
-					this.selector.select(500);
-				} catch (IOException e) {
-					LOG.error(e);
-				}
-	
-				final Iterator<SelectionKey> iter = this.selector.selectedKeys().iterator();
-	
-				while (iter.hasNext()) {
-					final SelectionKey key = iter.next();
-	
-					iter.remove();
-					if (key.isValid()) {
-						if (key.isReadable()) {
-							doRead(key);
-						} else if (key.isAcceptable()) {
-							doAccept(key);
-						} else {
-							LOG.error("Unknown key: " + key);
-						}
-					} else {
-						LOG.error("Received invalid key: " + key);
-					}
-				}
-			}
-	
-			// Do cleanup, if necessary
-			if (this.listeningSocket != null) {
-				try {
-					this.listeningSocket.close();
-				} catch (IOException ioe) {
-					// Actually, we can ignore this exception
-					LOG.debug(ioe);
-				}
-			}
-	
-			// Finally, close the selector
-			try {
-				this.selector.close();
-			} catch (IOException ioe) {
-				LOG.debug(StringUtils.stringifyException(ioe));
-			}
-		}
-		catch (Throwable t) {
-			// this is a disaster, this task manager cannot go on!
-			LOG.fatal("Incoming connection thread died with an exception: " + t.getMessage(), t);
-			System.exit(1);
-		}
-	}
-
-	private void doAccept(SelectionKey key) {
-
-		SocketChannel clientSocket = null;
-
-		try {
-			clientSocket = this.listeningSocket.accept();
-			if (clientSocket == null) {
-				LOG.error("Client socket is null");
-				return;
-			}
-		} catch (IOException ioe) {
-			LOG.error(ioe);
-			return;
-		}
-
-		final IncomingConnection incomingConnection = new IncomingConnection(
-			clientSocket, this.channelManager);
-		SelectionKey clientKey = null;
-		try {
-			clientSocket.configureBlocking(false);
-			clientKey = clientSocket.register(this.selector, SelectionKey.OP_READ);
-			clientKey.attach(incomingConnection);
-		} catch (IOException ioe) {
-			incomingConnection.reportTransmissionProblem(clientKey, ioe);
-		}
-	}
-
-	private void doRead(SelectionKey key) {
-
-		final IncomingConnection incomingConnection = (IncomingConnection) key.attachment();
-		try {
-			incomingConnection.read();
-		} catch (EOFException eof) {
-			if (incomingConnection.isCloseUnexpected()) {
-				final SocketChannel socketChannel = (SocketChannel) key.channel();
-				LOG.error("Connection from " + socketChannel.socket().getRemoteSocketAddress()
-					+ " was closed unexpectedly");
-				incomingConnection.reportTransmissionProblem(key, eof);
-			} else {
-				incomingConnection.closeConnection(key);
-			}
-		} catch (IOException ioe) {
-			incomingConnection.reportTransmissionProblem(key, ioe);
-		} catch (InterruptedException e) {
-			// Nothing to do here
-		} catch (NoBufferAvailableException e) {
-			// There are no buffers available, unsubscribe from read event
-			final SocketChannel socketChannel = (SocketChannel) key.channel();
-			try {
-				final SelectionKey newKey = socketChannel.register(this.selector, 0);
-				newKey.attach(incomingConnection);
-			} catch (ClosedChannelException e1) {
-				incomingConnection.reportTransmissionProblem(key, e1);
-			}
-
-			final BufferAvailabilityListener bal = new IncomingConnectionBufferAvailListener(
-				this.pendingReadEventSubscribeRequests, key);
-			if (!e.getBufferProvider().registerBufferAvailabilityListener(bal)) {
-				// In the meantime, a buffer has become available again, subscribe to read event again
-
-				try {
-					final SelectionKey newKey = socketChannel.register(this.selector, SelectionKey.OP_READ);
-					newKey.attach(incomingConnection);
-				} catch (ClosedChannelException e1) {
-					incomingConnection.reportTransmissionProblem(key, e1);
-				}
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnection.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnection.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnection.java
deleted file mode 100644
index 7df1901..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnection.java
+++ /dev/null
@@ -1,529 +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.network.tcp;
-
-import eu.stratosphere.runtime.io.channels.ChannelID;
-import eu.stratosphere.runtime.io.network.RemoteReceiver;
-import eu.stratosphere.runtime.io.network.envelope.Envelope;
-import eu.stratosphere.runtime.io.network.envelope.EnvelopeWriter;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.nio.channels.SelectionKey;
-import java.nio.channels.SocketChannel;
-import java.nio.channels.WritableByteChannel;
-import java.util.ArrayDeque;
-import java.util.Iterator;
-import java.util.Queue;
-
-/**
- * This class represents an outgoing TCP connection through which {@link eu.stratosphere.runtime.io.network.envelope.Envelope} objects can be sent.
- * {@link eu.stratosphere.runtime.io.network.envelope.Envelope} objects are received from the {@link eu.stratosphere.nephele.taskmanager.io.bytebuffered.ChannelManager} and added to a queue. An
- * additional network thread then takes the envelopes from the queue and transmits them to the respective destination
- * host.
- * 
- */
-public class OutgoingConnection {
-
-	/**
-	 * The log object used to report debug information and possible errors.
-	 */
-	private static final Log LOG = LogFactory.getLog(OutgoingConnection.class);
-
-	/**
-	 * The address this outgoing connection is connected to.
-	 */
-	private final RemoteReceiver remoteReceiver;
-
-	/**
-	 * The outgoing connection thread which actually transmits the queued transfer envelopes.
-	 */
-	private final OutgoingConnectionThread connectionThread;
-
-	/**
-	 * The queue of transfer envelopes to be transmitted.
-	 */
-	private final Queue<Envelope> queuedEnvelopes = new ArrayDeque<Envelope>();
-
-	/**
-	 * The {@link eu.stratosphere.runtime.io.network.envelope.Envelope} that is currently processed.
-	 */
-	private Envelope currentEnvelope = null;
-
-	/**
-	 * Stores whether the underlying TCP connection is established. As this variable is accessed by the byte buffered
-	 * channel manager and the outgoing connection thread, it must be protected by a monitor.
-	 */
-	private boolean isConnected = false;
-
-	/**
-	 * Stores whether is underlying TCP connection is subscribed to the NIO write event. As this variable is accessed by
-	 * the byte buffered channel and the outgoing connection thread, it must be protected by a monitor.
-	 */
-	private boolean isSubscribedToWriteEvent = false;
-
-	/**
-	 * The overall number of connection retries which shall be performed before a connection error is reported.
-	 */
-	private final int numberOfConnectionRetries;
-
-	/**
-	 * The number of connection retries left before an I/O error is reported.
-	 */
-	private int retriesLeft = 0;
-
-	/**
-	 * The timestamp of the last connection retry.
-	 */
-	private long timstampOfLastRetry = 0;
-
-	/**
-	 * The current selection key representing the interest set of the underlying TCP NIO connection. This variable may
-	 * only be accessed the the outgoing connection thread.
-	 */
-	private SelectionKey selectionKey = null;
-
-	/**
-	 * The period of time in milliseconds that shall be waited before a connection attempt is considered to be failed.
-	 */
-	private static long RETRYINTERVAL = 1000L; // 1 second
-
-	private EnvelopeWriter writer;
-
-	/**
-	 * Constructs a new outgoing connection object.
-	 * 
-	 * @param remoteReceiver
-	 *        the address of the destination host this outgoing connection object is supposed to connect to
-	 * @param connectionThread
-	 *        the connection thread which actually handles the network transfer
-	 * @param numberOfConnectionRetries
-	 *        the number of connection retries allowed before an I/O error is reported
-	 */
-	public OutgoingConnection(RemoteReceiver remoteReceiver, OutgoingConnectionThread connectionThread,
-			int numberOfConnectionRetries) {
-
-		this.remoteReceiver = remoteReceiver;
-		this.connectionThread = connectionThread;
-		this.numberOfConnectionRetries = numberOfConnectionRetries;
-		this.writer = new EnvelopeWriter();
-	}
-
-	/**
-	 * Adds a new {@link eu.stratosphere.runtime.io.network.envelope.Envelope} to the queue of envelopes to be transmitted to the destination host of this
-	 * connection.
-	 * <p>
-	 * This method should only be called by the {@link eu.stratosphere.nephele.taskmanager.io.bytebuffered.ChannelManager} object.
-	 * 
-	 * @param envelope
-	 *        the envelope to be added to the transfer queue
-	 */
-	public void queueEnvelope(Envelope envelope) {
-
-		synchronized (this.queuedEnvelopes) {
-
-			checkConnection();
-			this.queuedEnvelopes.add(envelope);
-		}
-	}
-
-	private void checkConnection() {
-
-		synchronized (this.queuedEnvelopes) {
-
-			if (!this.isConnected) {
-
-				this.retriesLeft = this.numberOfConnectionRetries;
-				this.timstampOfLastRetry = System.currentTimeMillis();
-				this.connectionThread.triggerConnect(this);
-				this.isConnected = true;
-				this.isSubscribedToWriteEvent = true;
-			} else {
-
-				if (!this.isSubscribedToWriteEvent) {
-					this.connectionThread.subscribeToWriteEvent(this.selectionKey);
-					this.isSubscribedToWriteEvent = true;
-				}
-			}
-
-		}
-	}
-
-	/**
-	 * Returns the {@link InetSocketAddress} to the destination host this outgoing connection is supposed to be
-	 * connected to.
-	 * <p>
-	 * This method should be called by the {@link OutgoingConnectionThread} object only.
-	 * 
-	 * @return the {@link InetSocketAddress} to the destination host this outgoing connection is supposed to be
-	 *         connected to
-	 */
-	public InetSocketAddress getConnectionAddress() {
-
-		return this.remoteReceiver.getConnectionAddress();
-	}
-
-	/**
-	 * Reports a problem which occurred while establishing the underlying TCP connection to this outgoing connection
-	 * object. Depending on the number of connection retries left, this method will either try to reestablish the TCP
-	 * connection or report an I/O error to all tasks which have queued envelopes for this connection. In the latter
-	 * case all queued envelopes will be dropped and all included buffers will be freed.
-	 * <p>
-	 * This method should only be called by the {@link OutgoingConnectionThread} object.
-	 * 
-	 * @param ioe
-	 *        thrown if an error occurs while reseting the underlying TCP connection
-	 */
-	public void reportConnectionProblem(IOException ioe) {
-
-		// First, write exception to log
-		final long currentTime = System.currentTimeMillis();
-		if (currentTime - this.timstampOfLastRetry >= RETRYINTERVAL) {
-			LOG.error("Cannot connect to " + this.remoteReceiver + ", " + this.retriesLeft + " retries left");
-		}
-
-		synchronized (this.queuedEnvelopes) {
-
-			if (this.selectionKey != null) {
-
-				final SocketChannel socketChannel = (SocketChannel) this.selectionKey.channel();
-				if (socketChannel != null) {
-					try {
-						socketChannel.close();
-					} catch (IOException e) {
-						LOG.debug("Error while trying to close the socket channel to " + this.remoteReceiver);
-					}
-				}
-
-				this.selectionKey.cancel();
-				this.selectionKey = null;
-				this.isConnected = false;
-				this.isSubscribedToWriteEvent = false;
-			}
-
-			if (hasRetriesLeft(currentTime)) {
-				this.connectionThread.triggerConnect(this);
-				this.isConnected = true;
-				this.isSubscribedToWriteEvent = true;
-				return;
-			}
-
-			// Error is fatal
-			LOG.error(ioe);
-
-			// Notify source of current envelope and release buffer
-			if (this.currentEnvelope != null) {
-				if (this.currentEnvelope.getBuffer() != null) {
-					this.currentEnvelope.getBuffer().recycleBuffer();
-					this.currentEnvelope = null;
-				}
-			}
-
-			// Notify all other tasks which are waiting for data to be transmitted
-			final Iterator<Envelope> iter = this.queuedEnvelopes.iterator();
-			while (iter.hasNext()) {
-				final Envelope envelope = iter.next();
-				iter.remove();
-				// Recycle the buffer inside the envelope
-				if (envelope.getBuffer() != null) {
-					envelope.getBuffer().recycleBuffer();
-				}
-			}
-
-			this.queuedEnvelopes.clear();
-		}
-	}
-
-	/**
-	 * Reports an I/O error which occurred while writing data to the TCP connection. As a result of the I/O error the
-	 * connection is closed and the interest keys are canceled. Moreover, the task which queued the currently
-	 * transmitted transfer envelope is notified about the error and the current envelope is dropped. If the current
-	 * envelope contains a buffer, the buffer is freed.
-	 * <p>
-	 * This method should only be called by the {@link OutgoingConnectionThread} object.
-	 * 
-	 * @param ioe
-	 *        thrown if an error occurs while reseting the connection
-	 */
-	public void reportTransmissionProblem(IOException ioe) {
-
-		final SocketChannel socketChannel = (SocketChannel) this.selectionKey.channel();
-
-		// First, write exception to log
-		if (this.currentEnvelope != null) {
-			LOG.error("The connection between " + socketChannel.socket().getLocalAddress() + " and "
-				+ socketChannel.socket().getRemoteSocketAddress()
-				+ " experienced an IOException for transfer envelope " + this.currentEnvelope.getSequenceNumber());
-		} else {
-			LOG.error("The connection between " + socketChannel.socket().getLocalAddress() + " and "
-				+ socketChannel.socket().getRemoteSocketAddress() + " experienced an IOException");
-		}
-
-		// Close the connection and cancel the interest key
-		synchronized (this.queuedEnvelopes) {
-			try {
-				LOG.debug("Closing connection to " + socketChannel.socket().getRemoteSocketAddress());
-				socketChannel.close();
-			} catch (IOException e) {
-				LOG.debug("An error occurred while responding to an IOException");
-				LOG.debug(e);
-			}
-
-			this.selectionKey.cancel();
-
-			// Error is fatal
-			LOG.error(ioe);
-
-			// Trigger new connection if there are more envelopes to be transmitted
-			if (this.queuedEnvelopes.isEmpty()) {
-				this.isConnected = false;
-				this.isSubscribedToWriteEvent = false;
-			} else {
-				this.connectionThread.triggerConnect(this);
-				this.isConnected = true;
-				this.isSubscribedToWriteEvent = true;
-			}
-
-			// We must assume the current envelope is corrupted so we notify the task which created it.
-			if (this.currentEnvelope != null) {
-				if (this.currentEnvelope.getBuffer() != null) {
-					this.currentEnvelope.getBuffer().recycleBuffer();
-					this.currentEnvelope = null;
-				}
-			}
-		}
-	}
-
-	/**
-	 * Checks whether further retries are left for establishing the underlying TCP connection.
-	 * 
-	 * @param currentTime
-	 *        the current system time in milliseconds since January 1st, 1970
-	 * @return <code>true</code> if there are retries left, <code>false</code> otherwise
-	 */
-	private boolean hasRetriesLeft(long currentTime) {
-
-		if (currentTime - this.timstampOfLastRetry >= RETRYINTERVAL) {
-			this.retriesLeft--;
-			this.timstampOfLastRetry = currentTime;
-			if (this.retriesLeft == 0) {
-				return false;
-			}
-		}
-
-		return true;
-	}
-
-	/**
-	 * Writes the content of the current {@link eu.stratosphere.runtime.io.network.envelope.Envelope} object to the underlying TCP connection.
-	 * <p>
-	 * This method should only be called by the {@link OutgoingConnectionThread} object.
-	 * 
-	 * @return <code>true</code> if there is more data from this/other queued envelopes to be written to this channel
-	 * @throws IOException
-	 *         thrown if an error occurs while writing the data to the channel
-	 */
-	public boolean write() throws IOException {
-
-		final WritableByteChannel writableByteChannel = (WritableByteChannel) this.selectionKey.channel();
-
-		if (this.currentEnvelope == null) {
-			synchronized (this.queuedEnvelopes) {
-				if (this.queuedEnvelopes.isEmpty()) {
-					return false;
-				} else {
-					this.currentEnvelope = this.queuedEnvelopes.peek();
-
-					this.writer.setEnvelopeForWriting(this.currentEnvelope);
-				}
-			}
-		}
-
-		if (!this.writer.writeNextChunk(writableByteChannel)) {
-			// Make sure we recycle the attached memory or file buffers correctly
-			if (this.currentEnvelope.getBuffer() != null) {
-				this.currentEnvelope.getBuffer().recycleBuffer();
-			}
-
-			synchronized (this.queuedEnvelopes) {
-				this.queuedEnvelopes.poll();
-				this.currentEnvelope = null;
-			}
-		}
-
-		return true;
-	}
-
-	/**
-	 * Requests to close the underlying TCP connection. The request is ignored if at least one {@link eu.stratosphere.runtime.io.network.envelope.Envelope}
-	 * is queued.
-	 * <p>
-	 * This method should only be called by the {@link OutgoingConnectionThread} object.
-	 * 
-	 * @throws IOException
-	 *         thrown if an error occurs while closing the TCP connection
-	 */
-	public void requestClose() throws IOException {
-
-		synchronized (this.queuedEnvelopes) {
-
-			if (this.queuedEnvelopes.isEmpty()) {
-
-				if (this.isSubscribedToWriteEvent) {
-
-					this.connectionThread.unsubscribeFromWriteEvent(this.selectionKey);
-					this.isSubscribedToWriteEvent = false;
-				}
-			}
-		}
-	}
-
-	/**
-	 * Closes the underlying TCP connection if no more {@link eu.stratosphere.runtime.io.network.envelope.Envelope} objects are in the transmission queue.
-	 * <p>
-	 * This method should only be called by the {@link OutgoingConnectionThread} object.
-	 * 
-	 * @throws IOException
-	 */
-	public void closeConnection() throws IOException {
-
-		synchronized (this.queuedEnvelopes) {
-
-			if (!this.queuedEnvelopes.isEmpty()) {
-				return;
-			}
-
-			if (this.selectionKey != null) {
-
-				final SocketChannel socketChannel = (SocketChannel) this.selectionKey.channel();
-				socketChannel.close();
-				this.selectionKey.cancel();
-				this.selectionKey = null;
-			}
-
-			this.isConnected = false;
-			this.isSubscribedToWriteEvent = false;
-		}
-	}
-
-	/**
-	 * Returns the number of queued {@link eu.stratosphere.runtime.io.network.envelope.Envelope} objects with the given source channel ID.
-	 * 
-	 * @param sourceChannelID
-	 *        the source channel ID to count the queued envelopes for
-	 * @return the number of queued transfer envelopes with the given source channel ID
-	 */
-	public int getNumberOfQueuedEnvelopesFromChannel(final ChannelID sourceChannelID) {
-
-		synchronized (this.queuedEnvelopes) {
-
-			int number = 0;
-
-			final Iterator<Envelope> it = this.queuedEnvelopes.iterator();
-			while (it.hasNext()) {
-				final Envelope te = it.next();
-				if (sourceChannelID.equals(te.getSource())) {
-					number++;
-				}
-			}
-
-			return number;
-		}
-	}
-
-	/**
-	 * Removes all queued {@link eu.stratosphere.runtime.io.network.envelope.Envelope} objects from the transmission which match the given source channel
-	 * ID.
-	 * 
-	 * @param sourceChannelID
-	 *        the source channel ID of the transfered transfer envelopes to be dropped
-	 */
-	public void dropAllQueuedEnvelopesFromChannel(final ChannelID sourceChannelID) {
-
-		synchronized (this.queuedEnvelopes) {
-
-			final Iterator<Envelope> it = this.queuedEnvelopes.iterator();
-			while (it.hasNext()) {
-				final Envelope te = it.next();
-				if (sourceChannelID.equals(te.getSource())) {
-					it.remove();
-					if (te.getBuffer() != null) {
-						te.getBuffer().recycleBuffer();
-					}
-				}
-			}
-		}
-	}
-
-	/**
-	 * Checks whether this outgoing connection object manages an active connection or can be removed by the
-	 * {@link eu.stratosphere.nephele.taskmanager.io.bytebuffered.ChannelManager} object.
-	 * <p>
-	 * This method should only be called by the byte buffered channel manager.
-	 * 
-	 * @return <code>true</code> if this object is no longer manages an active connection and can be removed,
-	 *         <code>false</code> otherwise.
-	 */
-	public boolean canBeRemoved() {
-
-		synchronized (this.queuedEnvelopes) {
-
-			if (this.isConnected) {
-				return false;
-			}
-
-			if (this.currentEnvelope != null) {
-				return false;
-			}
-
-			return this.queuedEnvelopes.isEmpty();
-		}
-	}
-
-	/**
-	 * Sets the selection key representing the interest set of the underlying TCP NIO connection.
-	 * 
-	 * @param selectionKey
-	 *        the selection of the underlying TCP connection
-	 */
-	public void setSelectionKey(SelectionKey selectionKey) {
-		this.selectionKey = selectionKey;
-	}
-
-	/**
-	 * Returns the number of currently queued envelopes which contain a write buffer.
-	 * 
-	 * @return the number of currently queued envelopes which contain a write buffer
-	 */
-	public int getNumberOfQueuedWriteBuffers() {
-
-		int retVal = 0;
-
-		synchronized (this.queuedEnvelopes) {
-
-			final Iterator<Envelope> it = this.queuedEnvelopes.iterator();
-			while (it.hasNext()) {
-
-				final Envelope envelope = it.next();
-				if (envelope.getBuffer() != null) {
-					++retVal;
-				}
-			}
-		}
-
-		return retVal;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnectionThread.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnectionThread.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnectionThread.java
deleted file mode 100644
index dde26e3..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnectionThread.java
+++ /dev/null
@@ -1,276 +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.network.tcp;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.SelectionKey;
-import java.nio.channels.Selector;
-import java.nio.channels.SocketChannel;
-import java.util.ArrayDeque;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Queue;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.util.StringUtils;
-
-public class OutgoingConnectionThread extends Thread {
-
-	/**
-	 * The minimum time a TCP connection must be idle it is closed.
-	 */
-	private static final long MIN_IDLE_TIME_BEFORE_CLOSE = 80000L; // 80 seconds
-
-	private static final Log LOG = LogFactory.getLog(OutgoingConnectionThread.class);
-
-	private final Selector selector;
-
-	private final Queue<OutgoingConnection> pendingConnectionRequests = new ArrayDeque<OutgoingConnection>();
-
-	private final Queue<SelectionKey> pendingWriteEventSubscribeRequests = new ArrayDeque<SelectionKey>();
-
-	private final Map<OutgoingConnection, Long> connectionsToClose = new HashMap<OutgoingConnection, Long>();
-
-	public OutgoingConnectionThread() throws IOException {
-		super("Outgoing Connection Thread");
-
-		this.selector = Selector.open();
-	}
-
-
-	@Override
-	public void run() {
-		try {
-			while (!isInterrupted()) {
-	
-				synchronized (this.pendingConnectionRequests) {
-	
-					if (!this.pendingConnectionRequests.isEmpty()) {
-	
-						final OutgoingConnection outgoingConnection = this.pendingConnectionRequests.poll();
-						try {
-							final SocketChannel socketChannel = SocketChannel.open();
-							socketChannel.configureBlocking(false);
-							final SelectionKey key = socketChannel.register(this.selector, SelectionKey.OP_CONNECT);
-							socketChannel.connect(outgoingConnection.getConnectionAddress());
-							key.attach(outgoingConnection);
-						} catch (final IOException ioe) {
-							// IOException is reported by separate thread to avoid deadlocks
-							final Runnable reporterThread = new Runnable() {
-	
-								@Override
-								public void run() {
-									outgoingConnection.reportConnectionProblem(ioe);
-								}
-							};
-							new Thread(reporterThread).start();
-						}
-					}
-				}
-	
-				synchronized (this.pendingWriteEventSubscribeRequests) {
-	
-					if (!this.pendingWriteEventSubscribeRequests.isEmpty()) {
-						final SelectionKey oldSelectionKey = this.pendingWriteEventSubscribeRequests.poll();
-						final OutgoingConnection outgoingConnection = (OutgoingConnection) oldSelectionKey.attachment();
-						final SocketChannel socketChannel = (SocketChannel) oldSelectionKey.channel();
-	
-						try {
-							final SelectionKey newSelectionKey = socketChannel.register(this.selector, SelectionKey.OP_READ
-								| SelectionKey.OP_WRITE);
-							newSelectionKey.attach(outgoingConnection);
-							outgoingConnection.setSelectionKey(newSelectionKey);
-						} catch (final IOException ioe) {
-							// IOException is reported by separate thread to avoid deadlocks
-							final Runnable reporterThread = new Runnable() {
-	
-								@Override
-								public void run() {
-									outgoingConnection.reportTransmissionProblem(ioe);
-								}
-							};
-							new Thread(reporterThread).start();
-						}
-					}
-				}
-	
-				synchronized (this.connectionsToClose) {
-	
-					final Iterator<Map.Entry<OutgoingConnection, Long>> closeIt = this.connectionsToClose.entrySet()
-						.iterator();
-					final long now = System.currentTimeMillis();
-					while (closeIt.hasNext()) {
-	
-						final Map.Entry<OutgoingConnection, Long> entry = closeIt.next();
-						if ((entry.getValue().longValue() + MIN_IDLE_TIME_BEFORE_CLOSE) < now) {
-							final OutgoingConnection outgoingConnection = entry.getKey();
-							closeIt.remove();
-							// Create new thread to close connection to avoid deadlocks
-							final Runnable closeThread = new Runnable() {
-	
-								@Override
-								public void run() {
-									try {
-										outgoingConnection.closeConnection();
-									} catch (IOException ioe) {
-										outgoingConnection.reportTransmissionProblem(ioe);
-									}
-								}
-							};
-	
-							new Thread(closeThread).start();
-						}
-	
-					}
-				}
-	
-				try {
-					this.selector.select(10);
-				} catch (IOException e) {
-					LOG.error(e);
-				}
-	
-				final Iterator<SelectionKey> iter = this.selector.selectedKeys().iterator();
-	
-				while (iter.hasNext()) {
-					final SelectionKey key = iter.next();
-	
-					iter.remove();
-					if (key.isValid()) {
-						if (key.isConnectable()) {
-							doConnect(key);
-						} else {
-							if (key.isReadable()) {
-								doRead(key);
-								// A read will always result in an exception, so the write key will not be valid anymore
-								continue;
-							}
-							if (key.isWritable()) {
-								doWrite(key);
-							}
-						}
-					} else {
-						LOG.error("Received invalid key: " + key);
-					}
-				}
-			}
-	
-			// Finally, try to close the selector
-			try {
-				this.selector.close();
-			} catch (IOException ioe) {
-				LOG.debug(StringUtils.stringifyException(ioe));
-			}
-		}
-		catch (Throwable t) {
-			// this is a disaster, this task manager cannot go on!
-			LOG.fatal("Outgoing connection thread died with an exception: " + t.getMessage(), t);
-			System.exit(1);
-		}
-	}
-
-	private void doConnect(SelectionKey key) {
-
-		final OutgoingConnection outgoingConnection = (OutgoingConnection) key.attachment();
-		final SocketChannel socketChannel = (SocketChannel) key.channel();
-		try {
-			while (!socketChannel.finishConnect()) {
-				try {
-					Thread.sleep(100);
-				} catch (InterruptedException e1) {
-					LOG.error(e1);
-				}
-			}
-
-			final SelectionKey channelKey = socketChannel.register(selector, SelectionKey.OP_WRITE
-				| SelectionKey.OP_READ);
-			outgoingConnection.setSelectionKey(channelKey);
-			channelKey.attach(outgoingConnection);
-
-		} catch (IOException ioe) {
-			outgoingConnection.reportConnectionProblem(ioe);
-		}
-	}
-
-	private void doWrite(SelectionKey key) {
-
-		final OutgoingConnection outgoingConnection = (OutgoingConnection) key.attachment();
-
-		try {
-
-			if (!outgoingConnection.write()) {
-				// Try to close the connection
-				outgoingConnection.requestClose();
-			}
-
-		} catch (IOException ioe) {
-			outgoingConnection.reportTransmissionProblem(ioe);
-		}
-	}
-
-	private void doRead(SelectionKey key) {
-
-		final SocketChannel socketChannel = (SocketChannel) key.channel();
-		final OutgoingConnection outgoingConnection = (OutgoingConnection) key.attachment();
-		final ByteBuffer buf = ByteBuffer.allocate(8);
-
-		try {
-
-			if (socketChannel.read(buf) == -1) {
-				outgoingConnection.reportTransmissionProblem(new IOException(
-					"Read unexpected EOF from channel"));
-			} else {
-				LOG.error("Outgoing connection read real data from channel");
-			}
-		} catch (IOException ioe) {
-			outgoingConnection.reportTransmissionProblem(ioe);
-		}
-	}
-
-	public void triggerConnect(OutgoingConnection outgoingConnection) {
-
-		synchronized (this.pendingConnectionRequests) {
-			this.pendingConnectionRequests.add(outgoingConnection);
-		}
-	}
-
-	public void unsubscribeFromWriteEvent(SelectionKey selectionKey) throws IOException {
-
-		final SocketChannel socketChannel = (SocketChannel) selectionKey.channel();
-		final OutgoingConnection outgoingConnection = (OutgoingConnection) selectionKey.attachment();
-
-		final SelectionKey newSelectionKey = socketChannel.register(this.selector, SelectionKey.OP_READ);
-		newSelectionKey.attach(outgoingConnection);
-		outgoingConnection.setSelectionKey(newSelectionKey);
-
-		synchronized (this.connectionsToClose) {
-			this.connectionsToClose.put(outgoingConnection, Long.valueOf(System.currentTimeMillis()));
-		}
-	}
-
-	public void subscribeToWriteEvent(SelectionKey selectionKey) {
-
-		synchronized (this.pendingWriteEventSubscribeRequests) {
-			this.pendingWriteEventSubscribeRequests.add(selectionKey);
-		}
-		synchronized (this.connectionsToClose) {
-			this.connectionsToClose.remove((OutgoingConnection) selectionKey.attachment());
-		}
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/AdaptiveSpanningRecordDeserializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/AdaptiveSpanningRecordDeserializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/AdaptiveSpanningRecordDeserializer.java
index f4c8aec..696915f 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/AdaptiveSpanningRecordDeserializer.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/AdaptiveSpanningRecordDeserializer.java
@@ -15,9 +15,6 @@ package eu.stratosphere.runtime.io.serialization;
 
 import eu.stratosphere.core.io.IOReadableWritable;
 import eu.stratosphere.core.memory.MemorySegment;
-import eu.stratosphere.runtime.io.serialization.DataInputDeserializer;
-import eu.stratosphere.runtime.io.serialization.DataOutputSerializer;
-import eu.stratosphere.runtime.io.serialization.RecordDeserializer;
 
 import java.io.DataInput;
 import java.io.EOFException;
@@ -62,6 +59,7 @@ public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> im
 		// check if we can get a full length;
 		if (nonSpanningRemaining >= 4) {
 			int len = this.nonSpanningWrapper.readInt();
+
 			if (len <= nonSpanningRemaining - 4) {
 				// we can get a full record from here
 				target.read(this.nonSpanningWrapper);
@@ -156,8 +154,9 @@ public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> im
 
 		@Override
 		public final void readFully(byte[] b, int off, int len) throws IOException {
-			if (off < 0 || len < 0 || off + len > b.length)
+			if (off < 0 || len < 0 || off + len > b.length) {
 				throw new IndexOutOfBoundsException();
+			}
 			
 			this.segment.get(this.position, b, off, len);
 			this.position += len;
@@ -230,14 +229,16 @@ public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> im
 			try {
 				int b;
 				while ((b = readUnsignedByte()) != '\n') {
-					if (b != '\r')
+					if (b != '\r') {
 						bld.append((char) b);
+					}
 				}
 			}
 			catch (EOFException eofex) {}
 
-			if (bld.length() == 0)
+			if (bld.length() == 0) {
 				return null;
+			}
 			
 			// trim a trailing carriage return
 			int len = bld.length();
@@ -275,8 +276,9 @@ public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> im
 
 			while (count < utflen) {
 				c = (int) bytearr[count] & 0xff;
-				if (c > 127)
+				if (c > 127) {
 					break;
+				}
 				count++;
 				chararr[chararr_count++] = (char) c;
 			}
@@ -298,21 +300,25 @@ public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> im
 				case 12:
 				case 13:
 					count += 2;
-					if (count > utflen)
+					if (count > utflen) {
 						throw new UTFDataFormatException("malformed input: partial character at end");
+					}
 					char2 = (int) bytearr[count - 1];
-					if ((char2 & 0xC0) != 0x80)
+					if ((char2 & 0xC0) != 0x80) {
 						throw new UTFDataFormatException("malformed input around byte " + count);
+					}
 					chararr[chararr_count++] = (char) (((c & 0x1F) << 6) | (char2 & 0x3F));
 					break;
 				case 14:
 					count += 3;
-					if (count > utflen)
+					if (count > utflen) {
 						throw new UTFDataFormatException("malformed input: partial character at end");
+					}
 					char2 = (int) bytearr[count - 2];
 					char3 = (int) bytearr[count - 1];
-					if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80))
+					if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80)) {
 						throw new UTFDataFormatException("malformed input around byte " + (count - 1));
+					}
 					chararr[chararr_count++] = (char) (((c & 0x0F) << 12) | ((char2 & 0x3F) << 6) | ((char3 & 0x3F) << 0));
 					break;
 				default:
@@ -325,8 +331,9 @@ public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> im
 		
 		@Override
 		public final int skipBytes(int n) throws IOException {
-			if (n < 0)
+			if (n < 0) {
 				throw new IllegalArgumentException();
+			}
 			
 			int toSkip = Math.min(n, remaining());
 			this.position += toSkip;
@@ -390,6 +397,7 @@ public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> im
 					return;
 				} else {
 					this.recordLength = this.lengthBuffer.getInt(0);
+
 					this.lengthBuffer.clear();
 					segmentPosition = toPut;
 				}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataInputDeserializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataInputDeserializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataInputDeserializer.java
index e6479fe..a8a53fe 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataInputDeserializer.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataInputDeserializer.java
@@ -63,11 +63,13 @@ public class DataInputDeserializer implements DataInput {
 	}
 
 	public void setBuffer(byte[] buffer, int start, int len) {
-		if (buffer == null)
+		if (buffer == null) {
 			throw new NullPointerException();
+		}
 
-		if (start < 0 || len < 0 || start + len >= buffer.length)
+		if (start < 0 || len < 0 || start + len >= buffer.length) {
 			throw new IllegalArgumentException();
+		}
 
 		this.buffer = buffer;
 		this.position = start;
@@ -144,7 +146,7 @@ public class DataInputDeserializer implements DataInput {
 			@SuppressWarnings("restriction")
 			int value = UNSAFE.getInt(this.buffer, BASE_OFFSET + this.position);
 			if (LITTLE_ENDIAN) {
-				 value = Integer.reverseBytes(value);
+				value = Integer.reverseBytes(value);
 			}
 			
 			this.position += 4;
@@ -183,7 +185,7 @@ public class DataInputDeserializer implements DataInput {
 			@SuppressWarnings("restriction")
 			long value = UNSAFE.getLong(this.buffer, BASE_OFFSET + this.position);
 			if (LITTLE_ENDIAN) {
-				 value = Long.reverseBytes(value);
+				value = Long.reverseBytes(value);
 			}
 			this.position += 8;
 			return value;
@@ -215,8 +217,9 @@ public class DataInputDeserializer implements DataInput {
 
 		while (count < utflen) {
 			c = (int) bytearr[count] & 0xff;
-			if (c > 127)
+			if (c > 127) {
 				break;
+			}
 			count++;
 			chararr[chararr_count++] = (char) c;
 		}
@@ -240,22 +243,26 @@ public class DataInputDeserializer implements DataInput {
 			case 13:
 				/* 110x xxxx 10xx xxxx */
 				count += 2;
-				if (count > utflen)
+				if (count > utflen) {
 					throw new UTFDataFormatException("malformed input: partial character at end");
+				}
 				char2 = (int) bytearr[count - 1];
-				if ((char2 & 0xC0) != 0x80)
+				if ((char2 & 0xC0) != 0x80) {
 					throw new UTFDataFormatException("malformed input around byte " + count);
+				}
 				chararr[chararr_count++] = (char) (((c & 0x1F) << 6) | (char2 & 0x3F));
 				break;
 			case 14:
 				/* 1110 xxxx 10xx xxxx 10xx xxxx */
 				count += 3;
-				if (count > utflen)
+				if (count > utflen) {
 					throw new UTFDataFormatException("malformed input: partial character at end");
+				}
 				char2 = (int) bytearr[count - 2];
 				char3 = (int) bytearr[count - 1];
-				if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80))
+				if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80)) {
 					throw new UTFDataFormatException("malformed input around byte " + (count - 1));
+				}
 				chararr[chararr_count++] = (char) (((c & 0x0F) << 12) | ((char2 & 0x3F) << 6) | ((char3 & 0x3F) << 0));
 				break;
 			default:

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataOutputSerializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataOutputSerializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataOutputSerializer.java
index b5171b9..ce088f0 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataOutputSerializer.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataOutputSerializer.java
@@ -196,9 +196,9 @@ public class DataOutputSerializer implements DataOutput {
 			}
 		}
 
-		if (utflen > 65535)
+		if (utflen > 65535) {
 			throw new UTFDataFormatException("Encoded string is too long: " + utflen);
-		
+		}
 		else if (this.position > this.buffer.length - utflen - 2) {
 			resize(utflen + 2);
 		}
@@ -212,8 +212,9 @@ public class DataOutputSerializer implements DataOutput {
 		int i = 0;
 		for (i = 0; i < strlen; i++) {
 			c = str.charAt(i);
-			if (!((c >= 0x0001) && (c <= 0x007F)))
+			if (!((c >= 0x0001) && (c <= 0x007F))) {
 				break;
+			}
 			bytearr[count++] = (byte) c;
 		}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializer.java
index 443f8d8..b54496a 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializer.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializer.java
@@ -68,6 +68,7 @@ public class SpanningRecordSerializer<T extends IOReadableWritable> implements R
 
 		// write data and length
 		record.write(this.serializationBuffer);
+
 		this.lengthBuffer.putInt(0, this.serializationBuffer.length());
 
 		this.dataBuffer = this.serializationBuffer.wrapAsByteBuffer();
@@ -103,8 +104,9 @@ public class SpanningRecordSerializer<T extends IOReadableWritable> implements R
 	 * @param source the {@link ByteBuffer} to copy data from
 	 */
 	private void copyToTargetBufferFrom(ByteBuffer source) {
-		if (this.targetBuffer == null)
+		if (this.targetBuffer == null) {
 			return;
+		}
 
 		int needed = source.remaining();
 		int available = this.limit - this.position;
@@ -127,8 +129,9 @@ public class SpanningRecordSerializer<T extends IOReadableWritable> implements R
 
 	@Override
 	public Buffer getCurrentBuffer() {
-		if (targetBuffer == null)
+		if (targetBuffer == null) {
 			return null;
+		}
 
 		this.targetBuffer.limitSize(this.position);
 		return this.targetBuffer;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/TestBufferProvider.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/TestBufferProvider.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/TestBufferProvider.java
index 09b244f..3b2ad69 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/TestBufferProvider.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/TestBufferProvider.java
@@ -70,7 +70,7 @@ public class TestBufferProvider implements BufferProvider {
 	}
 
 	@Override
-	public boolean registerBufferAvailabilityListener(BufferAvailabilityListener bufferAvailabilityListener) {
+	public BufferAvailabilityRegistration registerBufferAvailabilityListener(BufferAvailabilityListener bufferAvailabilityListener) {
 		throw new UnsupportedOperationException();
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/OutputEmitterTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/OutputEmitterTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/OutputEmitterTest.java
index c66d821..13fbfbc 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/OutputEmitterTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/OutputEmitterTest.java
@@ -33,7 +33,6 @@ import eu.stratosphere.api.java.typeutils.runtime.record.RecordSerializerFactory
 import eu.stratosphere.core.memory.DataInputView;
 import eu.stratosphere.core.memory.DataOutputView;
 import eu.stratosphere.core.memory.MemorySegment;
-import eu.stratosphere.nephele.io.ChannelSelector;
 import eu.stratosphere.pact.runtime.plugable.SerializationDelegate;
 import eu.stratosphere.pact.runtime.shipping.OutputEmitter;
 import eu.stratosphere.pact.runtime.shipping.ShipStrategyType;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/RecordOutputEmitterTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/RecordOutputEmitterTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/RecordOutputEmitterTest.java
index 0b968d8..2bd5d98 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/RecordOutputEmitterTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/RecordOutputEmitterTest.java
@@ -28,8 +28,6 @@ import org.junit.Test;
 import eu.stratosphere.api.common.distributions.DataDistribution;
 import eu.stratosphere.api.common.distributions.UniformIntegerDistribution;
 import eu.stratosphere.runtime.io.api.ChannelSelector;
-import eu.stratosphere.pact.runtime.plugable.pactrecord.RecordComparator;
-import eu.stratosphere.nephele.io.ChannelSelector;
 import eu.stratosphere.api.java.typeutils.runtime.record.RecordComparator;
 import eu.stratosphere.pact.runtime.shipping.RecordOutputEmitter;
 import eu.stratosphere.pact.runtime.shipping.ShipStrategyType;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/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 a397312..b715a4e 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
@@ -44,7 +44,9 @@ import eu.stratosphere.util.MutableObjectIterator;
 import java.io.IOException;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.FutureTask;
 
 public class MockEnvironment implements Environment, BufferProvider, LocalBufferPoolOwner {
 	
@@ -124,8 +126,8 @@ public class MockEnvironment implements Environment, BufferProvider, LocalBuffer
 	}
 
 	@Override
-	public boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
-		return false;
+	public BufferAvailabilityRegistration registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
+		return BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_POOL_DESTROYED;
 	}
 
 	@Override
@@ -338,4 +340,9 @@ public class MockEnvironment implements Environment, BufferProvider, LocalBuffer
 	public BufferProvider getOutputBufferProvider() {
 		return this;
 	}
+
+	@Override
+	public Map<String, FutureTask<Path>> getCopyTask() {
+		return null;
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReaderWriterTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReaderWriterTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReaderWriterTest.java
deleted file mode 100644
index b8914a8..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReaderWriterTest.java
+++ /dev/null
@@ -1,394 +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.network.envelope;
-
-import eu.stratosphere.core.memory.MemorySegment;
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.util.DiscardingRecycler;
-import eu.stratosphere.nephele.util.TestBufferProvider;
-import eu.stratosphere.runtime.io.Buffer;
-import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
-import eu.stratosphere.runtime.io.network.bufferprovider.BufferProviderBroker;
-import eu.stratosphere.runtime.io.BufferRecycler;
-import eu.stratosphere.runtime.io.channels.ChannelID;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.File;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.nio.channels.ReadableByteChannel;
-import java.nio.channels.WritableByteChannel;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Random;
-
-public class EnvelopeReaderWriterTest {
-	
-	private final long RANDOM_SEED = 520346508276087l;
-
-	private static final int BUFFER_SIZE = 32768;
-	
-	private static final byte BUFFER_CONTENT = 13;
-	
-	private final int[] BUFFER_SIZES = { 0, 2, BUFFER_SIZE, 3782, 88, 0, 23};
-	
-	private final AbstractEvent[][] EVENT_LISTS = {
-		{},
-		{},
-		{},
-		{ new TestEvent1(34872527) },
-		{ new TestEvent1(8749653), new TestEvent1(365345) },
-		{ new TestEvent2(34563456), new TestEvent1(598432), new TestEvent2(976293845) },
-		{}
-	};
-
-	@Test
-	public void testWriteAndRead() {
-		
-		Assert.assertTrue("Test broken.", BUFFER_SIZES.length == EVENT_LISTS.length);
-
-		File testFile = null;
-		RandomAccessFile raf = null;
-		try {
-			testFile = File.createTempFile("envelopes", ".tmp");
-			raf = new RandomAccessFile(testFile, "rw");
-			
-			// write
-			FileChannel c = raf.getChannel();
-			writeEnvelopes(c);
-			
-			// read
-			c.position(0);
-			readEnvelopes(c, -1.0f);
-			c.close();
-		}
-		catch (Exception e) {
-			System.err.println(e.getMessage());
-			e.printStackTrace();
-			Assert.fail(e.getMessage());
-		}
-		finally {
-			if (raf != null)
-				try { raf.close(); } catch (Throwable t) {}
-			
-			if (testFile != null)
-				testFile.delete();
-		}
-	}
-	
-	@Test
-	public void testWriteAndReadChunked() {
-		
-		Assert.assertTrue("Test broken.", BUFFER_SIZES.length == EVENT_LISTS.length);
-
-		File testFile = null;
-		RandomAccessFile raf = null;
-		try {
-			testFile = File.createTempFile("envelopes", ".tmp");
-			raf = new RandomAccessFile(testFile, "rw");
-			
-			// write
-			FileChannel c = raf.getChannel();
-			writeEnvelopes(new ChunkedWriteableChannel(c));
-			
-			// read
-			c.position(0);
-			readEnvelopes(new ChunkedReadableChannel(c), 0.75f);
-			c.close();
-		}
-		catch (Exception e) {
-			System.err.println(e.getMessage());
-			e.printStackTrace();
-			Assert.fail(e.getMessage());
-		}
-		finally {
-			if (raf != null)
-				try { raf.close(); } catch (Throwable t) {}
-			
-			if (testFile != null)
-				testFile.delete();
-		}
-	}
-
-	private void writeEnvelopes(WritableByteChannel channel) throws IOException {
-
-		final BufferRecycler recycler = new DiscardingRecycler();
-		final Random rand = new Random(RANDOM_SEED);
-		
-		final EnvelopeWriter serializer = new EnvelopeWriter();
-		
-		final int NUM_ENVS = BUFFER_SIZES.length;
-		
-		for (int i = 0; i < NUM_ENVS; i++) {
-			int seqNum = Math.abs(rand.nextInt());
-			JobID jid = new JobID(rand.nextLong(), rand.nextLong());
-			ChannelID sid = new ChannelID(rand.nextLong(), rand.nextLong());
-			
-			Envelope env = new Envelope(seqNum, jid, sid);
-			if (EVENT_LISTS[i].length > 0) {
-				env.serializeEventList(Arrays.asList(EVENT_LISTS[i]));
-			}
-			
-			int bufferSize = BUFFER_SIZES[i];
-			if (bufferSize > 0) {
-				MemorySegment ms = new MemorySegment(new byte[BUFFER_SIZE]);
-				for (int x = 0; x < bufferSize; x++) {
-					ms.put(x, BUFFER_CONTENT);
-				}
-				
-				Buffer mb = new Buffer(ms, bufferSize, recycler);
-				env.setBuffer(mb);
-			}
-			
-			serializer.setEnvelopeForWriting(env);
-			
-			while (serializer.writeNextChunk(channel));
-		}
-	}
-	
-	private void readEnvelopes(ReadableByteChannel channel, float probabilityForNoBufferCurrently) throws IOException {
-		
-		final Random rand = new Random(RANDOM_SEED);
-		
-		final EnvelopeReader reader = new EnvelopeReader(new OneForAllBroker(BUFFER_SIZE, probabilityForNoBufferCurrently));
-		
-		final int NUM_ENVS = BUFFER_SIZES.length;
-		
-		for (int i = 0; i < NUM_ENVS; i++) {
-			int expectedSeqNum = Math.abs(rand.nextInt());
-			JobID expectedJid = new JobID(rand.nextLong(), rand.nextLong());
-			ChannelID expectedSid = new ChannelID(rand.nextLong(), rand.nextLong());
-			
-			// read the next envelope
-			while (reader.readNextChunk(channel) != EnvelopeReader.DeserializationState.COMPLETE);
-			Envelope env = reader.getFullyDeserializedTransferEnvelope();
-			
-			// check the basic fields from the header
-			Assert.assertEquals(expectedSeqNum, env.getSequenceNumber());
-			Assert.assertEquals(expectedJid, env.getJobID());
-			Assert.assertEquals(expectedSid, env.getSource());
-			
-			// check the events
-			List<? extends AbstractEvent> events = env.deserializeEvents();
-			Assert.assertEquals(EVENT_LISTS[i].length, events.size());
-			
-			for (int n = 0; n < EVENT_LISTS[i].length; n++) {
-				AbstractEvent expectedEvent = EVENT_LISTS[i][n];
-				AbstractEvent actualEvent = events.get(n);
-				
-				Assert.assertEquals(expectedEvent.getClass(), actualEvent.getClass());
-				Assert.assertEquals(expectedEvent, actualEvent);
-			}
-			
-			// check the buffer
-			Buffer buf = env.getBuffer();
-			if (buf == null) {
-				Assert.assertTrue(BUFFER_SIZES[i] == 0);
-			} else {
-				Assert.assertEquals(BUFFER_SIZES[i], buf.size());
-				for (int k = 0; k < BUFFER_SIZES[i]; k++) {
-					Assert.assertEquals(BUFFER_CONTENT, buf.getMemorySegment().get(k));
-				}
-			}
-			
-			reader.reset();
-		}
-		
-	}
-	
-	
-	public  static final class TestEvent1 extends AbstractEvent {
-
-		private long id;
-		
-		public TestEvent1() {}
-		
-		public TestEvent1(long id) {
-			this.id = id;
-		}
-		
-		@Override
-		public void write(DataOutput out) throws IOException {
-			out.writeLong(id);
-		}
-
-		@Override
-		public void read(DataInput in) throws IOException {
-			id = in.readLong();
-		}
-		
-
-		@Override
-		public boolean equals(Object obj) {
-			if (obj.getClass() == TestEvent1.class) {
-				return ((TestEvent1) obj).id == this.id;
-			} else {
-				return false;
-			}
-		}
-		
-		@Override
-		public int hashCode() {
-			return ((int) id) ^ ((int) (id >>> 32));
-		}
-		
-		@Override
-		public String toString() {
-			return "TestEvent1 (" + id + ")";
-		}
-	}
-	
-	public static final class TestEvent2 extends AbstractEvent {
-
-		private long id;
-		
-		public TestEvent2() {}
-		
-		public TestEvent2(long id) {
-			this.id = id;
-		}
-		
-		@Override
-		public void write(DataOutput out) throws IOException {
-			out.writeLong(id);
-		}
-
-		@Override
-		public void read(DataInput in) throws IOException {
-			id = in.readLong();
-		}
-		
-
-		@Override
-		public boolean equals(Object obj) {
-			if (obj.getClass() == TestEvent2.class) {
-				return ((TestEvent2) obj).id == this.id;
-			} else {
-				return false;
-			}
-		}
-		
-		@Override
-		public int hashCode() {
-			return ((int) id) ^ ((int) (id >>> 32));
-		}
-		
-		@Override
-		public String toString() {
-			return "TestEvent2 (" + id + ")";
-		}
-	}
-	
-	private static final class ChunkedWriteableChannel implements WritableByteChannel {
-		
-		private final WritableByteChannel delegate;
-		
-		private final Random rnd;
-		
-		private ChunkedWriteableChannel(WritableByteChannel delegate) {
-			this.delegate = delegate;
-			this.rnd = new Random();
-		}
-
-		@Override
-		public boolean isOpen() {
-			return this.delegate.isOpen();
-		}
-
-		@Override
-		public void close() throws IOException {
-			this.delegate.close();
-		}
-
-		@Override
-		public int write(ByteBuffer src) throws IOException {
-			final int available = src.remaining();
-			final int oldLimit = src.limit();
-			
-			int toWrite = rnd.nextInt(available) + 1;
-			toWrite = Math.min(Math.max(toWrite, 8), available);
-			
-			src.limit(src.position() + toWrite);
-			
-			int written = this.delegate.write(src);
-			
-			src.limit(oldLimit);
-			
-			return written;
-		}
-	}
-	
-	private static final class ChunkedReadableChannel implements ReadableByteChannel {
-		
-		private final ReadableByteChannel delegate;
-		
-		private final Random rnd;
-		
-		private ChunkedReadableChannel(ReadableByteChannel delegate) {
-			this.delegate = delegate;
-			this.rnd = new Random();
-		}
-
-		@Override
-		public boolean isOpen() {
-			return this.delegate.isOpen();
-		}
-
-		@Override
-		public void close() throws IOException {
-			this.delegate.close();
-		}
-
-		@Override
-		public int read(ByteBuffer dst) throws IOException {
-			final int available = dst.remaining();
-			final int oldLimit = dst.limit();
-			
-			int toRead = rnd.nextInt(available) + 1;
-			toRead = Math.min(Math.max(toRead, 8), available);
-			
-			dst.limit(dst.position() + toRead);
-			
-			int read = this.delegate.read(dst);
-			
-			dst.limit(oldLimit);
-			
-			return read;
-		}
-	}
-	
-	private static final class OneForAllBroker implements BufferProviderBroker {
-
-		private final TestBufferProvider provider;
-
-		private OneForAllBroker(int sizeOfMemorySegments) {
-			this.provider = new TestBufferProvider(sizeOfMemorySegments);
-		}
-		
-		private OneForAllBroker(int sizeOfMemorySegments, float probabilityForNoBufferCurrently) {
-			this.provider = new TestBufferProvider(sizeOfMemorySegments, probabilityForNoBufferCurrently);
-		}
-		
-		@Override
-		public BufferProvider getBufferProvider(JobID jobID, ChannelID sourceChannelID) {
-			return this.provider;
-		}
-	}
-}


[21/30] Offer buffer-oriented API for I/O (#25)

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/LocalChannelWithAccessInfo.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/LocalChannelWithAccessInfo.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/LocalChannelWithAccessInfo.java
deleted file mode 100644
index 173021f..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/LocalChannelWithAccessInfo.java
+++ /dev/null
@@ -1,170 +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.io.channels;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.nio.channels.FileChannel;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-final class LocalChannelWithAccessInfo implements ChannelWithAccessInfo {
-
-	/**
-	 * The logging object.
-	 */
-	private static final Log LOG = LogFactory.getLog(LocalChannelWithAccessInfo.class);
-
-	private final File file;
-
-	private final FileChannel channel;
-
-	private final AtomicLong reservedWritePosition;
-
-	private final AtomicInteger referenceCounter;
-
-	private final AtomicBoolean deleteOnClose;
-
-	LocalChannelWithAccessInfo(final File file, final boolean deleteOnClose) throws IOException {
-
-		this.file = file;
-		this.channel = new RandomAccessFile(file, "rw").getChannel();
-		this.reservedWritePosition = new AtomicLong(0L);
-		this.referenceCounter = new AtomicInteger(0);
-		this.deleteOnClose = new AtomicBoolean(deleteOnClose);
-	}
-
-
-	@Override
-	public FileChannel getChannel() {
-
-		return this.channel;
-	}
-
-
-	@Override
-	public FileChannel getAndIncrementReferences() {
-
-		if (incrementReferences()) {
-			return this.channel;
-		} else {
-			return null;
-		}
-	}
-
-	@Override
-	public ChannelWithPosition reserveWriteSpaceAndIncrementReferences(final int spaceToReserve) {
-
-		if (incrementReferences()) {
-			return new ChannelWithPosition(this.channel, this.reservedWritePosition.getAndAdd(spaceToReserve));
-		} else {
-			return null;
-		}
-	}
-
-
-	@Override
-	public int decrementReferences() {
-
-		int current = this.referenceCounter.get();
-		while (true) {
-			if (current <= 0) {
-				// this is actually an error case, because the channel was deleted before
-				throw new IllegalStateException("The references to the file were already at zero.");
-			}
-
-			if (current == 1) {
-				// this call decrements to zero, so mark it as deleted
-				if (this.referenceCounter.compareAndSet(current, Integer.MIN_VALUE)) {
-					current = 0;
-					break;
-				}
-			} else if (this.referenceCounter.compareAndSet(current, current - 1)) {
-				current = current - 1;
-				break;
-			}
-			current = this.referenceCounter.get();
-		}
-
-		if (current > 0) {
-			return current;
-		} else if (current == 0) {
-			// delete the channel
-			this.referenceCounter.set(Integer.MIN_VALUE);
-			this.reservedWritePosition.set(Long.MIN_VALUE);
-			try {
-				this.channel.close();
-			} catch (IOException ioex) {
-				if (LOG.isErrorEnabled()) {
-					LOG.error("Error while closing spill file for file buffers: " + ioex.getMessage(), ioex);
-				}
-			}
-			if (this.deleteOnClose.get()) {
-				this.file.delete();
-			}
-			return current;
-		} else {
-			throw new IllegalStateException("The references to the file were already at zero.");
-		}
-	}
-
-
-	@Override
-	public boolean incrementReferences() {
-
-		int current = this.referenceCounter.get();
-		while (true) {
-			// check whether it was disposed in the meantime
-			if (current < 0) {
-				return false;
-			}
-			// atomically check and increment
-			if (this.referenceCounter.compareAndSet(current, current + 1)) {
-				return true;
-			}
-			current = this.referenceCounter.get();
-		}
-	}
-
-
-	@Override
-	public void disposeSilently() {
-
-		this.referenceCounter.set(Integer.MIN_VALUE);
-		this.reservedWritePosition.set(Long.MIN_VALUE);
-
-		if (this.channel.isOpen()) {
-			try {
-				this.channel.close();
-			} catch (Throwable t) {
-			}
-		}
-
-		if (this.deleteOnClose.get()) {
-			this.file.delete();
-		}
-	}
-
-
-	@Override
-	public void updateDeleteOnCloseFlag(final boolean deleteOnClose) {
-
-		this.deleteOnClose.compareAndSet(true, deleteOnClose);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBuffer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBuffer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBuffer.java
deleted file mode 100644
index 8b2c0fb..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBuffer.java
+++ /dev/null
@@ -1,249 +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.io.channels;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.ReadableByteChannel;
-import java.nio.channels.WritableByteChannel;
-
-import eu.stratosphere.core.memory.MemorySegment;
-
-public final class MemoryBuffer extends Buffer {
-
-	private final MemoryBufferRecycler bufferRecycler;
-
-	private final MemorySegment internalMemorySegment;
-	
-	/**
-	 * Internal index that points to the next byte to write
-	 */
-	private int index = 0;
-	
-	/**
-	 * Internal limit to simulate ByteBuffer behavior of MemorySegment. index > limit is not allowed.
-	 */
-	private int limit = 0;
-
-	MemoryBuffer(final int bufferSize, final MemorySegment memory, final MemoryBufferPoolConnector bufferPoolConnector) {
-		if (bufferSize > memory.size()) {
-			throw new IllegalArgumentException("Requested segment size is " + bufferSize
-				+ ", but provided MemorySegment only has a capacity of " + memory.size());
-		}
-
-		this.bufferRecycler = new MemoryBufferRecycler(memory, bufferPoolConnector);
-		this.internalMemorySegment = memory;
-		this.position(0);
-		this.limit(bufferSize);
-	}
-
-	private MemoryBuffer(final int bufferSize, final int pos, final MemorySegment memory, final MemoryBufferRecycler bufferRecycler) {
-		this.bufferRecycler = bufferRecycler;
-		this.internalMemorySegment = memory;
-		this.position(pos);
-		this.limit(bufferSize);
-	}
-
-	@Override
-	public int read(ByteBuffer dst) throws IOException {
-
-		if (!this.hasRemaining()) {
-			return -1;
-		}
-		int numBytes = dst.remaining();
-		final int remaining = this.remaining();
-		if (numBytes == 0) {
-			return 0;
-		}
-		if(numBytes > remaining) {
-			numBytes = remaining;
-		}
-		internalMemorySegment.get(position(), dst, numBytes);
-		index += numBytes;
-		return numBytes;
-	}
-	
-
-	@Override
-	public int writeTo(WritableByteChannel writableByteChannel) throws IOException {
-		if (!this.hasRemaining()) {
-			return -1;
-		}
-		
-		final ByteBuffer wrapped = this.internalMemorySegment.wrap(index, limit-index);
-		final int written = writableByteChannel.write(wrapped);
-		position(wrapped.position());
-		return written;
-	}
-
-
-	@Override
-	public void close() throws IOException {
-
-		this.position(this.limit());
-	}
-
-
-	@Override
-	public boolean isOpen() {
-
-		return this.hasRemaining();
-	}
-
-	/**
-	 * Resets the memory buffer.
-	 * 
-	 * @param bufferSize
-	 *        the size of buffer in bytes after the reset
-	 */
-	public final void reset(final int bufferSize) {
-		if(bufferSize > this.internalMemorySegment.size()) {
-			throw new RuntimeException("Given buffer size exceeds underlying buffer size");
-		}
-		this.position(0);
-		this.limit(bufferSize);
-	}
-
-	public final void position(final int i) {
-		if(i > limit) {
-			throw new IndexOutOfBoundsException("new position is larger than the limit");
-		}
-		index = i;
-	}
-	
-	@Override
-	public final int position() {
-		return index;
-	}
-	
-	public final void limit(final int l) {
-		if(limit > internalMemorySegment.size()) {
-			throw new RuntimeException("Limit is larger than MemoryBuffer size");
-		}
-		if (index > limit) {
-			index = limit;
-		}
-		limit = l;
-	}
-	
-	public final int limit() {
-		return limit;
-	}
-	
-	public final boolean hasRemaining() {
-		return index < limit;
-	}
-	
-	public final int remaining() {
-		return limit - index;
-	}
-
-	/**
-	 * Put MemoryBuffer into read mode
-	 */
-	public final void flip() {
-		limit = position();
-		position(0);
-	}
-
-	public void clear() {
-		this.limit = getTotalSize();
-		this.position(0);
-	}
-
-	/**
-	 * 
-	 * @return Returns the size of the underlying MemorySegment
-	 */
-	public int getTotalSize() {
-		return this.internalMemorySegment.size();
-	}
-	
-	@Override
-	public final int size() {
-		return this.limit();
-	}
-
-	public MemorySegment getMemorySegment() {
-		return this.internalMemorySegment;
-	}
-
-
-	@Override
-	protected void recycle() {
-		this.bufferRecycler.decreaseReferenceCounter();
-		if(bufferRecycler.referenceCounter.get() == 0) {
-			clear();
-		}
-	}
-
-
-
-	@Override
-	public boolean isBackedByMemory() {
-		return true;
-	}
-
-
-	@Override
-	public MemoryBuffer duplicate() {
-		final MemoryBuffer duplicatedMemoryBuffer = new MemoryBuffer(this.limit(), this.position(), this.internalMemorySegment, this.bufferRecycler);
-		this.bufferRecycler.increaseReferenceCounter();
-		return duplicatedMemoryBuffer;
-	}
-
-
-	@Override
-	public void copyToBuffer(final Buffer destinationBuffer) throws IOException {
-		if (size() > destinationBuffer.size()) {
-			throw new IllegalArgumentException("Destination buffer is too small to store content of source buffer: "
-				+ size() + " vs. " + destinationBuffer.size());
-		}
-		final MemoryBuffer target = (MemoryBuffer) destinationBuffer;
-		this.internalMemorySegment.copyTo(this.position(), target.getMemorySegment(), destinationBuffer.position(), limit()-position());
-		target.position(limit()-position()); // even if we do not change the source (this), we change the destination!!
-		destinationBuffer.flip();
-	}
-
-	
-
-	@Override
-	public int write(final ByteBuffer src) throws IOException {
-		int numBytes = src.remaining();
-		final int thisRemaining = this.remaining();
-		if(thisRemaining == 0) {
-			return 0;
-		}
-		if(numBytes > thisRemaining) {
-			numBytes = thisRemaining;
-		}
-		this.internalMemorySegment.put(position(), src, numBytes);
-		this.index += numBytes;
-		return numBytes;
-	}
-
-
-	@Override
-	public int write(final ReadableByteChannel readableByteChannel) throws IOException {
-
-		if (!this.hasRemaining()) {
-			return 0;
-		}
-		ByteBuffer wrapper = this.internalMemorySegment.wrap(index, limit-index);
-		final int written = readableByteChannel.read(wrapper);
-		this.position(wrapper.position());
-		this.limit(wrapper.limit());
-		return written;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBufferPoolConnector.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBufferPoolConnector.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBufferPoolConnector.java
deleted file mode 100644
index bd8519d..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBufferPoolConnector.java
+++ /dev/null
@@ -1,32 +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.io.channels;
-
-import eu.stratosphere.core.memory.MemorySegment;
-
-/**
- * The memory buffer pool connector provides a connection between {@link MemoryBuffer} and the {@link LocalBufferPool}
- * the memory buffer's encapsulated byte buffer has originally been taken from.
- * 
- */
-public interface MemoryBufferPoolConnector {
-
-	/**
-	 * Called by the {@link MemoryBufferRecycler} to return a buffer to its original buffer pool.
-	 * 
-	 * @param byteBuffer
-	 *        the buffer to be recycled
-	 */
-	void recycle(MemorySegment memSeg);
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBufferRecycler.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBufferRecycler.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBufferRecycler.java
deleted file mode 100644
index 141e3a1..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBufferRecycler.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.io.channels;
-
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.core.memory.MemorySegment;
-
-/**
- * A memory buffer recycler takes care of the correct recycling of the internal byte buffer which backs a memory buffer.
- * Since buffer objects can be duplicated, i.e. multiple buffer objects point to the same physical buffer, it is
- * necessary to coordinate the recycling of the physical buffer.
- * <p>
- * This class is thread-safe.
- * 
- */
-public final class MemoryBufferRecycler {
-
-	/**
-	 * The log object used to report debug information and possible errors.
-	 */
-	private static final Log LOG = LogFactory.getLog(MemoryBufferRecycler.class);
-
-	/**
-	 * The original memory segment which has been taken from byte buffered channel manager's buffer pool.
-	 */
-	private final MemorySegment originalSegment;
-
-	/**
-	 * The connection to the pool from which the byte buffer has originally been taken.
-	 */
-	private final MemoryBufferPoolConnector bufferPoolConnector;
-
-	/**
-	 * The number of memory buffer objects which may still access the physical buffer.
-	 */
-	public final AtomicInteger referenceCounter = new AtomicInteger(1);
-
-	/**
-	 * Constructs a new memory buffer recycler.
-	 * 
-	 * @param originalBuffer
-	 *        the original byte buffer
-	 * @param bufferPoolConnector
-	 *        the connection to the pool from which the byte buffer has originally been taken
-	 */
-	MemoryBufferRecycler(final MemorySegment originalSegment, final MemoryBufferPoolConnector bufferPoolConnector) {
-
-		this.originalSegment = originalSegment;
-		this.bufferPoolConnector = bufferPoolConnector;
-	}
-
-	/**
-	 * Increases the number of references to the physical buffer by one.
-	 */
-	void increaseReferenceCounter() {
-
-		if (this.referenceCounter.getAndIncrement() == 0) {
-			LOG.error("Increasing reference counter from 0 to 1");
-		}
-	}
-
-	/**
-	 * Decreases the number of references to the physical buffer by one. If the number of references becomes zero the
-	 * physical buffer is recycled.
-	 */
-	void decreaseReferenceCounter() {
-
-		final int val = this.referenceCounter.decrementAndGet();
-		if (val == 0) {
-			this.bufferPoolConnector.recycle(this.originalSegment);
-
-		} else if (val < 0) {
-			LOG.error("reference counter is negative");
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/SerializationBuffer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/SerializationBuffer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/SerializationBuffer.java
deleted file mode 100644
index 55bcb56..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/SerializationBuffer.java
+++ /dev/null
@@ -1,141 +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.io.channels;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.WritableByteChannel;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.io.DataOutputBuffer;
-
-/**
- * A class for serializing a record to its binary representation.
- * 
- * @param <T>
- *        the type of the record this serialization buffer can be used for
- */
-public class SerializationBuffer<T extends IOReadableWritable> {
-
-	private static final int SIZEOFINT = 4;
-
-	private DataOutputBuffer serializationBuffer = new DataOutputBuffer();
-
-	private ByteBuffer lengthBuf = ByteBuffer.allocate(SIZEOFINT);
-
-	private int bytesReadFromBuffer = 0;
-
-	/**
-	 * Translates an integer into an array of bytes.
-	 * 
-	 * @param val
-	 *        The integer to be translated
-	 * @param arr
-	 *        The byte buffer to store the data of the integer
-	 */
-	private void integerToByteBuffer(final int val, final ByteBuffer byteBuffer) {
-
-		for (int i = 0; i < SIZEOFINT; ++i) {
-			final int shift = i << (SIZEOFINT - 1); // i * 8
-			byteBuffer.put(SIZEOFINT - 1 - i, (byte) ((val & (0xff << shift)) >>> shift));
-		}
-
-		byteBuffer.position(0);
-		byteBuffer.limit(SIZEOFINT);
-	}
-
-	/**
-	 * Return <code>true</code> if the internal serialization buffer still contains data.
-	 * In this case the method serialize must not be called. If the internal buffer is empty
-	 * the method return <code>false</code>
-	 * 
-	 * @return <code>true</code> if the internal serialization buffer still contains data, <code>false</code> it it is
-	 *         empty
-	 */
-	public boolean dataLeftFromPreviousSerialization() {
-		return leftInSerializationBuffer() > 0;
-	}
-
-	/**
-	 * Reads the internal serialization buffer and writes the data to the given {@link WritableByteChannel} byte
-	 * channel.
-	 * 
-	 * @param writableByteChannel
-	 *        the byte channel to write the serialized data to
-	 * @return the number of bytes written the to given byte channel
-	 * @throws IOException
-	 *         thrown if an error occurs while writing to serialized data to the channel
-	 */
-	public int read(final WritableByteChannel writableByteChannel) throws IOException {
-
-		int bytesReadFromLengthBuf = 0;
-
-		// Deal with length buffer first
-		if (this.lengthBuf.hasRemaining()) { // There is data from the length buffer to be written
-			bytesReadFromLengthBuf = writableByteChannel.write(this.lengthBuf);
-		}
-
-		final int bytesReadFromSerializationBuf = writableByteChannel.write(this.serializationBuffer.getData());
-		// byteBuffer.put(this.serializationBuffer.getData(), this.bytesReadFromBuffer, length);
-		this.bytesReadFromBuffer += bytesReadFromSerializationBuf;
-
-		if (leftInSerializationBuffer() == 0) { // Record is entirely written to byteBuffer
-			this.serializationBuffer.reset();
-			this.bytesReadFromBuffer = 0;
-		}
-
-		return (bytesReadFromSerializationBuf + bytesReadFromLengthBuf);
-	}
-
-	/**
-	 * Return the number of bytes that have not been read from the internal serialization
-	 * buffer so far.
-	 * 
-	 * @return the number of bytes that have not been read from the internal serialization buffer so far
-	 */
-	private int leftInSerializationBuffer() {
-
-		return (this.serializationBuffer.getLength() - this.bytesReadFromBuffer);
-	}
-
-	/**
-	 * Serializes the record and writes it to an internal buffer. The buffer grows dynamically
-	 * in case more memory is required to serialization.
-	 * 
-	 * @param record
-	 *        The record to the serialized
-	 * @throws IOException
-	 *         Thrown if data from a previous serialization process is still in the internal buffer and has not yet been
-	 *         transfered to a byte buffer
-	 */
-	public void serialize(final T record) throws IOException {
-
-		// Check if there is data left in the buffer
-		if (dataLeftFromPreviousSerialization()) {
-			throw new IOException("Cannot write new data, " + leftInSerializationBuffer()
-				+ " bytes still left from previous call");
-		}
-
-		record.write(this.serializationBuffer); // serializationBuffer grows dynamically
-
-		// Now record is completely in serializationBuffer;
-		integerToByteBuffer(this.serializationBuffer.getLength(), this.lengthBuf);
-	}
-
-	public void clear() {
-		this.bytesReadFromBuffer = 0;
-		this.lengthBuf.clear();
-		this.serializationBuffer.reset();
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedInputChannel.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedInputChannel.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedInputChannel.java
deleted file mode 100644
index 0009d5a..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedInputChannel.java
+++ /dev/null
@@ -1,243 +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.io.channels.bytebuffered;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
-import eu.stratosphere.nephele.io.InputChannelResult;
-import eu.stratosphere.nephele.io.InputGate;
-import eu.stratosphere.nephele.io.RecordDeserializer;
-import eu.stratosphere.nephele.io.channels.AbstractInputChannel;
-import eu.stratosphere.nephele.io.channels.Buffer;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-
-/**
- * @param <T> The type of record that can be transported through this channel.
- */
-public abstract class AbstractByteBufferedInputChannel<T extends IOReadableWritable> extends AbstractInputChannel<T> {
-
-	/**
-	 * The log object used to report warnings and errors.
-	 */
-	private static final Log LOG = LogFactory.getLog(AbstractByteBufferedInputChannel.class);
-
-	/**
-	 * The deserializer used to deserialize records.
-	 */
-	private final RecordDeserializer<T> deserializer;
-
-	/**
-	 * Buffer for the uncompressed (raw) data.
-	 */
-	private Buffer dataBuffer;
-
-	private ByteBufferedInputChannelBroker inputChannelBroker;
-	
-	private AbstractTaskEvent currentEvent;
-
-	/**
-	 * The exception observed in this channel while processing the buffers. Checked and thrown
-	 * per-buffer.
-	 */
-	private volatile IOException ioException;
-
-	/**
-	 * Stores the number of bytes read through this input channel since its instantiation.
-	 */
-	private long amountOfDataTransmitted;
-	
-
-	private volatile boolean brokerAggreedToCloseChannel;
-
-	/**
-	 * Creates a new input channel.
-	 * 
-	 * @param inputGate
-	 *        the input gate this channel is wired to
-	 * @param channelIndex
-	 *        the channel's index at the associated input gate
-	 * @param type
-	 *        the type of record transported through this channel
-	 * @param channelID
-	 *        the ID of the channel
-	 * @param connectedChannelID
-	 *        the ID of the channel this channel is connected to
-	 */
-	public AbstractByteBufferedInputChannel(final InputGate<T> inputGate, final int channelIndex,
-			final RecordDeserializer<T> deserializer, final ChannelID channelID, final ChannelID connectedChannelID) {
-		super(inputGate, channelIndex, channelID, connectedChannelID);
-		this.deserializer = deserializer;
-	}
-
-	@Override
-	public InputChannelResult readRecord(T target) throws IOException {
-		if (this.dataBuffer == null) {
-			if (isClosed()) {
-				return InputChannelResult.END_OF_STREAM;
-			}
-
-			// get the next element we need to handle (buffer or event)
-			BufferOrEvent boe = this.inputChannelBroker.getNextBufferOrEvent();
-			
-			if (boe == null) {
-				throw new IllegalStateException("Input channel was queries for data even though none was announced available.");
-			}
-			
-			// handle events
-			if (boe.isEvent())
-			{
-				// sanity check: an event may only come after a complete record.
-				if (this.deserializer.hasUnfinishedData()) {
-					throw new IOException("Channel received an event before completing the current partial record.");
-				}
-				
-				AbstractEvent evt = boe.getEvent();
-				if (evt.getClass() == ByteBufferedChannelCloseEvent.class) {
-					this.brokerAggreedToCloseChannel = true;
-					return InputChannelResult.END_OF_STREAM;
-				}
-				else if (evt.getClass() == EndOfSuperstepEvent.class) {
-					return InputChannelResult.END_OF_SUPERSTEP;
-				}
-				else if (evt instanceof AbstractTaskEvent) {
-					this.currentEvent = (AbstractTaskEvent) evt;
-					return InputChannelResult.TASK_EVENT;
-				}
-				else {
-					LOG.error("Received unknown event: " + evt);
-					return InputChannelResult.NONE;
-				}
-			} else {
-				// buffer case
-				this.dataBuffer = boe.getBuffer();
-			}
-		}
-
-		// get the next record form the buffer
-		T nextRecord = this.deserializer.readData(target, this.dataBuffer);
-
-		// release the buffer if it is empty
-		if (this.dataBuffer.remaining() == 0) {
-			releasedConsumedReadBuffer(this.dataBuffer);
-			this.dataBuffer = null;
-			return nextRecord == null ? InputChannelResult.NONE : InputChannelResult.LAST_RECORD_FROM_BUFFER;
-		} else {
-			return nextRecord == null ? InputChannelResult.NONE : InputChannelResult.INTERMEDIATE_RECORD_FROM_BUFFER;
-		}
-	}
-
-	@Override
-	public boolean isClosed() throws IOException{
-		if (this.ioException != null) {
-			throw new IOException("An error occurred in the channel: " + this.ioException.getMessage(), this.ioException);
-		} else {
-			return this.brokerAggreedToCloseChannel;
-		}
-	}
-
-
-	@Override
-	public void close() throws IOException, InterruptedException {
-
-		this.deserializer.clear();
-		if (this.dataBuffer != null) {
-			releasedConsumedReadBuffer(this.dataBuffer);
-			this.dataBuffer = null;
-		}
-
-		// This code fragment makes sure the isClosed method works in case the channel input has not been fully consumed
-		while (!this.brokerAggreedToCloseChannel)
-		{
-			BufferOrEvent next = this.inputChannelBroker.getNextBufferOrEvent();
-			if (next != null) {
-				if (next.isEvent()) {
-					if (next.getEvent() instanceof ByteBufferedChannelCloseEvent) {
-						this.brokerAggreedToCloseChannel = true;
-					}
-				} else {
-					releasedConsumedReadBuffer(next.getBuffer());
-				}
-			} else {
-				Thread.sleep(200);
-			}
-		}
-
-		// Send close event to indicate the input channel has successfully
-		// processed all data it is interested in.
-		transferEvent(new ByteBufferedChannelCloseEvent());
-	}
-
-	
-	private void releasedConsumedReadBuffer(Buffer buffer) {
-		this.amountOfDataTransmitted += buffer.size();
-		buffer.recycleBuffer();
-	}
-	
-
-	public void setInputChannelBroker(ByteBufferedInputChannelBroker inputChannelBroker) {
-		this.inputChannelBroker = inputChannelBroker;
-	}
-
-
-	public void notifyGateThatInputIsAvailable() {
-		this.getInputGate().notifyRecordIsAvailable(getChannelIndex());
-	}
-
-	
-	@Override
-	public void transferEvent(AbstractEvent event) throws IOException, InterruptedException {
-		this.inputChannelBroker.transferEventToOutputChannel(event);
-	}
-
-	
-	public void reportIOException(IOException ioe) {
-		this.ioException = ioe;
-	}
-
-	
-	@Override
-	public void releaseAllResources() {
-		this.brokerAggreedToCloseChannel = true;
-		this.deserializer.clear();
-
-		// The buffers are recycled by the input channel wrapper
-	}
-
-	
-	@Override
-	public long getAmountOfDataTransmitted() {
-		return this.amountOfDataTransmitted;
-	}
-
-	
-	/**
-	 * Notify the channel that a data unit has been consumed.
-	 */
-	public void notifyDataUnitConsumed() {
-		this.getInputGate().notifyDataUnitConsumed(getChannelIndex());
-	}
-	
-	@Override
-	public AbstractTaskEvent getCurrentEvent() {
-		AbstractTaskEvent e = this.currentEvent;
-		this.currentEvent = null;
-		return e;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java
deleted file mode 100644
index 50d4a50..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java
+++ /dev/null
@@ -1,255 +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.io.channels.bytebuffered;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
-import eu.stratosphere.nephele.io.OutputGate;
-import eu.stratosphere.nephele.io.channels.AbstractOutputChannel;
-import eu.stratosphere.nephele.io.channels.Buffer;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-import eu.stratosphere.nephele.io.channels.SerializationBuffer;
-
-public abstract class AbstractByteBufferedOutputChannel<T extends IOReadableWritable> extends AbstractOutputChannel<T> {
-
-	/**
-	 * The serialization buffer used to serialize records.
-	 */
-	private final SerializationBuffer<T> serializationBuffer = new SerializationBuffer<T>();
-
-	/**
-	 * Buffer for the serialized output data.
-	 */
-	private Buffer dataBuffer = null;
-
-	/**
-	 * Stores whether the channel is requested to be closed.
-	 */
-	private boolean closeRequested = false;
-
-	/**
-	 * The output channel broker the channel should contact to request and release write buffers.
-	 */
-	private ByteBufferedOutputChannelBroker outputChannelBroker = null;
-
-
-	/**
-	 * Stores the number of bytes transmitted through this output channel since its instantiation.
-	 */
-	private long amountOfDataTransmitted = 0L;
-
-	private static final Log LOG = LogFactory.getLog(AbstractByteBufferedOutputChannel.class);
-
-	/**
-	 * Creates a new byte buffered output channel.
-	 * 
-	 * @param outputGate
-	 *        the output gate this channel is wired to
-	 * @param channelIndex
-	 *        the channel's index at the associated output gate
-	 * @param channelID
-	 *        the ID of the channel
-	 * @param connectedChannelID
-	 *        the ID of the channel this channel is connected to
-	 */
-	protected AbstractByteBufferedOutputChannel(final OutputGate<T> outputGate, final int channelIndex,
-			final ChannelID channelID, final ChannelID connectedChannelID) {
-		super(outputGate, channelIndex, channelID, connectedChannelID);
-	}
-
-
-	@Override
-	public boolean isClosed() throws IOException, InterruptedException {
-
-		if (this.closeRequested && this.dataBuffer == null
-			&& !this.serializationBuffer.dataLeftFromPreviousSerialization()) {
-
-			if (!this.outputChannelBroker.hasDataLeftToTransmit()) {
-				return true;
-			}
-		}
-
-		return false;
-	}
-
-
-	@Override
-	public void requestClose() throws IOException, InterruptedException {
-
-		if (!this.closeRequested) {
-			this.closeRequested = true;
-			if (this.serializationBuffer.dataLeftFromPreviousSerialization()) {
-				// make sure we serialized all data before we send the close event
-				flush();
-			}
-
-			if (getType() == ChannelType.INMEMORY || !isBroadcastChannel() || getChannelIndex() == 0) {
-				transferEvent(new ByteBufferedChannelCloseEvent());
-				flush();
-			}
-		}
-	}
-
-	/**
-	 * Requests a new write buffer from the framework. This method blocks until the requested buffer is available.
-	 * 
-	 * @throws InterruptedException
-	 *         thrown if the thread is interrupted while waiting for the buffer
-	 * @throws IOException
-	 *         thrown if an I/O error occurs while waiting for the buffer
-	 */
-	private void requestWriteBufferFromBroker() throws InterruptedException, IOException {
-		if (Thread.interrupted()) {
-			throw new InterruptedException();
-		}
-		this.dataBuffer = this.outputChannelBroker.requestEmptyWriteBuffer();
-	}
-
-	/**
-	 * Returns the filled buffer to the framework and triggers further processing.
-	 * 
-	 * @throws IOException
-	 *         thrown if an I/O error occurs while releasing the buffers
-	 * @throws InterruptedException
-	 *         thrown if the thread is interrupted while releasing the buffers
-	 */
-	private void releaseWriteBuffer() throws IOException, InterruptedException {
-		// Keep track of number of bytes transmitted through this channel
-		this.amountOfDataTransmitted += this.dataBuffer.size();
-
-		this.outputChannelBroker.releaseWriteBuffer(this.dataBuffer);
-		this.dataBuffer = null;
-	}
-
-
-	@Override
-	public void writeRecord(T record) throws IOException, InterruptedException {
-
-		// Get a write buffer from the broker
-		if (this.dataBuffer == null) {
-			requestWriteBufferFromBroker();
-		}
-
-		if (this.closeRequested) {
-			throw new IOException("Channel is aready requested to be closed");
-		}
-
-		// Check if we can accept new records or if there are still old
-		// records to be transmitted
-		while (this.serializationBuffer.dataLeftFromPreviousSerialization()) {
-
-			this.serializationBuffer.read(this.dataBuffer);
-			if (this.dataBuffer.remaining() == 0) {
-				releaseWriteBuffer();
-				requestWriteBufferFromBroker();
-			}
-		}
-
-		if (this.serializationBuffer.dataLeftFromPreviousSerialization()) {
-			throw new IOException("Serialization buffer is expected to be empty!");
-		}
-
-		this.serializationBuffer.serialize(record);
-
-		this.serializationBuffer.read(this.dataBuffer);
-		if (this.dataBuffer.remaining() == 0) {
-			releaseWriteBuffer();
-		}
-	}
-
-	/**
-	 * Sets the output channel broker this channel should contact to request and release write buffers.
-	 * 
-	 * @param byteBufferedOutputChannelBroker
-	 *        the output channel broker the channel should contact to request and release write buffers
-	 */
-	public void setByteBufferedOutputChannelBroker(ByteBufferedOutputChannelBroker byteBufferedOutputChannelBroker) {
-
-		this.outputChannelBroker = byteBufferedOutputChannelBroker;
-	}
-
-
-	public void processEvent(AbstractEvent event) {
-
-		if (event instanceof AbstractTaskEvent) {
-			getOutputGate().deliverEvent((AbstractTaskEvent) event);
-		} else {
-			LOG.error("Channel " + getID() + " received unknown event " + event);
-		}
-	}
-
-
-	@Override
-	public void transferEvent(AbstractEvent event) throws IOException, InterruptedException {
-
-		flush();
-		this.outputChannelBroker.transferEventToInputChannel(event);
-	}
-
-
-	@Override
-	public void flush() throws IOException, InterruptedException {
-
-		// Get rid of remaining data in the serialization buffer
-		while (this.serializationBuffer.dataLeftFromPreviousSerialization()) {
-
-			if (this.dataBuffer == null) {
-
-				try {
-					requestWriteBufferFromBroker();
-				} catch (InterruptedException e) {
-					LOG.error(e);
-				}
-			}
-			this.serializationBuffer.read(this.dataBuffer);
-			if (this.dataBuffer.remaining() == 0) {
-				releaseWriteBuffer();
-			}
-		}
-
-		// Get rid of the leased write buffer
-		if (this.dataBuffer != null) {
-			releaseWriteBuffer();
-		}
-	}
-
-
-	@Override
-	public void releaseAllResources() {
-
-		// TODO: Reconsider release of broker's resources here
-		this.closeRequested = true;
-
-		this.serializationBuffer.clear();
-
-		if (this.dataBuffer != null) {
-			this.dataBuffer.recycleBuffer();
-			this.dataBuffer = null;
-		}
-	}
-
-
-	@Override
-	public long getAmountOfDataTransmitted() {
-
-		return this.amountOfDataTransmitted;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/BufferOrEvent.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/BufferOrEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/BufferOrEvent.java
deleted file mode 100644
index f89c7fb..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/BufferOrEvent.java
+++ /dev/null
@@ -1,52 +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.io.channels.bytebuffered;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.io.channels.Buffer;
-
-/**
- * Either type for {@link Buffer} and {@link AbstractEvent}.
- */
-public class BufferOrEvent {
-	
-	private final Buffer buffer;
-	
-	private final AbstractEvent event;
-	
-	public BufferOrEvent(Buffer buffer) {
-		this.buffer = buffer;
-		this.event = null;
-	}
-	
-	public BufferOrEvent(AbstractEvent event) {
-		this.buffer = null;
-		this.event = event;
-	}
-	
-	public boolean isBuffer() {
-		return this.buffer != null;
-	}
-	
-	public boolean isEvent() {
-		return this.event != null;
-	}
-	
-	public Buffer getBuffer() {
-		return this.buffer;
-	}
-	
-	public AbstractEvent getEvent() {
-		return this.event;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedChannelCloseEvent.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedChannelCloseEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedChannelCloseEvent.java
deleted file mode 100644
index f427b70..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedChannelCloseEvent.java
+++ /dev/null
@@ -1,35 +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.io.channels.bytebuffered;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-
-public class ByteBufferedChannelCloseEvent extends AbstractEvent {
-
-	@Override
-	public void read(DataInput in) throws IOException {
-
-		// Nothing to do here
-	}
-
-	@Override
-	public void write(DataOutput out) throws IOException {
-		// Nothing to do here
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedInputChannelBroker.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedInputChannelBroker.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedInputChannelBroker.java
deleted file mode 100644
index 4d259ec..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedInputChannelBroker.java
+++ /dev/null
@@ -1,36 +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.io.channels.bytebuffered;
-
-import java.io.IOException;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-
-
-public interface ByteBufferedInputChannelBroker {
-
-	public BufferOrEvent getNextBufferOrEvent() throws IOException;
-
-	/**
-	 * Forwards the given event to the connected network output channel on a best effort basis.
-	 * 
-	 * @param event
-	 *        the event to be transferred
-	 * @throws InterruptedException
-	 *         thrown if the thread is interrupted while waiting for the event to be transfered
-	 * @throws IOException
-	 *         thrown if an I/O error occurs while transferring the event
-	 */
-	void transferEventToOutputChannel(AbstractEvent event) throws IOException, InterruptedException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedOutputChannelBroker.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedOutputChannelBroker.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedOutputChannelBroker.java
deleted file mode 100644
index af45e4c..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedOutputChannelBroker.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.io.channels.bytebuffered;
-
-import java.io.IOException;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.io.channels.Buffer;
-
-public interface ByteBufferedOutputChannelBroker {
-
-	/**
-	 * Requests an empty write buffer from the broker. This method will block
-	 * until the requested write buffer is available.
-	 * 
-	 * @return the byte buffers to write in
-	 * @throws InterruptedException
-	 *         thrown if the connected task is interrupted while waiting for the buffer
-	 * @throws IOException
-	 *         thrown if an error occurs while requesting the empty write buffer.
-	 */
-	Buffer requestEmptyWriteBuffer() throws InterruptedException, IOException;
-
-	/**
-	 * Returns a filled write buffer to the broker. The broker will take care
-	 * of the buffers and transfer the user data to the connected input channel on a best effort basis.
-	 * 
-	 * @param buffer
-	 *        the buffer to be returned to the broker
-	 * @throws InterruptedException
-	 *         thrown if the thread is interrupted while waiting for the buffers to be released
-	 * @throws IOException
-	 *         thrown if an I/O error occurs while releasing the buffers
-	 */
-	void releaseWriteBuffer(Buffer buffer) throws IOException, InterruptedException;
-
-	/**
-	 * Checks if there is still data created by this output channel that must be transfered to the corresponding input
-	 * channel.
-	 * 
-	 * @return <code>true</code> if the channel has data left to transmit, <code>false</code> otherwise
-	 * @throws InterruptedException
-	 *         thrown if the connected task is interrupted while waiting for the remaining data to be transmitted
-	 * @throws IOException
-	 *         thrown if an error occurs while transmitting the remaining data
-	 */
-	boolean hasDataLeftToTransmit() throws IOException, InterruptedException;
-
-	/**
-	 * Forwards the given event to the connected network input channel on a best effort basis.
-	 * 
-	 * @param event
-	 *        the event to be transferred
-	 * @throws InterruptedException
-	 *         thrown if the thread is interrupted while waiting for the event to be transfered
-	 * @throws IOException
-	 *         thrown if an I/O error occurs while transfering the event
-	 */
-	void transferEventToInputChannel(AbstractEvent event) throws IOException, InterruptedException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/EndOfSuperstepEvent.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/EndOfSuperstepEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/EndOfSuperstepEvent.java
deleted file mode 100644
index b0b20d4..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/EndOfSuperstepEvent.java
+++ /dev/null
@@ -1,34 +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.io.channels.bytebuffered;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-
-/**
- * Marks the end of a superstep of one particular iteration head
- */
-public class EndOfSuperstepEvent extends AbstractEvent {
-	
-	public static final EndOfSuperstepEvent INSTANCE = new EndOfSuperstepEvent();
-
-	@Override
-	public void write(DataOutput out) throws IOException {}
-
-	@Override
-	public void read(DataInput in) throws IOException {}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/InMemoryInputChannel.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/InMemoryInputChannel.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/InMemoryInputChannel.java
deleted file mode 100644
index 0609fcb..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/InMemoryInputChannel.java
+++ /dev/null
@@ -1,34 +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.io.channels.bytebuffered;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.io.InputGate;
-import eu.stratosphere.nephele.io.RecordDeserializer;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-
-public final class InMemoryInputChannel<T extends IOReadableWritable> extends AbstractByteBufferedInputChannel<T> {
-
-	public InMemoryInputChannel(InputGate<T> inputGate, int channelIndex, RecordDeserializer<T> deserializer,
-			ChannelID channelID, ChannelID connectedChannelID) {
-		super(inputGate, channelIndex, deserializer, channelID, connectedChannelID);
-	}
-
-	@Override
-	public ChannelType getType() {
-
-		return ChannelType.INMEMORY;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/InMemoryOutputChannel.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/InMemoryOutputChannel.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/InMemoryOutputChannel.java
deleted file mode 100644
index 0113d6f..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/InMemoryOutputChannel.java
+++ /dev/null
@@ -1,34 +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.io.channels.bytebuffered;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.io.OutputGate;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-
-public final class InMemoryOutputChannel<T extends IOReadableWritable> extends AbstractByteBufferedOutputChannel<T> {
-
-	public InMemoryOutputChannel(OutputGate<T> outputGate, int channelIndex, ChannelID channelID,
-			ChannelID connectedChannelID) {
-		super(outputGate, channelIndex, channelID, connectedChannelID);
-	}
-
-	@Override
-	public ChannelType getType() {
-
-		return ChannelType.INMEMORY;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/NetworkInputChannel.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/NetworkInputChannel.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/NetworkInputChannel.java
deleted file mode 100644
index ee11c0e..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/NetworkInputChannel.java
+++ /dev/null
@@ -1,34 +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.io.channels.bytebuffered;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.io.InputGate;
-import eu.stratosphere.nephele.io.RecordDeserializer;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-
-public final class NetworkInputChannel<T extends IOReadableWritable> extends AbstractByteBufferedInputChannel<T> {
-
-	public NetworkInputChannel(InputGate<T> inputGate, int channelIndex, RecordDeserializer<T> deserializer,
-			ChannelID channelID, ChannelID connectedChannelID) {
-		super(inputGate, channelIndex, deserializer, channelID, connectedChannelID);
-	}
-
-	@Override
-	public ChannelType getType() {
-
-		return ChannelType.NETWORK;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/NetworkOutputChannel.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/NetworkOutputChannel.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/NetworkOutputChannel.java
deleted file mode 100644
index c98f41a..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/NetworkOutputChannel.java
+++ /dev/null
@@ -1,34 +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.io.channels.bytebuffered;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.io.OutputGate;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-
-public final class NetworkOutputChannel<T extends IOReadableWritable> extends AbstractByteBufferedOutputChannel<T> {
-
-	public NetworkOutputChannel(OutputGate<T> outputGate, int channelIndex, ChannelID channelID,
-			ChannelID connectedChannelID) {
-		super(outputGate, channelIndex, channelID, connectedChannelID);
-	}
-
-	@Override
-	public ChannelType getType() {
-
-		return ChannelType.NETWORK;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/DirectoryReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/DirectoryReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/DirectoryReader.java
deleted file mode 100644
index acfddde..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/DirectoryReader.java
+++ /dev/null
@@ -1,103 +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.io.library;
-
-import java.util.Iterator;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.core.fs.FSDataInputStream;
-import eu.stratosphere.core.fs.FileInputSplit;
-import eu.stratosphere.core.fs.FileSystem;
-import eu.stratosphere.nephele.io.RecordWriter;
-import eu.stratosphere.nephele.template.AbstractFileInputTask;
-import eu.stratosphere.nephele.types.FileRecord;
-
-public class DirectoryReader extends AbstractFileInputTask {
-
-	/**
-	 * The record writer to write the output strings to.
-	 */
-	private RecordWriter<FileRecord> output = null;
-
-	// buffer
-	private byte[] buffer;
-
-	private static final Log LOG = LogFactory.getLog(DirectoryReader.class);
-
-
-	@Override
-	public void invoke() throws Exception {
-
-		final Iterator<FileInputSplit> splitIterator = getFileInputSplits();
-		FileRecord fr = null;
-
-		while (splitIterator.hasNext()) {
-
-			final FileInputSplit split = splitIterator.next();
-
-			final long start = split.getStart();
-			final long end = start + split.getLength();
-
-			if (buffer == null || buffer.length < end - start) {
-				buffer = new byte[(int) (end - start)];
-			}
-
-			if (fr == null || fr.getFileName().compareTo(split.getPath().getName()) != 0) {
-				if (fr != null) {
-					try {
-						output.emit(fr);
-					} catch (InterruptedException e) {
-						// TODO: Respond to interruption properly
-						LOG.error(e);
-					}
-				}
-				fr = new FileRecord(split.getPath().getName());
-			}
-
-			final FileSystem fs = FileSystem.get(split.getPath().toUri());
-
-			final FSDataInputStream fdis = fs.open(split.getPath());
-			fdis.seek(split.getStart());
-
-			int read = fdis.read(buffer, 0, buffer.length);
-			if (read == -1) {
-				continue;
-			}
-
-			fr.append(buffer, 0, read);
-
-			if (read != end - start) {
-				System.err.println("Unexpected number of bytes read! Expected: " + (end - start) + " Read: " + read);
-			}
-		}
-
-		if (fr != null) {
-			try {
-				output.emit(fr);
-			} catch (InterruptedException e) {
-				// TODO: Respond to interruption properly
-				LOG.error(e);
-			}
-		}
-	}
-
-
-	@Override
-	public void registerInputOutput() {
-		output = new RecordWriter<FileRecord>(this, FileRecord.class);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/DirectoryWriter.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/DirectoryWriter.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/DirectoryWriter.java
deleted file mode 100644
index 915aaa7..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/DirectoryWriter.java
+++ /dev/null
@@ -1,67 +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.io.library;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.core.fs.FSDataOutputStream;
-import eu.stratosphere.core.fs.FileSystem;
-import eu.stratosphere.core.fs.Path;
-import eu.stratosphere.nephele.io.RecordReader;
-import eu.stratosphere.nephele.template.AbstractFileOutputTask;
-import eu.stratosphere.nephele.types.FileRecord;
-
-public class DirectoryWriter extends AbstractFileOutputTask {
-
-	/**
-	 * The record reader to read the incoming strings from.
-	 */
-	private RecordReader<FileRecord> input = null;
-
-	private static final Log LOG = LogFactory.getLog(DirectoryWriter.class);
-
-	@Override
-	public void invoke() throws Exception {
-
-		final Path path = getFileOutputPath();
-		final FileSystem fs = path.getFileSystem();
-
-		try {
-			while (input.hasNext()) {
-
-				final FileRecord record = input.next();
-				Path newPath = new Path(path + Path.SEPARATOR + record.getFileName());
-				FSDataOutputStream outputStream = fs.create(newPath, true);
-
-				outputStream.write(record.getDataBuffer(), 0, record.getDataBuffer().length);
-				outputStream.close();
-				// TODO: Implement me
-				// System.out.println(input.next());
-				// ODO Auto-generated catch block
-			}
-		} catch (InterruptedException e) {
-			// TODO Auto-generated catch block
-			LOG.error(e);
-		}
-
-	}
-
-	@Override
-	public void registerInputOutput() {
-		input = new RecordReader<FileRecord>(this, FileRecord.class);
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/FileLineReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/FileLineReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/FileLineReader.java
deleted file mode 100644
index a26c56c..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/FileLineReader.java
+++ /dev/null
@@ -1,76 +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.io.library;
-
-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.nephele.io.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 {
-
-		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();
-		}
-	}
-
-	@Override
-	public void registerInputOutput() {
-		output = new RecordWriter<StringRecord>(this, StringRecord.class);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/FileLineWriter.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/FileLineWriter.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/FileLineWriter.java
deleted file mode 100644
index 5be3e1b..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/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.io.library;
-
-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.io.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/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/ipc/Client.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/ipc/Client.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/ipc/Client.java
index 0204ec6..ea7bd4a 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/ipc/Client.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/ipc/Client.java
@@ -19,6 +19,16 @@
 
 package eu.stratosphere.nephele.ipc;
 
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.core.io.StringRecord;
+import eu.stratosphere.nephele.net.NetUtils;
+import eu.stratosphere.nephele.util.IOUtils;
+import eu.stratosphere.runtime.io.serialization.DataOutputSerializer;
+import eu.stratosphere.util.ClassUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import javax.net.SocketFactory;
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
 import java.io.DataInputStream;
@@ -31,24 +41,13 @@ import java.net.InetSocketAddress;
 import java.net.Socket;
 import java.net.SocketTimeoutException;
 import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
 import java.util.Hashtable;
 import java.util.Iterator;
 import java.util.Map.Entry;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 
-import javax.net.SocketFactory;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.DataOutputBuffer;
-import eu.stratosphere.nephele.net.NetUtils;
-import eu.stratosphere.nephele.util.IOUtils;
-import eu.stratosphere.util.ClassUtils;
-
 /**
  * A client for an IPC service. IPC calls take a single {@link Writable} as a
  * parameter, and return a {@link Writable} as their value. A service runs on
@@ -379,13 +378,14 @@ public class Client {
 			out.write(Server.HEADER.array());
 
 			// Write out the ConnectionHeader
-			DataOutputBuffer buf = new DataOutputBuffer();
+			DataOutputSerializer buf = new DataOutputSerializer(4 + header.getProtocol().getBytes().length + 1);
 			header.write(buf);
 
 			// Write out the payload length
-			int bufLen = buf.getLength();
+			ByteBuffer wrapper = buf.wrapAsByteBuffer();
+			int bufLen = wrapper.limit();
 			out.writeInt(bufLen);
-			out.write(buf.getData().array(), 0, bufLen);
+			out.write(wrapper.array(), 0, bufLen);
 		}
 
 		/*
@@ -456,30 +456,27 @@ public class Client {
 				return;
 			}
 
-			DataOutputBuffer d = null;
+			DataOutputSerializer d = null;
 			try {
 				synchronized (this.out) {
 
 					// for serializing the
 					// data to be written
-					d = new DataOutputBuffer();
+					d = new DataOutputSerializer(64);
 					// First, write call id to buffer d
 					d.writeInt(call.id);
 					// Then write RPC data (the actual call) to buffer d
 					call.param.write(d);
 
-					byte[] data = d.getData().array();
-					int dataLength = d.getLength();
+					ByteBuffer wrapper = d.wrapAsByteBuffer();
+					byte[] data = wrapper.array();
+					int dataLength = wrapper.limit();
 					out.writeInt(dataLength); // first put the data length
 					out.write(data, 0, dataLength);// write the data
 					out.flush();
 				}
 			} catch (IOException e) {
 				markClosed(e);
-			} finally {
-				// the buffer is just an in-memory buffer, but it is still polite to
-				// close early
-				IOUtils.closeStream(d);
 			}
 		}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 6af9d74..b4c51f2 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
@@ -23,8 +23,7 @@ import eu.stratosphere.configuration.IllegalConfigurationException;
 import eu.stratosphere.core.io.IOReadableWritable;
 import eu.stratosphere.core.io.StringRecord;
 import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
-import eu.stratosphere.nephele.io.DistributionPattern;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.channels.ChannelType;
 import eu.stratosphere.nephele.template.AbstractInvokable;
 import eu.stratosphere.nephele.util.EnumUtils;
 import eu.stratosphere.util.StringUtils;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/DistributionPattern.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/DistributionPattern.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/DistributionPattern.java
new file mode 100644
index 0000000..3b8c9a0
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/DistributionPattern.java
@@ -0,0 +1,33 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ **********************************************************************************************************************/
+
+package eu.stratosphere.nephele.jobgraph;
+
+/**
+ * A distribution pattern determines which subtasks of a producing Nephele task a wired to which
+ * subtasks of a consuming subtask.
+ * 
+ */
+
+public enum DistributionPattern {
+
+	/**
+	 * Each subtask of the producing Nephele task is wired to each subtask of the consuming Nephele task.
+	 */
+	BIPARTITE,
+
+	/**
+	 * The i-th subtask of the producing Nephele task is wired to the i-th subtask of the consuming Nephele task.
+	 */
+	POINTWISE
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobEdge.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobEdge.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobEdge.java
index 51396ff..45788dd 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobEdge.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobEdge.java
@@ -13,8 +13,7 @@
 
 package eu.stratosphere.nephele.jobgraph;
 
-import eu.stratosphere.nephele.io.DistributionPattern;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.channels.ChannelType;
 
 /**
  * Objects of this class represent edges in the user's job graph.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobID.java
index e4467a0..2ec2ed6 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobID.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobID.java
@@ -1,5 +1,5 @@
 /***********************************************************************************************************************
- * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ * Copyright (C) 2010-2014 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
@@ -13,81 +13,44 @@
 
 package eu.stratosphere.nephele.jobgraph;
 
+import java.nio.ByteBuffer;
+
 import javax.xml.bind.DatatypeConverter;
 
-import eu.stratosphere.nephele.io.AbstractID;
+import eu.stratosphere.nephele.AbstractID;
 
-/**
- * A class for statistically unique job IDs.
- * <p>
- * This class is thread-safe.
- * 
- */
 public final class JobID extends AbstractID {
 
-	/**
-	 * Constructs a new random ID from a uniform distribution.
-	 */
 	public JobID() {
 		super();
 	}
 
-	/**
-	 * Constructs a new job ID.
-	 * 
-	 * @param lowerPart
-	 *        the lower bytes of the ID
-	 * @param upperPart
-	 *        the higher bytes of the ID
-	 */
-	private JobID(final long lowerPart, final long upperPart) {
+	public JobID(long lowerPart, long upperPart) {
 		super(lowerPart, upperPart);
 	}
 
-	/**
-	 * Constructs a new job ID from the given bytes.
-	 * 
-	 * @param bytes
-	 *        the bytes to initialize the job ID with
-	 */
-	public JobID(final byte[] bytes) {
+	public JobID(byte[] bytes) {
 		super(bytes);
 	}
 
-	/**
-	 * Generates a new statistically unique job ID.
-	 * 
-	 * @return a new statistically unique job ID
-	 */
 	public static JobID generate() {
-
-		final long lowerPart = AbstractID.generateRandomBytes();
-		final long upperPart = AbstractID.generateRandomBytes();
+		long lowerPart = AbstractID.generateRandomLong();
+		long upperPart = AbstractID.generateRandomLong();
 
 		return new JobID(lowerPart, upperPart);
 	}
 
-	/**
-	 * Constructs a new job ID and initializes it with the given bytes.
-	 * 
-	 * @param bytes
-	 *        the bytes to initialize the new job ID with
-	 * @return the new job ID
-	 */
-	public static JobID fromByteArray(final byte[] bytes) {
-
+	public static JobID fromByteArray(byte[] bytes) {
 		return new JobID(bytes);
 	}
-	
-	/**
-	 * Constructs a new job ID and initializes it with the given bytes.
-	 * 
-	 * @param bytes
-	 *        the bytes to initialize the new job ID with
-	 * @return the new job ID
-	 */
-	public static JobID fromHexString(final String hexString) {
 
+	public static JobID fromByteBuffer(ByteBuffer buf, int offset) {
+		long lower = buf.getLong(offset);
+		long upper = buf.getLong(offset + 8);
+		return new JobID(lower, upper);
+	}
+
+	public static JobID fromHexString(String hexString) {
 		return new JobID(DatatypeConverter.parseHexBinary(hexString));
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobVertexID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobVertexID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobVertexID.java
index c6f63ca..689cc02 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobVertexID.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobVertexID.java
@@ -13,7 +13,7 @@
 
 package eu.stratosphere.nephele.jobgraph;
 
-import eu.stratosphere.nephele.io.AbstractID;
+import eu.stratosphere.nephele.AbstractID;
 
 /**
  * A class for statistically unique job vertex IDs.


[20/30] Offer buffer-oriented API for I/O (#25)

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 eba81a2..846ca2e 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
@@ -75,7 +75,7 @@ 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.nephele.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.channels.ChannelID;
 import eu.stratosphere.nephele.ipc.RPC;
 import eu.stratosphere.nephele.ipc.Server;
 import eu.stratosphere.nephele.jobgraph.AbstractJobVertex;
@@ -90,8 +90,6 @@ import eu.stratosphere.nephele.jobmanager.splitassigner.InputSplitManager;
 import eu.stratosphere.nephele.jobmanager.splitassigner.InputSplitWrapper;
 import eu.stratosphere.nephele.jobmanager.web.WebInfoServer;
 import eu.stratosphere.nephele.managementgraph.ManagementGraph;
-import eu.stratosphere.nephele.managementgraph.ManagementVertexID;
-import eu.stratosphere.nephele.multicast.MulticastManager;
 import eu.stratosphere.nephele.profiling.JobManagerProfiler;
 import eu.stratosphere.nephele.profiling.ProfilingUtils;
 import eu.stratosphere.nephele.protocols.AccumulatorProtocol;
@@ -103,11 +101,10 @@ import eu.stratosphere.nephele.services.accumulators.AccumulatorEvent;
 import eu.stratosphere.nephele.taskmanager.AbstractTaskResult;
 import eu.stratosphere.nephele.taskmanager.TaskCancelResult;
 import eu.stratosphere.nephele.taskmanager.TaskExecutionState;
-import eu.stratosphere.nephele.taskmanager.TaskKillResult;
 import eu.stratosphere.nephele.taskmanager.TaskSubmissionResult;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.ConnectionInfoLookupResponse;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.RemoteReceiver;
-import eu.stratosphere.nephele.taskmanager.runtime.ExecutorThreadFactory;
+import eu.stratosphere.runtime.io.network.ConnectionInfoLookupResponse;
+import eu.stratosphere.runtime.io.network.RemoteReceiver;
+import eu.stratosphere.nephele.taskmanager.ExecutorThreadFactory;
 import eu.stratosphere.nephele.topology.NetworkTopology;
 import eu.stratosphere.nephele.types.IntegerRecord;
 import eu.stratosphere.nephele.util.SerializableArrayList;
@@ -141,8 +138,6 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 	private final InputSplitManager inputSplitManager;
 
 	private final AbstractScheduler scheduler;
-
-	private final MulticastManager multicastManager;
 	
 	private AccumulatorManager accumulatorManager;
 
@@ -246,9 +241,6 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 			throw new Exception("Unable to load scheduler " + schedulerClassName);
 		}
 
-		// Create multicastManager
-		this.multicastManager = new MulticastManager(this.scheduler);
-
 		// Load profiler if it should be used
 		if (GlobalConfiguration.getBoolean(ProfilingUtils.ENABLE_PROFILING_KEY, false)) {
 			final String profilerClassName = GlobalConfiguration.getString(ProfilingUtils.JOBMANAGER_CLASSNAME_KEY,
@@ -732,8 +724,7 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 
 
 	@Override
-	public ConnectionInfoLookupResponse lookupConnectionInfo(final InstanceConnectionInfo caller, final JobID jobID,
-			final ChannelID sourceChannelID) {
+	public ConnectionInfoLookupResponse lookupConnectionInfo(InstanceConnectionInfo caller, JobID jobID, ChannelID sourceChannelID) {
 
 		final ExecutionGraph eg = this.scheduler.getExecutionGraphByID(jobID);
 		if (eg == null) {
@@ -754,7 +745,6 @@ 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();
@@ -768,9 +758,11 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 			// Check execution state
 			final ExecutionState executionState = connectedVertex.getExecutionState();
 			if (executionState == ExecutionState.FINISHED) {
-				return ConnectionInfoLookupResponse.createReceiverFoundAndReady();
+				// that should not happen. if there is data pending, the receiver cannot be ready
+				return ConnectionInfoLookupResponse.createReceiverNotFound();
 			}
 
+			// running is common, finishing is happens when the lookup is for the close event
 			if (executionState != ExecutionState.RUNNING && executionState != ExecutionState.FINISHING) {
 				// LOG.info("Created receiverNotReady for " + connectedVertex + " in state " + executionState + " 2");
 				return ConnectionInfoLookupResponse.createReceiverNotReady();
@@ -781,74 +773,53 @@ 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.getAddress(), ici.getDataPort());
+				final InetSocketAddress isa = new InetSocketAddress(ici.address(), ici.dataPort());
 
-				return ConnectionInfoLookupResponse.createReceiverFoundAndReady(new RemoteReceiver(isa, edge
-					.getConnectionID()));
+				return ConnectionInfoLookupResponse.createReceiverFoundAndReady(new RemoteReceiver(isa, edge.getConnectionID()));
 			}
 		}
+		// else, the request is for an output channel
+		// Find vertex of connected input channel
+		final ExecutionVertex targetVertex = edge.getInputGate().getVertex();
 
-		if (edge.isBroadcast()) {
+		// Check execution state
+		final ExecutionState executionState = targetVertex.getExecutionState();
 
-			return multicastManager.lookupConnectionInfo(caller, jobID, sourceChannelID);
+		// check whether the task needs to be deployed
+		if (executionState != ExecutionState.RUNNING && executionState != ExecutionState.FINISHING && executionState != ExecutionState.FINISHED) {
 
-		} else {
-
-			// Find vertex of connected input channel
-			final ExecutionVertex targetVertex = edge.getInputGate().getVertex();
-
-			// Check execution state
-			final ExecutionState executionState = targetVertex.getExecutionState();
-
-			if (executionState != ExecutionState.RUNNING && executionState != ExecutionState.FINISHING
-					&& executionState != ExecutionState.FINISHED) {
-
-				if (executionState == ExecutionState.ASSIGNED) {
-
-					final Runnable command = new Runnable() {
-
-						/**
-						 * {@inheritDoc}
-						 */
-						@Override
-						public void run() {
-							scheduler.deployAssignedVertices(targetVertex);
-						}
-					};
-
-					eg.executeCommand(command);
-				}
-
-				// LOG.info("Created receiverNotReady for " + targetVertex + " in state " + executionState + " 3");
-				return ConnectionInfoLookupResponse.createReceiverNotReady();
-			}
-
-			final AbstractInstance assignedInstance = targetVertex.getAllocatedResource().getInstance();
-			if (assignedInstance == null) {
-				LOG.error("Cannot resolve lookup: vertex found for channel ID " + edge.getInputChannelID()
-					+ " but no instance assigned");
-				// LOG.info("Created receiverNotReady for " + targetVertex + " in state " + executionState + " 4");
-				return ConnectionInfoLookupResponse.createReceiverNotReady();
+			if (executionState == ExecutionState.ASSIGNED) {
+				final Runnable command = new Runnable() {
+					@Override
+					public void run() {
+						scheduler.deployAssignedVertices(targetVertex);
+					}
+				};
+				eg.executeCommand(command);
 			}
 
-			if (assignedInstance.getInstanceConnectionInfo().equals(caller)) {
-				// Receiver runs on the same task manager
-				return ConnectionInfoLookupResponse.createReceiverFoundAndReady(edge.getInputChannelID());
-			} else {
-				// Receiver runs on a different task manager
-				final InstanceConnectionInfo ici = assignedInstance.getInstanceConnectionInfo();
-				final InetSocketAddress isa = new InetSocketAddress(ici.getAddress(), ici.getDataPort());
+			// LOG.info("Created receiverNotReady for " + targetVertex + " in state " + executionState + " 3");
+			return ConnectionInfoLookupResponse.createReceiverNotReady();
+		}
 
-				return ConnectionInfoLookupResponse.createReceiverFoundAndReady(new RemoteReceiver(isa, edge
-					.getConnectionID()));
-			}
+		final AbstractInstance assignedInstance = targetVertex.getAllocatedResource().getInstance();
+		if (assignedInstance == null) {
+			LOG.error("Cannot resolve lookup: vertex found for channel ID " + edge.getInputChannelID() + " but no instance assigned");
+			// LOG.info("Created receiverNotReady for " + targetVertex + " in state " + executionState + " 4");
+			return ConnectionInfoLookupResponse.createReceiverNotReady();
 		}
 
-		// LOG.error("Receiver(s) not found");
+		if (assignedInstance.getInstanceConnectionInfo().equals(caller)) {
+			// Receiver runs on the same task manager
+			return ConnectionInfoLookupResponse.createReceiverFoundAndReady(edge.getInputChannelID());
+		} else {
+			// Receiver runs on a different task manager
+			final InstanceConnectionInfo ici = assignedInstance.getInstanceConnectionInfo();
+			final InetSocketAddress isa = new InetSocketAddress(ici.address(), ici.dataPort());
 
-		// return ConnectionInfoLookupResponse.createReceiverNotFound();
+			return ConnectionInfoLookupResponse.createReceiverFoundAndReady(new RemoteReceiver(isa, edge.getConnectionID()));
+		}
 	}
 
 	/**
@@ -921,40 +892,6 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 		return eventList;
 	}
 
-
-	@Override
-	public void killTask(final JobID jobID, final ManagementVertexID id) throws IOException {
-
-		final ExecutionGraph eg = this.scheduler.getExecutionGraphByID(jobID);
-		if (eg == null) {
-			LOG.error("Cannot find execution graph for job " + jobID);
-			return;
-		}
-
-		final ExecutionVertex vertex = eg.getVertexByID(ExecutionVertexID.fromManagementVertexID(id));
-		if (vertex == null) {
-			LOG.error("Cannot find execution vertex with ID " + id);
-			return;
-		}
-
-		LOG.info("Killing task " + vertex + " of job " + jobID);
-
-		final Runnable runnable = new Runnable() {
-
-			@Override
-			public void run() {
-
-				final TaskKillResult result = vertex.killTask();
-				if (result.getReturnCode() != AbstractTaskResult.ReturnCode.SUCCESS) {
-					LOG.error(result.getDescription());
-				}
-			}
-		};
-
-		eg.executeCommand(runnable);
-	}
-
-
 	@Override
 	public void killInstance(final StringRecord instanceName) throws IOException {
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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
index c9e02d3..24e2970 100644
--- 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
@@ -210,7 +210,7 @@ public abstract class AbstractScheduler implements InstanceListener {
 			case NETWORK:
 				deployTarget = false;
 				break;
-			case INMEMORY:
+			case IN_MEMORY:
 				deployTarget = true;
 				break;
 			default:

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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
index efbaf93..762b494 100644
--- 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
@@ -32,8 +32,7 @@ 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.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.taskmanager.AbstractTaskResult.ReturnCode;
+import eu.stratosphere.runtime.io.channels.ChannelID;
 import eu.stratosphere.nephele.taskmanager.TaskCancelResult;
 import eu.stratosphere.nephele.util.SerializableHashSet;
 import eu.stratosphere.util.StringUtils;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdge.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdge.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdge.java
index 8ecd99f..58486da 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdge.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdge.java
@@ -13,7 +13,7 @@
 
 package eu.stratosphere.nephele.managementgraph;
 
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.channels.ChannelType;
 
 /**
  * This class implements a directed edge of a {@link ManagementGraph}. The edge is derived from a channel of the actual

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdgeID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdgeID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdgeID.java
index 20107a7..d263999 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdgeID.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdgeID.java
@@ -13,8 +13,8 @@
 
 package eu.stratosphere.nephele.managementgraph;
 
-import eu.stratosphere.nephele.io.AbstractID;
-import eu.stratosphere.nephele.io.channels.ChannelID;
+import eu.stratosphere.nephele.AbstractID;
+import eu.stratosphere.runtime.io.channels.ChannelID;
 
 /**
  * A management edge ID uniquely identifies a {@link ManagementEdge}.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGateID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGateID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGateID.java
index 73548be..63aa6f7 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGateID.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGateID.java
@@ -13,7 +13,7 @@
 
 package eu.stratosphere.nephele.managementgraph;
 
-import eu.stratosphere.nephele.io.AbstractID;
+import eu.stratosphere.nephele.AbstractID;
 
 /**
  * A management gate ID uniquely identifies a {@link ManagementGate}.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 f4a9855..374656b 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
@@ -28,7 +28,7 @@ import java.util.Map;
 
 import eu.stratosphere.core.io.IOReadableWritable;
 import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.channels.ChannelType;
 import eu.stratosphere.nephele.jobgraph.JobID;
 import eu.stratosphere.nephele.util.EnumUtils;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupEdge.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupEdge.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupEdge.java
index 01a0903..7e6d554 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupEdge.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupEdge.java
@@ -13,7 +13,7 @@
 
 package eu.stratosphere.nephele.managementgraph;
 
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.channels.ChannelType;
 
 /**
  * This class implements a directed edge of between two {@link ManagementGroupVertex} objects. The edge is derived from

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupVertex.java
index bbf5fd2..b98a153 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupVertex.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupVertex.java
@@ -24,7 +24,7 @@ import java.util.Map;
 
 import eu.stratosphere.core.io.IOReadableWritable;
 import eu.stratosphere.nephele.execution.ExecutionState;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.channels.ChannelType;
 import eu.stratosphere.nephele.util.EnumUtils;
 import eu.stratosphere.util.StringUtils;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupVertexID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupVertexID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupVertexID.java
index b5c1055..aef64af 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupVertexID.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupVertexID.java
@@ -15,7 +15,7 @@ package eu.stratosphere.nephele.managementgraph;
 
 import javax.xml.bind.DatatypeConverter;
 
-import eu.stratosphere.nephele.io.AbstractID;
+import eu.stratosphere.nephele.AbstractID;
 
 /**
  * A management group vertex ID uniquely identifies a {@link ManagementGroupVertex}.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertexID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertexID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertexID.java
index 716ae7f..486b3fa 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertexID.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertexID.java
@@ -13,7 +13,7 @@
 
 package eu.stratosphere.nephele.managementgraph;
 
-import eu.stratosphere.nephele.io.AbstractID;
+import eu.stratosphere.nephele.AbstractID;
 
 /**
  * A management vertex ID uniquely identifies a {@link ManagementVertex}.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/MulticastCluster.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/MulticastCluster.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/MulticastCluster.java
deleted file mode 100644
index 87e0181..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/MulticastCluster.java
+++ /dev/null
@@ -1,210 +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.multicast;
-
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedList;
-
-/**
- * This class represents a cluster of hosts within a multicast-tree.
- * 
- */
-
-public class MulticastCluster {
-	TreeNode master = null;
-
-	HashSet<TreeNode> clusternodes = new HashSet<TreeNode>();
-
-	public void addNode(TreeNode node) {
-		this.clusternodes.add(node);
-	}
-
-	public int getSize() {
-		return this.clusternodes.size();
-	}
-
-	public HashSet<TreeNode> getNodes() {
-		return this.clusternodes;
-	}
-
-	/**
-	 * Returns the master-node of the current cluster.
-	 * 
-	 * @return
-	 */
-	public TreeNode getMaster() {
-
-		if (this.master == null) {
-
-			// TODO: topology-aware!!
-			if (clusternodes.size() != 0) {
-				this.master = clusternodes.iterator().next();
-			} else {
-				System.out.println("cluster is empty.");
-				return null;
-			}
-
-		}
-
-		return this.master;
-	}
-
-	/**
-	 * Splits the cluster into an arbitrary number of clusters not exceeding maxsize.
-	 * 
-	 * @param maxsize
-	 * @return
-	 */
-	public HashSet<MulticastCluster> splitCluster(int maxsize) {
-		// TODO: topology-aware!
-		HashSet<MulticastCluster> newClusters = new HashSet<MulticastCluster>();
-
-		MulticastCluster actualcluster = new MulticastCluster();
-
-		for (Iterator<TreeNode> i = this.clusternodes.iterator(); i.hasNext();) {
-			if (actualcluster.getSize() < maxsize) {
-				actualcluster.addNode(i.next());
-			} else {
-				// cluster is full.. add old cluster to list
-				newClusters.add(actualcluster);
-				// and create new cluster object
-				actualcluster = new MulticastCluster();
-				actualcluster.addNode(i.next());
-			}
-		}
-
-		newClusters.add(actualcluster);
-
-		return newClusters;
-
-	}
-
-	public static MulticastCluster createInitialCluster(Collection<TreeNode> nodes) {
-		// TODO: topology-aware? in here?
-
-		MulticastCluster cluster = new MulticastCluster();
-
-		for (TreeNode n : nodes) {
-			cluster.addNode(n);
-		}
-
-		return cluster;
-
-	}
-
-	public static MulticastForwardingTable createClusteredTree(LinkedList<TreeNode> nodes, int maxclustersize) {
-
-		return null;
-		/*
-		// List to store all levels of the clustered multicast tree
-		LinkedList<HashSet<MulticastCluster>> clusterlist = new LinkedList<HashSet<MulticastCluster>>();
-
-		// Poll off the sending node first..
-		TreeNode source = nodes.pollFirst();
-
-		// Create an initital multicast cluster containing all receivers
-		MulticastCluster initialcluster = createInitialCluster(nodes);
-
-		// Create a first layer of clusters with arbitrary size by splitting the initital cluster
-		HashSet<MulticastCluster> firstlaycluster = initialcluster.splitCluster(maxclustersize);
-
-		// add to the list of cluster layers
-		clusterlist.add(firstlaycluster);
-
-		// we want the top layer to consist of max. maxclustersize clusters
-		while (clusterlist.getFirst().size() > maxclustersize) {
-
-			// creating a new cluster-layer...
-			MulticastCluster nextlayercluster = new MulticastCluster();
-
-			HashSet<MulticastCluster> lowerlayer = clusterlist.getFirst();
-
-			// add all master nodes from current layer to next-layer cluster...
-			for (MulticastCluster c : lowerlayer) {
-				nextlayercluster.addNode(c.getMaster());
-			}
-
-			// if our next-layer cluster is still too big, we need to split it again
-			HashSet<MulticastCluster> nextlayerclusters = nextlayercluster.splitCluster(maxclustersize);
-
-			// and finally ad the new layer of clusters
-			clusterlist.addFirst(nextlayerclusters);
-
-		}
-
-		// now we can create the tree...
-
-		MulticastForwardingTable table = new MulticastForwardingTable();
-
-		HashSet<MulticastCluster> initialclusterlevel = clusterlist.getFirst();
-
-		ConnectionInfoLookupResponse sourceentry = ConnectionInfoLookupResponse.createReceiverFoundAndReady();
-
-		// add all local targets
-		for (ChannelID id : source.getLocalTargets()) {
-			System.out.println("local target: " + id);
-			sourceentry.addLocalTarget(id);
-		}
-
-		// connect source node with all master nodes in top-level clusters
-		for (MulticastCluster c : initialclusterlevel) {
-			sourceentry.addRemoteTarget(c.getMaster().getConnectionInfo());
-		}
-
-		table.addConnectionInfo(source.getConnectionInfo(), sourceentry);
-		System.out.println("forwards for node: " + source.getConnectionInfo());
-		System.out.println(sourceentry);
-		// now we have connected the source node to the initial cluster layer. iterate through cluster layers and
-		// connect
-
-		while (clusterlist.size() > 0) {
-			HashSet<MulticastCluster> actualclusterlevel = clusterlist.pollFirst();
-
-			// add remote targets!
-
-			for (MulticastCluster c : actualclusterlevel) {
-				TreeNode master = c.getMaster();
-				for (Iterator<TreeNode> i = c.getNodes().iterator(); i.hasNext();) {
-					TreeNode actualnode = i.next();
-					if (!actualnode.equals(master)) {
-						// add remote target at master of current cluster
-						master.addRemoteTarget(actualnode.getConnectionInfo());
-					}
-				}
-			}
-
-		}
-
-		// now iterate through all nodes and create forwarding table...
-		// we already have the entry for the source node..
-		for (TreeNode n : nodes) {
-			ConnectionInfoLookupResponse actualentry = ConnectionInfoLookupResponse.createReceiverFoundAndReady();
-			for (ChannelID localTarget : n.getLocalTargets()) {
-				actualentry.addLocalTarget(localTarget);
-			}
-			for (InstanceConnectionInfo remoteTarget : n.getRemoteTargets()) {
-				actualentry.addRemoteTarget(remoteTarget);
-			}
-			table.addConnectionInfo(n.getConnectionInfo(), actualentry);
-			System.out.println("forwards for node: " + n.getConnectionInfo());
-			System.out.println(actualentry);
-		}
-
-		return table;
-		*/
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/MulticastForwardingTable.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/MulticastForwardingTable.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/MulticastForwardingTable.java
deleted file mode 100644
index d08821e..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/MulticastForwardingTable.java
+++ /dev/null
@@ -1,49 +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.multicast;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import eu.stratosphere.nephele.instance.InstanceConnectionInfo;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.ConnectionInfoLookupResponse;
-
-/**
- * This class contains ConnectionInfoLookupResponse objects containing local, as well as remote receivers for all
- * instances within a certain job-specific multicast tree.
- * 
- */
-public class MulticastForwardingTable {
-
-	private final Map<InstanceConnectionInfo, ConnectionInfoLookupResponse> forwardingTable = new HashMap<InstanceConnectionInfo, ConnectionInfoLookupResponse>();
-
-	/**
-	 * Returns the related ConnectionInfoLookupResponse for the calling Instance.
-	 * 
-	 * @param caller
-	 * @return
-	 */
-	public ConnectionInfoLookupResponse getConnectionInfo(InstanceConnectionInfo caller) {
-		if (this.forwardingTable.containsKey(caller)) {
-			return this.forwardingTable.get(caller);
-		} else {
-			return null;
-		}
-	}
-
-	protected void addConnectionInfo(InstanceConnectionInfo caller, ConnectionInfoLookupResponse response) {
-		this.forwardingTable.put(caller, response);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/MulticastManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/MulticastManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/MulticastManager.java
deleted file mode 100644
index 0a7b471..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/MulticastManager.java
+++ /dev/null
@@ -1,463 +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.multicast;
-
-import java.io.BufferedReader;
-import java.io.DataInputStream;
-import java.io.FileInputStream;
-import java.io.InputStreamReader;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.configuration.GlobalConfiguration;
-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.instance.InstanceConnectionInfo;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.jobmanager.JobManager;
-import eu.stratosphere.nephele.jobmanager.scheduler.AbstractScheduler;
-import eu.stratosphere.nephele.protocols.ChannelLookupProtocol;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.ConnectionInfoLookupResponse;
-
-/**
- * The MulticastManager is responsible for the creation and storage of application-layer multicast trees used to
- * broadcast records to multiple target vertices.
- * 
- */
-
-public final class MulticastManager implements ChannelLookupProtocol {
-
-	/**
-	 * The log object used to report errors and warnings.
-	 */
-	private static final Log LOG = LogFactory.getLog(JobManager.class);
-
-	/**
-	 * Indicates if the arrangement of nodes within the overlay-tree should be randomized or not. If set to false,
-	 * arrangement of the same set of receiver nodes is guaranteed to be the same
-	 */
-	private final boolean randomized;
-
-	/**
-	 * Indicates if the tree should be constructed with a given topology stored in a file.
-	 */
-	private final boolean useHardCodedTree;
-
-	/**
-	 * File containing the hard-coded tree topology, if desired should contain node names (e.g. hostnames) with
-	 * corresponding children per line.
-	 * For example, a line "vm1.local vm2.local vm3.local" would result in vm1.local connecting to vm2.local and
-	 * vm3.local as children no further checking for connectivity of the given topology is done!
-	 */
-	private final String hardCodedTreeFilePath;
-
-	/**
-	 * Indicates the desired branching of the generated multicast-tree. 0 means unicast transmisison, 1 sequential tree,
-	 * 2 binomial tree, 3+ clustered tree
-	 */
-	private final int treeBranching;
-
-	/**
-	 * Reference to the scheduler.
-	 */
-	private final AbstractScheduler scheduler;
-
-	/**
-	 * Map caching already computed multicast forwarding tables.
-	 */
-	private final Map<ChannelID, MulticastForwardingTable> cachedTrees = new HashMap<ChannelID, MulticastForwardingTable>();
-
-	/**
-	 * Constructs a new multicast manager.
-	 * 
-	 * @param scheduler
-	 *        reference to the scheduler
-	 */
-	public MulticastManager(final AbstractScheduler scheduler) {
-
-		this.scheduler = scheduler;
-
-		this.randomized = GlobalConfiguration.getBoolean("multicast.randomize", false);
-		this.treeBranching = GlobalConfiguration.getInteger("multicast.branching", 1);
-		this.useHardCodedTree = GlobalConfiguration.getBoolean("multicast.usehardcodedtree", false);
-		this.hardCodedTreeFilePath = GlobalConfiguration.getString("multicast.hardcodedtreefile", null);
-	}
-
-	/**
-	 * Retrieves all recipients of a data for the given <code>sourceChannelID</code>. Returns both local recipients as
-	 * well as next-hop remote instances within the multicast-tree.
-	 * 
-	 * @param caller
-	 *        the {@link InstanceConnectionInfo} object of the task manager which calls this method
-	 * @param jobID
-	 *        the ID of the job the channel ID belongs to
-	 * @param sourceChannelID
-	 *        the ID of the channel to resolve
-	 * @return the lookup response containing the connection info and a return code
-	 */
-	public synchronized ConnectionInfoLookupResponse lookupConnectionInfo(final InstanceConnectionInfo caller,
-			final JobID jobID, final ChannelID sourceChannelID) {
-
-		if (LOG.isInfoEnabled()) {
-			LOG.info("Receiving multicast receiver request from " + caller + " channel ID: " + sourceChannelID);
-		}
-
-		// Check if the tree is already created and cached
-		if (this.cachedTrees.containsKey(sourceChannelID)) {
-
-			LOG.info("Replying with cached entry...");
-			return cachedTrees.get(sourceChannelID).getConnectionInfo(caller);
-
-		} else {
-
-			// No tree exists, so we assume that this is the sending node initiating a multicast
-
-			if (!checkIfAllTargetVerticesReady(caller, jobID, sourceChannelID)) {
-				LOG.info("Received multicast request but not all receivers ready.");
-
-				return ConnectionInfoLookupResponse.createReceiverNotReady();
-			}
-
-			// Receivers are up and running.. extract tree nodes...
-			LinkedList<TreeNode> treeNodes = extractTreeNodes(caller, jobID, sourceChannelID, this.randomized);
-
-			// Do we want to use a hard-coded tree topology?
-			if (this.useHardCodedTree) {
-				LOG.info("Creating a hard-coded tree topology from file: " + hardCodedTreeFilePath);
-				cachedTrees.put(sourceChannelID, createHardCodedTree(treeNodes));
-				return cachedTrees.get(sourceChannelID).getConnectionInfo(caller);
-			}
-
-			// Otherwise we create a default tree and put it into the tree-cache
-			cachedTrees.put(sourceChannelID, createDefaultTree(treeNodes, this.treeBranching));
-			return cachedTrees.get(sourceChannelID).getConnectionInfo(caller);
-
-		}
-
-	}
-
-	/**
-	 * Returns and removes the TreeNode which is closest to the given indicator.
-	 * 
-	 * @param indicator
-	 * @param nodes
-	 * @return
-	 */
-	private TreeNode pollClosestNode(final TreeNode indicator, final LinkedList<TreeNode> nodes) {
-
-		TreeNode closestnode = getClosestNode(indicator, nodes);
-
-		nodes.remove(closestnode);
-
-		return closestnode;
-
-	}
-
-	/**
-	 * Returns the TreeNode which is closest to the given indicator Node. Proximity is determined
-	 * either using topology-information (if given), penalty information (if given) or it returns
-	 * the first node in the list.
-	 * 
-	 * @param indicator
-	 * @param nodes
-	 * @return
-	 */
-	private TreeNode getClosestNode(final TreeNode indicator, final LinkedList<TreeNode> nodes) {
-
-		if (indicator == null) {
-			return nodes.getFirst();
-		}
-
-		TreeNode closestNode = null;
-		for (TreeNode n : nodes) {
-			if (closestNode == null || n.getDistance(indicator) < closestNode.getDistance(indicator)) {
-				closestNode = n;
-			}
-		}
-
-		return closestNode;
-	}
-
-	/**
-	 * This method creates a tree with an arbitrary fan out (two means binary tree).
-	 * If topology information or penalties are available, it considers that.
-	 * If fanout is set to 1, it creates a sequential tree.
-	 * if fanout is set to Integer.MAXVALUE, it creates a unicast tree.
-	 * 
-	 * @param nodes
-	 * @param fanout
-	 * @return
-	 */
-	private MulticastForwardingTable createDefaultTree(LinkedList<TreeNode> nodes, int fanout) {
-
-		// Store nodes that already have a parent, but no children
-		LinkedList<TreeNode> connectedNodes = new LinkedList<TreeNode>();
-
-		final TreeNode rootnode = nodes.pollFirst();
-		TreeNode actualnode = rootnode;
-
-		while (nodes.size() > 0) { // We still have unconnected nodes...
-
-			for (int i = 0; i < fanout; i++) {
-
-				if (nodes.size() > 0) {
-					// pick the closest one and attach to actualnode
-					TreeNode child = pollClosestNode(actualnode, nodes);
-					actualnode.addChild(child);
-
-					// The child is now connected and can be used as forwarder in the next iteration..
-					connectedNodes.add(child);
-				} else {
-					break;
-				}
-			}
-
-			// OK.. take the next node to attach children to it..
-			// TODO: Optimization? "pollBest()" ?
-			actualnode = connectedNodes.pollFirst();
-
-		}
-		LOG.info("created multicast tree with following topology:\n" + rootnode.printTree());
-
-		return rootnode.createForwardingTable();
-
-	}
-
-	/**
-	 * Reads a hard-coded tree topology from file and creates a tree according to the hard-coded
-	 * topology from the file.
-	 * 
-	 * @param nodes
-	 * @return
-	 */
-	private MulticastForwardingTable createHardCodedTree(LinkedList<TreeNode> nodes) {
-		try {
-			FileInputStream fstream = new FileInputStream(this.hardCodedTreeFilePath);
-			DataInputStream in = new DataInputStream(fstream);
-			BufferedReader br = new BufferedReader(new InputStreamReader(in));
-			String strLine;
-			while ((strLine = br.readLine()) != null) {
-				String[] values = strLine.split(" ");
-				String actualhostname = values[0];
-				for (TreeNode n : nodes) {
-					if (n.toString().equals(actualhostname)) {
-						// we found the node.. connect the children
-						for (int i = 1; i < values.length; i++) {
-							for (TreeNode childnode : nodes) {
-								if (childnode.toString().equals(values[i])) {
-									n.addChild(childnode);
-								}
-							}
-						}
-					}
-				}
-			}
-			br.close();
-			// First node is root.. create tree. easy
-			return nodes.getFirst().createForwardingTable();
-
-		} catch (Exception e) {
-			System.out.println("Error reading hard-coded topology file for multicast tree: " + e.getMessage());
-			return null;
-		}
-	}
-
-	/**
-	 * Checks, if all target vertices for multicast transmisison are ready. If vertices are in state ASSIGNED, it will
-	 * deploy those vertices.
-	 * 
-	 * @param caller
-	 * @param jobID
-	 * @param sourceChannelID
-	 * @return
-	 */
-	private boolean checkIfAllTargetVerticesReady(InstanceConnectionInfo caller, JobID jobID, ChannelID sourceChannelID) {
-
-		final ExecutionGraph eg = this.scheduler.getExecutionGraphByID(jobID);
-
-		final ExecutionEdge outputChannel = eg.getEdgeByID(sourceChannelID);
-
-		final ExecutionGate broadcastGate = outputChannel.getOutputGate();
-
-		List<ExecutionVertex> verticesToDeploy = null;
-
-		// get all broadcast output channels
-		final int numberOfOutputChannels = broadcastGate.getNumberOfEdges();
-		for (int i = 0; i < numberOfOutputChannels; ++i) {
-
-			final ExecutionEdge c = broadcastGate.getEdge(i);
-
-			if (c.isBroadcast()) {
-
-				final ExecutionVertex targetVertex = c.getInputGate().getVertex();
-
-				if (targetVertex.getExecutionState() == ExecutionState.ASSIGNED) {
-					if (verticesToDeploy == null) {
-						verticesToDeploy = new ArrayList<ExecutionVertex>();
-					}
-					verticesToDeploy.add(targetVertex);
-				} else {
-
-					if (targetVertex.getExecutionState() != ExecutionState.RUNNING
-						&& targetVertex.getExecutionState() != ExecutionState.FINISHING) {
-						return false;
-					}
-				}
-			}
-		}
-
-		if (verticesToDeploy != null) {
-			this.scheduler.deployAssignedVertices(verticesToDeploy);
-			return false;
-		}
-
-		return true;
-	}
-
-	/**
-	 * Returns a list of (physical) Nodes (=hosts) within the multicast tree. Each node contains the local ChannelIDs,
-	 * records
-	 * must be forwarded to. The first node in the List is the only multicast sender.
-	 * 
-	 * @param sourceChannelID
-	 * @return
-	 */
-	private LinkedList<TreeNode> extractTreeNodes(final InstanceConnectionInfo source, final JobID jobID,
-			final ChannelID sourceChannelID, final boolean randomize) {
-
-		final ExecutionGraph eg = this.scheduler.getExecutionGraphByID(jobID);
-
-		final ExecutionEdge outputChannel = eg.getEdgeByID(sourceChannelID);
-
-		final ExecutionGate broadcastGate = outputChannel.getOutputGate();
-
-		final LinkedList<ExecutionEdge> outputChannels = new LinkedList<ExecutionEdge>();
-
-		// Get all broadcast output channels
-		final int numberOfOutputChannels = broadcastGate.getNumberOfEdges();
-		for (int i = 0; i < numberOfOutputChannels; ++i) {
-			final ExecutionEdge c = broadcastGate.getEdge(i);
-
-			if (c.isBroadcast()) {
-				outputChannels.add(c);
-			}
-		}
-
-		final LinkedList<TreeNode> treeNodes = new LinkedList<TreeNode>();
-
-		LinkedList<ChannelID> actualLocalTargets = new LinkedList<ChannelID>();
-
-		int firstConnectionID = 0;
-		// search for local targets for the tree node
-		for (Iterator<ExecutionEdge> iter = outputChannels.iterator(); iter.hasNext();) {
-
-			final ExecutionEdge actualOutputChannel = iter.next();
-
-			// the connection ID should not be needed for the root node (as it is not set as remote receiver)
-			// but in order to maintain consistency, it also gets the connectionID of the first channel pointing to it
-			firstConnectionID = actualOutputChannel.getConnectionID();
-
-			final ExecutionVertex targetVertex = actualOutputChannel.getInputGate().getVertex();
-
-			// is the target vertex running on the same instance?
-			if (targetVertex.getAllocatedResource().getInstance().getInstanceConnectionInfo().equals(source)) {
-
-				actualLocalTargets.add(actualOutputChannel.getInputChannelID());
-				iter.remove();
-			}
-
-		}
-
-		// create sender node (root) with source instance
-		TreeNode actualNode = new TreeNode(eg.getVertexByChannelID(sourceChannelID).getAllocatedResource()
-			.getInstance(), source, firstConnectionID, actualLocalTargets);
-
-		treeNodes.add(actualNode);
-
-		// now we have the root-node.. lets extract all other nodes
-
-		LinkedList<TreeNode> receiverNodes = new LinkedList<TreeNode>();
-
-		while (outputChannels.size() > 0) {
-
-			final ExecutionEdge firstChannel = outputChannels.pollFirst();
-
-			// each receiver nodes' endpoint is associated with the connection ID
-			// of the first channel pointing to this node.
-			final int connectionID = firstChannel.getConnectionID();
-
-			final ExecutionVertex firstTarget = firstChannel.getInputGate().getVertex();
-
-			final InstanceConnectionInfo actualInstance = firstTarget.getAllocatedResource().getInstance()
-				.getInstanceConnectionInfo();
-
-			actualLocalTargets = new LinkedList<ChannelID>();
-
-			// add first local target
-			actualLocalTargets.add(firstChannel.getInputChannelID());
-
-			// now we iterate through the remaining channels to find other local targets...
-			for (Iterator<ExecutionEdge> iter = outputChannels.iterator(); iter.hasNext();) {
-
-				final ExecutionEdge actualOutputChannel = iter.next();
-
-				final ExecutionVertex actualTarget = actualOutputChannel.getInputGate().getVertex();
-
-				// is the target vertex running on the same instance?
-				if (actualTarget.getAllocatedResource().getInstance().getInstanceConnectionInfo()
-					.equals(actualInstance)) {
-					actualLocalTargets.add(actualOutputChannel.getInputChannelID());
-
-					iter.remove();
-
-				}
-
-			}// end for
-
-			// create tree node for current instance
-			actualNode = new TreeNode(firstTarget.getAllocatedResource().getInstance(), actualInstance, connectionID,
-				actualLocalTargets);
-
-			receiverNodes.add(actualNode);
-
-		}// end while
-
-		// Do we want to shuffle the receiver nodes?
-		// Only randomize the receivers, as the sender (the first one) has to stay the same
-		if (randomize) {
-			Collections.shuffle(receiverNodes);
-		} else {
-			// Sort Tree Nodes according to host name..
-			Collections.sort(receiverNodes);
-		}
-
-		treeNodes.addAll(receiverNodes);
-
-		return treeNodes;
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/TopologyInformationSupplier.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/TopologyInformationSupplier.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/TopologyInformationSupplier.java
deleted file mode 100644
index 0e80ba2..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/TopologyInformationSupplier.java
+++ /dev/null
@@ -1,30 +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.multicast;
-
-public class TopologyInformationSupplier {
-	
-	
-	/**
-	 * Returns a float representing the network distance between two nodes.
-	 * @param node1
-	 * @param node2
-	 * @return
-	 */
-	public float getNetworkDistance(String node1, String node2){
-		//TODO: Implement me
-		return 0;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/TreeNode.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/TreeNode.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/TreeNode.java
deleted file mode 100644
index 12ee998..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/TreeNode.java
+++ /dev/null
@@ -1,246 +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.multicast;
-
-import java.net.InetSocketAddress;
-import java.util.LinkedList;
-
-import eu.stratosphere.nephele.instance.AbstractInstance;
-import eu.stratosphere.nephele.instance.InstanceConnectionInfo;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.ConnectionInfoLookupResponse;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.RemoteReceiver;
-
-/**
- * Each physical node (instance) within a multicast tree is represented by a TreeNode object.
- * It contains the connection info for the certain node and a list of the local output channels.
- * 
- */
-
-public class TreeNode implements Comparable<TreeNode> {
-
-	private TreeNode parentnode = null;
-
-	private final AbstractInstance instance;
-
-	private final InstanceConnectionInfo nodeConnectionInfo;
-
-	private final int connectionID;
-
-	private final LinkedList<ChannelID> localTargets;
-
-	private final LinkedList<TreeNode> children = new LinkedList<TreeNode>();
-
-	private final LinkedList<IntegerProperty> properties = new LinkedList<TreeNode.IntegerProperty>();
-
-	private int penalty = 0;
-
-	public TreeNode(AbstractInstance instance, InstanceConnectionInfo nodeConnectionInfo, int connectionID,
-			LinkedList<ChannelID> localTargets) {
-		this.instance = instance;
-		this.connectionID = connectionID;
-		this.nodeConnectionInfo = nodeConnectionInfo;
-		this.localTargets = localTargets;
-	}
-
-	public void setProperty(String key, int value) {
-		boolean exists = false;
-		for (IntegerProperty property : this.properties) {
-			if (property.getKey().equals(key)) {
-				property.setValue(value);
-				exists = true;
-				break;
-			}
-		}
-		if (!exists) {
-			this.properties.add(new IntegerProperty(key, value));
-		}
-	}
-
-	public int getProperty(String key) {
-		for (IntegerProperty property : this.properties) {
-			if (property.getKey().equals(key)) {
-				return property.getValue();
-			}
-		}
-		return -1;
-	}
-
-	public void removeChild(TreeNode child) {
-		if (this.children.contains(child)) {
-			child.setParent(null);
-			this.children.remove(child);
-		}
-	}
-
-	public void addChild(TreeNode child) {
-		this.children.add(child);
-		child.setParent(this);
-	}
-
-	public LinkedList<TreeNode> getChildren() {
-		return this.children;
-	}
-
-	public TreeNode getParent() {
-		return this.parentnode;
-	}
-
-	private InstanceConnectionInfo getConnectionInfo() {
-		return this.nodeConnectionInfo;
-	}
-
-	private int getConnectionID() {
-		return this.connectionID;
-	}
-
-	private void setParent(TreeNode parent) {
-		this.parentnode = parent;
-	}
-
-	@Override
-	public int compareTo(TreeNode o) {
-		return this.nodeConnectionInfo.compareTo(o.nodeConnectionInfo);
-	}
-
-	@Override
-	public boolean equals(Object o) {
-		if (o instanceof TreeNode) {
-			return this.nodeConnectionInfo.equals(((TreeNode) o).nodeConnectionInfo);
-		} else {
-			return false;
-		}
-	}
-
-	public int getDistance(TreeNode o) {
-		return this.instance.getDistance(o.instance);
-	}
-
-	public String toString() {
-		return this.nodeConnectionInfo.toString();
-	}
-
-	public int getPenalty() {
-		return this.penalty;
-	}
-
-	public void setPenalty(int penalty) {
-		this.penalty = penalty;
-	}
-
-	/**
-	 * This method should be called on the root node (sender node).
-	 * It traverses the Tree and returns a full forwarding table
-	 * including all local and remote receivers.
-	 * 
-	 * @return
-	 */
-	public MulticastForwardingTable createForwardingTable() {
-		MulticastForwardingTable table = new MulticastForwardingTable();
-		this.generateRecursiveForwardingTable(table);
-		return table;
-	}
-
-	/**
-	 * Private recursive method to generate forwarding table
-	 * 
-	 * @param table
-	 */
-	private void generateRecursiveForwardingTable(MulticastForwardingTable table) {
-
-		final ConnectionInfoLookupResponse lookupResponse = ConnectionInfoLookupResponse.createReceiverFoundAndReady();
-
-		// add local targets
-		for (final ChannelID i : this.localTargets) {
-			lookupResponse.addLocalTarget(i);
-		}
-
-		// add remote targets
-		for (final TreeNode n : this.children) {
-
-			// Instance Connection info associated with the remote target
-			final InstanceConnectionInfo ici = n.getConnectionInfo();
-
-			// get the connection ID associated with the remote target endpoint
-			final int icid = n.getConnectionID();
-
-			final InetSocketAddress isa = new InetSocketAddress(ici.getAddress(), ici.getDataPort());
-
-			lookupResponse.addRemoteTarget(new RemoteReceiver(isa, icid));
-		}
-
-		table.addConnectionInfo(this.nodeConnectionInfo, lookupResponse);
-
-		for (final TreeNode n : this.children) {
-			n.generateRecursiveForwardingTable(table);
-		}
-	}
-
-	/**
-	 * Prints the tree in a human readable format, starting with the actual node as root.
-	 * 
-	 * @return
-	 */
-	public String printTree() {
-
-		StringBuilder sb = new StringBuilder();
-		this.printRecursiveTree(sb);
-		return sb.toString();
-	}
-
-	private void printRecursiveTree(StringBuilder sb) {
-
-		if (this.children.size() > 0) {
-			sb.append("STRUCT ");
-
-			sb.append(this.nodeConnectionInfo);
-
-			for (TreeNode n : this.children) {
-				sb.append(" ");
-				sb.append(n.getConnectionInfo().toString());
-			}
-
-			sb.append("\n");
-
-			for (TreeNode n : this.children) {
-				n.printRecursiveTree(sb);
-			}
-		}
-	}
-
-	private static class IntegerProperty {
-
-		private String key = null;
-
-		private int value = 0;
-
-		public IntegerProperty(final String key, final int value) {
-			this.key = key;
-			this.value = value;
-		}
-
-		public String getKey() {
-			return this.key;
-		}
-
-		public int getValue() {
-			return this.value;
-		}
-
-		public void setValue(final int value) {
-			this.value = value;
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/TaskManagerProfiler.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/TaskManagerProfiler.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/TaskManagerProfiler.java
index b1d6ff6..302192b 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/TaskManagerProfiler.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/TaskManagerProfiler.java
@@ -15,7 +15,7 @@ package eu.stratosphere.nephele.profiling;
 
 import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
-import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTask;
+import eu.stratosphere.nephele.taskmanager.Task;
 
 /**
  * This interface must be implemented by profiling components
@@ -32,8 +32,7 @@ public interface TaskManagerProfiler {
 	 * @param jobConfiguration
 	 *        the job configuration sent with the task
 	 */
-	void registerExecutionListener(RuntimeTask task, Configuration jobConfiguration);
-
+	void registerExecutionListener(Task task, Configuration jobConfiguration);
 
 	/**
 	 * Unregisters all previously register {@link ExecutionListener} objects for

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/TaskManagerProfilerImpl.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/TaskManagerProfilerImpl.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/TaskManagerProfilerImpl.java
index 49fa8d1..58f0f38 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/TaskManagerProfilerImpl.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/TaskManagerProfilerImpl.java
@@ -13,20 +13,6 @@
 
 package eu.stratosphere.nephele.profiling.impl;
 
-import java.io.IOException;
-import java.lang.management.ManagementFactory;
-import java.lang.management.ThreadMXBean;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.util.HashMap;
-import java.util.Iterator;
-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.Configuration;
 import eu.stratosphere.configuration.GlobalConfiguration;
 import eu.stratosphere.nephele.execution.Environment;
@@ -40,8 +26,21 @@ import eu.stratosphere.nephele.profiling.TaskManagerProfiler;
 import eu.stratosphere.nephele.profiling.impl.types.InternalExecutionVertexThreadProfilingData;
 import eu.stratosphere.nephele.profiling.impl.types.InternalInstanceProfilingData;
 import eu.stratosphere.nephele.profiling.impl.types.ProfilingDataContainer;
-import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTask;
+import eu.stratosphere.nephele.taskmanager.Task;
 import eu.stratosphere.util.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.lang.management.ThreadMXBean;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Timer;
+import java.util.TimerTask;
 
 public class TaskManagerProfilerImpl extends TimerTask implements TaskManagerProfiler {
 
@@ -99,7 +98,7 @@ public class TaskManagerProfilerImpl extends TimerTask implements TaskManagerPro
 
 
 	@Override
-	public void registerExecutionListener(final RuntimeTask task, final Configuration jobConfiguration) {
+	public void registerExecutionListener(final Task task, final Configuration jobConfiguration) {
 
 		// Register profiling hook for the environment
 		task.registerExecutionListener(new EnvironmentListenerImpl(this, task.getRuntimeEnvironment()));

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/ChannelLookupProtocol.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/ChannelLookupProtocol.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/ChannelLookupProtocol.java
index 3f1d669..8fdaf55 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/ChannelLookupProtocol.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/ChannelLookupProtocol.java
@@ -17,9 +17,9 @@ import java.io.IOException;
 
 import eu.stratosphere.core.protocols.VersionedProtocol;
 import eu.stratosphere.nephele.instance.InstanceConnectionInfo;
-import eu.stratosphere.nephele.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.channels.ChannelID;
 import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.ConnectionInfoLookupResponse;
+import eu.stratosphere.runtime.io.network.ConnectionInfoLookupResponse;
 
 /**
  * The channel lookup protocol can be used to resolve the ID of an output channel to all recipients which shall receive

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 a2578a5..461c797 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
@@ -24,7 +24,6 @@ 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;
 import eu.stratosphere.nephele.topology.NetworkTopology;
 
 /**
@@ -82,18 +81,6 @@ public interface ExtendedManagementProtocol extends JobManagementProtocol {
 	List<AbstractEvent> getEvents(JobID jobID) throws IOException;
 
 	/**
-	 * Kills the task with the given vertex ID.
-	 * 
-	 * @param jobID
-	 *        the ID of the job the vertex to be killed belongs to
-	 * @param id
-	 *        the vertex ID which identified the task be killed
-	 * @throws IOException
-	 *         thrown if an error occurs while transmitting the kill request
-	 */
-	void killTask(JobID jobID, ManagementVertexID id) throws IOException;
-
-	/**
 	 * Kills the instance with the given name (i.e. shuts down its task manager).
 	 * 
 	 * @param instanceName

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java
index 63509ab..19522db 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java
@@ -23,9 +23,8 @@ 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.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.channels.ChannelID;
 import eu.stratosphere.nephele.taskmanager.TaskCancelResult;
-import eu.stratosphere.nephele.taskmanager.TaskKillResult;
 import eu.stratosphere.nephele.taskmanager.TaskSubmissionResult;
 
 /**
@@ -59,17 +58,6 @@ public interface TaskOperationProtocol extends VersionedProtocol {
 	TaskCancelResult cancelTask(ExecutionVertexID id) throws IOException;
 
 	/**
-	 * Advises the task manager to kill the task with the given ID.
-	 * 
-	 * @param id
-	 *        the ID of the task to kill
-	 * @return the result of the task kill attempt
-	 * @throws IOException
-	 *         thrown if an error occurs during this remote procedure call
-	 */
-	TaskKillResult killTask(ExecutionVertexID id) throws IOException;
-
-	/**
 	 * Queries the task manager about the cache status of the libraries stated in the {@link LibraryCacheProfileRequest}
 	 * object.
 	 * 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/iomanager/BlockChannelAccess.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/iomanager/BlockChannelAccess.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/iomanager/BlockChannelAccess.java
index be24922..8fdaa5d 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/iomanager/BlockChannelAccess.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/iomanager/BlockChannelAccess.java
@@ -134,9 +134,7 @@ public abstract class BlockChannelAccess<R extends IORequest, C extends Collecti
 						this.closeLock.wait(1000);
 						checkErroneous();
 					}
-					catch (InterruptedException iex) {
-						throw new IOException("Block channel access was interrupted while closing.");
-					}
+					catch (InterruptedException iex) {}
 				}
 			}
 			finally {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 7680843..a8fe096 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
@@ -56,8 +56,7 @@ public interface MemoryManager {
 	
 	/**
 	 * Releases all memory segments for the given task. 
-	 * 
-	 * @param <T> The type of memory segment.
+	 *
 	 * @param task The task whose memory segments are to be released.
 	 */
 	void releaseAll(AbstractInvokable task);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 5a74bab..8bc7b13 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
@@ -13,7 +13,6 @@
 
 package eu.stratosphere.nephele.services.memorymanager.spi;
 
-
 import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -125,7 +124,6 @@ public class DefaultMemoryManager implements MemoryManager {
 		}
 	}
 
-
 	@Override
 	public void shutdown() {
 		// -------------------- BEGIN CRITICAL SECTION -------------------
@@ -150,7 +148,6 @@ public class DefaultMemoryManager implements MemoryManager {
 		}
 		// -------------------- END CRITICAL SECTION -------------------
 	}
-	
 
 	public boolean verifyEmpty() {
 		synchronized (this.lock) {
@@ -161,7 +158,7 @@ public class DefaultMemoryManager implements MemoryManager {
 	// ------------------------------------------------------------------------
 	//                 MemoryManager interface implementation
 	// ------------------------------------------------------------------------
-	
+
 	@Override
 	public List<MemorySegment> allocatePages(AbstractInvokable owner, int numPages) throws MemoryAllocationException {
 		final ArrayList<MemorySegment> segs = new ArrayList<MemorySegment>(numPages);
@@ -212,7 +209,6 @@ public class DefaultMemoryManager implements MemoryManager {
 	}
 	
 	// ------------------------------------------------------------------------
-	
 
 	@Override
 	public void release(MemorySegment segment) {
@@ -254,7 +250,6 @@ public class DefaultMemoryManager implements MemoryManager {
 		// -------------------- END CRITICAL SECTION -------------------
 	}
 
-
 	@Override
 	public <T extends MemorySegment> void release(Collection<T> segments) {
 		
@@ -317,7 +312,6 @@ public class DefaultMemoryManager implements MemoryManager {
 		// -------------------- END CRITICAL SECTION -------------------
 	}
 
-
 	@Override
 	public void releaseAll(AbstractInvokable owner) {
 		// -------------------- BEGIN CRITICAL SECTION -------------------
@@ -326,28 +320,27 @@ public class DefaultMemoryManager implements MemoryManager {
 			if (this.isShutDown) {
 				throw new IllegalStateException("Memory manager has been shut down.");
 			}
-			
+
 			// get all segments
 			final Set<DefaultMemorySegment> segments = this.allocatedSegments.remove(owner);
-			
+
 			// all segments may have been freed previously individually
 			if (segments == null || segments.isEmpty()) {
 				return;
 			}
-			
+
 			// free each segment
 			for (DefaultMemorySegment seg : segments) {
 				final byte[] buffer = seg.destroy();
 				this.freeSegments.add(buffer);
 			}
-			
+
 			segments.clear();
 		}
 		// -------------------- END CRITICAL SECTION -------------------
 	}
 	
 	// ------------------------------------------------------------------------
-	
 
 	@Override
 	public int getPageSize() {

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

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 df9bbde..06eec0c 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
@@ -15,42 +15,92 @@ package eu.stratosphere.nephele.taskmanager;
 
 import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.nephele.execution.Environment;
+import eu.stratosphere.nephele.execution.ExecutionListener;
+import eu.stratosphere.nephele.execution.ExecutionObserver;
 import eu.stratosphere.nephele.execution.ExecutionState;
+import eu.stratosphere.nephele.execution.ExecutionStateTransition;
+import eu.stratosphere.nephele.execution.RuntimeEnvironment;
 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.taskmanager.bufferprovider.LocalBufferPoolOwner;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.TaskContext;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher;
+import eu.stratosphere.nephele.template.AbstractInvokable;
+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;
+
+public final class Task implements ExecutionObserver {
+
+	/**
+	 * The log object used for debugging.
+	 */
+	private static final Log LOG = LogFactory.getLog(Task.class);
+
+	private final ExecutionVertexID vertexID;
+
+	private final RuntimeEnvironment environment;
+
+	private final TaskManager taskManager;
+
+	/**
+	 * Stores whether the task has been canceled.
+	 */
+	private volatile boolean isCanceled = false;
+
+	/**
+	 * The current execution state of the task
+	 */
+	private volatile ExecutionState executionState = ExecutionState.STARTING;
+
+	private Queue<ExecutionListener> registeredListeners = new ConcurrentLinkedQueue<ExecutionListener>();
+
+	public Task(final ExecutionVertexID vertexID, final RuntimeEnvironment environment,
+					   final TaskManager taskManager) {
+
+		this.vertexID = vertexID;
+		this.environment = environment;
+		this.taskManager = taskManager;
+
+		this.environment.setExecutionObserver(this);
+	}
 
-public interface Task {
 
 	/**
 	 * Returns the ID of the job this task belongs to.
 	 * 
 	 * @return the ID of the job this task belongs to
 	 */
-	JobID getJobID();
+	public JobID getJobID() {
+		return this.environment.getJobID();
+	}
 
 	/**
 	 * Returns the ID of this task.
 	 * 
 	 * @return the ID of this task
 	 */
-	ExecutionVertexID getVertexID();
+	public ExecutionVertexID getVertexID() {
+		return this.vertexID;
+	}
 
 	/**
 	 * Returns the environment associated with this task.
 	 * 
 	 * @return the environment associated with this task
 	 */
-	Environment getEnvironment();
+	public Environment getEnvironment() {
+		return this.environment;
+	}
 
 	/**
 	 * Marks the task as failed and triggers the appropriate state changes.
 	 */
-	void markAsFailed();
+	public void markAsFailed() {
+		executionStateChanged(ExecutionState.FAILED, "Execution thread died unexpectedly");
+	}
 
 	/**
 	 * Checks if the state of the thread which is associated with this task is <code>TERMINATED</code>.
@@ -58,22 +108,72 @@ public interface Task {
 	 * @return <code>true</code> if the state of this thread which is associated with this task is
 	 *         <code>TERMINATED</code>, <code>false</code> otherwise
 	 */
-	boolean isTerminated();
+	public boolean isTerminated() {
+		final Thread executingThread = this.environment.getExecutingThread();
+		if (executingThread.getState() == Thread.State.TERMINATED) {
+			return true;
+		}
+
+		return false;
+	}
 
 	/**
 	 * Starts the execution of this task.
 	 */
-	void startExecution();
+	public void startExecution() {
+
+		final Thread thread = this.environment.getExecutingThread();
+		thread.start();
+	}
 
 	/**
 	 * Cancels the execution of the task (i.e. interrupts the execution thread).
 	 */
-	void cancelExecution();
+	public void cancelExecution() {
+		final Thread executingThread = this.environment.getExecutingThread();
+
+		if (executingThread == null) {
+			return;
+		}
+
+		LOG.info("Canceling " + this.environment.getTaskNameWithIndex());
+
+		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("Error while canceling task", 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) {}
+
+			if (!executingThread.isAlive()) {
+				break;
+			}
+
+			if (LOG.isDebugEnabled())
+				LOG.debug("Sending repeated canceling  signal to " +
+						this.environment.getTaskName() + " with state " + this.executionState);
+		}
+	}
 
-	/**
-	 * Kills the task (i.e. interrupts the execution thread).
-	 */
-	void killExecution();
 
 	/**
 	 * Registers the task manager profiler with the task.
@@ -83,7 +183,9 @@ public interface Task {
 	 * @param jobConfiguration
 	 *        the configuration attached to the job
 	 */
-	void registerProfiler(TaskManagerProfiler taskManagerProfiler, Configuration jobConfiguration);
+	public void registerProfiler(final TaskManagerProfiler taskManagerProfiler, final Configuration jobConfiguration) {
+		taskManagerProfiler.registerExecutionListener(this, jobConfiguration);
+	}
 
 	/**
 	 * Unregisters the task from the central memory manager.
@@ -91,7 +193,11 @@ public interface Task {
 	 * @param memoryManager
 	 *        the central memory manager
 	 */
-	void unregisterMemoryManager(MemoryManager memoryManager);
+	public void unregisterMemoryManager(final MemoryManager memoryManager) {
+		if (memoryManager != null) {
+			memoryManager.releaseAll(this.environment.getInvokable());
+		}
+	}
 
 	/**
 	 * Unregisters the task from the task manager profiler.
@@ -99,15 +205,124 @@ public interface Task {
 	 * @param taskManagerProfiler
 	 *        the task manager profiler
 	 */
-	void unregisterProfiler(TaskManagerProfiler taskManagerProfiler);
+	public void unregisterProfiler(final TaskManagerProfiler taskManagerProfiler) {
+		if (taskManagerProfiler != null) {
+			taskManagerProfiler.unregisterExecutionListener(this.vertexID);
+		}
+	}
 
 	/**
 	 * Returns the current execution state of the task.
 	 * 
 	 * @return the current execution state of the task
 	 */
-	ExecutionState getExecutionState();
+	public ExecutionState getExecutionState() {
+		return this.executionState;
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+	//                                        ExecutionObserver methods
+	// -----------------------------------------------------------------------------------------------------------------
+	@Override
+	public void executionStateChanged(final ExecutionState newExecutionState, final String optionalMessage) {
+
+		// Check the state transition
+		ExecutionStateTransition.checkTransition(false, getTaskName(), this.executionState, newExecutionState);
+
+		// Make sure the reason for a transition to FAILED appears in the log files
+		if (newExecutionState == ExecutionState.FAILED) {
+			LOG.error(optionalMessage);
+		}
+
+		// Notify all listener objects
+		final Iterator<ExecutionListener> it = this.registeredListeners.iterator();
+		while (it.hasNext()) {
+			it.next().executionStateChanged(this.environment.getJobID(), this.vertexID, newExecutionState,
+					optionalMessage);
+		}
+
+		// Store the new execution state
+		this.executionState = newExecutionState;
+
+		// Finally propagate the state change to the job manager
+		this.taskManager.executionStateChanged(this.environment.getJobID(), this.vertexID, newExecutionState,
+				optionalMessage);
+	}
+
+	/**
+	 * Returns the name of the task associated with this observer object.
+	 *
+	 * @return the name of the task associated with this observer object
+	 */
+	private String getTaskName() {
+
+		return this.environment.getTaskName() + " (" + (this.environment.getIndexInSubtaskGroup() + 1) + "/"
+				+ this.environment.getCurrentNumberOfSubtasks() + ")";
+	}
+
+
+	@Override
+	public void userThreadStarted(final Thread userThread) {
+
+		// Notify the listeners
+		final Iterator<ExecutionListener> it = this.registeredListeners.iterator();
+		while (it.hasNext()) {
+			it.next().userThreadStarted(this.environment.getJobID(), this.vertexID, userThread);
+		}
+	}
+
+
+	@Override
+	public void userThreadFinished(final Thread userThread) {
+
+		// Notify the listeners
+		final Iterator<ExecutionListener> it = this.registeredListeners.iterator();
+		while (it.hasNext()) {
+			it.next().userThreadFinished(this.environment.getJobID(), this.vertexID, userThread);
+		}
+	}
+
+	/**
+	 * Registers the {@link ExecutionListener} object for this task. This object
+	 * will be notified about important events during the task execution.
+	 *
+	 * @param executionListener
+	 *        the object to be notified for important events during the task execution
+	 */
+
+	public void registerExecutionListener(final ExecutionListener executionListener) {
+
+		this.registeredListeners.add(executionListener);
+	}
+
+	/**
+	 * Unregisters the {@link ExecutionListener} object for this environment. This object
+	 * will no longer be notified about important events during the task execution.
+	 *
+	 * @param executionListener
+	 *        the lister object to be unregistered
+	 */
+
+	public void unregisterExecutionListener(final ExecutionListener executionListener) {
+
+		this.registeredListeners.remove(executionListener);
+	}
+
+
+	@Override
+	public boolean isCanceled() {
+
+		return this.isCanceled;
+	}
+
+	/**
+	 * Returns the runtime environment associated with this task.
+	 *
+	 * @return the runtime environment associated with this task
+	 */
+	public RuntimeEnvironment getRuntimeEnvironment() {
+
+		return this.environment;
+	}
 
-	TaskContext createTaskContext(TransferEnvelopeDispatcher transferEnvelopeDispatcher,
-			LocalBufferPoolOwner previousBufferPoolOwner);
 }

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


[14/30] Offer buffer-oriented API for I/O (#25)

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/Gate.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/Gate.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/Gate.java
new file mode 100644
index 0000000..c9db615
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/Gate.java
@@ -0,0 +1,174 @@
+/***********************************************************************************************************************
+ * 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.gates;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.nephele.event.task.AbstractEvent;
+import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
+import eu.stratosphere.nephele.event.task.EventListener;
+import eu.stratosphere.nephele.event.task.EventNotificationManager;
+import eu.stratosphere.runtime.io.channels.ChannelType;
+import eu.stratosphere.nephele.jobgraph.JobID;
+
+import java.io.IOException;
+
+/**
+ * In Nephele a gate represents the connection between a user program and the processing framework. A gate
+ * must be connected to exactly one record reader/writer and to at least one channel. The <code>Gate</code> class itself
+ * is abstract. A gate automatically created for every record reader/writer in the user program. A gate can only be used
+ * to transport one specific type of records.
+ * <p>
+ * This class in general is not thread-safe.
+ * 
+ * @param <T>
+ *        the record type to be transported from this gate
+ *
+ *  TODO refactor with changes to input side
+ */
+public abstract class Gate<T extends IOReadableWritable> {
+
+	/**
+	 * The ID of the job this gate belongs to.
+	 */
+	private final JobID jobID;
+
+	/**
+	 * The ID of this gate.
+	 */
+	private final GateID gateID;
+
+	/**
+	 * The index of the gate in the list of available input/output gates.
+	 */
+	private final int index;
+
+	/**
+	 * The event notification manager used to dispatch events.
+	 */
+	private final EventNotificationManager eventNotificationManager = new EventNotificationManager();
+
+	/**
+	 * The type of input/output channels connected to this gate.
+	 */
+	private ChannelType channelType = ChannelType.NETWORK;
+
+	/**
+	 * Constructs a new abstract gate
+	 * 
+	 * @param jobID
+	 *        the ID of the job this gate belongs to
+	 * @param gateID
+	 *        the ID of this gate
+	 * @param index
+	 *        the index of the gate in the list of available input/output gates.
+	 */
+	protected Gate(final JobID jobID, final GateID gateID, final int index) {
+		this.jobID = jobID;
+		this.gateID = gateID;
+		this.index = index;
+	}
+
+	public final int getIndex() {
+		return this.index;
+	}
+
+	/**
+	 * Returns the event notification manager used to dispatch events.
+	 * 
+	 * @return the event notification manager used to dispatch events
+	 */
+	protected final EventNotificationManager getEventNotificationManager() {
+		return this.eventNotificationManager;
+	}
+
+	public String toString() {
+
+		return "Gate " + this.index;
+	}
+
+	public final void subscribeToEvent(EventListener eventListener, Class<? extends AbstractTaskEvent> eventType) {
+
+		this.eventNotificationManager.subscribeToEvent(eventListener, eventType);
+	}
+
+	public final void unsubscribeFromEvent(final EventListener eventListener,
+			final Class<? extends AbstractTaskEvent> eventType) {
+
+		this.eventNotificationManager.unsubscribeFromEvent(eventListener, eventType);
+	}
+
+	public final void deliverEvent(final AbstractTaskEvent event) {
+
+		this.eventNotificationManager.deliverEvent((AbstractTaskEvent) event);
+	}
+
+	public final void setChannelType(final ChannelType channelType) {
+
+		this.channelType = channelType;
+	}
+
+	public final ChannelType getChannelType() {
+
+		return this.channelType;
+	}
+
+	public JobID getJobID() {
+
+		return this.jobID;
+	}
+
+	public GateID getGateID() {
+
+		return this.gateID;
+	}
+
+	// FROM GATE INTERFACE
+
+	/**
+	 * Publishes an event.
+	 *
+	 * @param event
+	 *        the event to be published
+	 * @throws IOException
+	 *         thrown if an error occurs while transmitting the event
+	 * @throws InterruptedException
+	 *         thrown if the thread is interrupted while waiting for the event to be published
+	 */
+	abstract public void publishEvent(AbstractEvent event) throws IOException, InterruptedException;
+
+	/**
+	 * Releases the allocated resources (particularly buffer) of all channels attached to this gate. This method
+	 * should only be called after the respected task has stopped running.
+	 */
+	abstract public void releaseAllChannelResources();
+
+	/**
+	 * Checks if the gate is closed. The gate is closed if all this associated channels are closed.
+	 *
+	 * @return <code>true</code> if the gate is closed, <code>false</code> otherwise
+	 * @throws IOException
+	 *         thrown if any error occurred while closing the gate
+	 * @throws InterruptedException
+	 *         thrown if the gate is interrupted while waiting for this operation to complete
+	 */
+	abstract public boolean isClosed() throws IOException, InterruptedException;
+
+	/**
+	 * Checks if the considered gate is an input gate.
+	 *
+	 * @return <code>true</code> if the considered gate is an input gate, <code>false</code> if it is an output gate
+	 */
+	abstract public boolean isInputGate();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/GateID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/GateID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/GateID.java
new file mode 100644
index 0000000..8375c88
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/GateID.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.runtime.io.gates;
+
+import eu.stratosphere.nephele.AbstractID;
+
+/**
+ * A class for statistically unique gate IDs.
+ * 
+ */
+public final class GateID extends AbstractID {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/InputChannelResult.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/InputChannelResult.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/InputChannelResult.java
new file mode 100644
index 0000000..e2083e5
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/InputChannelResult.java
@@ -0,0 +1,23 @@
+/***********************************************************************************************************************
+ * 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.gates;
+
+public enum InputChannelResult {
+
+	NONE,
+	INTERMEDIATE_RECORD_FROM_BUFFER,
+	LAST_RECORD_FROM_BUFFER,
+	END_OF_SUPERSTEP,
+	TASK_EVENT,
+	END_OF_STREAM;
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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
new file mode 100644
index 0000000..bdac7a2
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/InputGate.java
@@ -0,0 +1,384 @@
+/***********************************************************************************************************************
+ * 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.gates;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicReference;
+
+import eu.stratosphere.nephele.deployment.ChannelDeploymentDescriptor;
+import eu.stratosphere.nephele.deployment.GateDeploymentDescriptor;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferAvailabilityListener;
+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;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.nephele.event.task.AbstractEvent;
+import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
+import eu.stratosphere.nephele.execution.Environment;
+import eu.stratosphere.runtime.io.channels.InputChannel;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.nephele.jobgraph.JobID;
+
+/**
+ * In Nephele input gates are a specialization of general gates and connect input channels and record readers. As
+ * channels, input gates are always parameterized to a specific type of record which they can transport. In contrast to
+ * output gates input gates can be associated with a {@link eu.stratosphere.runtime.io.serialization.io.DistributionPattern} object which dictates the concrete
+ * wiring between two groups of vertices.
+ * 
+ * @param <T> The type of record that can be transported through this gate.
+ */
+public class InputGate<T extends IOReadableWritable> extends Gate<T> implements BufferProvider, LocalBufferPoolOwner {
+	
+	/**
+	 * The log object used for debugging.
+	 */
+	private static final Log LOG = LogFactory.getLog(InputGate.class);
+
+	/**
+	 * The array of input channels attached to this input gate.
+	 */
+	private InputChannel<T>[] channels;
+
+	/**
+	 * Queue with indices of channels that store at least one available record.
+	 */
+	private final BlockingQueue<Integer> availableChannels = new LinkedBlockingQueue<Integer>();
+
+	/**
+	 * The listener object to be notified when a channel has at least one record available.
+	 */
+	private final AtomicReference<RecordAvailabilityListener<T>> recordAvailabilityListener = new AtomicReference<RecordAvailabilityListener<T>>(null);
+	
+	
+	private AbstractTaskEvent currentEvent;
+
+	/**
+	 * If the value of this variable is set to <code>true</code>, the input gate is closed.
+	 */
+	private boolean isClosed = false;
+
+	/**
+	 * The channel to read from next.
+	 */
+	private int channelToReadFrom = -1;
+
+	private LocalBufferPool bufferPool;
+
+	/**
+	 * Constructs a new runtime input gate.
+	 * 
+	 * @param jobID
+	 *        the ID of the job this input gate belongs to
+	 * @param gateID
+	 *        the ID of the gate
+	 * @param index
+	 *        the index assigned to this input gate at the {@link Environment} object
+	 */
+	public InputGate(final JobID jobID, final GateID gateID, final int index) {
+		super(jobID, gateID, index);
+	}
+
+	public void initializeChannels(GateDeploymentDescriptor inputGateDescriptor){
+		channels = new InputChannel[inputGateDescriptor.getNumberOfChannelDescriptors()];
+
+		setChannelType(inputGateDescriptor.getChannelType());
+
+		final int nicdd = inputGateDescriptor.getNumberOfChannelDescriptors();
+
+		for(int i = 0; i < nicdd; i++){
+			final ChannelDeploymentDescriptor cdd = inputGateDescriptor.getChannelDescriptor(i);
+			channels[i] = new InputChannel<T>(this, i, cdd.getInputChannelID(),
+					cdd.getOutputChannelID(), getChannelType());
+		}
+	}
+
+	@Override
+	public boolean isInputGate() {
+		return true;
+	}
+
+	/**
+	 * Returns the number of input channels associated with this input gate.
+	 *
+	 * @return the number of input channels associated with this input gate
+	 */
+	public int getNumberOfInputChannels() {
+		return this.channels.length;
+	}
+
+	/**
+	 * Returns the input channel from position <code>pos</code> of the gate's internal channel list.
+	 *
+	 * @param pos
+	 *        the position to retrieve the channel from
+	 * @return the channel from the given position or <code>null</code> if such position does not exist.
+	 */
+	public InputChannel<T> getInputChannel(int pos) {
+		return this.channels[pos];
+	}
+
+	public InputChannel<T>[] channels() {
+		return this.channels;
+	}
+
+	/**
+	 * Reads a record from one of the associated input channels. Channels are read such that one buffer from a channel is
+	 * consecutively consumed. The buffers in turn are consumed in the order in which they arrive.
+	 * Note that this method is not guaranteed to return a record, because the currently available channel data may not always
+	 * constitute an entire record, when events or partial records are part of the data.
+	 *
+	 * When called even though no data is available, this call will block until data is available, so this method should be called
+	 * when waiting is desired (such as when synchronously consuming a single gate) or only when it is known that data is available
+	 * (such as when reading a union of multiple input gates).
+	 *
+	 * @param target The record object into which to construct the complete record.
+	 * @return The result indicating whether a complete record is available, a event is available, only incomplete data
+	 *         is available (NONE), or the gate is exhausted.
+	 * @throws IOException Thrown when an error occurred in the network stack relating to this channel.
+	 * @throws InterruptedException Thrown, when the thread working on this channel is interrupted.
+	 */
+	public InputChannelResult readRecord(T target) throws IOException, InterruptedException {
+
+		if (this.channelToReadFrom == -1) {
+			if (this.isClosed()) {
+				return InputChannelResult.END_OF_STREAM;
+			}
+				
+			if (Thread.interrupted()) {
+				throw new InterruptedException();
+			}
+				
+			this.channelToReadFrom = waitForAnyChannelToBecomeAvailable();
+		}
+			
+		InputChannelResult result = this.getInputChannel(this.channelToReadFrom).readRecord(target);
+		switch (result) {
+			case INTERMEDIATE_RECORD_FROM_BUFFER: // full record and we can stay on the same channel
+				return InputChannelResult.INTERMEDIATE_RECORD_FROM_BUFFER;
+				
+			case LAST_RECORD_FROM_BUFFER: // full record, but we must switch the channel afterwards
+				this.channelToReadFrom = -1;
+				return InputChannelResult.LAST_RECORD_FROM_BUFFER;
+				
+			case END_OF_SUPERSTEP:
+				this.channelToReadFrom = -1;
+				return InputChannelResult.END_OF_SUPERSTEP;
+				
+			case TASK_EVENT: // task event
+				this.currentEvent = this.getInputChannel(this.channelToReadFrom).getCurrentEvent();
+				this.channelToReadFrom = -1;	// event always marks a unit as consumed
+				return InputChannelResult.TASK_EVENT;
+					
+			case NONE: // internal event or an incomplete record that needs further chunks
+				// the current unit is exhausted
+				this.channelToReadFrom = -1;
+				return InputChannelResult.NONE;
+				
+			case END_OF_STREAM: // channel is done
+				this.channelToReadFrom = -1;
+				return isClosed() ? InputChannelResult.END_OF_STREAM : InputChannelResult.NONE;
+				
+			default:   // silence the compiler
+				throw new RuntimeException();
+		}
+	}
+
+	public AbstractTaskEvent getCurrentEvent() {
+		AbstractTaskEvent e = this.currentEvent;
+		this.currentEvent = null;
+		return e;
+	}
+
+	/**
+	 * Notify the gate that the channel with the given index has
+	 * at least one record available.
+	 *
+	 * @param channelIndex
+	 *        the index of the channel which has at least one record available
+	 */
+	public void notifyRecordIsAvailable(int channelIndex) {
+		this.availableChannels.add(Integer.valueOf(channelIndex));
+
+		RecordAvailabilityListener<T> listener = this.recordAvailabilityListener.get();
+		if (listener != null) {
+			listener.reportRecordAvailability(this);
+		}
+	}
+
+	/**
+	 * This method returns the index of a channel which has at least
+	 * one record available. The method may block until at least one
+	 * channel has become ready.
+	 * 
+	 * @return the index of the channel which has at least one record available
+	 */
+	public int waitForAnyChannelToBecomeAvailable() throws InterruptedException {
+		return this.availableChannels.take().intValue();
+	}
+
+
+	@Override
+	public boolean isClosed() throws IOException, InterruptedException {
+
+		if (this.isClosed) {
+			return true;
+		}
+
+		for (int i = 0; i < this.getNumberOfInputChannels(); i++) {
+			final InputChannel<T> inputChannel = this.channels[i];
+			if (!inputChannel.isClosed()) {
+				return false;
+			}
+		}
+
+		this.isClosed = true;
+		
+		return true;
+	}
+
+
+	/**
+	 * Immediately closes the input gate and all its input channels. The corresponding
+	 * output channels are notified. Any remaining records in any buffers or queue is considered
+	 * irrelevant and is discarded.
+	 *
+	 * @throws IOException
+	 *         thrown if an I/O error occurs while closing the gate
+	 * @throws InterruptedException
+	 *         thrown if the thread is interrupted while waiting for the gate to be closed
+	 */
+	public void close() throws IOException, InterruptedException {
+
+		for (int i = 0; i < this.getNumberOfInputChannels(); i++) {
+			final InputChannel<T> inputChannel = this.channels[i];
+			inputChannel.close();
+		}
+
+	}
+
+
+	@Override
+	public String toString() {
+		return "Input " + super.toString();
+	}
+
+
+	@Override
+	public void publishEvent(AbstractEvent event) throws IOException, InterruptedException {
+
+		// Copy event to all connected channels
+		for(int i=0; i< getNumberOfChannels(); i++){
+			channels[i].transferEvent(event);
+		}
+	}
+
+
+	@Override
+	public void releaseAllChannelResources() {
+
+		for(int i=0; i< getNumberOfChannels(); i++){
+			channels[i].releaseAllResources();
+		}
+	}
+
+	/**
+	 * Registers a {@link RecordAvailabilityListener} with this input gate.
+	 *
+	 * @param listener
+	 *        the listener object to be registered
+	 */
+	public void registerRecordAvailabilityListener(final RecordAvailabilityListener<T> listener) {
+		if (!this.recordAvailabilityListener.compareAndSet(null, listener)) {
+			throw new IllegalStateException(this.recordAvailabilityListener
+				+ " is already registered as a record availability listener");
+		}
+	}
+
+	/**
+	 * Notify the gate that is has consumed a data unit from the channel with the given index
+	 *
+	 * @param channelIndex
+	 *        the index of the channel from which a data unit has been consumed
+	 */
+	public void notifyDataUnitConsumed(int channelIndex) {
+		this.channelToReadFrom = -1;
+	}
+
+	//
+
+	@Override
+	public Buffer requestBuffer(int minBufferSize) throws IOException {
+		return this.bufferPool.requestBuffer(minBufferSize);
+	}
+
+	@Override
+	public Buffer requestBufferBlocking(int minBufferSize) throws IOException, InterruptedException {
+		return this.bufferPool.requestBufferBlocking(minBufferSize);
+	}
+
+	@Override
+	public int getBufferSize() {
+		return this.bufferPool.getBufferSize();
+	}
+
+	@Override
+	public int getNumberOfChannels() {
+		return getNumberOfInputChannels();
+	}
+
+	@Override
+	public void setDesignatedNumberOfBuffers(int numBuffers) {
+		this.bufferPool.setNumDesignatedBuffers(numBuffers);
+	}
+
+	@Override
+	public void clearLocalBufferPool() {
+		this.bufferPool.destroy();
+	}
+
+	@Override
+	public void registerGlobalBufferPool(GlobalBufferPool globalBufferPool) {
+		this.bufferPool = new LocalBufferPool(globalBufferPool, 1);
+	}
+
+	@Override
+	public void logBufferUtilization() {
+		LOG.info(String.format("\t%s: %d available, %d requested, %d designated",
+				this,
+				this.bufferPool.numAvailableBuffers(),
+				this.bufferPool.numRequestedBuffers(),
+				this.bufferPool.numDesignatedBuffers()));
+	}
+
+	@Override
+	public void reportAsynchronousEvent() {
+		this.bufferPool.reportAsynchronousEvent();
+	}
+
+	@Override
+	public boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
+		return this.bufferPool.registerBufferAvailabilityListener(listener);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/OutputGate.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/OutputGate.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/OutputGate.java
new file mode 100644
index 0000000..d3eaea1
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/OutputGate.java
@@ -0,0 +1,165 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.gates;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.nephele.deployment.ChannelDeploymentDescriptor;
+import eu.stratosphere.nephele.deployment.GateDeploymentDescriptor;
+import eu.stratosphere.nephele.event.task.AbstractEvent;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.channels.OutputChannel;
+import eu.stratosphere.nephele.jobgraph.JobID;
+
+import java.io.IOException;
+
+public class OutputGate extends Gate<IOReadableWritable> {
+
+	private OutputChannel[] channels;
+
+	private boolean closed;
+	
+	/**
+	 * Constructs a new output gate.
+	 *
+	 * @param jobId the ID of the job this input gate belongs to
+	 * @param gateId the ID of the gate
+	 * @param index the index assigned to this output gate at the Environment object
+	 */
+	public OutputGate(JobID jobId, GateID gateId, int index) {
+		super(jobId, gateId, index);
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+	//                                             Data processing
+	// -----------------------------------------------------------------------------------------------------------------
+
+	public void sendBuffer(Buffer buffer, int targetChannel) throws IOException, InterruptedException {
+		this.channels[targetChannel].sendBuffer(buffer);
+	}
+
+	public void sendEvent(AbstractEvent event, int targetChannel) throws IOException, InterruptedException {
+		this.channels[targetChannel].sendEvent(event);
+	}
+
+	public void sendBufferAndEvent(Buffer buffer, AbstractEvent event, int targetChannel) throws IOException, InterruptedException {
+		this.channels[targetChannel].sendBufferAndEvent(buffer, event);
+	}
+
+	public void broadcastBuffer(Buffer buffer) throws IOException, InterruptedException {
+		for (int i = 1; i < this.channels.length; i++) {
+			channels[i].sendBuffer(buffer.duplicate());
+		}
+		channels[0].sendBuffer(buffer);
+	}
+
+	public void broadcastEvent(AbstractEvent event) throws IOException, InterruptedException {
+		for (OutputChannel channel : this.channels) {
+			channel.sendEvent(event);
+		}
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+	//                                              Channels
+	// -----------------------------------------------------------------------------------------------------------------
+
+	public void initializeChannels(GateDeploymentDescriptor descriptor) {
+		int numChannels = descriptor.getNumberOfChannelDescriptors();
+		this.channels = new OutputChannel[numChannels];
+
+		setChannelType(descriptor.getChannelType());
+
+		for (int i = 0; i < numChannels; i++) {
+			ChannelDeploymentDescriptor channelDescriptor = descriptor.getChannelDescriptor(i);
+
+			ChannelID id = channelDescriptor.getOutputChannelID();
+			ChannelID connectedId = channelDescriptor.getInputChannelID();
+
+			this.channels[i] = new OutputChannel(this, i, id, connectedId, getChannelType());
+		}
+	}
+
+	public OutputChannel[] channels() {
+		return this.channels;
+	}
+
+	public OutputChannel getChannel(int index) {
+		return (index < this.channels.length) ? this.channels[index] : null;
+	}
+
+	public int getNumChannels() {
+		return this.channels.length;
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+	//                                              Shutdown
+	// -----------------------------------------------------------------------------------------------------------------
+
+	public void requestClose() throws IOException, InterruptedException {
+		for (OutputChannel channel : this.channels) {
+			channel.requestClose();
+		}
+	}
+
+	@Override
+	public boolean isClosed() {
+		if (this.closed) {
+			return true;
+		}
+		
+		for (OutputChannel channel : this.channels) {
+			if (!channel.isClosed()) {
+				return false;
+			}
+		}
+		
+		this.closed = true;
+		return true;
+	}
+	
+	public void waitForGateToBeClosed() throws InterruptedException {
+		if (this.closed) {
+			return;
+		}
+		
+		for (OutputChannel channel : this.channels) {
+			channel.waitForChannelToBeClosed();
+		}
+		
+		this.closed = true;
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+
+	@Override
+	public boolean isInputGate() {
+		return false;
+	}
+
+	@Override
+	public String toString() {
+		return "Output " + super.toString();
+	}
+
+	@Override
+	public void publishEvent(AbstractEvent event) throws IOException, InterruptedException {
+		// replaced by broadcastEvent(AbstractEvent) => TODO will be removed with changes to input side
+	}
+
+	@Override
+	public void releaseAllChannelResources() {
+		// nothing to do for buffer oriented runtime => TODO will be removed with changes to input side
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/RecordAvailabilityListener.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/RecordAvailabilityListener.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/RecordAvailabilityListener.java
new file mode 100644
index 0000000..ea1d865
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/RecordAvailabilityListener.java
@@ -0,0 +1,36 @@
+/***********************************************************************************************************************
+ * 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.gates;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.runtime.io.gates.InputGate;
+
+/**
+ * This interface can be implemented by a class which shall be notified by an input gate when one of the its connected
+ * input channels has at least one record available for reading.
+ * 
+ * @param <T>
+ *        the type of record transported through the corresponding input gate
+ */
+public interface RecordAvailabilityListener<T extends IOReadableWritable> {
+
+	/**
+	 * This method is called by an input gate when one of its connected input channels has at least one record available
+	 * for reading.
+	 * 
+	 * @param inputGate
+	 *        the input gate which has at least one record available
+	 */
+	void reportRecordAvailability(InputGate<T> inputGate);
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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
new file mode 100644
index 0000000..405d79e
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/ChannelManager.java
@@ -0,0 +1,646 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.network;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.nephele.execution.CancelTaskException;
+import eu.stratosphere.nephele.execution.Environment;
+import eu.stratosphere.nephele.execution.RuntimeEnvironment;
+import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
+import eu.stratosphere.nephele.instance.InstanceConnectionInfo;
+import eu.stratosphere.runtime.io.channels.Channel;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.channels.InputChannel;
+import eu.stratosphere.runtime.io.channels.OutputChannel;
+import eu.stratosphere.nephele.jobgraph.JobID;
+import eu.stratosphere.nephele.protocols.ChannelLookupProtocol;
+import eu.stratosphere.nephele.taskmanager.Task;
+import eu.stratosphere.nephele.AbstractID;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferProviderBroker;
+import eu.stratosphere.runtime.io.network.bufferprovider.GlobalBufferPool;
+import eu.stratosphere.runtime.io.network.bufferprovider.LocalBufferPoolOwner;
+import eu.stratosphere.runtime.io.network.bufferprovider.SerialSingleBufferPool;
+import eu.stratosphere.runtime.io.network.envelope.Envelope;
+import eu.stratosphere.runtime.io.network.envelope.EnvelopeDispatcher;
+import eu.stratosphere.runtime.io.network.envelope.EnvelopeReceiverList;
+import eu.stratosphere.runtime.io.gates.GateID;
+import eu.stratosphere.runtime.io.gates.InputGate;
+import eu.stratosphere.runtime.io.gates.OutputGate;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The channel manager sets up the network buffers and dispatches data between channels.
+ */
+public final class ChannelManager implements EnvelopeDispatcher, BufferProviderBroker {
+
+	private static final Log LOG = LogFactory.getLog(ChannelManager.class);
+
+	private final ChannelLookupProtocol channelLookupService;
+
+	private final InstanceConnectionInfo connectionInfo;
+
+	private final Map<ChannelID, Channel> channels;
+
+	private final Map<AbstractID, LocalBufferPoolOwner> localBuffersPools;
+
+	private final Map<ChannelID, EnvelopeReceiverList> receiverCache;
+
+	private final GlobalBufferPool globalBufferPool;
+
+	private final NetworkConnectionManager networkConnectionManager;
+	
+	private final InetSocketAddress ourAddress;
+	
+	private final SerialSingleBufferPool discardingDataPool;
+
+	// -----------------------------------------------------------------------------------------------------------------
+
+	public ChannelManager(ChannelLookupProtocol channelLookupService, InstanceConnectionInfo connectionInfo,
+						  int numNetworkBuffers, int networkBufferSize) throws IOException {
+		this.channelLookupService = channelLookupService;
+		this.connectionInfo = connectionInfo;
+		this.globalBufferPool = new GlobalBufferPool(numNetworkBuffers, networkBufferSize);
+		this.networkConnectionManager = new NetworkConnectionManager(this, connectionInfo.address(), connectionInfo.dataPort());
+
+		// management data structures
+		this.channels = new ConcurrentHashMap<ChannelID, Channel>();
+		this.receiverCache = new ConcurrentHashMap<ChannelID, EnvelopeReceiverList>();
+		this.localBuffersPools = new ConcurrentHashMap<AbstractID, LocalBufferPoolOwner>();
+		
+		this.ourAddress = new InetSocketAddress(connectionInfo.address(), connectionInfo.dataPort());
+		
+		// a special pool if the data is to be discarded
+		this.discardingDataPool = new SerialSingleBufferPool(networkBufferSize);
+	}
+
+	public void shutdown() {
+		this.networkConnectionManager.shutDown();
+		this.globalBufferPool.destroy();
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+	//                                               Task registration
+	// -----------------------------------------------------------------------------------------------------------------
+
+	/**
+	 * Registers the given task with the channel manager.
+	 *
+	 * @param task the task to be registered
+	 * @throws InsufficientResourcesException thrown if not enough buffers available to safely run this task
+	 */
+	public void register(Task task) throws InsufficientResourcesException {
+		// Check if we can safely run this task with the given buffers
+		ensureBufferAvailability(task);
+
+		RuntimeEnvironment environment = task.getRuntimeEnvironment();
+
+		// -------------------------------------------------------------------------------------------------------------
+		//                                       Register output channels
+		// -------------------------------------------------------------------------------------------------------------
+
+		environment.registerGlobalBufferPool(this.globalBufferPool);
+
+		if (this.localBuffersPools.containsKey(task.getVertexID())) {
+			throw new IllegalStateException("Vertex " + task.getVertexID() + " has a previous buffer pool owner");
+		}
+
+		for (OutputGate gate : environment.outputGates()) {
+			// add receiver list hints
+			for (OutputChannel channel : gate.channels()) {
+				// register envelope dispatcher with the channel
+				channel.registerEnvelopeDispatcher(this);
+
+				switch (channel.getChannelType()) {
+					case IN_MEMORY:
+						addReceiverListHint(channel.getID(), channel.getConnectedId());
+						break;
+					case NETWORK:
+						addReceiverListHint(channel.getConnectedId(), channel.getID());
+						break;
+				}
+
+				this.channels.put(channel.getID(), channel);
+			}
+		}
+
+		this.localBuffersPools.put(task.getVertexID(), environment);
+
+		// -------------------------------------------------------------------------------------------------------------
+		//                                       Register input channels
+		// -------------------------------------------------------------------------------------------------------------
+
+		// register global
+		for (InputGate<?> gate : environment.inputGates()) {
+			gate.registerGlobalBufferPool(this.globalBufferPool);
+
+			for (int i = 0; i < gate.getNumberOfInputChannels(); i++) {
+				InputChannel<? extends IOReadableWritable> channel = gate.getInputChannel(i);
+				channel.registerEnvelopeDispatcher(this);
+
+				if (channel.getChannelType() == ChannelType.IN_MEMORY) {
+					addReceiverListHint(channel.getID(), channel.getConnectedId());
+				}
+
+				this.channels.put(channel.getID(), channel);
+			}
+
+			this.localBuffersPools.put(gate.getGateID(), gate);
+		}
+
+		// the number of channels per buffers has changed after unregistering the task
+		// => redistribute the number of designated buffers of the registered local buffer pools
+		redistributeBuffers();
+	}
+
+	/**
+	 * Unregisters the given task from the channel manager.
+	 *
+	 * @param vertexId the ID of the task to be unregistered
+	 * @param task the task to be unregistered
+	 */
+	public void unregister(ExecutionVertexID vertexId, Task task) {
+		final Environment environment = task.getEnvironment();
+
+		// destroy and remove OUTPUT channels from registered channels and cache
+		for (ChannelID id : environment.getOutputChannelIDs()) {
+			Channel channel = this.channels.remove(id);
+			if (channel != null) {
+				channel.destroy();
+			}
+
+			this.receiverCache.remove(channel);
+		}
+
+		// destroy and remove INPUT channels from registered channels and cache
+		for (ChannelID id : environment.getInputChannelIDs()) {
+			Channel channel = this.channels.remove(id);
+			if (channel != null) {
+				channel.destroy();
+			}
+
+			this.receiverCache.remove(channel);
+		}
+
+		// clear and remove INPUT side buffer pools
+		for (GateID id : environment.getInputGateIDs()) {
+			LocalBufferPoolOwner bufferPool = this.localBuffersPools.remove(id);
+			if (bufferPool != null) {
+				bufferPool.clearLocalBufferPool();
+			}
+		}
+
+		// clear and remove OUTPUT side buffer pool
+		LocalBufferPoolOwner bufferPool = this.localBuffersPools.remove(vertexId);
+		if (bufferPool != null) {
+			bufferPool.clearLocalBufferPool();
+		}
+
+		// the number of channels per buffers has changed after unregistering the task
+		// => redistribute the number of designated buffers of the registered local buffer pools
+		redistributeBuffers();
+	}
+
+	/**
+	 * Ensures that the channel manager has enough buffers to execute the given task.
+	 * <p>
+	 * If there is less than one buffer per channel available, an InsufficientResourcesException will be thrown,
+	 * because of possible deadlocks. With more then one buffer per channel, deadlock-freedom is guaranteed.
+	 *
+	 * @param task task to be executed
+	 * @throws InsufficientResourcesException thrown if not enough buffers available to execute the task
+	 */
+	private void ensureBufferAvailability(Task task) throws InsufficientResourcesException {
+		Environment env = task.getEnvironment();
+
+		int numBuffers = this.globalBufferPool.numBuffers();
+		// existing channels + channels of the task
+		int numChannels = this.channels.size() + env.getNumberOfOutputChannels() + env.getNumberOfInputChannels();
+
+		// need at least one buffer per channel
+		if (numBuffers / numChannels < 1) {
+			String msg = String.format("%s has not enough buffers to safely execute %s (%d buffers missing)",
+					this.connectionInfo.hostname(), env.getTaskName(), numChannels - numBuffers);
+
+			throw new InsufficientResourcesException(msg);
+		}
+	}
+
+	/**
+	 * Redistributes the buffers among the registered buffer pools. This method is called after each task registration
+	 * and unregistration.
+	 * <p>
+	 * Every registered buffer pool gets buffers according to its number of channels weighted by the current buffer to
+	 * channel ratio.
+	 */
+	private void redistributeBuffers() {
+		if (this.localBuffersPools.isEmpty() | this.channels.size() == 0) {
+			return;
+		}
+
+		int numBuffers = this.globalBufferPool.numBuffers();
+		int numChannels = this.channels.size();
+
+		double buffersPerChannel = numBuffers / (double) numChannels;
+
+		if (buffersPerChannel < 1.0) {
+			throw new RuntimeException("System has not enough buffers to execute tasks.");
+		}
+
+		// redistribute number of designated buffers per buffer pool
+		for (LocalBufferPoolOwner bufferPool : this.localBuffersPools.values()) {
+			int numDesignatedBuffers = (int) Math.ceil(buffersPerChannel * bufferPool.getNumberOfChannels());
+			bufferPool.setDesignatedNumberOfBuffers(numDesignatedBuffers);
+		}
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+	//                                           Envelope processing
+	// -----------------------------------------------------------------------------------------------------------------
+
+	private void releaseEnvelope(Envelope envelope) {
+		Buffer buffer = envelope.getBuffer();
+		if (buffer != null) {
+			buffer.recycleBuffer();
+		}
+	}
+
+	private void addReceiverListHint(ChannelID source, ChannelID localReceiver) {
+		EnvelopeReceiverList receiverList = new EnvelopeReceiverList(localReceiver);
+
+		if (this.receiverCache.put(source, receiverList) != null) {
+			LOG.warn("Receiver cache already contained entry for " + source);
+		}
+	}
+
+	private void addReceiverListHint(ChannelID source, RemoteReceiver remoteReceiver) {
+		EnvelopeReceiverList receiverList = new EnvelopeReceiverList(remoteReceiver);
+
+		if (this.receiverCache.put(source, receiverList) != null) {
+			LOG.warn("Receiver cache already contained entry for " + source);
+		}
+	}
+
+	private void generateSenderHint(Envelope envelope, RemoteReceiver receiver) {
+		Channel channel = this.channels.get(envelope.getSource());
+		if (channel == null) {
+			LOG.error("Cannot find channel for channel ID " + envelope.getSource());
+			return;
+		}
+
+		// Only generate sender hints for output channels
+		if (channel.isInputChannel()) {
+			return;
+		}
+
+		final ChannelID targetChannelID = channel.getConnectedId();
+		final int connectionIndex = receiver.getConnectionIndex();
+
+		final RemoteReceiver ourAddress = new RemoteReceiver(this.ourAddress, connectionIndex);
+		final Envelope senderHint = SenderHintEvent.createEnvelopeWithEvent(envelope, targetChannelID, ourAddress);
+
+		this.networkConnectionManager.queueEnvelopeForTransfer(receiver, senderHint);
+	}
+
+	/**
+	 * Returns the list of receivers for transfer envelopes produced by the channel with the given source channel ID.
+	 *
+	 * @param jobID
+	 *        the ID of the job the given channel ID belongs to
+	 * @param sourceChannelID
+	 *        the source channel ID for which the receiver list shall be retrieved
+	 * @return the list of receivers or <code>null</code> if the receiver could not be determined
+	 * @throws IOException
+	 * @throws InterruptedException
+	 */
+	private EnvelopeReceiverList getReceiverList(JobID jobID, ChannelID sourceChannelID, boolean reportException) throws IOException {
+		EnvelopeReceiverList receiverList = this.receiverCache.get(sourceChannelID);
+
+		if (receiverList != null) {
+			return receiverList;
+		}
+
+		while (true) {
+			ConnectionInfoLookupResponse lookupResponse;
+			synchronized (this.channelLookupService) {
+				lookupResponse = this.channelLookupService.lookupConnectionInfo(this.connectionInfo, jobID, sourceChannelID);
+			}
+
+			if (lookupResponse.receiverReady()) {
+				receiverList = new EnvelopeReceiverList(lookupResponse);
+				break;
+			}
+			else if (lookupResponse.receiverNotReady()) {
+				try {
+					Thread.sleep(500);
+				} catch (InterruptedException e) {
+					if (reportException) {
+						throw new IOException("Lookup was interrupted.");
+					} else {
+						return null;
+					}
+				}
+			}
+			else if (lookupResponse.isJobAborting()) {
+				if (reportException) {
+					throw new CancelTaskException();
+				} else {
+					return null;
+				}
+			}
+			else if (lookupResponse.receiverNotFound()) {
+				if (reportException) {
+					throw new IOException("Could not find the receiver for Job " + jobID + ", channel with source id " + sourceChannelID);
+				} else {
+					return null;
+				}
+			}
+			else {
+				throw new IllegalStateException("Unrecognized response to channel lookup.");
+			}
+		}
+
+		this.receiverCache.put(sourceChannelID, receiverList);
+
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("Receivers for source channel ID " + sourceChannelID + " at task manager " + this.connectionInfo +
+				": " + receiverList);
+		}
+
+		return receiverList;
+	}
+
+	/**
+	 * Invalidates the entries identified by the given channel IDs from the receiver lookup cache.
+	 *
+	 * @param channelIDs channel IDs for entries to invalidate
+	 */
+	public void invalidateLookupCacheEntries(Set<ChannelID> channelIDs) {
+		for (ChannelID id : channelIDs) {
+			this.receiverCache.remove(id);
+		}
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+	//                                       EnvelopeDispatcher methods
+	// -----------------------------------------------------------------------------------------------------------------
+
+	@Override
+	public void dispatchFromOutputChannel(Envelope envelope) throws IOException, InterruptedException {
+		EnvelopeReceiverList receiverList = getReceiverListForEnvelope(envelope, true);
+
+		Buffer srcBuffer = envelope.getBuffer();
+		Buffer destBuffer = null;
+		
+		boolean success = false;
+		
+		try {
+			if (receiverList.hasLocalReceiver()) {
+				ChannelID receiver = receiverList.getLocalReceiver();
+				Channel channel = this.channels.get(receiver);
+
+				if (channel == null) {
+					throw new LocalReceiverCancelledException(receiver);
+				}
+
+				if (!channel.isInputChannel()) {
+					throw new IOException("Local receiver " + receiver + " is not an input channel.");
+				}
+
+				InputChannel<?> inputChannel = (InputChannel<?>) channel;
+				
+				// copy the buffer into the memory space of the receiver 
+				if (srcBuffer != null) {
+					try {
+						destBuffer = inputChannel.requestBufferBlocking(srcBuffer.size());
+					} catch (InterruptedException e) {
+						throw new IOException(e.getMessage());
+					}
+					
+					srcBuffer.copyToBuffer(destBuffer);
+					envelope.setBuffer(destBuffer);
+					srcBuffer.recycleBuffer();
+				}
+				
+				inputChannel.queueEnvelope(envelope);
+				success = true;
+			}
+			else if (receiverList.hasRemoteReceiver()) {
+				RemoteReceiver remoteReceiver = receiverList.getRemoteReceiver();
+
+				// Generate sender hint before sending the first envelope over the network
+				if (envelope.getSequenceNumber() == 0) {
+					generateSenderHint(envelope, remoteReceiver);
+				}
+
+				this.networkConnectionManager.queueEnvelopeForTransfer(remoteReceiver, envelope);
+				success = true;
+			}
+		} finally {
+			if (!success) {
+				if (srcBuffer != null) {
+					srcBuffer.recycleBuffer();
+				}
+				if (destBuffer != null) {
+					destBuffer.recycleBuffer();
+				}
+			}
+		}
+	}
+
+	@Override
+	public void dispatchFromInputChannel(Envelope envelope) throws IOException, InterruptedException {
+		// this method sends only events back from input channels to output channels
+		// sanity check that we have no buffer
+		if (envelope.getBuffer() != null) {
+			throw new RuntimeException("Error: This method can only process envelopes without buffers.");
+		}
+		
+		EnvelopeReceiverList receiverList = getReceiverListForEnvelope(envelope, true);
+
+		if (receiverList.hasLocalReceiver()) {
+			ChannelID receiver = receiverList.getLocalReceiver();
+			Channel channel = this.channels.get(receiver);
+
+			if (channel == null) {
+				throw new LocalReceiverCancelledException(receiver);
+			}
+
+			if (channel.isInputChannel()) {
+				throw new IOException("Local receiver " + receiver + " of backward event is not an output channel.");
+			}
+
+			OutputChannel outputChannel = (OutputChannel) channel;
+			outputChannel.queueEnvelope(envelope);
+		}
+		else if (receiverList.hasRemoteReceiver()) {
+			RemoteReceiver remoteReceiver = receiverList.getRemoteReceiver();
+
+			// Generate sender hint before sending the first envelope over the network
+			if (envelope.getSequenceNumber() == 0) {
+				generateSenderHint(envelope, remoteReceiver);
+			}
+
+			this.networkConnectionManager.queueEnvelopeForTransfer(remoteReceiver, envelope);
+		}
+	}
+
+	/**
+	 * 
+	 */
+	@Override
+	public void dispatchFromNetwork(Envelope envelope) throws IOException, InterruptedException {
+		// ========================================================================================
+		//  IMPORTANT
+		//  
+		//  This method is called by the network I/O thread that reads the incoming TCP 
+		//  connections. This method must have minimal overhead and not throw exception if
+		//  something is wrong with a job or individual transmission, but only when something
+		//  is fundamentally broken in the system.
+		// ========================================================================================
+		
+		// the sender hint event is to let the receiver know where exactly the envelope came from.
+		// the receiver will cache the sender id and its connection info in its local lookup table
+		// that allows the receiver to send envelopes to the sender without first pinging the job manager
+		// for the sender's connection info
+		
+		// Check if the envelope is the special envelope with the sender hint event
+		if (SenderHintEvent.isSenderHintEvent(envelope)) {
+			// Check if this is the final destination of the sender hint event before adding it
+			final SenderHintEvent seh = (SenderHintEvent) envelope.deserializeEvents().get(0);
+			if (this.channels.get(seh.getSource()) != null) {
+				addReceiverListHint(seh.getSource(), seh.getRemoteReceiver());
+				return;
+			}
+		}
+		
+		// try and get the receiver list. if we cannot get it anymore, the task has been cleared
+		// the code frees the envelope on exception, so we need not to anything
+		EnvelopeReceiverList receiverList = getReceiverListForEnvelope(envelope, false);
+		if (receiverList == null) {
+			// receiver is cancelled and cleaned away
+			releaseEnvelope(envelope);
+			if (LOG.isDebugEnabled()) {
+				LOG.debug("Dropping envelope for cleaned up receiver.");
+			}
+
+			return;
+		}
+
+		if (!receiverList.hasLocalReceiver() || receiverList.hasRemoteReceiver()) {
+			throw new IOException("Bug in network stack: Envelope dispatched from the incoming network pipe has no local receiver or has a remote receiver");
+		}
+
+		ChannelID localReceiver = receiverList.getLocalReceiver();
+		Channel channel = this.channels.get(localReceiver);
+		
+		// if the channel is null, it means that receiver has been cleared already (cancelled or failed).
+		// release the buffer immediately
+		if (channel == null) {
+			releaseEnvelope(envelope);
+			if (LOG.isDebugEnabled()) {
+				LOG.debug("Dropping envelope for cancelled receiver " + localReceiver);
+			}
+		}
+		else {
+			channel.queueEnvelope(envelope);
+		}
+	}
+
+	/**
+	 * 
+	 * Upon an exception, this method frees the envelope.
+	 * 
+	 * @param envelope
+	 * @return
+	 * @throws IOException
+	 */
+	private final EnvelopeReceiverList getReceiverListForEnvelope(Envelope envelope, boolean reportException) throws IOException {
+		try {
+			return getReceiverList(envelope.getJobID(), envelope.getSource(), reportException);
+		} catch (IOException e) {
+			releaseEnvelope(envelope);
+			throw e;
+		} catch (CancelTaskException e) {
+			releaseEnvelope(envelope);
+			throw e;
+		}
+	}
+	
+	// -----------------------------------------------------------------------------------------------------------------
+	//                                       BufferProviderBroker methods
+	// -----------------------------------------------------------------------------------------------------------------
+
+	@Override
+	public BufferProvider getBufferProvider(JobID jobID, ChannelID sourceChannelID) throws IOException {
+		EnvelopeReceiverList receiverList = getReceiverList(jobID, sourceChannelID, false);
+		
+		// check if the receiver is already gone
+		if (receiverList == null) {
+			return this.discardingDataPool;
+		}
+
+		if (!receiverList.hasLocalReceiver() || receiverList.hasRemoteReceiver()) {
+			throw new IOException("The destination to be looked up is not a single local endpoint.");
+		}
+		
+
+		ChannelID localReceiver = receiverList.getLocalReceiver();
+		Channel channel = this.channels.get(localReceiver);
+		
+		if (channel == null) {
+			// receiver is already canceled
+			return this.discardingDataPool;
+		}
+
+		if (!channel.isInputChannel()) {
+			throw new IOException("Channel context for local receiver " + localReceiver + " is not an input channel context");
+		}
+
+		return (InputChannel<?>) channel;
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+
+	public void logBufferUtilization() {
+		System.out.println("Buffer utilization at " + System.currentTimeMillis());
+
+		System.out.println("\tUnused global buffers: " + this.globalBufferPool.numAvailableBuffers());
+
+		System.out.println("\tLocal buffer pool status:");
+
+		for (LocalBufferPoolOwner bufferPool : this.localBuffersPools.values()) {
+			bufferPool.logBufferUtilization();
+		}
+
+		this.networkConnectionManager.logBufferUtilization();
+
+		System.out.println("\tIncoming connections:");
+
+		for (Channel channel : this.channels.values()) {
+			if (channel.isInputChannel()) {
+				((InputChannel<?>) channel).logQueuedEnvelopes();
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/ConnectionInfoLookupResponse.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/ConnectionInfoLookupResponse.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/ConnectionInfoLookupResponse.java
new file mode 100644
index 0000000..aeb5025
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/ConnectionInfoLookupResponse.java
@@ -0,0 +1,143 @@
+/***********************************************************************************************************************
+ *
+ * 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.network;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+
+public class ConnectionInfoLookupResponse implements IOReadableWritable {
+
+	private enum ReturnCode {
+		NOT_FOUND, FOUND_AND_RECEIVER_READY, FOUND_BUT_RECEIVER_NOT_READY, JOB_IS_ABORTING
+	};
+
+	// was request successful?
+	private ReturnCode returnCode;
+	
+	private RemoteReceiver remoteTarget;
+
+	private ChannelID localTarget;
+	
+	
+	public ConnectionInfoLookupResponse() {}
+
+	public ConnectionInfoLookupResponse(ReturnCode code) {
+		this.returnCode = code;
+		this.remoteTarget = null;
+		this.localTarget = null;
+	}
+	
+	public ConnectionInfoLookupResponse(ReturnCode code, ChannelID localTarget) {
+		this.returnCode = code;
+		this.remoteTarget = null;
+		this.localTarget = localTarget;
+	}
+	
+	public ConnectionInfoLookupResponse(ReturnCode code, RemoteReceiver receiver) {
+		this.returnCode = code;
+		this.remoteTarget = receiver;
+		this.localTarget = null;
+	}
+
+	public RemoteReceiver getRemoteTarget() {
+		return this.remoteTarget;
+	}
+
+	public ChannelID getLocalTarget() {
+		return this.localTarget;
+	}
+
+	@Override
+	public void read(DataInput in) throws IOException {
+		this.returnCode = ReturnCode.values()[in.readInt()];
+		
+		if (in.readBoolean()) {
+			this.remoteTarget = new RemoteReceiver();
+			this.remoteTarget.read(in);
+		}
+		if (in.readBoolean()) {
+			this.localTarget = new ChannelID();
+			this.localTarget.read(in);
+		}
+	}
+
+	@Override
+	public void write(DataOutput out) throws IOException {
+		out.writeInt(this.returnCode.ordinal());
+		
+		if (this.remoteTarget != null) {
+			out.writeBoolean(true);
+			this.remoteTarget.write(out);
+		} else {
+			out.writeBoolean(false);
+		}
+
+		if (this.localTarget != null) {
+			out.writeBoolean(true);
+			this.localTarget.write(out);
+		} else {
+			out.writeBoolean(false);
+		}
+	}
+
+	public boolean receiverNotFound() {
+		return (this.returnCode == ReturnCode.NOT_FOUND);
+	}
+
+	public boolean receiverNotReady() {
+		return (this.returnCode == ReturnCode.FOUND_BUT_RECEIVER_NOT_READY);
+	}
+
+	public boolean receiverReady() {
+		return (this.returnCode == ReturnCode.FOUND_AND_RECEIVER_READY);
+	}
+
+	public boolean isJobAborting() {
+		return (this.returnCode == ReturnCode.JOB_IS_ABORTING);
+	}
+
+	
+	public static ConnectionInfoLookupResponse createReceiverFoundAndReady(ChannelID targetChannelID) {
+		return new ConnectionInfoLookupResponse(ReturnCode.FOUND_AND_RECEIVER_READY, targetChannelID);
+	}
+
+	public static ConnectionInfoLookupResponse createReceiverFoundAndReady(RemoteReceiver remoteReceiver) {
+		return new ConnectionInfoLookupResponse(ReturnCode.FOUND_AND_RECEIVER_READY, remoteReceiver);
+	}
+
+	public static ConnectionInfoLookupResponse createReceiverNotFound() {
+		return new ConnectionInfoLookupResponse(ReturnCode.NOT_FOUND);
+	}
+
+	public static ConnectionInfoLookupResponse createReceiverNotReady() {
+		return new ConnectionInfoLookupResponse(ReturnCode.FOUND_BUT_RECEIVER_NOT_READY);
+	}
+
+	public static ConnectionInfoLookupResponse createJobIsAborting() {
+		return new ConnectionInfoLookupResponse(ReturnCode.JOB_IS_ABORTING);
+	}
+
+	
+	@Override
+	public String toString() {
+		return this.returnCode.name() + ", local target: " + this.localTarget + ", remoteTarget: " + this.remoteTarget;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/InsufficientResourcesException.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/InsufficientResourcesException.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/InsufficientResourcesException.java
new file mode 100644
index 0000000..0d2fcd4
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/InsufficientResourcesException.java
@@ -0,0 +1,37 @@
+/***********************************************************************************************************************
+ * 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.network;
+
+/**
+ * This exception is thrown by the {@link ChannelManager} to indicate that a task cannot be accepted because
+ * there are not enough resources available to safely execute it.
+ * 
+ */
+public final class InsufficientResourcesException extends Exception {
+
+	/**
+	 * The generated serial version UID.
+	 */
+	private static final long serialVersionUID = -8977049569413215169L;
+
+	/**
+	 * Constructs a new insufficient resources exception.
+	 * 
+	 * @param msg
+	 *        the message describing the exception
+	 */
+	InsufficientResourcesException(final String msg) {
+		super(msg);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/LocalReceiverCancelledException.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/LocalReceiverCancelledException.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/LocalReceiverCancelledException.java
new file mode 100644
index 0000000..769ada5
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/LocalReceiverCancelledException.java
@@ -0,0 +1,37 @@
+/***********************************************************************************************************************
+ *
+ * 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.network;
+
+import eu.stratosphere.nephele.execution.CancelTaskException;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+
+
+/**
+ *
+ */
+public class LocalReceiverCancelledException extends CancelTaskException {
+	private static final long serialVersionUID = 1L;
+
+	private final ChannelID receiver;
+
+	public LocalReceiverCancelledException(ChannelID receiver) {
+		this.receiver = receiver;
+	}
+	
+	
+	public ChannelID getReceiver() {
+		return receiver;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/NetworkConnectionManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/NetworkConnectionManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/NetworkConnectionManager.java
new file mode 100644
index 0000000..44ec642
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/NetworkConnectionManager.java
@@ -0,0 +1,176 @@
+/***********************************************************************************************************************
+ * 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.network;
+
+import eu.stratosphere.configuration.Configuration;
+import eu.stratosphere.configuration.GlobalConfiguration;
+import eu.stratosphere.runtime.io.network.envelope.Envelope;
+import eu.stratosphere.runtime.io.network.tcp.IncomingConnectionThread;
+import eu.stratosphere.runtime.io.network.tcp.OutgoingConnection;
+import eu.stratosphere.runtime.io.network.tcp.OutgoingConnectionThread;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+
+/**
+ * The network connection manager manages incoming and outgoing network connection from and to other hosts.
+ * <p>
+ * This class is thread-safe.
+ * 
+ */
+public final class NetworkConnectionManager {
+
+	/**
+	 * The default number of threads dealing with outgoing connections.
+	 */
+	private static final int DEFAULT_NUMBER_OF_OUTGOING_CONNECTION_THREADS = 1;
+
+	/**
+	 * The default number of connection retries before giving up.
+	 */
+	private static final int DEFAULT_NUMBER_OF_CONNECTION_RETRIES = 10;
+
+	/**
+	 * List of active threads dealing with outgoing connections.
+	 */
+	private final List<OutgoingConnectionThread> outgoingConnectionThreads = new CopyOnWriteArrayList<OutgoingConnectionThread>();
+
+	/**
+	 * Thread dealing with incoming connections.
+	 */
+	private final IncomingConnectionThread incomingConnectionThread;
+
+	/**
+	 * Map containing currently active outgoing connections.
+	 */
+	private final ConcurrentMap<RemoteReceiver, OutgoingConnection> outgoingConnections = new ConcurrentHashMap<RemoteReceiver, OutgoingConnection>();
+
+	/**
+	 * The number of connection retries before giving up.
+	 */
+	private final int numberOfConnectionRetries;
+
+	/**
+	 * A buffer provider for read buffers
+	 */
+	private final ChannelManager channelManager;
+
+	public NetworkConnectionManager(final ChannelManager channelManager,
+			final InetAddress bindAddress, final int dataPort) throws IOException {
+
+		final Configuration configuration = GlobalConfiguration.getConfiguration();
+
+		this.channelManager = channelManager;
+
+		// Start the connection threads
+		final int numberOfOutgoingConnectionThreads = configuration.getInteger(
+			"channel.network.numberOfOutgoingConnectionThreads", DEFAULT_NUMBER_OF_OUTGOING_CONNECTION_THREADS);
+
+		for (int i = 0; i < numberOfOutgoingConnectionThreads; i++) {
+			final OutgoingConnectionThread outgoingConnectionThread = new OutgoingConnectionThread();
+			outgoingConnectionThread.start();
+			this.outgoingConnectionThreads.add(outgoingConnectionThread);
+		}
+
+		this.incomingConnectionThread = new IncomingConnectionThread(
+			this.channelManager, true, new InetSocketAddress(bindAddress, dataPort));
+		this.incomingConnectionThread.start();
+
+		this.numberOfConnectionRetries = configuration.getInteger("channel.network.numberOfConnectionRetries",
+			DEFAULT_NUMBER_OF_CONNECTION_RETRIES);
+	}
+
+	/**
+	 * Randomly selects one of the active threads dealing with outgoing connections.
+	 * 
+	 * @return one of the active threads dealing with outgoing connections
+	 */
+	private OutgoingConnectionThread getOutgoingConnectionThread() {
+
+		return this.outgoingConnectionThreads.get((int) (this.outgoingConnectionThreads.size() * Math.random()));
+	}
+
+	/**
+	 * Queues an envelope for transfer to a particular target host.
+	 * 
+	 * @param remoteReceiver
+	 *        the address of the remote receiver
+	 * @param envelope
+	 *        the envelope to be transfered
+	 */
+	public void queueEnvelopeForTransfer(final RemoteReceiver remoteReceiver, final Envelope envelope) {
+
+		getOutgoingConnection(remoteReceiver).queueEnvelope(envelope);
+	}
+
+	/**
+	 * Returns (and possibly creates) the outgoing connection for the given target address.
+	 * 
+	 * @param targetAddress
+	 *        the address of the connection target
+	 * @return the outgoing connection object
+	 */
+	private OutgoingConnection getOutgoingConnection(final RemoteReceiver remoteReceiver) {
+
+		OutgoingConnection outgoingConnection = this.outgoingConnections.get(remoteReceiver);
+
+		if (outgoingConnection == null) {
+
+			outgoingConnection = new OutgoingConnection(remoteReceiver, getOutgoingConnectionThread(),
+				this.numberOfConnectionRetries);
+
+			final OutgoingConnection oldEntry = this.outgoingConnections
+				.putIfAbsent(remoteReceiver, outgoingConnection);
+
+			// We had a race, use the old value
+			if (oldEntry != null) {
+				outgoingConnection = oldEntry;
+			}
+		}
+
+		return outgoingConnection;
+	}
+
+	public void shutDown() {
+
+		// Interrupt the threads we started
+		this.incomingConnectionThread.interrupt();
+
+		final Iterator<OutgoingConnectionThread> it = this.outgoingConnectionThreads.iterator();
+		while (it.hasNext()) {
+			it.next().interrupt();
+		}
+	}
+
+	public void logBufferUtilization() {
+
+		System.out.println("\tOutgoing connections:");
+
+		final Iterator<Map.Entry<RemoteReceiver, OutgoingConnection>> it = this.outgoingConnections.entrySet()
+			.iterator();
+
+		while (it.hasNext()) {
+
+			final Map.Entry<RemoteReceiver, OutgoingConnection> entry = it.next();
+			System.out.println("\t\tOC " + entry.getKey() + ": " + entry.getValue().getNumberOfQueuedWriteBuffers());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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
new file mode 100644
index 0000000..da36ad0
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/RemoteReceiver.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.io.network;
+
+import java.io.DataInput;
+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}.
+ * 
+ */
+public final class RemoteReceiver implements IOReadableWritable {
+
+	/**
+	 * The address of the connection to the remote {@link TaskManager}.
+	 */
+	private InetSocketAddress connectionAddress;
+
+	/**
+	 * The index of the connection to the remote {@link TaskManager}.
+	 */
+	private int connectionIndex;
+
+	/**
+	 * Constructs a new remote receiver object.
+	 * 
+	 * @param connectionAddress
+	 *        the address of the connection to the remote {@link TaskManager}
+	 * @param connectionIndex
+	 *        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");
+		}
+
+		this.connectionAddress = connectionAddress;
+		this.connectionIndex = connectionIndex;
+	}
+
+	/**
+	 * Default constructor for serialization/deserialization.
+	 */
+	public RemoteReceiver() {
+		this.connectionAddress = null;
+		this.connectionIndex = -1;
+	}
+
+	/**
+	 * Returns the address of the connection to the remote {@link TaskManager}.
+	 * 
+	 * @return the address of the connection to the remote {@link TaskManager}
+	 */
+	public InetSocketAddress getConnectionAddress() {
+
+		return this.connectionAddress;
+	}
+
+	/**
+	 * Returns the index of the connection to the remote {@link TaskManager}.
+	 * 
+	 * @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);
+	}
+
+
+	@Override
+	public boolean equals(final Object obj) {
+
+		if (!(obj instanceof RemoteReceiver)) {
+			return false;
+		}
+
+		final RemoteReceiver rr = (RemoteReceiver) obj;
+		if (!this.connectionAddress.equals(rr.connectionAddress)) {
+			return false;
+		}
+
+		if (this.connectionIndex != rr.connectionIndex) {
+			return false;
+		}
+
+		return true;
+	}
+
+
+	@Override
+	public void write(final DataOutput out) throws IOException {
+
+		final InetAddress ia = this.connectionAddress.getAddress();
+		out.writeInt(ia.getAddress().length);
+		out.write(ia.getAddress());
+		out.writeInt(this.connectionAddress.getPort());
+
+		out.writeInt(this.connectionIndex);
+	}
+
+
+	@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();
+		this.connectionAddress = new InetSocketAddress(ia, port);
+
+		this.connectionIndex = in.readInt();
+	}
+
+
+	@Override
+	public String toString() {
+
+		return this.connectionAddress + " (" + this.connectionIndex + ")";
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/SenderHintEvent.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/SenderHintEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/SenderHintEvent.java
new file mode 100644
index 0000000..32be058
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/SenderHintEvent.java
@@ -0,0 +1,117 @@
+/***********************************************************************************************************************
+ * 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.network;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+
+import eu.stratosphere.nephele.event.task.AbstractEvent;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.network.envelope.Envelope;
+
+public final class SenderHintEvent extends AbstractEvent {
+
+	/**
+	 * The sequence number that will be set for transfer envelopes which contain the sender hint event.
+	 */
+	private static final int SENDER_HINT_SEQUENCE_NUMBER = 0;
+
+	private final ChannelID source;
+
+	private final RemoteReceiver remoteReceiver;
+
+	SenderHintEvent(final ChannelID source, final RemoteReceiver remoteReceiver) {
+
+		if (source == null) {
+			throw new IllegalArgumentException("Argument source must not be null");
+		}
+
+		if (remoteReceiver == null) {
+			throw new IllegalArgumentException("Argument remoteReceiver must not be null");
+		}
+
+		this.source = source;
+		this.remoteReceiver = remoteReceiver;
+	}
+
+	public SenderHintEvent() {
+
+		this.source = new ChannelID();
+		this.remoteReceiver = new RemoteReceiver();
+	}
+
+	public ChannelID getSource() {
+
+		return this.source;
+	}
+
+	public RemoteReceiver getRemoteReceiver() {
+
+		return this.remoteReceiver;
+	}
+
+
+	@Override
+	public void write(final DataOutput out) throws IOException {
+
+		this.source.write(out);
+		this.remoteReceiver.write(out);
+	}
+
+
+	@Override
+	public void read(final DataInput in) throws IOException {
+
+		this.source.read(in);
+		this.remoteReceiver.read(in);
+	}
+
+	public static Envelope createEnvelopeWithEvent(final Envelope originalEnvelope,
+			final ChannelID source, final RemoteReceiver remoteReceiver) {
+
+		final Envelope envelope = new Envelope(SENDER_HINT_SEQUENCE_NUMBER,
+			originalEnvelope.getJobID(), originalEnvelope.getSource());
+
+		final SenderHintEvent senderEvent = new SenderHintEvent(source, remoteReceiver);
+		envelope.serializeEventList(Arrays.asList(senderEvent));
+
+		return envelope;
+	}
+
+	static boolean isSenderHintEvent(final Envelope envelope) {
+
+		if (envelope.getSequenceNumber() != SENDER_HINT_SEQUENCE_NUMBER) {
+			return false;
+		}
+
+		if (envelope.getBuffer() != null) {
+			return false;
+		}
+
+		List<? extends AbstractEvent> events = envelope.deserializeEvents();
+
+		if (events.size() != 1) {
+			return false;
+		}
+
+		if (!(events.get(0) instanceof SenderHintEvent)) {
+			return false;
+		}
+
+		return true;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferAvailabilityListener.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferAvailabilityListener.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferAvailabilityListener.java
new file mode 100644
index 0000000..1d23e93
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferAvailabilityListener.java
@@ -0,0 +1,28 @@
+/***********************************************************************************************************************
+ * 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.network.bufferprovider;
+
+/**
+ * This interface must be implemented to receive a notification from a {@link BufferProvider} when an empty
+ * {@link eu.stratosphere.runtime.io.Buffer} has
+ * become available again.
+ * 
+ */
+public interface BufferAvailabilityListener {
+
+	/**
+	 * Indicates that at least one {@link eu.stratosphere.runtime.io.Buffer} has become available again.
+	 */
+	void bufferAvailable();
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProvider.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProvider.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProvider.java
new file mode 100644
index 0000000..e3085ee
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProvider.java
@@ -0,0 +1,69 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.network.bufferprovider;
+
+import eu.stratosphere.runtime.io.Buffer;
+
+import java.io.IOException;
+
+public interface BufferProvider {
+
+	/**
+	 * Requests a buffer with a minimum size of <code>minBufferSize</code>. The method returns immediately, even if the
+	 * request could not be fulfilled.
+	 *
+	 * @param minBufferSize minimum size of the requested buffer (in bytes)
+	 * @return buffer with at least the requested size or <code>null</code> if no such buffer is currently available
+	 * @throws IOException
+	 */
+	Buffer requestBuffer(int minBufferSize) throws IOException;
+
+	/**
+	 * Requests a buffer with a minimum size of <code>minBufferSize</code>. The method blocks until the request has
+	 * been fulfilled or {@link #reportAsynchronousEvent()} has been called.
+	 *
+	 * @param minBufferSize minimum size of the requested buffer (in bytes)
+	 * @return buffer with at least the requested size
+	 * @throws IOException
+	 * @throws InterruptedException
+	 */
+	Buffer requestBufferBlocking(int minBufferSize) throws IOException, InterruptedException;
+
+	/**
+	 * Returns the size of buffers (in bytes) available at this buffer provider.
+	 * 
+	 * @return size of buffers (in bytes) available at this buffer provider
+	 */
+	int getBufferSize();
+
+	/**
+	 * Reports an asynchronous event and interrupts each blocking method of this buffer provider in order to allow the
+	 * blocked thread to respond to the event.
+	 */
+	void reportAsynchronousEvent();
+
+	/**
+	 * Registers the given {@link BufferAvailabilityListener} with an empty buffer pool.
+	 * <p>
+	 * The registration only succeeds, if the buffer pool is empty and has not been destroyed yet.
+	 * <p>
+	 * The registered listener will receive a notification when at least one buffer has become available again. After
+	 * the notification, the listener will be unregistered.
+	 *
+	 * @param listener the listener to be registered
+	 * @return <code>true</code> if the registration has been successful; <code>false</code> if the registration
+	 *         failed, because the buffer pool was not empty or has already been destroyed
+	 */
+	boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener);
+}


[22/30] Offer buffer-oriented API for I/O (#25)

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java
deleted file mode 100644
index 71cad15..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java
+++ /dev/null
@@ -1,330 +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.io;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.atomic.AtomicReference;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
-import eu.stratosphere.nephele.io.channels.AbstractInputChannel;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.bytebuffered.InMemoryInputChannel;
-import eu.stratosphere.nephele.io.channels.bytebuffered.NetworkInputChannel;
-import eu.stratosphere.nephele.jobgraph.JobID;
-
-/**
- * In Nephele input gates are a specialization of general gates and connect input channels and record readers. As
- * channels, input gates are always parameterized to a specific type of record which they can transport. In contrast to
- * output gates input gates can be associated with a {@link DistributionPattern} object which dictates the concrete
- * wiring between two groups of vertices.
- * 
- * @param <T> The type of record that can be transported through this gate.
- */
-public class RuntimeInputGate<T extends IOReadableWritable> extends AbstractGate<T> implements InputGate<T> {
-	
-	/**
-	 * The log object used for debugging.
-	 */
-	private static final Log LOG = LogFactory.getLog(InputGate.class);
-
-	/**
-	 * The deserializer factory used to instantiate the deserializers that construct records from byte streams.
-	 */
-	private final RecordDeserializerFactory<T> deserializerFactory;
-
-	/**
-	 * The list of input channels attached to this input gate.
-	 */
-	private final ArrayList<AbstractInputChannel<T>> inputChannels = new ArrayList<AbstractInputChannel<T>>();
-
-	/**
-	 * Queue with indices of channels that store at least one available record.
-	 */
-	private final BlockingQueue<Integer> availableChannels = new LinkedBlockingQueue<Integer>();
-
-	/**
-	 * The listener object to be notified when a channel has at least one record available.
-	 */
-	private final AtomicReference<RecordAvailabilityListener<T>> recordAvailabilityListener = new AtomicReference<RecordAvailabilityListener<T>>(null);
-	
-	
-	private AbstractTaskEvent currentEvent;
-
-	/**
-	 * If the value of this variable is set to <code>true</code>, the input gate is closed.
-	 */
-	private boolean isClosed = false;
-
-	/**
-	 * The channel to read from next.
-	 */
-	private int channelToReadFrom = -1;
-
-	/**
-	 * Constructs a new runtime input gate.
-	 * 
-	 * @param jobID
-	 *        the ID of the job this input gate belongs to
-	 * @param gateID
-	 *        the ID of the gate
-	 * @param deserializerFactory
-	 *        The factory used to instantiate the deserializers that construct records from byte streams.
-	 * @param index
-	 *        the index assigned to this input gate at the {@link Environment} object
-	 */
-	public RuntimeInputGate(final JobID jobID, final GateID gateID,
-						final RecordDeserializerFactory<T> deserializerFactory, final int index) {
-		super(jobID, gateID, index);
-		this.deserializerFactory = deserializerFactory;
-	}
-
-	/**
-	 * Adds a new input channel to the input gate.
-	 * 
-	 * @param inputChannel
-	 *        the input channel to be added.
-	 */
-	private void addInputChannel(AbstractInputChannel<T> inputChannel) {
-		// in high DOPs, this can be a serious performance issue, as adding all channels and checking linearly has a
-		// quadratic complexity!
-		if (!this.inputChannels.contains(inputChannel)) {
-			this.inputChannels.add(inputChannel);
-		}
-	}
-
-	/**
-	 * Removes the input channel with the given ID from the input gate if it exists.
-	 * 
-	 * @param inputChannelID
-	 *        the ID of the channel to be removed
-	 */
-	public void removeInputChannel(ChannelID inputChannelID) {
-
-		for (int i = 0; i < this.inputChannels.size(); i++) {
-
-			final AbstractInputChannel<T> inputChannel = this.inputChannels.get(i);
-			if (inputChannel.getID().equals(inputChannelID)) {
-				this.inputChannels.remove(i);
-				return;
-			}
-		}
-		
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("Cannot find output channel with ID " + inputChannelID + " to remove");
-		}
-	}
-
-	@Override
-	public boolean isInputGate() {
-		return true;
-	}
-
-	@Override
-	public int getNumberOfInputChannels() {
-		return this.inputChannels.size();
-	}
-
-	@Override
-	public AbstractInputChannel<T> getInputChannel(int pos) {
-		return this.inputChannels.get(pos);
-	}
-
-
-	@Override
-	public NetworkInputChannel<T> createNetworkInputChannel(final InputGate<T> inputGate, final ChannelID channelID,
-			final ChannelID connectedChannelID) {
-
-		final NetworkInputChannel<T> enic = new NetworkInputChannel<T>(inputGate, this.inputChannels.size(),
-			this.deserializerFactory.createDeserializer(), channelID, connectedChannelID);
-		addInputChannel(enic);
-
-		return enic;
-	}
-
-
-	@Override
-	public InMemoryInputChannel<T> createInMemoryInputChannel(final InputGate<T> inputGate, final ChannelID channelID,
-			final ChannelID connectedChannelID) {
-
-		final InMemoryInputChannel<T> eimic = new InMemoryInputChannel<T>(inputGate, this.inputChannels.size(),
-			this.deserializerFactory.createDeserializer(), channelID, connectedChannelID);
-		addInputChannel(eimic);
-
-		return eimic;
-	}
-
-
-	@Override
-	public InputChannelResult readRecord(T target) throws IOException, InterruptedException {
-
-		if (this.channelToReadFrom == -1) {
-			if (this.isClosed()) {
-				return InputChannelResult.END_OF_STREAM;
-			}
-				
-			if (Thread.interrupted()) {
-				throw new InterruptedException();
-			}
-				
-			this.channelToReadFrom = waitForAnyChannelToBecomeAvailable();
-		}
-			
-		InputChannelResult result = this.getInputChannel(this.channelToReadFrom).readRecord(target);
-		switch (result) {
-			case INTERMEDIATE_RECORD_FROM_BUFFER: // full record and we can stay on the same channel
-				return InputChannelResult.INTERMEDIATE_RECORD_FROM_BUFFER;
-				
-			case LAST_RECORD_FROM_BUFFER: // full record, but we must switch the channel afterwards
-				this.channelToReadFrom = -1;
-				return InputChannelResult.LAST_RECORD_FROM_BUFFER;
-				
-			case END_OF_SUPERSTEP:
-				this.channelToReadFrom = -1;
-				return InputChannelResult.END_OF_SUPERSTEP;
-				
-			case TASK_EVENT: // task event
-				this.currentEvent = this.getInputChannel(this.channelToReadFrom).getCurrentEvent();
-				this.channelToReadFrom = -1;	// event always marks a unit as consumed
-				return InputChannelResult.TASK_EVENT;
-					
-			case NONE: // internal event or an incomplete record that needs further chunks
-				// the current unit is exhausted
-				this.channelToReadFrom = -1;
-				return InputChannelResult.NONE;
-				
-			case END_OF_STREAM: // channel is done
-				this.channelToReadFrom = -1;
-				return isClosed() ? InputChannelResult.END_OF_STREAM : InputChannelResult.NONE;
-				
-			default:   // silence the compiler
-				throw new RuntimeException();
-		}
-	}
-	
-	@Override
-	public AbstractTaskEvent getCurrentEvent() {
-		AbstractTaskEvent e = this.currentEvent;
-		this.currentEvent = null;
-		return e;
-	}
-
-	@Override
-	public void notifyRecordIsAvailable(int channelIndex) {
-		this.availableChannels.add(Integer.valueOf(channelIndex));
-
-		RecordAvailabilityListener<T> listener = this.recordAvailabilityListener.get();
-		if (listener != null) {
-			listener.reportRecordAvailability(this);
-		}
-	}
-
-	/**
-	 * This method returns the index of a channel which has at least
-	 * one record available. The method may block until at least one
-	 * channel has become ready.
-	 * 
-	 * @return the index of the channel which has at least one record available
-	 */
-	public int waitForAnyChannelToBecomeAvailable() throws InterruptedException {
-		return this.availableChannels.take().intValue();
-	}
-
-
-	@Override
-	public boolean isClosed() throws IOException, InterruptedException {
-
-		if (this.isClosed) {
-			return true;
-		}
-
-		for (int i = 0; i < this.getNumberOfInputChannels(); i++) {
-			final AbstractInputChannel<T> inputChannel = this.inputChannels.get(i);
-			if (!inputChannel.isClosed()) {
-				return false;
-			}
-		}
-
-		this.isClosed = true;
-		
-		return true;
-	}
-
-
-	@Override
-	public void close() throws IOException, InterruptedException {
-
-		for (int i = 0; i < this.getNumberOfInputChannels(); i++) {
-			final AbstractInputChannel<T> inputChannel = this.inputChannels.get(i);
-			inputChannel.close();
-		}
-
-	}
-
-
-	@Override
-	public String toString() {
-		return "Input " + super.toString();
-	}
-
-
-	@Override
-	public void publishEvent(AbstractEvent event) throws IOException, InterruptedException {
-
-		// Copy event to all connected channels
-		final Iterator<AbstractInputChannel<T>> it = this.inputChannels.iterator();
-		while (it.hasNext()) {
-			it.next().transferEvent(event);
-		}
-	}
-
-	/**
-	 * Returns the {@link RecordDeserializerFactory} used by this input gate.
-	 * 
-	 * @return The {@link RecordDeserializerFactory} used by this input gate.
-	 */
-	public RecordDeserializerFactory<T> getRecordDeserializerFactory() {
-		return this.deserializerFactory;
-	}
-
-
-	@Override
-	public void releaseAllChannelResources() {
-
-		final Iterator<AbstractInputChannel<T>> it = this.inputChannels.iterator();
-		while (it.hasNext()) {
-			it.next().releaseAllResources();
-		}
-	}
-
-	@Override
-	public void registerRecordAvailabilityListener(final RecordAvailabilityListener<T> listener) {
-		if (!this.recordAvailabilityListener.compareAndSet(null, listener)) {
-			throw new IllegalStateException(this.recordAvailabilityListener
-				+ " is already registered as a record availability listener");
-		}
-	}
-
-	public void notifyDataUnitConsumed(int channelIndex) {
-		this.channelToReadFrom = -1;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java
deleted file mode 100644
index 20efd94..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java
+++ /dev/null
@@ -1,333 +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.io;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.io.channels.AbstractOutputChannel;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-import eu.stratosphere.nephele.io.channels.bytebuffered.InMemoryOutputChannel;
-import eu.stratosphere.nephele.io.channels.bytebuffered.NetworkOutputChannel;
-import eu.stratosphere.nephele.jobgraph.JobID;
-
-/**
- * In Nephele output gates are a specialization of general gates and connect
- * record writers and output channels. As channels, output gates are always
- * parameterized to a specific type of record which they can transport.
- * <p>
- * This class is in general not thread-safe.
- * 
- * @param <T>
- *        the type of record that can be transported through this gate
- */
-public class RuntimeOutputGate<T extends IOReadableWritable> extends AbstractGate<T> implements OutputGate<T> {
-
-	/**
-	 * The log object used for debugging.
-	 */
-	private static final Log LOG = LogFactory.getLog(OutputGate.class);
-
-	/**
-	 * The list of output channels attached to this gate.
-	 */
-	private final ArrayList<AbstractOutputChannel<T>> outputChannels = new ArrayList<AbstractOutputChannel<T>>();
-
-	/**
-	 * Channel selector to determine which channel is supposed receive the next record.
-	 */
-	private final ChannelSelector<T> channelSelector;
-
-	/**
-	 * The class of the record transported through this output gate.
-	 */
-	private final Class<T> type;
-
-	/**
-	 * Stores whether all records passed to this output gate shall be transmitted through all connected output channels.
-	 */
-	private final boolean isBroadcast;
-
-	/**
-	 * Constructs a new runtime output gate.
-	 * 
-	 * @param jobID
-	 *        the ID of the job this input gate belongs to
-	 * @param gateID
-	 *        the ID of the gate
-	 * @param inputClass
-	 *        the class of the record that can be transported through this
-	 *        gate
-	 * @param index
-	 *        the index assigned to this output gate at the {@link Environment} object
-	 * @param channelSelector
-	 *        the channel selector to be used for this output gate
-	 * @param isBroadcast
-	 *        <code>true</code> if every records passed to this output gate shall be transmitted through all connected
-	 *        output channels, <code>false</code> otherwise
-	 */
-	public RuntimeOutputGate(final JobID jobID, final GateID gateID, final Class<T> inputClass, final int index,
-			final ChannelSelector<T> channelSelector, final boolean isBroadcast) {
-
-		super(jobID, gateID, index);
-
-		this.isBroadcast = isBroadcast;
-		this.type = inputClass;
-
-		if (this.isBroadcast) {
-			this.channelSelector = null;
-		} else {
-			if (channelSelector == null) {
-				this.channelSelector = new DefaultChannelSelector<T>();
-			} else {
-				this.channelSelector = channelSelector;
-			}
-		}
-	}
-
-
-	@Override
-	public final Class<T> getType() {
-		return this.type;
-	}
-
-	/**
-	 * Adds a new output channel to the output gate.
-	 * 
-	 * @param outputChannel
-	 *        the output channel to be added.
-	 */
-	private void addOutputChannel(AbstractOutputChannel<T> outputChannel) {
-		if (!this.outputChannels.contains(outputChannel)) {
-			this.outputChannels.add(outputChannel);
-		}
-	}
-
-	/**
-	 * Removes the output channel with the given ID from the output gate if it
-	 * exists.
-	 * 
-	 * @param outputChannelID
-	 *        the ID of the channel to be removed
-	 */
-	public void removeOutputChannel(ChannelID outputChannelID) {
-
-		for (int i = 0; i < this.outputChannels.size(); i++) {
-
-			final AbstractOutputChannel<T> outputChannel = this.outputChannels.get(i);
-			if (outputChannel.getID().equals(outputChannelID)) {
-				this.outputChannels.remove(i);
-				return;
-			}
-		}
-
-		LOG.debug("Cannot find output channel with ID " + outputChannelID + " to remove");
-	}
-
-	/**
-	 * Removes all output channels from the output gate.
-	 */
-	public void removeAllOutputChannels() {
-
-		this.outputChannels.clear();
-	}
-
-
-	@Override
-	public boolean isInputGate() {
-
-		return false;
-	}
-
-
-	@Override
-	public int getNumberOfOutputChannels() {
-
-		return this.outputChannels.size();
-	}
-
-	/**
-	 * Returns the output channel from position <code>pos</code> of the gate's
-	 * internal channel list.
-	 * 
-	 * @param pos
-	 *        the position to retrieve the channel from
-	 * @return the channel from the given position or <code>null</code> if such
-	 *         position does not exist.
-	 */
-	public AbstractOutputChannel<T> getOutputChannel(int pos) {
-
-		if (pos < this.outputChannels.size()) {
-			return this.outputChannels.get(pos);
-		} else {
-			return null;
-		}
-	}
-
-
-	@Override
-	public NetworkOutputChannel<T> createNetworkOutputChannel(final OutputGate<T> outputGate,
-			final ChannelID channelID, final ChannelID connectedChannelID) {
-
-		final NetworkOutputChannel<T> enoc = new NetworkOutputChannel<T>(outputGate, this.outputChannels.size(),
-			channelID, connectedChannelID);
-		addOutputChannel(enoc);
-
-		return enoc;
-	}
-
-
-	@Override
-	public InMemoryOutputChannel<T> createInMemoryOutputChannel(final OutputGate<T> outputGate,
-			final ChannelID channelID, final ChannelID connectedChannelID) {
-
-		final InMemoryOutputChannel<T> einoc = new InMemoryOutputChannel<T>(outputGate, this.outputChannels.size(),
-			channelID, connectedChannelID);
-		addOutputChannel(einoc);
-
-		return einoc;
-	}
-
-
-	@Override
-	public void requestClose() throws IOException, InterruptedException {
-		// Close all output channels
-		for (int i = 0; i < this.getNumberOfOutputChannels(); i++) {
-			final AbstractOutputChannel<T> outputChannel = this.getOutputChannel(i);
-			outputChannel.requestClose();
-		}
-	}
-
-
-	@Override
-	public boolean isClosed() throws IOException, InterruptedException {
-
-		boolean allClosed = true;
-
-		for (int i = 0; i < this.getNumberOfOutputChannels(); i++) {
-			final AbstractOutputChannel<T> outputChannel = this.getOutputChannel(i);
-			if (!outputChannel.isClosed()) {
-				allClosed = false;
-			}
-		}
-
-		return allClosed;
-	}
-
-
-	@Override
-	public void writeRecord(final T record) throws IOException, InterruptedException {
-
-		if (this.isBroadcast) {
-
-			if (getChannelType() == ChannelType.INMEMORY) {
-
-				final int numberOfOutputChannels = this.outputChannels.size();
-				for (int i = 0; i < numberOfOutputChannels; ++i) {
-					this.outputChannels.get(i).writeRecord(record);
-				}
-
-			} else {
-
-				// Use optimization for byte buffered channels
-				this.outputChannels.get(0).writeRecord(record);
-			}
-
-		} else {
-
-			// Non-broadcast gate, use channel selector to select output channels
-			final int numberOfOutputChannels = this.outputChannels.size();
-			final int[] selectedOutputChannels = this.channelSelector.selectChannels(record, numberOfOutputChannels);
-			
-			if (selectedOutputChannels == null) {
-				return;
-			}
-
-			
-			for (int i = 0; i < selectedOutputChannels.length; ++i) {
-				if (selectedOutputChannels[i] < numberOfOutputChannels) {
-					final AbstractOutputChannel<T> outputChannel = this.outputChannels.get(selectedOutputChannels[i]);
-					outputChannel.writeRecord(record);
-				}
-			}
-		}
-	}
-
-
-	@Override
-	public List<AbstractOutputChannel<T>> getOutputChannels() {
-		return this.outputChannels;
-	}
-
-
-	@Override
-	public String toString() {
-		return "Output " + super.toString();
-	}
-
-
-	@Override
-	public void publishEvent(AbstractEvent event) throws IOException, InterruptedException {
-
-		// Copy event to all connected channels
-		final Iterator<AbstractOutputChannel<T>> it = this.outputChannels.iterator();
-		while (it.hasNext()) {
-			it.next().transferEvent(event);
-		}
-	}
-
-
-	@Override
-	public void flush() throws IOException, InterruptedException {
-		// Flush all connected channels
-		final Iterator<AbstractOutputChannel<T>> it = this.outputChannels.iterator();
-		while (it.hasNext()) {
-			it.next().flush();
-		}
-	}
-
-
-	@Override
-	public boolean isBroadcast() {
-
-		return this.isBroadcast;
-	}
-
-
-	@Override
-	public ChannelSelector<T> getChannelSelector() {
-
-		return this.channelSelector;
-	}
-
-
-	@Override
-	public void releaseAllChannelResources() {
-
-		final Iterator<AbstractOutputChannel<T>> it = this.outputChannels.iterator();
-
-		while (it.hasNext()) {
-			it.next().releaseAllResources();
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/UnionRecordReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/UnionRecordReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/UnionRecordReader.java
deleted file mode 100644
index ec8ef0c..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/UnionRecordReader.java
+++ /dev/null
@@ -1,67 +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.io;
-
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-
-public final class UnionRecordReader<T extends IOReadableWritable> extends AbstractUnionRecordReader<T> implements Reader<T> {
-	
-	private final Class<T> recordType;
-	
-	private T lookahead;
-	
-
-	public UnionRecordReader(MutableRecordReader<T>[] recordReaders, Class<T> recordType) {
-		super(recordReaders);
-		this.recordType = recordType;
-	}
-
-	@Override
-	public boolean hasNext() throws IOException, InterruptedException {
-		if (this.lookahead != null) {
-			return true;
-		} else {
-			T record = instantiateRecordType();
-			if (getNextRecord(record)) {
-				this.lookahead = record;
-				return true;
-			} else {
-				return false;
-			}
-		}
-	}
-
-	@Override
-	public T next() throws IOException, InterruptedException {
-		if (hasNext()) {
-			T tmp = this.lookahead;
-			this.lookahead = null;
-			return tmp;
-		} else {
-			return null;
-		}
-	}
-	
-	private T instantiateRecordType() {
-		try {
-			return this.recordType.newInstance();
-		} catch (InstantiationException e) {
-			throw new RuntimeException("Cannot instantiate class '" + this.recordType.getName() + "'.", e);
-		} catch (IllegalAccessException e) {
-			throw new RuntimeException("Cannot instantiate class '" + this.recordType.getName() + "'.", e);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/Writer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/Writer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/Writer.java
deleted file mode 100644
index 91fa3b7..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/Writer.java
+++ /dev/null
@@ -1,28 +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.io;
-
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-
-/**
- * A writer that sends records.
- * 
- * @param <T> The type of the record that can be emitted with this record writer.
- */
-public interface Writer<T extends IOReadableWritable> {
-	
-	void emit(T record) throws IOException, InterruptedException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/AbstractChannel.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/AbstractChannel.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/AbstractChannel.java
deleted file mode 100644
index b48e5f0..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/AbstractChannel.java
+++ /dev/null
@@ -1,127 +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.io.channels;
-
-import java.io.IOException;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.jobgraph.JobID;
-
-/**
- * An abstract base class for channel objects.
- */
-public abstract class AbstractChannel {
-
-	/**
-	 * The ID of the channel.
-	 */
-	private final ChannelID channelID;
-
-	/**
-	 * The ID of the connected channel.
-	 */
-	private final ChannelID connectedChannelID;
-
-	private final int channelIndex;
-
-	/**
-	 * Auxiliary constructor for channels
-	 * 
-	 * @param channelIndex
-	 *        the index of the channel in either the output or input gate
-	 * @param channelID
-	 *        the ID of the channel
-	 * @param connectedChannelID
-	 *        the ID of the channel this channel is connected to
-	 * @param compressionLevel
-	 *        the level of compression to be used for this channel
-	 */
-	protected AbstractChannel(final int channelIndex, final ChannelID channelID, final ChannelID connectedChannelID) {
-		this.channelIndex = channelIndex;
-		this.channelID = channelID;
-		this.connectedChannelID = connectedChannelID;
-	}
-
-	/**
-	 * Returns the ID of the channel.
-	 * 
-	 * @return the ID of the channel.
-	 */
-	public ChannelID getID() {
-		return this.channelID;
-	}
-
-	/**
-	 * Returns the channel's input at the associated gate.
-	 * 
-	 * @return the channel's input at the associated gate
-	 */
-	public int getChannelIndex() {
-		return this.channelIndex;
-	}
-
-	/**
-	 * Returns the type of the channel.
-	 * 
-	 * @return the type of the channel.
-	 */
-	public abstract ChannelType getType();
-
-	/**
-	 * Checks if the channel is closed, i.e. no more records can be transported through the channel.
-	 * 
-	 * @return <code>true</code> if the channel is closed, <code>false</code> otherwise
-	 * @throws IOException
-	 *         thrown if an error occurred while closing the channel
-	 * @throws InterruptedException
-	 *         thrown if the channel is interrupted while waiting for this operation to complete
-	 */
-	public abstract boolean isClosed() throws IOException, InterruptedException;
-
-	
-	public ChannelID getConnectedChannelID() {
-		return this.connectedChannelID;
-	}
-
-
-	/**
-	 * Returns the ID of the job this channel belongs to.
-	 * 
-	 * @return the ID of the job this channel belongs to
-	 */
-	public abstract JobID getJobID();
-
-	/**
-	 * Returns <code>true</code> if this channel is an input channel, <code>false</code> otherwise.
-	 * 
-	 * @return <code>true</code> if this channel is an input channel, <code>false</code> otherwise
-	 */
-	public abstract boolean isInputChannel();
-
-	
-	public abstract void transferEvent(AbstractEvent event) throws IOException, InterruptedException;
-
-	/**
-	 * Releases all resources (especially buffers) which are currently allocated by this channel. This method should be
-	 * called in case of a task error or as a result of a cancel operation.
-	 */
-	public abstract void releaseAllResources();
-
-	/**
-	 * Returns the number of bytes which have been transmitted through this channel since its instantiation.
-	 * 
-	 * @return the number of bytes which have been transmitted through this channel since its instantiation
-	 */
-	public abstract long getAmountOfDataTransmitted();
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/AbstractInputChannel.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/AbstractInputChannel.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/AbstractInputChannel.java
deleted file mode 100644
index 4b88aff..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/AbstractInputChannel.java
+++ /dev/null
@@ -1,102 +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.io.channels;
-
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
-import eu.stratosphere.nephele.io.InputChannelResult;
-import eu.stratosphere.nephele.io.InputGate;
-import eu.stratosphere.nephele.jobgraph.JobID;
-
-/**
- * InputChannel is an abstract base class to all different kinds of concrete
- * input channels that can be used. Input channels are always parameterized to
- * a specific type that can be transported through the channel.
-
- * @param <T> The Type of the record that can be transported through the channel.
- */
-public abstract class AbstractInputChannel<T extends IOReadableWritable> extends AbstractChannel {
-
-	private final InputGate<T> inputGate;
-
-	/**
-	 * Constructs an input channel with a given input gate associated.
-	 * 
-	 * @param inputGate
-	 *        the input gate this channel is connected to
-	 * @param channelIndex
-	 *        the index of the channel in the input gate
-	 * @param channelID
-	 *        the ID of the channel
-	 * @param connectedChannelID
-	 *        the ID of the channel this channel is connected to
-	 * @param compressionLevel
-	 *        the level of compression to be used for this channel
-	 */
-	protected AbstractInputChannel(final InputGate<T> inputGate, final int channelIndex, final ChannelID channelID,
-			final ChannelID connectedChannelID) {
-		super(channelIndex, channelID, connectedChannelID);
-		this.inputGate = inputGate;
-	}
-
-	/**
-	 * Returns the input gate associated with the input channel.
-	 * 
-	 * @return the input gate associated with the input channel.
-	 */
-	public InputGate<T> getInputGate() {
-		return this.inputGate;
-	}
-
-	/**
-	 * Reads a record from the input channel. If currently no record is available the method
-	 * returns <code>null</code>. If the channel is closed (i.e. no more records will be received), the method
-	 * throws an {@link EOFException}.
-	 * 
-	 * @return a record that has been transported through the channel or <code>null</code> if currently no record is
-	 *         available
-	 * @throws IOException
-	 *         thrown if the input channel is already closed {@link EOFException} or a transmission error has occurred
-	 */
-	public abstract InputChannelResult readRecord(T target) throws IOException;
-
-	/**
-	 * Immediately closes the input channel. The corresponding output channels are
-	 * notified if necessary. Any remaining records in any buffers or queue is considered
-	 * irrelevant and is discarded.
-	 * 
-	 * @throws InterruptedException
-	 *         thrown if the thread is interrupted while waiting for the channel to close
-	 * @throws IOException
-	 *         thrown if an I/O error occurs while closing the channel
-	 */
-	public abstract void close() throws IOException, InterruptedException;
-
-
-
-	@Override
-	public boolean isInputChannel() {
-		return true;
-	}
-
-
-	@Override
-	public JobID getJobID() {
-		return this.inputGate.getJobID();
-	}
-	
-	public abstract AbstractTaskEvent getCurrentEvent();
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/AbstractOutputChannel.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/AbstractOutputChannel.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/AbstractOutputChannel.java
deleted file mode 100644
index 7974c24..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/AbstractOutputChannel.java
+++ /dev/null
@@ -1,111 +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.io.channels;
-
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.io.OutputGate;
-import eu.stratosphere.nephele.jobgraph.JobID;
-
-/**
- * OutputChannel is an abstract base class to all different kinds of concrete
- * output channels that can be used. Input channels are always parameterized to
- * a specific type that can be transported through the channel.
- * 
- * @param <T>
- *        The Type of the record that can be transported through the channel.
- */
-public abstract class AbstractOutputChannel<T extends IOReadableWritable> extends AbstractChannel {
-
-	private final OutputGate<T> outputGate;
-
-	/**
-	 * Creates a new output channel object.
-	 * 
-	 * @param outputGate
-	 *        the output gate this channel is connected to
-	 * @param channelIndex
-	 *        the index of the channel in the output gate
-	 * @param channelID
-	 *        the ID of the channel
-	 * @param connectedChannelID
-	 *        the ID of the channel this channel is connected to
-	 * @param compressionLevel
-	 *        the level of compression to be used for this channel
-	 */
-	public AbstractOutputChannel(final OutputGate<T> outputGate, final int channelIndex, final ChannelID channelID,
-			final ChannelID connectedChannelID) {
-		super(channelIndex, channelID, connectedChannelID);
-		this.outputGate = outputGate;
-	}
-
-	/**
-	 * Returns the output gate this channel is connected to.
-	 * 
-	 * @return the output gate this channel is connected to
-	 */
-	public OutputGate<T> getOutputGate() {
-		return this.outputGate;
-	}
-
-	/**
-	 * Writes a record to the channel. The operation may block until the record
-	 * is completely written to the channel.
-	 * 
-	 * @param record
-	 *        the record to be written to the channel
-	 * @throws IOException
-	 *         thrown if an error occurred while transmitting the record
-	 */
-	public abstract void writeRecord(T record) throws IOException, InterruptedException;
-
-	/**
-	 * Requests the output channel to close. After calling this method no more records can be written
-	 * to the channel. The channel is finally closed when all remaining data that may exist in internal buffers
-	 * are written to the channel.
-	 * 
-	 * @throws InterruptedException
-	 *         thrown if the thread is interrupted while requesting the close operation
-	 * @throws IOException
-	 *         thrown if an I/O error occurs while requesting the close operation
-	 */
-	public abstract void requestClose() throws IOException, InterruptedException;
-
-
-
-	@Override
-	public boolean isInputChannel() {
-		return false;
-	}
-
-	public abstract void flush() throws IOException, InterruptedException;
-
-
-	@Override
-	public JobID getJobID() {
-		return this.outputGate.getJobID();
-	}
-
-	/**
-	 * Returns <code>true</code> if this channel is connected to an output gate which operates in broadcast mode,
-	 * <code>false</code> otherwise.
-	 * 
-	 * @return <code>true</code> if the connected output gate operates in broadcase mode, <code>false</code> otherwise
-	 */
-	public boolean isBroadcastChannel() {
-
-		return this.outputGate.isBroadcast();
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/Buffer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/Buffer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/Buffer.java
deleted file mode 100644
index 80dfa8f..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/Buffer.java
+++ /dev/null
@@ -1,175 +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.io.channels;
-
-import java.io.IOException;
-import java.nio.channels.ReadableByteChannel;
-import java.nio.channels.WritableByteChannel;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-
-/**
- * This class represents the general buffer abstraction that is used by Nephele
- * to transport data through the network or the file system.
- * <p>
- * Buffers may be backed by actual main memory or files.
- * <p>
- * Each buffer is expected to be written and read exactly once. Initially, the every buffer is in write mode. Before
- * reading from the buffer, it must be explicitly switched to read mode.
- * <p>
- * This class is in general not thread-safe.
- * 
- */
-public abstract class Buffer implements ReadableByteChannel, WritableByteChannel
-{
-	/**
-	 * Stores whether this buffer has already been recycled.
-	 */
-	private final AtomicBoolean isRecycled = new AtomicBoolean(false);
-
-	/**
-	 * Constructs a new buffer object.
-	 * 
-	 * @param internalBuffer
-	 *        the concrete implementation which backs the buffer
-	 */
-	protected Buffer()
-	{}
-
-	/**
-	 * Reads data from the buffer and writes it to the
-	 * given {@link WritableByteChannel} object.
-	 * 
-	 * @param destination The {@link WritableByteChannel} object to write the data to
-	 * @return The number of bytes read from the buffer, potentially <code>0</code> or <code>-1</code to indicate the
-	 *         end of the stream.
-	 * @throws IOException Thrown if an error occurs while writing to the {@link WritableByteChannel} object.
-	 */
-
-	public abstract boolean isOpen();
-	
-	
-
-	@Override
-	public abstract void close() throws IOException;
-	
-	/**
-	 * Reads data from the given {@link ReadableByteChannel} object and
-	 * writes it to the buffer.
-	 * 
-	 * @param source The {@link ReadableByteChannel} object to read data from.
-	 * @return The number of bytes written to the buffer, possibly <code>0</code>.
-	 * @throws IOException Thrown if an error occurs while writing data to the buffer.
-	 */
-	public abstract int write(ReadableByteChannel source) throws IOException;
-	
-
-	/**
-	 * Returns the number of bytes which can be either still written to or read from
-	 * the buffer, depending whether the buffer is still in write mode or not.
-	 * <p>
-	 * If in write mode, the method returns the number of bytes which can be written to be buffer, before its capacity
-	 * limit is reached. In read mode, the method returns the number of bytes which can be read from the number until
-	 * all data previously written to the buffer is consumed.
-	 * 
-	 * @return the number of bytes which can be either written to or read from the buffer
-	 */
-	public abstract int remaining();
-
-	/**
-	 * Checks whether data can still be written to or read from the buffer.
-	 * 
-	 * @return <code>true</code> if data can be still written to or read from
-	 *         the buffer, <code>false</code> otherwise
-	 */
-	public boolean hasRemaining() {
-		return remaining() > 0;
-	}
-
-	/**
-	 * Returns the size of the buffer. In write mode, the size of the buffer is the initial capacity
-	 * of the buffer. In read mode, the size of the buffer is number of bytes which have been
-	 * previously written to the buffer.
-	 * 
-	 * @return the size of the buffer in bytes
-	 */
-	public abstract int size();
-
-	/**
-	 * Recycles the buffer. In case of a memory backed buffer, the internal memory buffer
-	 * is returned to a global buffer queue. In case of a file backed buffer, the temporary
-	 * file created for this buffer is deleted. A buffer can only be recycled once. Calling this method more than once
-	 * will therefore have no effect.
-	 */
-	public final void recycleBuffer()
-	{
-		if (this.isRecycled.compareAndSet(false, true)) {
-			recycle();
-		}
-	}
-	
-	protected abstract void recycle();
-
-
-	/**
-	 * Returns whether the buffer is backed by main memory or a file.
-	 * 
-	 * @return <code>true</code> if the buffer is backed by main memory
-	 *         or <code>false</code> if it is backed by a file
-	 */
-	public abstract boolean isBackedByMemory();
-
-	/**
-	 * Copies the content of the buffer to the given destination buffer. The state of the source buffer is not modified
-	 * by this operation.
-	 * 
-	 * @param destinationBuffer
-	 *        the destination buffer to copy this buffer's content to
-	 * @throws IOException
-	 *         thrown if an error occurs while copying the data
-	 */
-	public abstract void copyToBuffer(Buffer destinationBuffer) throws IOException;
-
-	/**
-	 * Duplicates the buffer. This operation does not duplicate the actual
-	 * content of the buffer, only the reading/writing state. As a result,
-	 * modifications to the original buffer will affect the duplicate and vice-versa.
-	 * 
-	 * @return the duplicated buffer
-	 */
-	public abstract Buffer duplicate() throws IOException, InterruptedException;
-
-	/**
-	 * Reads data from the buffer and writes it to the
-	 * given {@link WritableByteChannel} object.
-	 * 
-	 * @param destination The {@link WritableByteChannel} object to write the data to
-	 * @return The number of bytes read from the buffer, potentially <code>0</code> or <code>-1</code to indicate the
-	 *         end of the stream.
-	 * @throws IOException Thrown if an error occurs while writing to the {@link WritableByteChannel} object.
-	 */
-	public abstract int writeTo(WritableByteChannel writableByteChannel) throws IOException;
-
-	/**
-	 * Flip buffer (exchange limit and position).
-	 */
-	public abstract void flip();
-	
-	/**
-	 * Returns the current read/write position for relative operations.
-	 * @return
-	 */
-	public abstract int position();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/BufferFactory.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/BufferFactory.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/BufferFactory.java
deleted file mode 100644
index 001e5a7..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/BufferFactory.java
+++ /dev/null
@@ -1,32 +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.io.channels;
-
-import eu.stratosphere.core.memory.MemorySegment;
-
-
-public final class BufferFactory {
-
-	public static MemoryBuffer createFromMemory(final int bufferSize, final MemorySegment byteBuffer,
-			final MemoryBufferPoolConnector bufferPoolConnector) {
-
-		return new MemoryBuffer(bufferSize, byteBuffer, bufferPoolConnector);
-	}
-
-	/**
-	 * Private constructor to prevent instantiation.
-	 */
-	private BufferFactory() {
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelID.java
deleted file mode 100644
index a4e3ba2..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelID.java
+++ /dev/null
@@ -1,30 +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.io.channels;
-
-import eu.stratosphere.nephele.io.AbstractID;
-
-/**
- * A class for statistically unique channel IDs.
- * 
- */
-public class ChannelID extends AbstractID {
-
-	/**
-	 * Constructs a new, random channel ID.
-	 */
-	public ChannelID() {
-		super();
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelType.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelType.java
deleted file mode 100644
index 17fc980..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelType.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/***********************************************************************************************************************
- * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
- *
- * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- **********************************************************************************************************************/
-
-package eu.stratosphere.nephele.io.channels;
-
-/**
- * An enumeration for declaring the type of channel.
- * 
- */
-public enum ChannelType {
-	
-	/**
-	 * Enumeration type for network channels.
-	 */
-	NETWORK,
-
-	/**
-	 * Enumeration type for in-memory channels.
-	 */
-	INMEMORY
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithAccessInfo.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithAccessInfo.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithAccessInfo.java
deleted file mode 100644
index a44b1bd..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithAccessInfo.java
+++ /dev/null
@@ -1,58 +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.io.channels;
-
-import java.nio.channels.FileChannel;
-
-interface ChannelWithAccessInfo {
-
-	FileChannel getChannel();
-
-	FileChannel getAndIncrementReferences();
-
-	/**
-	 * Increments the references to this channel. Returns <code>true</code>, if successful, and <code>false</code>,
-	 * if the channel has been disposed in the meantime.
-	 * 
-	 * @return True, if successful, false, if the channel has been disposed.
-	 */
-	boolean incrementReferences();
-
-	ChannelWithPosition reserveWriteSpaceAndIncrementReferences(int spaceToReserve);
-
-	/**
-	 * Decrements the number of references to this channel. If the number of references is zero after the
-	 * decrement, the channel is deleted.
-	 * 
-	 * @return The number of references remaining after the decrement.
-	 * @throws IllegalStateException
-	 *         Thrown, if the number of references is already zero or below.
-	 */
-	int decrementReferences();
-
-	/**
-	 * Disposes the channel without further notice. Tries to close it (swallowing all exceptions) and tries
-	 * to delete the file.
-	 */
-	void disposeSilently();
-
-	/**
-	 * Updates the flag which indicates whether the underlying physical file shall be deleted when it is closed. Once
-	 * the flag was updated to <code>false</code> it cannot be set to <code>true</code> again.
-	 * 
-	 * @param deleteOnClose
-	 *        <code>true</code> to indicate the file shall be deleted when closed, <code>false</code> otherwise
-	 */
-	void updateDeleteOnCloseFlag(final boolean deleteOnClose);
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithPosition.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithPosition.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithPosition.java
deleted file mode 100644
index 5678439..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithPosition.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/***********************************************************************************************************************
- * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
- *
- * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- **********************************************************************************************************************/
-
-package eu.stratosphere.nephele.io.channels;
-
-import java.nio.channels.FileChannel;
-
-/**
- * A simple encapsulation of a file channel with an offset. This object is used for purposes, where
- * the channel is accessed by multiple threads and its internal position may be changed.
- */
-public class ChannelWithPosition {
-
-	private final FileChannel channel;
-
-	private final long offset;
-
-	ChannelWithPosition(final FileChannel channel, final long offset) {
-		this.channel = channel;
-		this.offset = offset;
-	}
-
-	public FileChannel getChannel() {
-
-		return this.channel;
-	}
-
-	public long getOffset() {
-
-		return this.offset;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/DefaultDeserializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/DefaultDeserializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/DefaultDeserializer.java
deleted file mode 100644
index 3f47fe1..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/DefaultDeserializer.java
+++ /dev/null
@@ -1,781 +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.io.channels;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.UTFDataFormatException;
-import java.nio.BufferUnderflowException;
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-import java.nio.channels.ReadableByteChannel;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.core.memory.DataInputView;
-import eu.stratosphere.nephele.io.RecordDeserializer;
-
-/**
- * A class for deserializing a portion of binary data into records of type <code>T</code>. The internal
- * buffer grows dynamically to the size that is required for deserialization.
- * 
- * @param <T>
- *        The type of the record this deserialization buffer can be used for.
- */
-public class DefaultDeserializer<T extends IOReadableWritable> implements RecordDeserializer<T> {
-	/**
-	 * The size of an integer in byte.
-	 */
-	private static final int SIZEOFINT = 4;
-
-	// --------------------------------------------------------------------------------------------
-
-	/**
-	 * The data input buffer used for deserialization.
-	 */
-	private final DataInputWrapper deserializationWrapper;
-
-	/**
-	 * Buffer to reconstruct the length field.
-	 */
-	private final ByteBuffer lengthBuf;
-
-	/**
-	 * Temporary buffer.
-	 */
-	private ByteBuffer tempBuffer;
-
-	/**
-	 * The type of the record to be deserialized.
-	 */
-	private final Class<? extends T> recordType;
-
-	/**
-	 * Size of the record to be deserialized in bytes.
-	 */
-	private int recordLength = -1;
-
-	/**
-	 * Flag indicating whether to throw an exception if nothing can be read any more.
-	 */
-	private final boolean propagateEndOfStream;
-
-	// --------------------------------------------------------------------------------------------
-
-	/**
-	 * Constructs a new deserialization buffer with the specified type.
-	 * 
-	 * @param recordType
-	 *        The type of the record to be deserialized.
-	 */
-	public DefaultDeserializer(final Class<? extends T> recordType) {
-		this(recordType, false);
-	}
-
-	/**
-	 * Constructs a new deserialization buffer with the specified type.
-	 * 
-	 * @param recordType
-	 *        The type of the record to be deserialized.
-	 * @param propagateEndOfStream
-	 *        <code>True</code>, if end of stream notifications during the
-	 *        deserialization process shall be propagated to the caller, <code>false</code> otherwise.
-	 */
-	public DefaultDeserializer(final Class<? extends T> recordType, final boolean propagateEndOfStream) {
-		this.recordType = recordType;
-		this.propagateEndOfStream = propagateEndOfStream;
-
-		this.lengthBuf = ByteBuffer.allocate(SIZEOFINT);
-		this.lengthBuf.order(ByteOrder.BIG_ENDIAN);
-
-		this.tempBuffer = ByteBuffer.allocate(128);
-		this.tempBuffer.order(ByteOrder.BIG_ENDIAN);
-
-		this.deserializationWrapper = new DataInputWrapper();
-		this.deserializationWrapper.setArray(this.tempBuffer.array());
-	}
-
-	// --------------------------------------------------------------------------------------------
-
-	/*
-	 * (non-Javadoc)
-	 * @see eu.stratosphere.nephele.io.RecordDeserializer#readData(java.lang.Object,
-	 * java.nio.channels.ReadableByteChannel)
-	 */
-	@Override
-	public T readData(T target, final ReadableByteChannel readableByteChannel) throws IOException {
-		// check whether the length has already been de-serialized
-		final int len;
-		if (this.recordLength < 0) {
-			if (readableByteChannel.read(this.lengthBuf) == -1 && this.propagateEndOfStream) {
-				if (this.lengthBuf.position() == 0) {
-					throw new EOFException();
-				} else {
-					throw new IOException("Deserialization error: Expected to read " + this.lengthBuf.remaining()
-						+ " more bytes of length information from the stream!");
-				}
-			}
-
-			if (this.lengthBuf.hasRemaining()) {
-				return null;
-			}
-
-			len = this.lengthBuf.getInt(0);
-			this.lengthBuf.clear();
-
-			if (this.tempBuffer.capacity() < len) {
-				this.tempBuffer = ByteBuffer.allocate(len);
-				this.tempBuffer.order(ByteOrder.BIG_ENDIAN);
-				this.deserializationWrapper.setArray(this.tempBuffer.array());
-			}
-
-			// Important: limit the number of bytes that can be read into the buffer
-			this.tempBuffer.position(0);
-			this.tempBuffer.limit(len);
-		} else {
-			len = this.recordLength;
-		}
-
-		if (readableByteChannel.read(this.tempBuffer) == -1 && this.propagateEndOfStream) {
-			throw new IOException("Deserilization error: Expected to read " + this.tempBuffer.remaining()
-				+ " more bytes from stream!");
-		}
-
-		if (this.tempBuffer.hasRemaining()) {
-			this.recordLength = len;
-			return null;
-		} else {
-			this.recordLength = -1;
-		}
-
-		this.deserializationWrapper.reset(len);
-
-		if (target == null) {
-			target = instantiateTarget();
-		}
-
-		// now de-serialize the target
-		try {
-			target.read(this.deserializationWrapper);
-			return target;
-		} catch (BufferUnderflowException buex) {
-			throw new EOFException();
-		}
-	}
-
-	private final T instantiateTarget() throws IOException {
-		try {
-			return this.recordType.newInstance();
-		} catch (Exception e) {
-			throw new IOException("Could not instantiate the given record type: " + e.getMessage(), e);
-		}
-	}
-
-	/*
-	 * (non-Javadoc)
-	 * @see eu.stratosphere.nephele.io.RecordDeserializer#clear()
-	 */
-	@Override
-	public void clear() {
-
-		this.recordLength = -1;
-		if (this.tempBuffer != null) {
-			this.tempBuffer.clear();
-		}
-		if (this.lengthBuf != null) {
-			this.lengthBuf.clear();
-		}
-	}
-
-	/*
-	 * (non-Javadoc)
-	 * @see eu.stratosphere.nephele.io.RecordDeserializer#hasUnfinishedData()
-	 */
-	@Override
-	public boolean hasUnfinishedData() {
-		if (this.recordLength != -1) {
-			return true;
-		}
-
-		if (this.lengthBuf.position() > 0) {
-			return true;
-		}
-
-		return false;
-	}
-
-	// --------------------------------------------------------------------------------------------
-
-	// private static final class DataInputWrapper implements DataInputView
-	// {
-	// private ByteBuffer source;
-	//
-	// private byte[] utfByteBuffer; // reusable byte buffer for utf-8 decoding
-	// private char[] utfCharBuffer; // reusable char buffer for utf-8 decoding
-	//
-	//
-	// void set(ByteBuffer source) {
-	// this.source = source;
-	// }
-	//
-	//
-	// /* (non-Javadoc)
-	// * @see java.io.DataInput#readFully(byte[])
-	// */
-	// @Override
-	// public void readFully(byte[] b) {
-	// this.source.get(b);
-	// }
-	//
-	// /* (non-Javadoc)
-	// * @see java.io.DataInput#readFully(byte[], int, int)
-	// */
-	// @Override
-	// public void readFully(byte[] b, int off, int len) {
-	// this.source.get(b, off, len);
-	// }
-	//
-	// /* (non-Javadoc)
-	// * @see java.io.DataInput#skipBytes(int)
-	// */
-	// @Override
-	// public int skipBytes(int n) {
-	// int newPos = this.source.position() + n;
-	// if (newPos > this.source.limit()) {
-	// newPos = this.source.limit();
-	// n = newPos - this.source.position();
-	// }
-	// this.source.position(newPos);
-	// return n;
-	// }
-	//
-	// /* (non-Javadoc)
-	// * @see java.io.DataInput#readBoolean()
-	// */
-	// @Override
-	// public boolean readBoolean() {
-	// return this.source.get() != 0;
-	// }
-	//
-	// /* (non-Javadoc)
-	// * @see java.io.DataInput#readByte()
-	// */
-	// @Override
-	// public byte readByte() {
-	// return this.source.get();
-	// }
-	//
-	// /* (non-Javadoc)
-	// * @see java.io.DataInput#readUnsignedByte()
-	// */
-	// @Override
-	// public int readUnsignedByte() {
-	// return this.source.get() & 0xff;
-	// }
-	//
-	// /* (non-Javadoc)
-	// * @see java.io.DataInput#readShort()
-	// */
-	// @Override
-	// public short readShort() {
-	// return this.source.getShort();
-	// }
-	//
-	// /* (non-Javadoc)
-	// * @see java.io.DataInput#readUnsignedShort()
-	// */
-	// @Override
-	// public int readUnsignedShort() {
-	// return this.source.getShort() & 0xffff;
-	// }
-	//
-	// /* (non-Javadoc)
-	// * @see java.io.DataInput#readChar()
-	// */
-	// @Override
-	// public char readChar() {
-	// return this.source.getChar();
-	// }
-	//
-	// /* (non-Javadoc)
-	// * @see java.io.DataInput#readInt()
-	// */
-	// @Override
-	// public int readInt() {
-	// return this.source.getInt();
-	// }
-	//
-	// /* (non-Javadoc)
-	// * @see java.io.DataInput#readLong()
-	// */
-	// @Override
-	// public long readLong() {
-	// return this.source.getLong();
-	// }
-	//
-	// /* (non-Javadoc)
-	// * @see java.io.DataInput#readFloat()
-	// */
-	// @Override
-	// public float readFloat() {
-	// return Float.intBitsToFloat(this.source.getInt());
-	// }
-	//
-	// /* (non-Javadoc)
-	// * @see java.io.DataInput#readDouble()
-	// */
-	// @Override
-	// public double readDouble() {
-	// return Double.longBitsToDouble(this.source.getLong());
-	// }
-	//
-	// /* (non-Javadoc)
-	// * @see java.io.DataInput#readLine()
-	// */
-	// @Override
-	// public String readLine()
-	// {
-	// if (this.source.hasRemaining()) {
-	// // read until a newline is found
-	// StringBuilder bld = new StringBuilder();
-	// char curr;
-	// while (this.source.hasRemaining() && (curr = (char) readUnsignedByte()) != '\n') {
-	// bld.append(curr);
-	// }
-	// // trim a trailing carriage return
-	// int len = bld.length();
-	// if (len > 0 && bld.charAt(len - 1) == '\r') {
-	// bld.setLength(len - 1);
-	// }
-	// String s = bld.toString();
-	// bld.setLength(0);
-	// return s;
-	// } else {
-	// return null;
-	// }
-	// }
-	//
-	// /* (non-Javadoc)
-	// * @see java.io.DataInput#readUTF()
-	// */
-	// @Override
-	// public String readUTF() throws IOException
-	// {
-	// final int utflen = readUnsignedShort();
-	//
-	// final byte[] bytearr;
-	// final char[] chararr;
-	//
-	// if (this.utfByteBuffer == null || this.utfByteBuffer.length < utflen) {
-	// bytearr = new byte[utflen];
-	// this.utfByteBuffer = bytearr;
-	// } else {
-	// bytearr = this.utfByteBuffer;
-	// }
-	// if (this.utfCharBuffer == null || this.utfCharBuffer.length < utflen) {
-	// chararr = new char[utflen];
-	// this.utfCharBuffer = chararr;
-	// } else {
-	// chararr = this.utfCharBuffer;
-	// }
-	//
-	// int c, char2, char3;
-	// int count = 0;
-	// int chararr_count = 0;
-	//
-	// readFully(bytearr, 0, utflen);
-	//
-	// while (count < utflen) {
-	// c = (int) bytearr[count] & 0xff;
-	// if (c > 127)
-	// break;
-	// count++;
-	// chararr[chararr_count++] = (char) c;
-	// }
-	//
-	// while (count < utflen) {
-	// c = (int) bytearr[count] & 0xff;
-	// switch (c >> 4) {
-	// case 0:
-	// case 1:
-	// case 2:
-	// case 3:
-	// case 4:
-	// case 5:
-	// case 6:
-	// case 7:
-	// /* 0xxxxxxx */
-	// count++;
-	// chararr[chararr_count++] = (char) c;
-	// break;
-	// case 12:
-	// case 13:
-	// /* 110x xxxx 10xx xxxx */
-	// count += 2;
-	// if (count > utflen)
-	// throw new UTFDataFormatException("malformed input: partial character at end");
-	// char2 = (int) bytearr[count - 1];
-	// if ((char2 & 0xC0) != 0x80)
-	// throw new UTFDataFormatException("malformed input around byte " + count);
-	// chararr[chararr_count++] = (char) (((c & 0x1F) << 6) | (char2 & 0x3F));
-	// break;
-	// case 14:
-	// /* 1110 xxxx 10xx xxxx 10xx xxxx */
-	// count += 3;
-	// if (count > utflen)
-	// throw new UTFDataFormatException("malformed input: partial character at end");
-	// char2 = (int) bytearr[count - 2];
-	// char3 = (int) bytearr[count - 1];
-	// if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80))
-	// throw new UTFDataFormatException("malformed input around byte " + (count - 1));
-	// chararr[chararr_count++] = (char) (((c & 0x0F) << 12) | ((char2 & 0x3F) << 6) | ((char3 & 0x3F) << 0));
-	// break;
-	// default:
-	// /* 10xx xxxx, 1111 xxxx */
-	// throw new UTFDataFormatException("malformed input around byte " + count);
-	// }
-	// }
-	// // The number of chars produced may be less than utflen
-	// return new String(chararr, 0, chararr_count);
-	// }
-	//
-	//
-	// /* (non-Javadoc)
-	// * @see eu.stratosphere.nephele.services.memorymanager.DataInputView#skipBytesToRead(int)
-	// */
-	// @Override
-	// public void skipBytesToRead(int numBytes) throws IOException {
-	// if (this.source.remaining() < numBytes) {
-	// throw new EOFException();
-	// } else {
-	// this.source.position(this.source.position() + numBytes);
-	// }
-	// }
-	// }
-
-	private static final class DataInputWrapper implements DataInputView {
-		private byte[] source;
-
-		private int position;
-
-		private int limit;
-
-		private char[] utfCharBuffer; // reusable char buffer for utf-8 decoding
-
-		void setArray(byte[] source) {
-			this.source = source;
-		}
-
-		void reset(int limit) {
-			this.position = 0;
-			this.limit = limit;
-		}
-
-		/*
-		 * (non-Javadoc)
-		 * @see java.io.DataInput#readFully(byte[])
-		 */
-		@Override
-		public void readFully(byte[] b) throws EOFException {
-			readFully(b, 0, b.length);
-		}
-
-		/*
-		 * (non-Javadoc)
-		 * @see java.io.DataInput#readFully(byte[], int, int)
-		 */
-		@Override
-		public void readFully(byte[] b, int off, int len) throws EOFException {
-			if (this.position <= this.limit - len) {
-				System.arraycopy(this.source, this.position, b, off, len);
-				this.position += len;
-			} else {
-				throw new EOFException();
-			}
-		}
-
-		/*
-		 * (non-Javadoc)
-		 * @see java.io.DataInput#skipBytes(int)
-		 */
-		@Override
-		public int skipBytes(int n) {
-			if (n < 0) {
-				throw new IllegalArgumentException("Number of bytes to skip must not be negative.");
-			}
-
-			int toSkip = Math.min(this.limit - this.position, n);
-			this.position += toSkip;
-			return toSkip;
-		}
-
-		/*
-		 * (non-Javadoc)
-		 * @see java.io.DataInput#readBoolean()
-		 */
-		@Override
-		public boolean readBoolean() throws EOFException {
-			return readByte() != 0;
-		}
-
-		/*
-		 * (non-Javadoc)
-		 * @see java.io.DataInput#readByte()
-		 */
-		@Override
-		public byte readByte() throws EOFException {
-			if (this.position < this.limit) {
-				return this.source[this.position++];
-			} else {
-				throw new EOFException();
-			}
-		}
-
-		/*
-		 * (non-Javadoc)
-		 * @see java.io.DataInput#readUnsignedByte()
-		 */
-		@Override
-		public int readUnsignedByte() throws EOFException {
-			return readByte() & 0xff;
-		}
-
-		/*
-		 * (non-Javadoc)
-		 * @see java.io.DataInput#readShort()
-		 */
-		@Override
-		public short readShort() throws EOFException {
-			if (this.position < this.limit - 1) {
-				short num = (short) (
-						((this.source[this.position + 0] & 0xff) << 8) |
-						((this.source[this.position + 1] & 0xff)));
-				this.position += 2;
-				return num;
-			} else {
-				throw new EOFException();
-			}
-		}
-
-		/*
-		 * (non-Javadoc)
-		 * @see java.io.DataInput#readUnsignedShort()
-		 */
-		@Override
-		public int readUnsignedShort() throws EOFException {
-			return readShort() & 0xffff;
-		}
-
-		/*
-		 * (non-Javadoc)
-		 * @see java.io.DataInput#readChar()
-		 */
-		@Override
-		public char readChar() throws EOFException {
-			if (this.position < this.limit - 1) {
-				char c = (char) (
-						((this.source[this.position + 0] & 0xff) << 8) |
-						((this.source[this.position + 1] & 0xff)));
-				this.position += 2;
-				return c;
-			} else {
-				throw new EOFException();
-			}
-		}
-
-		/*
-		 * (non-Javadoc)
-		 * @see java.io.DataInput#readInt()
-		 */
-		@Override
-		public int readInt() throws EOFException {
-			if (this.position < this.limit - 3) {
-				final int num = ((this.source[this.position + 0] & 0xff) << 24) |
-								((this.source[this.position + 1] & 0xff) << 16) |
-								((this.source[this.position + 2] & 0xff) << 8) |
-								((this.source[this.position + 3] & 0xff));
-				this.position += 4;
-				return num;
-			} else {
-				throw new EOFException();
-			}
-		}
-
-		/*
-		 * (non-Javadoc)
-		 * @see java.io.DataInput#readLong()
-		 */
-		@Override
-		public long readLong() throws EOFException {
-			if (this.position < this.limit - 7) {
-				final long num = (((long) this.source[this.position + 0] & 0xff) << 56) |
-									(((long) this.source[this.position + 1] & 0xff) << 48) |
-									(((long) this.source[this.position + 2] & 0xff) << 40) |
-									(((long) this.source[this.position + 3] & 0xff) << 32) |
-									(((long) this.source[this.position + 4] & 0xff) << 24) |
-									(((long) this.source[this.position + 5] & 0xff) << 16) |
-									(((long) this.source[this.position + 6] & 0xff) << 8) |
-									(((long) this.source[this.position + 7] & 0xff) << 0);
-				this.position += 8;
-				return num;
-			} else {
-				throw new EOFException();
-			}
-		}
-
-		/*
-		 * (non-Javadoc)
-		 * @see java.io.DataInput#readFloat()
-		 */
-		@Override
-		public float readFloat() throws EOFException {
-			return Float.intBitsToFloat(readInt());
-		}
-
-		/*
-		 * (non-Javadoc)
-		 * @see java.io.DataInput#readDouble()
-		 */
-		@Override
-		public double readDouble() throws EOFException {
-			return Double.longBitsToDouble(readLong());
-		}
-
-		/*
-		 * (non-Javadoc)
-		 * @see java.io.DataInput#readLine()
-		 */
-		@Override
-		public String readLine() {
-			if (this.position < this.limit) {
-				// read until a newline is found
-				StringBuilder bld = new StringBuilder();
-				char curr;
-				while (this.position < this.limit && (curr = (char) (this.source[this.position++] & 0xff)) != '\n') {
-					bld.append(curr);
-				}
-				// trim a trailing carriage return
-				int len = bld.length();
-				if (len > 0 && bld.charAt(len - 1) == '\r') {
-					bld.setLength(len - 1);
-				}
-				String s = bld.toString();
-				bld.setLength(0);
-				return s;
-			} else {
-				return null;
-			}
-		}
-
-		/*
-		 * (non-Javadoc)
-		 * @see java.io.DataInput#readUTF()
-		 */
-		@Override
-		public String readUTF() throws IOException {
-			final int utflen = readUnsignedShort();
-			final int utfLimit = this.position + utflen;
-
-			if (utfLimit > this.limit) {
-				throw new EOFException();
-			}
-
-			final byte[] bytearr = this.source;
-			final char[] chararr;
-			if (this.utfCharBuffer == null || this.utfCharBuffer.length < utflen) {
-				chararr = new char[utflen];
-				this.utfCharBuffer = chararr;
-			} else {
-				chararr = this.utfCharBuffer;
-			}
-
-			int c, char2, char3;
-			int count = this.position;
-			int chararr_count = 0;
-
-			while (count < utfLimit) {
-				c = (int) bytearr[count] & 0xff;
-				if (c > 127) {
-					break;
-				}
-				count++;
-				chararr[chararr_count++] = (char) c;
-			}
-
-			while (count < utfLimit) {
-				c = (int) bytearr[count] & 0xff;
-				switch (c >> 4) {
-				case 0:
-				case 1:
-				case 2:
-				case 3:
-				case 4:
-				case 5:
-				case 6:
-				case 7:
-					/* 0xxxxxxx */
-					count++;
-					chararr[chararr_count++] = (char) c;
-					break;
-				case 12:
-				case 13:
-					/* 110x xxxx 10xx xxxx */
-					count += 2;
-					if (count > utfLimit) {
-						throw new UTFDataFormatException("Malformed input: partial character at end");
-					}
-					char2 = (int) bytearr[count - 1];
-					if ((char2 & 0xC0) != 0x80) {
-						throw new UTFDataFormatException("Malformed input around byte " + count);
-					}
-					chararr[chararr_count++] = (char) (((c & 0x1F) << 6) | (char2 & 0x3F));
-					break;
-				case 14:
-					/* 1110 xxxx 10xx xxxx 10xx xxxx */
-					count += 3;
-					if (count > utfLimit) {
-						throw new UTFDataFormatException("Malformed input: partial character at end");
-					}
-					char2 = (int) bytearr[count - 2];
-					char3 = (int) bytearr[count - 1];
-					if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80)) {
-						throw new UTFDataFormatException("Malformed input around byte " + (count - 1));
-					}
-					chararr[chararr_count++] = (char) (((c & 0x0F) << 12) | ((char2 & 0x3F) << 6) | ((char3 & 0x3F) << 0));
-					break;
-				default:
-					/* 10xx xxxx, 1111 xxxx */
-					throw new UTFDataFormatException("Malformed input around byte " + count);
-				}
-			}
-			// The number of chars produced may be less than utflen
-			this.position += utflen;
-			return new String(chararr, 0, chararr_count);
-		}
-
-		/*
-		 * (non-Javadoc)
-		 * @see eu.stratosphere.nephele.services.memorymanager.DataInputView#skipBytesToRead(int)
-		 */
-		@Override
-		public void skipBytesToRead(int numBytes) throws EOFException {
-			if (numBytes < 0) {
-				throw new IllegalArgumentException("Number of bytes to skip must not be negative.");
-			} else if (this.limit - this.position < numBytes) {
-				throw new EOFException();
-			} else {
-				this.position += numBytes;
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/DistributedChannelWithAccessInfo.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/DistributedChannelWithAccessInfo.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/DistributedChannelWithAccessInfo.java
deleted file mode 100644
index 6873c44..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/DistributedChannelWithAccessInfo.java
+++ /dev/null
@@ -1,176 +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.io.channels;
-
-import java.io.IOException;
-import java.nio.channels.FileChannel;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.core.fs.FileChannelWrapper;
-import eu.stratosphere.core.fs.FileSystem;
-import eu.stratosphere.core.fs.Path;
-
-final class DistributedChannelWithAccessInfo implements ChannelWithAccessInfo {
-
-	/**
-	 * The logging object.
-	 */
-	private static final Log LOG = LogFactory.getLog(DistributedChannelWithAccessInfo.class);
-
-	private final FileSystem fs;
-
-	private final Path checkpointFile;
-
-	private final FileChannelWrapper channel;
-
-	private final AtomicLong reservedWritePosition;
-
-	private final AtomicInteger referenceCounter;
-
-	private final AtomicBoolean deleteOnClose;
-
-	DistributedChannelWithAccessInfo(final FileSystem fs, final Path checkpointFile, final int bufferSize,
-			final boolean deleteOnClose) throws IOException {
-
-		this.fs = fs;
-		this.checkpointFile = checkpointFile;
-		this.channel = new FileChannelWrapper(fs, checkpointFile, bufferSize, (short) 2);
-		this.reservedWritePosition = new AtomicLong(0L);
-		this.referenceCounter = new AtomicInteger(0);
-		this.deleteOnClose = new AtomicBoolean(deleteOnClose);
-	}
-
-
-	@Override
-	public FileChannel getChannel() {
-
-		return this.channel;
-	}
-
-
-	@Override
-	public FileChannel getAndIncrementReferences() {
-
-		if (incrementReferences()) {
-			return this.channel;
-		} else {
-			return null;
-		}
-	}
-
-	@Override
-	public ChannelWithPosition reserveWriteSpaceAndIncrementReferences(final int spaceToReserve) {
-
-		if (incrementReferences()) {
-			return new ChannelWithPosition(this.channel, this.reservedWritePosition.getAndAdd(spaceToReserve));
-		} else {
-			return null;
-		}
-	}
-
-
-	@Override
-	public int decrementReferences() {
-
-		int current = this.referenceCounter.get();
-		while (true) {
-			if (current <= 0) {
-				// this is actually an error case, because the channel was deleted before
-				throw new IllegalStateException("The references to the file were already at zero.");
-			}
-
-			if (current == 1) {
-				// this call decrements to zero, so mark it as deleted
-				if (this.referenceCounter.compareAndSet(current, Integer.MIN_VALUE)) {
-					current = 0;
-					break;
-				}
-			} else if (this.referenceCounter.compareAndSet(current, current - 1)) {
-				current = current - 1;
-				break;
-			}
-			current = this.referenceCounter.get();
-		}
-
-		if (current > 0) {
-			return current;
-		} else if (current == 0) {
-			// delete the channel
-			this.referenceCounter.set(Integer.MIN_VALUE);
-			this.reservedWritePosition.set(Long.MIN_VALUE);
-			try {
-				this.channel.close();
-				if (this.deleteOnClose.get()) {
-					this.fs.delete(this.checkpointFile, false);
-				}
-
-			} catch (IOException ioex) {
-				if (LOG.isErrorEnabled()) {
-					LOG.error("Error while closing spill file for file buffers: " + ioex.getMessage(), ioex);
-				}
-			}
-			return current;
-		} else {
-			throw new IllegalStateException("The references to the file were already at zero.");
-		}
-	}
-
-
-	@Override
-	public boolean incrementReferences() {
-
-		int current = this.referenceCounter.get();
-		while (true) {
-			// check whether it was disposed in the meantime
-			if (current < 0) {
-				return false;
-			}
-			// atomically check and increment
-			if (this.referenceCounter.compareAndSet(current, current + 1)) {
-				return true;
-			}
-			current = this.referenceCounter.get();
-		}
-	}
-
-
-	@Override
-	public void disposeSilently() {
-
-		this.referenceCounter.set(Integer.MIN_VALUE);
-		this.reservedWritePosition.set(Long.MIN_VALUE);
-
-		if (this.channel.isOpen()) {
-			try {
-				this.channel.close();
-				if (this.deleteOnClose.get()) {
-					this.fs.delete(this.checkpointFile, false);
-				}
-			} catch (Throwable t) {
-			}
-		}
-	}
-
-
-	@Override
-	public void updateDeleteOnCloseFlag(final boolean deleteOnClose) {
-
-		this.deleteOnClose.compareAndSet(true, deleteOnClose);
-	}
-}


[03/30] git commit: Fix dependency exclusion for examples in yarn-uberjar

Posted by rm...@apache.org.
Fix dependency exclusion for examples in yarn-uberjar


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

Branch: refs/heads/master
Commit: 0bb812e431f266a74ff9a668e055209ba5122c7d
Parents: 2a9ed09
Author: mingliang <qm...@gmail.com>
Authored: Fri Jun 6 15:43:42 2014 +0200
Committer: StephanEwen <st...@tu-berlin.de>
Committed: Fri Jun 6 16:00:59 2014 +0200

----------------------------------------------------------------------
 stratosphere-dist/src/main/assemblies/yarn-uberjar.xml | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0bb812e4/stratosphere-dist/src/main/assemblies/yarn-uberjar.xml
----------------------------------------------------------------------
diff --git a/stratosphere-dist/src/main/assemblies/yarn-uberjar.xml b/stratosphere-dist/src/main/assemblies/yarn-uberjar.xml
index 5803963..598936f 100644
--- a/stratosphere-dist/src/main/assemblies/yarn-uberjar.xml
+++ b/stratosphere-dist/src/main/assemblies/yarn-uberjar.xml
@@ -31,7 +31,8 @@
 			<unpack>true</unpack>
 			<scope>runtime</scope>
 			<excludes>
-				<exclude>eu.stratosphere:stratosphere-examples:*</exclude>
+				<exclude>eu.stratosphere:stratosphere-java-examples:*</exclude>
+				<exclude>eu.stratosphere:stratosphere-scala-examples:*</exclude>
 			</excludes>
 		</dependencySet>
 	</dependencySets>


[25/30] git commit: Offer buffer-oriented API for I/O (#25)

Posted by rm...@apache.org.
Offer buffer-oriented API for I/O (#25)

https://github.com/stratosphere/stratosphere/issues/25

The runtime offered a record-oriented API for data transfers, which
* resulted in unnecessary data (de)serialization,
* complicated the upcoming fault tolerance implementation, and
* blocked more efficient implementations of higher-level operators.

With this commit, the runtime offers a buffer-oriented API for the
output side (sending), which is oblivious to records. The buffer
oriented input side (receiving) is still to be implemented.


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

Branch: refs/heads/master
Commit: 2db78a8dc1a4664f3e384005d7e07bea594b835b
Parents: 0bb812e
Author: uce <u....@fu-berlin.de>
Authored: Tue Jan 14 17:45:21 2014 +0100
Committer: StephanEwen <st...@tu-berlin.de>
Committed: Sat Jun 7 09:41:21 2014 +0200

----------------------------------------------------------------------
 .../spargel/java/record/SpargelIteration.java   |   3 +-
 .../plantranslate/NepheleJobGraphGenerator.java |  13 +-
 .../configuration/ConfigConstants.java          |   6 -
 .../stratosphere/core/memory/MemorySegment.java |   5 +
 .../main/java/eu/stratosphere/types/Record.java |   2 +-
 .../eu/stratosphere/nephele/AbstractID.java     | 177 ++++
 .../nephele/annotations/ForceCheckpoint.java    |  30 -
 .../nephele/annotations/Stateful.java           |  28 -
 .../nephele/annotations/Stateless.java          |  28 -
 .../nephele/annotations/TaskAnnotation.java     |  29 -
 .../deployment/ChannelDeploymentDescriptor.java |   2 +-
 .../deployment/GateDeploymentDescriptor.java    |   4 +-
 .../nephele/example/events/EventExample.java    |  95 ---
 .../nephele/example/events/EventReceiver.java   |  49 --
 .../nephele/example/events/EventSender.java     |  68 --
 .../nephele/example/events/MyEventListener.java |  46 --
 .../stratosphere/nephele/example/grep/Grep.java | 108 ---
 .../nephele/example/grep/GrepTask.java          |  47 --
 .../nephele/example/speedtest/SpeedTest.java    | 224 -----
 .../example/speedtest/SpeedTestConsumer.java    |  45 -
 .../example/speedtest/SpeedTestForwarder.java   |  54 --
 .../example/speedtest/SpeedTestProducer.java    |  54 --
 .../example/speedtest/SpeedTestRecord.java      |  55 --
 .../nephele/example/union/ConsumerTask.java     |  44 -
 .../nephele/example/union/ProducerTask.java     |  43 -
 .../nephele/example/union/UnionJob.java         | 104 ---
 .../nephele/example/union/UnionTask.java        |  45 -
 .../nephele/execution/CancelTaskException.java  |  25 +
 .../nephele/execution/Environment.java          |  82 +-
 .../execution/ExecutionStateTransition.java     |  24 +-
 .../nephele/execution/RuntimeEnvironment.java   | 326 +++-----
 .../DistributionPatternProvider.java            |   2 +-
 .../nephele/executiongraph/ExecutionEdge.java   |  11 +-
 .../nephele/executiongraph/ExecutionGate.java   |   4 +-
 .../nephele/executiongraph/ExecutionGraph.java  |  20 +-
 .../executiongraph/ExecutionGroupEdge.java      |  28 +-
 .../executiongraph/ExecutionGroupVertex.java    |  27 +-
 .../nephele/executiongraph/ExecutionStage.java  |   6 +-
 .../nephele/executiongraph/ExecutionVertex.java |  38 +-
 .../executiongraph/ExecutionVertexID.java       |   2 +-
 .../executiongraph/ManagementGraphFactory.java  |   2 +-
 .../nephele/instance/AbstractInstance.java      |  23 +-
 .../nephele/instance/AllocationID.java          |   2 +-
 .../instance/InstanceConnectionInfo.java        |  25 +-
 .../nephele/instance/InstanceID.java            |   2 +-
 .../instance/cluster/ClusterManager.java        |  10 +-
 .../stratosphere/nephele/io/AbstractGate.java   | 148 ----
 .../eu/stratosphere/nephele/io/AbstractID.java  | 214 -----
 .../nephele/io/AbstractRecordReader.java        |  98 ---
 .../nephele/io/AbstractRecordWriter.java        | 140 ----
 .../io/AbstractSingleGateRecordReader.java      |  77 --
 .../nephele/io/AbstractUnionRecordReader.java   | 152 ----
 .../nephele/io/BroadcastRecordWriter.java       |  53 --
 .../nephele/io/ChannelSelector.java             |  39 -
 .../nephele/io/DataOutputBuffer.java            | 165 ----
 .../nephele/io/DefaultChannelSelector.java      |  47 --
 .../nephele/io/DistributionPattern.java         |  33 -
 .../java/eu/stratosphere/nephele/io/Gate.java   | 126 ---
 .../java/eu/stratosphere/nephele/io/GateID.java |  22 -
 .../io/ImmutableRecordDeserializerFactory.java  |  41 -
 .../nephele/io/InputChannelResult.java          |  23 -
 .../eu/stratosphere/nephele/io/InputGate.java   | 136 ----
 .../stratosphere/nephele/io/MutableReader.java  |  32 -
 .../io/MutableRecordDeserializerFactory.java    |  52 --
 .../nephele/io/MutableRecordReader.java         | 119 ---
 .../nephele/io/MutableUnionRecordReader.java    |  37 -
 .../eu/stratosphere/nephele/io/OutputGate.java  | 149 ----
 .../java/eu/stratosphere/nephele/io/Reader.java |  30 -
 .../eu/stratosphere/nephele/io/ReaderBase.java  |  67 --
 .../nephele/io/RecordAvailabilityListener.java  |  35 -
 .../nephele/io/RecordDeserializer.java          |  55 --
 .../nephele/io/RecordDeserializerFactory.java   |  31 -
 .../stratosphere/nephele/io/RecordReader.java   | 153 ----
 .../stratosphere/nephele/io/RecordWriter.java   |  82 --
 .../nephele/io/RuntimeInputGate.java            | 330 --------
 .../nephele/io/RuntimeOutputGate.java           | 333 --------
 .../nephele/io/UnionRecordReader.java           |  67 --
 .../java/eu/stratosphere/nephele/io/Writer.java |  28 -
 .../nephele/io/channels/AbstractChannel.java    | 127 ---
 .../io/channels/AbstractInputChannel.java       | 102 ---
 .../io/channels/AbstractOutputChannel.java      | 111 ---
 .../nephele/io/channels/Buffer.java             | 175 ----
 .../nephele/io/channels/BufferFactory.java      |  32 -
 .../nephele/io/channels/ChannelID.java          |  30 -
 .../nephele/io/channels/ChannelType.java        |  31 -
 .../io/channels/ChannelWithAccessInfo.java      |  58 --
 .../io/channels/ChannelWithPosition.java        |  42 -
 .../io/channels/DefaultDeserializer.java        | 781 ------------------
 .../DistributedChannelWithAccessInfo.java       | 176 ----
 .../io/channels/LocalChannelWithAccessInfo.java | 170 ----
 .../nephele/io/channels/MemoryBuffer.java       | 249 ------
 .../io/channels/MemoryBufferPoolConnector.java  |  32 -
 .../io/channels/MemoryBufferRecycler.java       |  91 ---
 .../io/channels/SerializationBuffer.java        | 141 ----
 .../AbstractByteBufferedInputChannel.java       | 243 ------
 .../AbstractByteBufferedOutputChannel.java      | 255 ------
 .../io/channels/bytebuffered/BufferOrEvent.java |  52 --
 .../ByteBufferedChannelCloseEvent.java          |  35 -
 .../ByteBufferedInputChannelBroker.java         |  36 -
 .../ByteBufferedOutputChannelBroker.java        |  71 --
 .../bytebuffered/EndOfSuperstepEvent.java       |  34 -
 .../bytebuffered/InMemoryInputChannel.java      |  34 -
 .../bytebuffered/InMemoryOutputChannel.java     |  34 -
 .../bytebuffered/NetworkInputChannel.java       |  34 -
 .../bytebuffered/NetworkOutputChannel.java      |  34 -
 .../nephele/io/library/DirectoryReader.java     | 103 ---
 .../nephele/io/library/DirectoryWriter.java     |  67 --
 .../nephele/io/library/FileLineReader.java      |  76 --
 .../nephele/io/library/FileLineWriter.java      |  75 --
 .../eu/stratosphere/nephele/ipc/Client.java     |  43 +-
 .../nephele/jobgraph/AbstractJobVertex.java     |   3 +-
 .../nephele/jobgraph/DistributionPattern.java   |  33 +
 .../stratosphere/nephele/jobgraph/JobEdge.java  |   3 +-
 .../eu/stratosphere/nephele/jobgraph/JobID.java |  69 +-
 .../nephele/jobgraph/JobVertexID.java           |   2 +-
 .../nephele/jobmanager/JobManager.java          | 147 +---
 .../jobmanager/scheduler/AbstractScheduler.java |   2 +-
 .../jobmanager/scheduler/RecoveryLogic.java     |   3 +-
 .../nephele/managementgraph/ManagementEdge.java |   2 +-
 .../managementgraph/ManagementEdgeID.java       |   4 +-
 .../managementgraph/ManagementGateID.java       |   2 +-
 .../managementgraph/ManagementGraph.java        |   2 +-
 .../managementgraph/ManagementGroupEdge.java    |   2 +-
 .../managementgraph/ManagementGroupVertex.java  |   2 +-
 .../ManagementGroupVertexID.java                |   2 +-
 .../managementgraph/ManagementVertexID.java     |   2 +-
 .../nephele/multicast/MulticastCluster.java     | 210 -----
 .../multicast/MulticastForwardingTable.java     |  49 --
 .../nephele/multicast/MulticastManager.java     | 463 -----------
 .../multicast/TopologyInformationSupplier.java  |  30 -
 .../nephele/multicast/TreeNode.java             | 246 ------
 .../nephele/profiling/TaskManagerProfiler.java  |   5 +-
 .../profiling/impl/TaskManagerProfilerImpl.java |  31 +-
 .../protocols/ChannelLookupProtocol.java        |   4 +-
 .../protocols/ExtendedManagementProtocol.java   |  13 -
 .../protocols/TaskOperationProtocol.java        |  14 +-
 .../services/iomanager/BlockChannelAccess.java  |   4 +-
 .../services/memorymanager/MemoryManager.java   |   3 +-
 .../memorymanager/spi/DefaultMemoryManager.java |  17 +-
 .../taskmanager/ExecutorThreadFactory.java      |  35 +
 .../stratosphere/nephele/taskmanager/Task.java  | 257 +++++-
 .../nephele/taskmanager/TaskKillResult.java     |  44 -
 .../nephele/taskmanager/TaskManager.java        |  56 +-
 .../BufferAvailabilityListener.java             |  28 -
 .../bufferprovider/BufferProvider.java          |  82 --
 .../bufferprovider/BufferProviderBroker.java    |  24 -
 .../bufferprovider/GlobalBufferPool.java        | 135 ---
 .../bufferprovider/LocalBufferPool.java         | 287 -------
 .../bufferprovider/LocalBufferPoolOwner.java    |  54 --
 .../AbstractOutputChannelContext.java           |  84 --
 .../AbstractOutputChannelForwarder.java         | 109 ---
 .../ByteBufferedChannelManager.java             | 816 -------------------
 .../bytebuffered/CanceledChannelSet.java        | 211 -----
 .../bytebuffered/ChannelContext.java            |  36 -
 .../ConnectionInfoLookupResponse.java           | 176 ----
 .../taskmanager/bytebuffered/GateContext.java   |  21 -
 .../bytebuffered/IncomingConnection.java        | 125 ---
 .../bytebuffered/IncomingConnectionThread.java  | 223 -----
 .../bytebuffered/InputChannelContext.java       |  21 -
 .../bytebuffered/InputGateContext.java          |  24 -
 .../InsufficientResourcesException.java         |  37 -
 .../bytebuffered/NetworkConnectionManager.java  | 173 ----
 .../bytebuffered/OutgoingConnection.java        | 531 ------------
 .../bytebuffered/OutgoingConnectionThread.java  | 270 ------
 .../bytebuffered/OutputChannelContext.java      |  17 -
 .../OutputChannelForwardingChain.java           |  84 --
 .../bytebuffered/OutputGateContext.java         |  22 -
 .../bytebuffered/ReceiverNotFoundEvent.java     | 169 ----
 .../bytebuffered/RemoteReceiver.java            | 157 ----
 .../bytebuffered/SenderHintEvent.java           | 119 ---
 .../taskmanager/bytebuffered/TaskContext.java   |  24 -
 .../bytebuffered/UnexpectedEnvelopeEvent.java   |  81 --
 .../runtime/ExecutorThreadFactory.java          |  35 -
 .../taskmanager/runtime/ForwardingBarrier.java  |  75 --
 .../taskmanager/runtime/RuntimeDispatcher.java  |  38 -
 .../runtime/RuntimeInputChannelContext.java     | 303 -------
 .../runtime/RuntimeInputGateContext.java        | 183 -----
 .../runtime/RuntimeOutputChannelBroker.java     | 206 -----
 .../runtime/RuntimeOutputChannelContext.java    |  76 --
 .../runtime/RuntimeOutputGateContext.java       | 159 ----
 .../taskmanager/runtime/RuntimeTask.java        | 346 --------
 .../taskmanager/runtime/RuntimeTaskContext.java | 211 -----
 .../transferenvelope/AbstractDeserializer.java  | 355 --------
 .../transferenvelope/AbstractSerializer.java    | 274 -------
 .../CapacityConstrainedArrayQueue.java          | 322 --------
 .../transferenvelope/DefaultDeserializer.java   |  94 ---
 .../transferenvelope/DefaultSerializer.java     |  38 -
 .../NoBufferAvailableException.java             |  53 --
 .../transferenvelope/TransferEnvelope.java      | 165 ----
 .../TransferEnvelopeDispatcher.java             |  37 -
 .../TransferEnvelopeReceiverList.java           |  86 --
 .../nephele/util/BufferPoolConnector.java       |  45 +
 .../eu/stratosphere/nephele/util/TaskUtils.java |  42 -
 .../runtime/iterative/io/FakeOutputTask.java    |   2 +-
 .../iterative/task/IterationHeadPactTask.java   |  16 +-
 .../task/IterationIntermediatePactTask.java     |   7 +-
 .../task/IterationSynchronizationSinkTask.java  |   2 +-
 .../iterative/task/IterationTailPactTask.java   |   1 +
 .../pact/runtime/shipping/OutputCollector.java  |  31 +-
 .../pact/runtime/shipping/OutputEmitter.java    |   2 +-
 .../runtime/shipping/RecordOutputCollector.java |  48 +-
 .../runtime/shipping/RecordOutputEmitter.java   |   2 +
 .../pact/runtime/task/DataSinkTask.java         |  19 +-
 .../pact/runtime/task/DataSourceTask.java       |  24 +-
 .../pact/runtime/task/RegularPactTask.java      | 250 +++---
 .../ExceptionInChainedStubException.java        |   8 +
 .../SynchronousChainedCombineDriver.java        |   2 +
 .../pact/runtime/task/util/ReaderIterator.java  |   2 +-
 .../runtime/task/util/RecordReaderIterator.java |   2 +-
 .../java/eu/stratosphere/runtime/io/Buffer.java |  93 +++
 .../stratosphere/runtime/io/BufferRecycler.java |  26 +
 .../runtime/io/api/AbstractRecordReader.java    |  98 +++
 .../io/api/AbstractSingleGateRecordReader.java  |  69 ++
 .../io/api/AbstractUnionRecordReader.java       | 155 ++++
 .../runtime/io/api/BufferWriter.java            |  67 ++
 .../runtime/io/api/ChannelSelector.java         |  39 +
 .../runtime/io/api/MutableReader.java           |  32 +
 .../runtime/io/api/MutableRecordReader.java     | 120 +++
 .../io/api/MutableUnionRecordReader.java        |  37 +
 .../eu/stratosphere/runtime/io/api/Reader.java  |  30 +
 .../stratosphere/runtime/io/api/ReaderBase.java |  67 ++
 .../runtime/io/api/RecordReader.java            | 154 ++++
 .../runtime/io/api/RecordWriter.java            | 151 ++++
 .../io/api/RoundRobinChannelSelector.java       |  47 ++
 .../runtime/io/api/UnionRecordReader.java       |  67 ++
 .../runtime/io/channels/BufferOrEvent.java      |  52 ++
 .../runtime/io/channels/Channel.java            |  97 +++
 .../runtime/io/channels/ChannelCloseEvent.java  |  33 +
 .../runtime/io/channels/ChannelID.java          |  39 +
 .../runtime/io/channels/ChannelType.java        |  26 +
 .../io/channels/EndOfSuperstepEvent.java        |  34 +
 .../runtime/io/channels/InputChannel.java       | 493 +++++++++++
 .../runtime/io/channels/OutputChannel.java      | 193 +++++
 .../ReceiverAlreadyClosedException.java         |  22 +
 .../eu/stratosphere/runtime/io/gates/Gate.java  | 174 ++++
 .../stratosphere/runtime/io/gates/GateID.java   |  24 +
 .../runtime/io/gates/InputChannelResult.java    |  23 +
 .../runtime/io/gates/InputGate.java             | 384 +++++++++
 .../runtime/io/gates/OutputGate.java            | 165 ++++
 .../io/gates/RecordAvailabilityListener.java    |  36 +
 .../runtime/io/network/ChannelManager.java      | 646 +++++++++++++++
 .../network/ConnectionInfoLookupResponse.java   | 143 ++++
 .../network/InsufficientResourcesException.java |  37 +
 .../LocalReceiverCancelledException.java        |  37 +
 .../io/network/NetworkConnectionManager.java    | 176 ++++
 .../runtime/io/network/RemoteReceiver.java      | 157 ++++
 .../runtime/io/network/SenderHintEvent.java     | 117 +++
 .../BufferAvailabilityListener.java             |  28 +
 .../network/bufferprovider/BufferProvider.java  |  69 ++
 .../bufferprovider/BufferProviderBroker.java    |  24 +
 .../bufferprovider/GlobalBufferPool.java        | 123 +++
 .../network/bufferprovider/LocalBufferPool.java | 306 +++++++
 .../bufferprovider/LocalBufferPoolOwner.java    |  56 ++
 .../bufferprovider/SerialSingleBufferPool.java  |  77 ++
 .../runtime/io/network/envelope/Envelope.java   | 169 ++++
 .../io/network/envelope/EnvelopeDispatcher.java |  46 ++
 .../io/network/envelope/EnvelopeReader.java     | 212 +++++
 .../network/envelope/EnvelopeReceiverList.java  |  75 ++
 .../io/network/envelope/EnvelopeWriter.java     | 134 +++
 .../envelope/NoBufferAvailableException.java    |  53 ++
 .../io/network/tcp/IncomingConnection.java      | 115 +++
 .../network/tcp/IncomingConnectionThread.java   | 226 +++++
 .../io/network/tcp/OutgoingConnection.java      | 529 ++++++++++++
 .../network/tcp/OutgoingConnectionThread.java   | 276 +++++++
 .../AdaptiveSpanningRecordDeserializer.java     | 521 ++++++++++++
 .../io/serialization/DataInputDeserializer.java | 307 +++++++
 .../io/serialization/DataOutputSerializer.java  | 259 ++++++
 .../io/serialization/RecordDeserializer.java    |  56 ++
 .../io/serialization/RecordSerializer.java      |  60 ++
 .../serialization/SpanningRecordSerializer.java | 153 ++++
 .../eu/stratosphere/nephele/AbstractIDTest.java |  62 ++
 .../ChannelDeploymentDescriptorTest.java        |   2 +-
 .../GateDeploymentDescriptorTest.java           |  12 +-
 .../TaskDeploymentDescriptorTest.java           |   2 +-
 .../executiongraph/ExecutionGraphTest.java      |  30 +-
 .../ForwardTask1Input1Output.java               |   9 +-
 .../ForwardTask1Input2Outputs.java              |  14 +-
 .../ForwardTask2Inputs1Output.java              |   9 +-
 .../executiongraph/SelfCrossForwardTask.java    |   6 +-
 .../executiongraph/SelfCrossInputTask.java      |   6 +-
 .../stratosphere/nephele/fs/LineReaderTest.java |  79 --
 .../nephele/fs/s3/S3FileSystemTest.java         | 461 -----------
 .../stratosphere/nephele/io/AbstractIDTest.java |  62 --
 .../nephele/io/DefaultChannelSelectorTest.java  |  46 --
 .../nephele/io/channels/BufferTest.java         | 192 -----
 .../nephele/io/channels/MemoryBufferTest.java   | 155 ----
 .../io/channels/serialization/BooleanType.java  |  77 --
 .../channels/serialization/ByteArrayType.java   |  88 --
 .../serialization/ByteSubArrayType.java         |  98 ---
 .../io/channels/serialization/ByteType.java     |  77 --
 .../io/channels/serialization/CharType.java     |  78 --
 .../serialization/DeSerializerTest.java         | 299 -------
 .../io/channels/serialization/DoubleType.java   |  78 --
 .../io/channels/serialization/FloatType.java    |  77 --
 .../io/channels/serialization/IntType.java      |  77 --
 .../io/channels/serialization/LongType.java     |  77 --
 .../serialization/SerializationTestType.java    |  52 --
 .../io/channels/serialization/ShortType.java    |  77 --
 .../channels/serialization/UTFStringType.java   |  86 --
 .../serialization/UnsignedByteType.java         |  77 --
 .../serialization/UnsignedShortType.java        |  77 --
 .../io/library/FileLineReadWriteTest.java       | 134 ---
 .../nephele/jobmanager/DoubleSourceTask.java    |  11 +-
 .../nephele/jobmanager/DoubleTargetTask.java    |  10 +-
 .../nephele/jobmanager/ExceptionTask.java       |   6 +-
 .../nephele/jobmanager/ForwardTask.java         |  10 +-
 .../nephele/jobmanager/JobManagerITCase.java    |  76 +-
 .../nephele/jobmanager/UnionTask.java           |  12 +-
 .../scheduler/queue/QueueSchedulerTest.java     |  10 +-
 .../managementgraph/ManagementGraphTest.java    |   6 +-
 .../profiling/impl/InstanceProfilerTest.java    |   2 +-
 .../DefaultDeserializerTest.java                | 358 --------
 .../transferenvelope/DefaultSerializerTest.java | 313 -------
 .../nephele/util/BufferPoolConnector.java       |  53 --
 .../nephele/util/DiscardingRecycler.java        |  24 +
 .../nephele/util/FileLineReader.java            |  80 ++
 .../nephele/util/FileLineWriter.java            |  75 ++
 .../nephele/util/TestBufferProvider.java        |  76 ++
 .../pact/runtime/task/DataSinkTaskTest.java     |  36 +-
 .../pact/runtime/task/DataSourceTaskTest.java   |  28 +-
 .../runtime/task/chaining/ChainTaskTest.java    |  11 +-
 .../runtime/task/util/OutputEmitterTest.java    |   1 +
 .../task/util/RecordOutputEmitterTest.java      |   2 +
 .../pact/runtime/test/util/MockEnvironment.java | 169 ++--
 .../pact/runtime/test/util/TaskTestBase.java    |   4 +-
 .../stratosphere/runtime/fs/LineReaderTest.java |  78 ++
 .../runtime/fs/s3/S3FileSystemTest.java         | 460 +++++++++++
 .../runtime/io/DefaultChannelSelectorTest.java  |  47 ++
 .../io/library/FileLineReadWriteTest.java       | 136 ++++
 .../envelope/EnvelopeReaderWriterTest.java      | 394 +++++++++
 .../DataInputOutputSerializerTest.java          | 115 +++
 .../io/serialization/PagedViewsTest.java        | 160 ++++
 .../SpanningRecordSerializationTest.java        | 164 ++++
 .../SpanningRecordSerializerTest.java           | 219 +++++
 .../io/serialization/types/AsciiStringType.java |  77 ++
 .../io/serialization/types/BooleanType.java     |  67 ++
 .../io/serialization/types/ByteArrayType.java   |  76 ++
 .../serialization/types/ByteSubArrayType.java   |  91 +++
 .../io/serialization/types/ByteType.java        |  67 ++
 .../io/serialization/types/CharType.java        |  68 ++
 .../io/serialization/types/DoubleType.java      |  68 ++
 .../io/serialization/types/FloatType.java       |  67 ++
 .../runtime/io/serialization/types/IntType.java |  67 ++
 .../io/serialization/types/LongType.java        |  67 ++
 .../types/SerializationTestType.java            |  26 +
 .../types/SerializationTestTypeFactory.java     |  40 +
 .../io/serialization/types/ShortType.java       |  67 ++
 .../serialization/types/UnsignedByteType.java   |  67 ++
 .../serialization/types/UnsignedShortType.java  |  67 ++
 .../runtime/io/serialization/types/Util.java    |  90 ++
 .../BroadcastVarsNepheleITCase.java             |   4 +-
 .../KMeansIterativeNepheleITCase.java           |   3 +-
 .../test/iterative/nephele/JobGraphUtils.java   |   4 +-
 .../CustomCompensatableDanglingPageRank.java    |  10 +-
 ...mpensatableDanglingPageRankWithCombiner.java |  10 +-
 .../CompensatableDanglingPageRank.java          |  10 +-
 .../query1Util/LineItemFilterTest.java          |   6 +-
 357 files changed, 12871 insertions(+), 20521 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-addons/spargel/src/main/java/eu/stratosphere/spargel/java/record/SpargelIteration.java
----------------------------------------------------------------------
diff --git a/stratosphere-addons/spargel/src/main/java/eu/stratosphere/spargel/java/record/SpargelIteration.java b/stratosphere-addons/spargel/src/main/java/eu/stratosphere/spargel/java/record/SpargelIteration.java
index 74bcaf3..14dcb35 100644
--- a/stratosphere-addons/spargel/src/main/java/eu/stratosphere/spargel/java/record/SpargelIteration.java
+++ b/stratosphere-addons/spargel/src/main/java/eu/stratosphere/spargel/java/record/SpargelIteration.java
@@ -155,6 +155,7 @@ public class SpargelIteration {
 
 		@Override
 		public void coGroup(Iterator<Record> messages, Iterator<Record> vertex, Collector<Record> out) throws Exception {
+
 			if (vertex.hasNext()) {
 				Record first = vertex.next();
 				first.getFieldInto(0, vertexKey);
@@ -270,4 +271,4 @@ public class SpargelIteration {
 			this.messagingFunction.postSuperstep();
 		}
 	}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 80ae308..53b4cc1 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
@@ -52,8 +52,9 @@ import eu.stratosphere.compiler.plan.WorksetPlanNode;
 import eu.stratosphere.configuration.ConfigConstants;
 import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.configuration.GlobalConfiguration;
-import eu.stratosphere.nephele.io.DistributionPattern;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.configuration.ConfigConstants;
+import eu.stratosphere.nephele.jobgraph.DistributionPattern;
+import eu.stratosphere.runtime.io.channels.ChannelType;
 import eu.stratosphere.nephele.jobgraph.AbstractJobOutputVertex;
 import eu.stratosphere.nephele.jobgraph.AbstractJobVertex;
 import eu.stratosphere.nephele.jobgraph.JobGraph;
@@ -1224,7 +1225,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 			
 			// connect the fake tail
 			try {
-				rootOfStepFunctionVertex.connectTo(fakeTail, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+				rootOfStepFunctionVertex.connectTo(fakeTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
 			} catch (JobGraphDefinitionException e) {
 				throw new CompilerException("Bug: Cannot connect iteration tail vertex fake tail task");
 			}
@@ -1267,7 +1268,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 		
 			// connect the fake tail
 			try {
-				rootOfTerminationCriterionVertex.connectTo(fakeTailTerminationCriterion, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+				rootOfTerminationCriterionVertex.connectTo(fakeTailTerminationCriterion, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
 			} catch (JobGraphDefinitionException e) {
 				throw new CompilerException("Bug: Cannot connect iteration tail vertex fake tail task for termination criterion");
 			}
@@ -1401,7 +1402,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 					
 					// connect the fake tail
 					try {
-						nextWorksetVertex.connectTo(fakeTail, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+						nextWorksetVertex.connectTo(fakeTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
 					} catch (JobGraphDefinitionException e) {
 						throw new CompilerException("Bug: Cannot connect iteration tail vertex fake tail task");
 					}
@@ -1440,7 +1441,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 					
 					// connect the fake tail
 					try {
-						solutionDeltaVertex.connectTo(fakeTail, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+						solutionDeltaVertex.connectTo(fakeTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
 					} catch (JobGraphDefinitionException e) {
 						throw new CompilerException("Bug: Cannot connect iteration tail vertex fake tail task");
 					}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 51c0a2f..3b9ba3d 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/configuration/ConfigConstants.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/configuration/ConfigConstants.java
@@ -344,12 +344,6 @@ public final class ConfigConstants {
 	 */
 	public static final int DEFAULT_FS_STREAM_OPENING_TIMEOUT = 0;
 	
-	/**
-	 * The config parameter defining whether to use the special multicast logic
-	 * for broadcasts.
-	 */
-	public static final boolean DEFAULT_USE_MULTICAST_FOR_BROADCAST = false;
-	
 	
 	// ------------------------ File System Bahavior ------------------------
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-core/src/main/java/eu/stratosphere/core/memory/MemorySegment.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/core/memory/MemorySegment.java b/stratosphere-core/src/main/java/eu/stratosphere/core/memory/MemorySegment.java
index 2698992..2d63fa4 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/core/memory/MemorySegment.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/core/memory/MemorySegment.java
@@ -103,6 +103,11 @@ public class MemorySegment {
 	public final boolean isFreed() {
 		return this.memory == null;
 	}
+
+	public final void free() {
+		this.wrapper = null;
+		this.memory = null;
+	}
 	
 	/**
 	 * Gets the size of the memory segment, in bytes. Because segments

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-core/src/main/java/eu/stratosphere/types/Record.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/types/Record.java b/stratosphere-core/src/main/java/eu/stratosphere/types/Record.java
index 73671fa..5b06547 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/types/Record.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/types/Record.java
@@ -70,7 +70,7 @@ public final class Record implements Value, CopyableValue<Record> {
 	
 	private Value[] writeFields;		// the cache for objects into which the binary representations are read
 	
-	private int binaryLen;				// the length of the contents in the binary buffer that is valid
+	public int binaryLen;				// the length of the contents in the binary buffer that is valid
 	
 	private int numFields;				// the number of fields in the record
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/AbstractID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/AbstractID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/AbstractID.java
new file mode 100644
index 0000000..476e22a
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/AbstractID.java
@@ -0,0 +1,177 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.util.StringUtils;
+
+/**
+ * A statistically unique identification number.
+ */
+public class AbstractID implements IOReadableWritable {
+
+	/** The size of a long in bytes */
+	private static final int SIZE_OF_LONG = 8;
+
+	/** The size of the ID in byte */
+	public static final int SIZE = 2 * SIZE_OF_LONG;
+
+	/** The upper part of the actual ID */
+	private long upperPart;
+
+	/** The lower part of the actual ID */
+	private long lowerPart;
+
+	/**
+	 * Constructs a new ID with a specific bytes value.
+	 */
+	public AbstractID(byte[] bytes) {
+		if (bytes.length != SIZE) {
+			throw new IllegalArgumentException("Argument bytes must by an array of " + SIZE + " bytes");
+		}
+
+		this.lowerPart = byteArrayToLong(bytes, 0);
+		this.upperPart = byteArrayToLong(bytes, SIZE_OF_LONG);
+	}
+
+	/**
+	 * Constructs a new abstract ID.
+	 *
+	 * @param lowerPart the lower bytes of the ID
+	 * @param upperPart the higher bytes of the ID
+	 */
+	public AbstractID(long lowerPart, long upperPart) {
+		this.lowerPart = lowerPart;
+		this.upperPart = upperPart;
+	}
+
+	/**
+	 * Creates a new abstract ID from the given one.
+	 * <p>
+	 * The given and the newly created abstract ID will be identical, i.e. a comparison by <code>equals</code> will
+	 * return <code>true</code> and both objects will have the same hash code.
+	 *
+	 * @param id the abstract ID to copy
+	 */
+	public AbstractID(AbstractID id) {
+		this.lowerPart = id.lowerPart;
+		this.upperPart = id.upperPart;
+	}
+
+	/**
+	 * Constructs a new random ID from a uniform distribution.
+	 */
+	public AbstractID() {
+		this.lowerPart = generateRandomLong();
+		this.upperPart = generateRandomLong();
+	}
+
+	/**
+	 * Generates a uniformly distributed random positive long.
+	 *
+	 * @return a uniformly distributed random positive long
+	 */
+	protected static long generateRandomLong() {
+		return (long) (Math.random() * Long.MAX_VALUE);
+	}
+
+	/**
+	 * Converts the given byte array to a long.
+	 *
+	 * @param ba the byte array to be converted
+	 * @param offset the offset indicating at which byte inside the array the conversion shall begin
+	 * @return the long variable
+	 */
+	private static long byteArrayToLong(byte[] ba, int offset) {
+		long l = 0;
+
+		for (int i = 0; i < SIZE_OF_LONG; ++i) {
+			l |= (ba[offset + SIZE_OF_LONG - 1 - i] & 0xffL) << (i << 3);
+		}
+
+		return l;
+	}
+
+	/**
+	 * Converts a long to a byte array.
+	 *
+	 * @param l the long variable to be converted
+	 * @param ba the byte array to store the result the of the conversion
+	 * @param offset offset indicating at what position inside the byte array the result of the conversion shall be stored
+	 */
+	private static void longToByteArray(final long l, final byte[] ba, final int offset) {
+		for (int i = 0; i < SIZE_OF_LONG; ++i) {
+			final int shift = i << 3; // i * 8
+			ba[offset + SIZE_OF_LONG - 1 - i] = (byte) ((l & (0xffL << shift)) >>> shift);
+		}
+	}
+
+	/**
+	 * Sets an ID from another ID by copying its internal byte representation.
+	 *
+	 * @param src source ID
+	 */
+	public void setID(AbstractID src) {
+		this.lowerPart = src.lowerPart;
+		this.upperPart = src.upperPart;
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj != null && obj instanceof AbstractID) {
+			AbstractID src = (AbstractID) obj;
+			return src.lowerPart == this.lowerPart && src.upperPart == this.upperPart;
+		} else {
+			return false;
+		}
+	}
+
+	@Override
+	public int hashCode() {
+		return ((int)  this.lowerPart) ^
+				((int) (this.lowerPart >>> 32)) ^
+				((int)  this.upperPart) ^
+				((int) (this.upperPart >>> 32));
+	}
+
+	@Override
+	public void read(DataInput in) throws IOException {
+		this.lowerPart = in.readLong();
+		this.upperPart = in.readLong();
+	}
+
+	@Override
+	public void write(DataOutput out) throws IOException {
+		out.writeLong(this.lowerPart);
+		out.writeLong(this.upperPart);
+	}
+
+	public void write(ByteBuffer buffer) {
+		buffer.putLong(this.lowerPart);
+		buffer.putLong(this.upperPart);
+	}
+
+	@Override
+	public String toString() {
+		final byte[] ba = new byte[SIZE];
+		longToByteArray(this.lowerPart, ba, 0);
+		longToByteArray(this.upperPart, ba, SIZE_OF_LONG);
+		return StringUtils.byteToHexString(ba);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/ForceCheckpoint.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/ForceCheckpoint.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/ForceCheckpoint.java
deleted file mode 100644
index 012dc02..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/ForceCheckpoint.java
+++ /dev/null
@@ -1,30 +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.annotations;
-
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Retention;
-import java.lang.annotation.RetentionPolicy;
-import java.lang.annotation.Target;
-
-/**
- * Annotation to force a checkpoint decision.
- * 
- */
-@Retention(RetentionPolicy.RUNTIME)
-@Target(ElementType.TYPE)
-public @interface ForceCheckpoint {
-
-	boolean checkpoint();
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/Stateful.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/Stateful.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/Stateful.java
deleted file mode 100644
index 433478b..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/Stateful.java
+++ /dev/null
@@ -1,28 +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.annotations;
-
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Retention;
-import java.lang.annotation.RetentionPolicy;
-import java.lang.annotation.Target;
-
-/**
- * This annotation can be used to indicate a stateful task.
- * 
- */
-@Retention(RetentionPolicy.RUNTIME)
-@Target(ElementType.TYPE)
-public @interface Stateful {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/Stateless.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/Stateless.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/Stateless.java
deleted file mode 100644
index b3d5e38..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/Stateless.java
+++ /dev/null
@@ -1,28 +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.annotations;
-
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Retention;
-import java.lang.annotation.RetentionPolicy;
-import java.lang.annotation.Target;
-
-/**
- * This annotation can be used to indicate a stateless task.
- * 
- */
-@Retention(RetentionPolicy.RUNTIME)
-@Target(ElementType.TYPE)
-public @interface Stateless {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/TaskAnnotation.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/TaskAnnotation.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/TaskAnnotation.java
deleted file mode 100644
index 317b112..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/TaskAnnotation.java
+++ /dev/null
@@ -1,29 +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.annotations;
-
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Retention;
-import java.lang.annotation.RetentionPolicy;
-import java.lang.annotation.Target;
-
-@Retention(RetentionPolicy.RUNTIME)
-@Target(ElementType.CONSTRUCTOR)
-public @interface TaskAnnotation {
-
-	boolean statefull();
-	double selektivity() default 1;
-	
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/ChannelDeploymentDescriptor.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/ChannelDeploymentDescriptor.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/ChannelDeploymentDescriptor.java
index c39aad0..4068e5b 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/ChannelDeploymentDescriptor.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/ChannelDeploymentDescriptor.java
@@ -18,7 +18,7 @@ import java.io.DataOutput;
 import java.io.IOException;
 
 import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.channels.ChannelID;
 
 /**
  * A channel deployment descriptor contains all the information necessary to deploy either an input or an output channel

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/GateDeploymentDescriptor.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/GateDeploymentDescriptor.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/GateDeploymentDescriptor.java
index 8a78154..02d6578 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/GateDeploymentDescriptor.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/GateDeploymentDescriptor.java
@@ -21,8 +21,8 @@ import java.util.Iterator;
 import java.util.List;
 
 import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.io.GateID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.gates.GateID;
+import eu.stratosphere.runtime.io.channels.ChannelType;
 import eu.stratosphere.nephele.util.EnumUtils;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/EventExample.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/EventExample.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/EventExample.java
deleted file mode 100644
index 4e6b2e4..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/EventExample.java
+++ /dev/null
@@ -1,95 +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.
- **********************************************************************************************************************/
-
-/*
- *  Copyright 2010 casp.
- * 
- *  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.
- *  under the License.
- */
-package eu.stratosphere.nephele.example.events;
-
-import java.io.IOException;
-
-import eu.stratosphere.configuration.ConfigConstants;
-import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.core.fs.Path;
-import eu.stratosphere.nephele.client.JobClient;
-import eu.stratosphere.nephele.client.JobSubmissionResult;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-import eu.stratosphere.nephele.io.library.FileLineReader;
-import eu.stratosphere.nephele.io.library.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.JobTaskVertex;
-
-/**
- */
-public class EventExample {
-
-	public static void main(String[] args) {
-
-		JobGraph jobGraph = new JobGraph("Grep Example Job");
-
-		JobFileInputVertex input = new JobFileInputVertex("Input 1", jobGraph);
-		input.setFileInputClass(FileLineReader.class);
-		input.setFilePath(new Path("file:///Users/casp/test2.txt"));
-
-		JobTaskVertex task1 = new JobTaskVertex("Task 1", jobGraph);
-		task1.setTaskClass(EventSender.class);
-
-		JobTaskVertex task2 = new JobTaskVertex("Task 2", jobGraph);
-		task2.setTaskClass(EventReceiver.class);
-
-		JobFileOutputVertex output = new JobFileOutputVertex("Output 1", jobGraph);
-		output.setFileOutputClass(FileLineWriter.class);
-		output.setFilePath(new Path("file:///Users/casp/output.txt"));
-
-		jobGraph.addJar(new Path("file:///Users/casp/EventTask.jar"));
-		jobGraph.addJar(new Path("file:///Users/casp/StringTaskEvent.jar"));
-		try {
-
-			input.connectTo(task1, ChannelType.INMEMORY);
-			task1.connectTo(task2, ChannelType.INMEMORY);
-			task2.connectTo(output, ChannelType.INMEMORY);
-
-		} catch (JobGraphDefinitionException e) {
-			e.printStackTrace();
-		}
-
-		Configuration conf = new Configuration();
-		conf.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "127.0.0.1");
-		conf.setString(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, "6023");
-
-		try {
-			JobClient jobClient = new JobClient(jobGraph, conf);
-			JobSubmissionResult result = jobClient.submitJob();
-			System.out.println(result.getDescription());
-		} catch (IOException ioe) {
-			ioe.printStackTrace();
-		}
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/EventReceiver.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/EventReceiver.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/EventReceiver.java
deleted file mode 100644
index 3e5d322..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/EventReceiver.java
+++ /dev/null
@@ -1,49 +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.example.events;
-
-import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.event.task.StringTaskEvent;
-import eu.stratosphere.nephele.io.RecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
-import eu.stratosphere.nephele.template.AbstractTask;
-
-/**
- */
-public class EventReceiver extends AbstractTask {
-
-	// this is just a dummy input gate...
-	private RecordReader<StringRecord> input = null;
-
-	private RecordWriter<StringRecord> output = null;
-
-	@Override
-	public void registerInputOutput() {
-		this.input = new RecordReader<StringRecord>(this, StringRecord.class);
-		this.output = new RecordWriter<StringRecord>(this, StringRecord.class);
-		this.input.subscribeToEvent(new MyEventListener(), StringTaskEvent.class);
-	}
-
-	@Override
-	public void invoke() throws Exception {
-
-		while (this.input.hasNext()) {
-
-			StringRecord s = input.next();
-			this.output.emit(s);
-		}
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/EventSender.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/EventSender.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/EventSender.java
deleted file mode 100644
index 4961dac..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/EventSender.java
+++ /dev/null
@@ -1,68 +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.example.events;
-
-/*
- *  Copyright 2010 casp.
- *
- *  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.
- *  under the License.
- */
-
-import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.event.task.StringTaskEvent;
-import eu.stratosphere.nephele.io.RecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
-import eu.stratosphere.nephele.template.AbstractTask;
-
-/**
- */
-public class EventSender extends AbstractTask {
-
-	// this is just a dummy output gate...
-	private RecordReader<StringRecord> input = null;
-
-	private RecordWriter<StringRecord> output = null;
-
-	@Override
-	public void registerInputOutput() {
-		this.input = new RecordReader<StringRecord>(this, StringRecord.class);
-		this.output = new RecordWriter<StringRecord>(this, StringRecord.class);
-	}
-
-	@Override
-	public void invoke() throws Exception {
-
-		int i = 1;
-		while (this.input.hasNext()) {
-			i++;
-			if (i % 1000 == 0) {
-				this.output.publishEvent(new StringTaskEvent("this is the " + i + "th message"));
-			}
-			StringRecord s = input.next();
-			this.output.emit(s);
-		}
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/MyEventListener.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/MyEventListener.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/MyEventListener.java
deleted file mode 100644
index f285d44..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/MyEventListener.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.
- **********************************************************************************************************************/
-
-/*
- *  Copyright 2010 casp.
- * 
- *  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.
- *  under the License.
- */
-package eu.stratosphere.nephele.example.events;
-
-import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
-import eu.stratosphere.nephele.event.task.EventListener;
-import eu.stratosphere.nephele.event.task.StringTaskEvent;
-
-/**
- */
-public class MyEventListener implements EventListener {
-
-	@Override
-	public void eventOccurred(AbstractTaskEvent event) {
-		if (event instanceof StringTaskEvent) {
-			System.out.println("Message Event received: " + ((StringTaskEvent) event).getString());
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/grep/Grep.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/grep/Grep.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/grep/Grep.java
deleted file mode 100644
index 9afef9b..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/grep/Grep.java
+++ /dev/null
@@ -1,108 +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.example.grep;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-
-import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.core.fs.Path;
-import eu.stratosphere.nephele.client.JobClient;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-import eu.stratosphere.nephele.io.library.FileLineReader;
-import eu.stratosphere.nephele.io.library.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.JobTaskVertex;
-import eu.stratosphere.nephele.util.JarFileCreator;
-
-public class Grep {
-
-	public static void main(String[] args) {
-
-		JobGraph jobGraph = new JobGraph("Grep Example Job");
-
-		JobFileInputVertex input = new JobFileInputVertex("Input 1", jobGraph);
-		input.setFileInputClass(FileLineReader.class);
-		input.setFilePath(new Path("file:///home/ec2-user/test.txt"));
-		input.setInstanceType("t1.micro");
-		
-		JobTaskVertex task1 = new JobTaskVertex("Task 1", jobGraph);
-		task1.setTaskClass(GrepTask.class);
-		task1.setInstanceType("t1.micro");
-
-		
-		JobFileOutputVertex output = new JobFileOutputVertex("Output 1", jobGraph);
-		output.setFileOutputClass(FileLineWriter.class);
-		output.setFilePath(new Path("file:///tmp/"));
-		output.setInstanceType("t1.micro");
-
-		try {
-
-			input.connectTo(task1, ChannelType.INMEMORY);
-			task1.connectTo(output, ChannelType.INMEMORY);
-
-		} catch (JobGraphDefinitionException e) {
-			e.printStackTrace();
-		}
-
-		// Create jar file and attach it
-		final File jarFile = new File("/tmp/grepJob.jar");
-		final JarFileCreator jarFileCreator = new JarFileCreator(jarFile);
-		jarFileCreator.addClass(GrepTask.class);
-
-		try {
-			jarFileCreator.createJarFile();
-			System.out.println("done creating!!");
-		} catch (IOException ioe) {
-
-			if (jarFile.exists()) {
-				jarFile.delete();
-			}
-
-			System.out.println("ERROR creating jar");
-			return;
-		}
-
-		jobGraph.addJar(new Path("file://" + jarFile.getAbsolutePath()));
-
-		// Submit job
-		Configuration conf = new Configuration();
-
-		jobGraph.getJobConfiguration().setString("job.cloud.awsaccessid", "xxx");
-		jobGraph.getJobConfiguration().setString("job.cloud.awssecretkey", "xxx");
-		jobGraph.getJobConfiguration().setString("job.cloud.sshkeypair", "caspeu");
-		jobGraph.getJobConfiguration().setString("job.ec2.image.id", "ami-d64474a2");		
-
-		InetSocketAddress jobmanager = new InetSocketAddress("127.0.0.1", 6123);
-		
-		
-		try {
-			final JobClient jobClient = new JobClient(jobGraph, conf, jobmanager);
-			System.out.println("submitting");
-			jobClient.submitJobAndWait();
-			System.out.println("done.");
-		} catch (Exception e) {
-			System.out.println(e);
-		}
-
-		if (jarFile.exists()) {
-			jarFile.delete();
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/grep/GrepTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/grep/GrepTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/grep/GrepTask.java
deleted file mode 100644
index ae72f1c..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/grep/GrepTask.java
+++ /dev/null
@@ -1,47 +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.example.grep;
-
-import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.RecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
-import eu.stratosphere.nephele.template.AbstractTask;
-
-public class GrepTask extends AbstractTask {
-
-	private RecordReader<StringRecord> input = null;
-
-	private RecordWriter<StringRecord> output = null;
-
-	private int i = 0;
-
-	@Override
-	public void invoke() throws Exception {
-
-		while (this.input.hasNext()) {
-
-			StringRecord s = input.next();
-			this.output.emit(s);
-			i++;
-		}
-
-		System.out.println("GREP: Emmited all " + i + " records");
-	}
-
-	@Override
-	public void registerInputOutput() {
-		this.input = new RecordReader<StringRecord>(this, StringRecord.class);
-		this.output = new RecordWriter<StringRecord>(this, StringRecord.class);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTest.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTest.java
deleted file mode 100644
index f85024c..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTest.java
+++ /dev/null
@@ -1,224 +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.example.speedtest;
-
-import java.io.File;
-import java.io.IOException;
-
-import eu.stratosphere.api.common.JobExecutionResult;
-import eu.stratosphere.configuration.ConfigConstants;
-import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.core.fs.Path;
-import eu.stratosphere.nephele.client.JobClient;
-import eu.stratosphere.nephele.client.JobExecutionException;
-import eu.stratosphere.nephele.io.DistributionPattern;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-import eu.stratosphere.nephele.jobgraph.JobGenericInputVertex;
-import eu.stratosphere.nephele.jobgraph.JobGenericOutputVertex;
-import eu.stratosphere.nephele.jobgraph.JobGraph;
-import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException;
-import eu.stratosphere.nephele.jobgraph.JobTaskVertex;
-import eu.stratosphere.nephele.util.JarFileCreator;
-
-/**
- * This class implements a speed test for Nephele. It's primary purpose is to benchmark the performance of Nephele's
- * network channels with different degrees of parallelism.
- * 
- */
-public final class SpeedTest {
-
-	/**
-	 * Configuration key to specify the amount of data to be send in GB.
-	 */
-	static final String DATA_VOLUME_CONFIG_KEY = "data.volume";
-
-	/**
-	 * Entry point to the application.
-	 * 
-	 * @param args
-	 *        the provided arguments
-	 */
-	public static void main(final String[] args) {
-
-		// Parse the arguments first
-		if (args.length < 4) {
-			System.err
-				.println("Insufficient number of arguments. Please provide <job manager address> <amount of data to send in GB> <number of subtasks> <number of subtasks per task> (<use forwarder>)");
-			System.exit(1);
-			return;
-		}
-
-		// Determine the job manager address
-		final String jobManagerAddress = args[0];
-
-		// Determine amount of data to send in GB
-		int amountOfDataToSend = 0;
-		try {
-			amountOfDataToSend = Integer.parseInt(args[1]);
-		} catch (NumberFormatException e) {
-			System.err.println("Cannot parse amount of data to send. Please provide a positive integer value.");
-			System.exit(1);
-			return;
-		}
-
-		if (amountOfDataToSend <= 0 || amountOfDataToSend > 1024) {
-			System.err
-				.println("Please provide an integer value between 1 and 1024 indicating the amount of data to send in GB.");
-			System.exit(1);
-			return;
-		}
-
-		// Determine the number of subtasks
-		int numberOfSubtasks = 0;
-		try {
-			numberOfSubtasks = Integer.parseInt(args[2]);
-		} catch (NumberFormatException e) {
-			System.err.println("Cannot parse the number of subtasks. Please provide a positive integer value.");
-			System.exit(1);
-			return;
-		}
-
-		if (numberOfSubtasks <= 0) {
-			System.err.println("Please provide a positive integer value indicating the number of subtasks.");
-			System.exit(1);
-			return;
-		}
-
-		// Determine the number of subtasks per instance
-		int numberOfSubtasksPerInstance = 0;
-		try {
-			numberOfSubtasksPerInstance = Integer.parseInt(args[3]);
-		} catch (NumberFormatException e) {
-			System.err
-				.println("Cannot parse the number of subtasks per instance. Please provide a positive integer value.");
-			System.exit(1);
-			return;
-		}
-
-		if (numberOfSubtasksPerInstance <= 0) {
-			System.err
-				.println("Please provide a positive integer value indicating the number of subtasks per instance.");
-			System.exit(1);
-			return;
-		}
-
-		// Determine whether to use a forwarder or not
-		boolean useForwarder = false;
-		if (args.length >= 5) {
-			useForwarder = Boolean.parseBoolean(args[4]);
-		}
-
-		final JobGraph jobGraph = new JobGraph("Nephele Speed Test");
-
-		final JobGenericInputVertex producer = new JobGenericInputVertex("Speed Test Producer", jobGraph);
-		producer.setInputClass(SpeedTestProducer.class);
-		producer.setNumberOfSubtasks(numberOfSubtasks);
-		producer.setNumberOfSubtasksPerInstance(numberOfSubtasksPerInstance);
-		producer.getConfiguration().setInteger(DATA_VOLUME_CONFIG_KEY, amountOfDataToSend);
-
-		JobTaskVertex forwarder = null;
-		if (useForwarder) {
-			forwarder = new JobTaskVertex("Speed Test Forwarder", jobGraph);
-			forwarder.setTaskClass(SpeedTestForwarder.class);
-			forwarder.setNumberOfSubtasks(numberOfSubtasks);
-			forwarder.setNumberOfSubtasksPerInstance(numberOfSubtasksPerInstance);
-		}
-
-		final JobGenericOutputVertex consumer = new JobGenericOutputVertex("Speed Test Consumer", jobGraph);
-		consumer.setOutputClass(SpeedTestConsumer.class);
-		consumer.setNumberOfSubtasks(numberOfSubtasks);
-		consumer.setNumberOfSubtasksPerInstance(numberOfSubtasksPerInstance);
-
-		// Set vertex sharing
-		producer.setVertexToShareInstancesWith(consumer);
-		if (forwarder != null) {
-			forwarder.setVertexToShareInstancesWith(consumer);
-		}
-
-		// Connect the vertices
-		try {
-			if (forwarder == null) {
-				producer.connectTo(consumer, ChannelType.NETWORK,
-					DistributionPattern.BIPARTITE);
-			} else {
-				producer.connectTo(forwarder, ChannelType.NETWORK,
-					DistributionPattern.BIPARTITE);
-				forwarder.connectTo(consumer, ChannelType.NETWORK,
-					DistributionPattern.BIPARTITE);
-			}
-
-		} catch (JobGraphDefinitionException e) {
-			e.printStackTrace();
-			System.exit(1);
-			return;
-		}
-
-		File jarFile = null;
-
-		try {
-
-			// Create jar file of job
-			jarFile = File.createTempFile("speedtest", "jar");
-			jarFile.deleteOnExit();
-
-			final JarFileCreator jfc = new JarFileCreator(jarFile);
-			jfc.addClass(SpeedTest.class);
-			jfc.addClass(SpeedTestProducer.class);
-			jfc.addClass(SpeedTestForwarder.class);
-			jfc.addClass(SpeedTestConsumer.class);
-			jfc.addClass(SpeedTestRecord.class);
-			jfc.createJarFile();
-
-			jobGraph.addJar(new Path("file://" + jarFile.getAbsolutePath()));
-
-			final Configuration clientConfiguration = new Configuration();
-			clientConfiguration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, jobManagerAddress);
-			clientConfiguration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY,
-				ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT);
-
-			final JobClient jobClient = new JobClient(jobGraph, clientConfiguration);
-			final JobExecutionResult jobResult = jobClient.submitJobAndWait();
-			final long executionTime = jobResult.getNetRuntime();
-
-			// Calculate throughput in MBit/s and output it
-			System.out.print("Job finished with a throughput of " + toMBitPerSecond(amountOfDataToSend, executionTime));
-
-		} catch (IOException ioe) {
-			ioe.printStackTrace();
-			System.exit(1);
-			return;
-		} catch (JobExecutionException jee) {
-			jee.printStackTrace();
-			System.exit(1);
-			return;
-		}
-	}
-
-	/**
-	 * Calculates the throughput in MBit/s from the given amount of data that has been sent and the execution time.
-	 * 
-	 * @param amountOfDataToSend
-	 *        the amount of data that has been sent in GB
-	 * @param executionTime
-	 *        the execution time in milliseconds
-	 * @return the resulting throughput in MBit/s
-	 */
-	private static int toMBitPerSecond(final int amountOfDataToSend, final long executionTime) {
-
-		final double dataVolumeInMBit = amountOfDataToSend * 8192.0;
-		final double executionTimeInSeconds = executionTime / 1000.0;
-
-		return (int) Math.round(dataVolumeInMBit / executionTimeInSeconds);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestConsumer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestConsumer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestConsumer.java
deleted file mode 100644
index eb40196..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestConsumer.java
+++ /dev/null
@@ -1,45 +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.example.speedtest;
-
-import eu.stratosphere.nephele.io.MutableRecordReader;
-import eu.stratosphere.nephele.template.AbstractOutputTask;
-
-/**
- * This class implements the consumer task of the speed test. The consumer task simply drops all received records.
- * 
- */
-public class SpeedTestConsumer extends AbstractOutputTask {
-
-	/**
-	 * The record reader used to read the incoming records.
-	 */
-	private MutableRecordReader<SpeedTestRecord> input;
-
-
-	@Override
-	public void registerInputOutput() {
-
-		this.input = new MutableRecordReader<SpeedTestRecord>(this);
-	}
-
-
-	@Override
-	public void invoke() throws Exception {
-
-		final SpeedTestRecord record = new SpeedTestRecord();
-		while (this.input.next(record)) {
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestForwarder.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestForwarder.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestForwarder.java
deleted file mode 100644
index 91090a8..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestForwarder.java
+++ /dev/null
@@ -1,54 +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.example.speedtest;
-
-import eu.stratosphere.nephele.io.MutableRecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
-import eu.stratosphere.nephele.template.AbstractTask;
-
-/**
- * This class implements a forwarder task for the speed test. The forwarder task immediately outputs every record it
- * reads.
- * 
- */
-public final class SpeedTestForwarder extends AbstractTask {
-
-	/**
-	 * The record reader used to read incoming records.
-	 */
-	private MutableRecordReader<SpeedTestRecord> input;
-
-	/**
-	 * The record writer used to forward incoming records.
-	 */
-	private RecordWriter<SpeedTestRecord> output;
-
-
-	@Override
-	public void registerInputOutput() {
-
-		this.input = new MutableRecordReader<SpeedTestRecord>(this);
-		this.output = new RecordWriter<SpeedTestRecord>(this, SpeedTestRecord.class);
-	}
-
-
-	@Override
-	public void invoke() throws Exception {
-
-		final SpeedTestRecord record = new SpeedTestRecord();
-		while (this.input.next(record)) {
-			this.output.emit(record);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestProducer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestProducer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestProducer.java
deleted file mode 100644
index 78ea23d..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestProducer.java
+++ /dev/null
@@ -1,54 +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.example.speedtest;
-
-import eu.stratosphere.nephele.io.RecordWriter;
-import eu.stratosphere.nephele.template.AbstractGenericInputTask;
-
-/**
- * This class implements the producer task which produces test records for the speed test.
- * 
- */
-public final class SpeedTestProducer extends AbstractGenericInputTask {
-
-	/**
-	 * The record writer to emit the produced records.
-	 */
-	private RecordWriter<SpeedTestRecord> writer;
-
-
-	@Override
-	public void registerInputOutput() {
-
-		this.writer = new RecordWriter<SpeedTestRecord>(this, SpeedTestRecord.class);
-	}
-
-
-	@Override
-	public void invoke() throws Exception {
-
-		// Determine the amount of data to send per subtask
-		final int dataVolumePerSubtaskInMB = getTaskConfiguration().getInteger(SpeedTest.DATA_VOLUME_CONFIG_KEY, 1)
-			* 1024 / getCurrentNumberOfSubtasks();
-
-		final long numberOfRecordsToEmit = dataVolumePerSubtaskInMB * 1024 * 1024 / SpeedTestRecord.RECORD_SIZE;
-
-		final SpeedTestRecord record = new SpeedTestRecord();
-
-		for (long i = 0; i < numberOfRecordsToEmit; ++i) {
-			this.writer.emit(record);
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestRecord.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestRecord.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestRecord.java
deleted file mode 100644
index b3522f3..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestRecord.java
+++ /dev/null
@@ -1,55 +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.example.speedtest;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-
-/**
- * This class implements the record type used for the speed test.
- */
-public final class SpeedTestRecord implements IOReadableWritable {
-
-	/**
-	 * The size of a single record in bytes.
-	 */
-	static final int RECORD_SIZE = 128;
-
-	/**
-	 * The byte buffer which actually stored the record's data.
-	 */
-	private final byte[] buf = new byte[RECORD_SIZE];
-
-	/**
-	 * Constructs a new record and initializes it.
-	 */
-	public SpeedTestRecord() {
-		for (int i = 0; i < RECORD_SIZE; ++i) {
-			this.buf[i] = (byte) (i % 128);
-		}
-	}
-
-	@Override
-	public void write(final DataOutput out) throws IOException {
-		out.write(this.buf);
-	}
-
-	@Override
-	public void read(final DataInput in) throws IOException {
-		in.readFully(this.buf);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/ConsumerTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/ConsumerTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/ConsumerTask.java
deleted file mode 100644
index 209a81e..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/ConsumerTask.java
+++ /dev/null
@@ -1,44 +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.example.union;
-
-import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.RecordReader;
-import eu.stratosphere.nephele.template.AbstractFileOutputTask;
-
-public class ConsumerTask extends AbstractFileOutputTask {
-
-	private RecordReader<StringRecord> input;
-
-	@Override
-	public void registerInputOutput() {
-
-		this.input = new RecordReader<StringRecord>(this, StringRecord.class);
-
-	}
-
-	@Override
-	public void invoke() throws Exception {
-
-		int count = 0;
-
-		while (this.input.hasNext()) {
-			this.input.next();
-			++count;
-		}
-
-		System.out.println("Consumer receiver " + count + " records in total");
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/ProducerTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/ProducerTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/ProducerTask.java
deleted file mode 100644
index 647a07b..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/ProducerTask.java
+++ /dev/null
@@ -1,43 +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.example.union;
-
-import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.RecordWriter;
-import eu.stratosphere.nephele.template.AbstractFileInputTask;
-
-public class ProducerTask extends AbstractFileInputTask {
-
-	private static final int NUMBER_OF_RECORDS_TO_PRODUCE = 1000000;
-
-	private RecordWriter<StringRecord> output;
-
-	@Override
-	public void registerInputOutput() {
-
-		this.output = new RecordWriter<StringRecord>(this, StringRecord.class);
-	}
-
-	@Override
-	public void invoke() throws Exception {
-
-		for (int i = 0; i < NUMBER_OF_RECORDS_TO_PRODUCE; ++i) {
-
-			final StringRecord record = new StringRecord("Record " + i + " of " + this);
-			this.output.emit(record);
-		}
-
-	}
-
-}


[19/30] Offer buffer-oriented API for I/O (#25)

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 1ea5b1b..5240fc8 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
@@ -65,7 +65,6 @@ import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
 import eu.stratosphere.nephele.instance.HardwareDescription;
 import eu.stratosphere.nephele.instance.HardwareDescriptionFactory;
 import eu.stratosphere.nephele.instance.InstanceConnectionInfo;
-import eu.stratosphere.nephele.io.channels.ChannelID;
 import eu.stratosphere.nephele.ipc.RPC;
 import eu.stratosphere.nephele.ipc.Server;
 import eu.stratosphere.nephele.jobgraph.JobID;
@@ -80,12 +79,11 @@ import eu.stratosphere.nephele.protocols.TaskOperationProtocol;
 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.taskmanager.bytebuffered.ByteBufferedChannelManager;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.InsufficientResourcesException;
-import eu.stratosphere.nephele.taskmanager.runtime.ExecutorThreadFactory;
-import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTask;
 import eu.stratosphere.nephele.util.SerializableArrayList;
 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.util.StringUtils;
 
 /**
@@ -128,10 +126,10 @@ public class TaskManager implements TaskOperationProtocol {
 	private final InstanceConnectionInfo localInstanceConnectionInfo;
 
 	/**
-	 * The instance of the {@link ByteBufferedChannelManager} which is responsible for
+	 * The instance of the {@link eu.stratosphere.nephele.taskmanager.io.bytebuffered.ChannelManager} which is responsible for
 	 * setting up and cleaning up the byte buffered channels of the tasks.
 	 */
-	private final ByteBufferedChannelManager byteBufferedChannelManager;
+	private final ChannelManager channelManager;
 
 	/**
 	 * Instance of the task manager profile if profiling is enabled.
@@ -279,14 +277,24 @@ public class TaskManager implements TaskOperationProtocol {
 		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);
+
+		int bufferSize = GlobalConfiguration.getInteger(
+				ConfigConstants.TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY,
+				ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_BUFFER_SIZE);
+
 		// Initialize the byte buffered channel manager
+		ChannelManager channelManager = null;
 		try {
-			this.byteBufferedChannelManager = new ByteBufferedChannelManager(this.lookupService,
-				this.localInstanceConnectionInfo);
-		} catch (Exception e) {
-			LOG.fatal("Cannot create byte channel manager:" + e.getMessage(), e);
-			throw new Exception("Failed to instantiate Byte-buffered channel manager. " + e.getMessage(), e);
+			channelManager = new ChannelManager(this.lookupService, this.localInstanceConnectionInfo, numBuffers, bufferSize);
+		} catch (IOException ioe) {
+			LOG.error(StringUtils.stringifyException(ioe));
+			throw new Exception("Failed to instantiate Byte-buffered channel manager. " + ioe.getMessage(), ioe);
 		}
+		this.channelManager = channelManager;
 		
 		{
 			HardwareDescription resources = HardwareDescriptionFactory.extractFromSystem();
@@ -598,7 +606,6 @@ public class TaskManager implements TaskOperationProtocol {
 		return new TaskCancelResult(id, AbstractTaskResult.ReturnCode.SUCCESS);
 	}
 
-
 	@Override
 	public TaskKillResult killTask(final ExecutionVertexID id) throws IOException {
 
@@ -661,13 +668,11 @@ public class TaskManager implements TaskOperationProtocol {
 			}
 
 			final Configuration jobConfiguration = tdd.getJobConfiguration();
-			final Set<ChannelID> activeOutputChannels = null; // TODO: Fix me
 
 			// Register the task
 			Task task;
 			try {
-				task = createAndRegisterTask(vertexID, jobConfiguration, re,
-					activeOutputChannels);
+				task = createAndRegisterTask(vertexID, jobConfiguration, re);
 			} catch (InsufficientResourcesException e) {
 				final TaskSubmissionResult result = new TaskSubmissionResult(vertexID,
 					AbstractTaskResult.ReturnCode.INSUFFICIENT_RESOURCES);
@@ -707,12 +712,10 @@ public class TaskManager implements TaskOperationProtocol {
 	 *        the job configuration that has been attached to the original job graph
 	 * @param environment
 	 *        the environment of the task to be registered
-	 * @param activeOutputChannels
-	 *        the set of initially active output channels
 	 * @return the task to be started or <code>null</code> if a task with the same ID was already running
 	 */
 	private Task createAndRegisterTask(final ExecutionVertexID id, final Configuration jobConfiguration,
-			final RuntimeEnvironment environment, final Set<ChannelID> activeOutputChannels)
+			final RuntimeEnvironment environment)
 					throws InsufficientResourcesException, IOException {
 
 		if (id == null) {
@@ -730,10 +733,10 @@ public class TaskManager implements TaskOperationProtocol {
 			final Task runningTask = this.runningTasks.get(id);
 			boolean registerTask = true;
 			if (runningTask == null) {
-				task = new RuntimeTask(id, environment, this);
+				task = new Task(id, environment, this);
 			} else {
 
-				if (runningTask instanceof RuntimeTask) {
+				if (runningTask instanceof Task) {
 					// Task is already running
 					return null;
 				} else {
@@ -746,7 +749,7 @@ public class TaskManager implements TaskOperationProtocol {
 
 			if (registerTask) {
 				// Register the task with the byte buffered channel manager
-				this.byteBufferedChannelManager.register(task, activeOutputChannels);
+				this.channelManager.register(task);
 
 				boolean enableProfiling = false;
 				if (this.profiler != null && jobConfiguration.getBoolean(ProfilingUtils.PROFILE_JOB_KEY, true)) {
@@ -786,7 +789,7 @@ public class TaskManager implements TaskOperationProtocol {
 				this.fileCache.deleteTmpFile(e.getKey(), task.getJobID());
 			}
 			// Unregister task from the byte buffered channel manager
-			this.byteBufferedChannelManager.unregister(id, task);
+			this.channelManager.unregister(id, task);
 
 			// Unregister task from profiling
 			task.unregisterProfiler(this.profiler);
@@ -892,7 +895,7 @@ public class TaskManager implements TaskOperationProtocol {
 		}
 
 		// Shut down the network channel manager
-		this.byteBufferedChannelManager.shutdown();
+		this.channelManager.shutdown();
 
 		// Shut down the memory manager
 		if (this.ioManager != null) {
@@ -930,8 +933,9 @@ public class TaskManager implements TaskOperationProtocol {
 	}
 
 	@Override
-	public void logBufferUtilization() {
-		this.byteBufferedChannelManager.logBufferUtilization();
+	public void logBufferUtilization() throws IOException {
+
+		this.channelManager.logBufferUtilization();
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/BufferAvailabilityListener.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/BufferAvailabilityListener.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/BufferAvailabilityListener.java
deleted file mode 100644
index 49e774b..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/BufferAvailabilityListener.java
+++ /dev/null
@@ -1,28 +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.taskmanager.bufferprovider;
-
-/**
- * This interface must be implemented to receive a notification from a {@link BufferProvider} when an empty
- * {@link Buffer} has
- * become available again.
- * 
- */
-public interface BufferAvailabilityListener {
-
-	/**
-	 * Indicates that at least one {@link Buffer} has become available again.
-	 */
-	void bufferAvailable();
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/BufferProvider.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/BufferProvider.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/BufferProvider.java
deleted file mode 100644
index 1b47b57..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/BufferProvider.java
+++ /dev/null
@@ -1,82 +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.taskmanager.bufferprovider;
-
-import java.io.IOException;
-
-import eu.stratosphere.nephele.io.channels.Buffer;
-
-public interface BufferProvider {
-
-	/**
-	 * Requests an empty buffer with a minimum size of <code>minimumSizeOfBuffer</code>. The method returns
-	 * immediately, even if the request could not be fulfilled. Note that <code>minimumSizeOfBuffer</code> must not
-	 * exceed the value returned by the method <code>getMaximumBufferSize()</code>.
-	 * 
-	 * @param minimumSizeOfBuffer
-	 *        the minimum size of the requested read buffer in bytes
-	 * @return the buffer with at least the requested size or <code>null</code> if no such buffer is currently available
-	 * @throws IOException
-	 *         thrown if an I/O error occurs while allocating the buffer
-	 */
-	Buffer requestEmptyBuffer(int minimumSizeOfBuffer) throws IOException;
-
-	/**
-	 * Requests an empty buffer with a minimum size of <code>minimumSizeOfBuffer</code>. The method blocks
-	 * until the request can be fulfilled. Note that <code>minimumSizeOfBuffer</code> must not
-	 * exceed the value returned by the method <code>getMaximumBufferSize()</code>.
-	 * 
-	 * @param minimumSizeOfBuffer
-	 *        the minimum size of the requested read buffer in bytes
-	 * @return the buffer with at least the requested size
-	 * @throws IOException
-	 *         thrown if an I/O error occurs while allocating the buffer
-	 * @throws InterruptedException
-	 *         thrown if the thread waiting for the buffer is interrupted
-	 */
-	Buffer requestEmptyBufferBlocking(int minimumSizeOfBuffer) throws IOException,
-			InterruptedException;
-
-	/**
-	 * Returns the maximum buffer size in bytes available at this buffer provider.
-	 * 
-	 * @return the maximum buffer size in bytes available at this buffer provider
-	 */
-	int getMaximumBufferSize();
-
-	/**
-	 * Returns if this buffer provider is shared between different entities (for examples tasks).
-	 * 
-	 * @return <code>true</code> if this buffer provider is shared, <code>false</code> otherwise
-	 */
-	boolean isShared();
-
-	/**
-	 * Reports an asynchronous event. Calling this method interrupts each blocking method of the buffer provider and
-	 * allows the blocked thread to respond to the event.
-	 */
-	void reportAsynchronousEvent();
-
-	/**
-	 * Registers the given {@link BufferAvailabilityListener} with an empty buffer pool to receive a notification when
-	 * at least one buffer has become available again. After the notification, the listener is automatically
-	 * unregistered again.
-	 * 
-	 * @param bufferAvailabilityListener
-	 *        the listener to be registered
-	 * @return <code>true</code> if the registration has been successful or <code>false</code> if the registration
-	 *         failed because the buffer pool has not been empty or has already been destroyed
-	 */
-	boolean registerBufferAvailabilityListener(BufferAvailabilityListener bufferAvailabilityListener);
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/BufferProviderBroker.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/BufferProviderBroker.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/BufferProviderBroker.java
deleted file mode 100644
index 6474386..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/BufferProviderBroker.java
+++ /dev/null
@@ -1,24 +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.taskmanager.bufferprovider;
-
-import java.io.IOException;
-
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.jobgraph.JobID;
-
-public interface BufferProviderBroker {
-
-	BufferProvider getBufferProvider(JobID jobID, ChannelID sourceChannelID) throws IOException, InterruptedException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/GlobalBufferPool.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/GlobalBufferPool.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/GlobalBufferPool.java
deleted file mode 100644
index 0fc25eb..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/GlobalBufferPool.java
+++ /dev/null
@@ -1,135 +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.taskmanager.bufferprovider;
-
-import java.util.Queue;
-import java.util.concurrent.ArrayBlockingQueue;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.configuration.ConfigConstants;
-import eu.stratosphere.configuration.GlobalConfiguration;
-import eu.stratosphere.core.memory.MemorySegment;
-
-public final class GlobalBufferPool {
-
-	private final static Log LOG = LogFactory.getLog(GlobalBufferPool.class);
-	
-	/**
-	 * The singleton instance of the global buffer pool.
-	 */
-	private static GlobalBufferPool instance = null;
-
-	/**
-	 * The number of buffers created at startup.
-	 */
-	private final int numberOfBuffers;
-
-	/**
-	 * The size of read/write buffers in bytes.
-	 */
-	private final int bufferSizeInBytes;
-
-	private final Queue<MemorySegment> buffers;
-
-	/**
-	 * Returns the singleton instance of the global buffer pool. If the instance does not already exist, it is also
-	 * created by calling this method.
-	 * 
-	 * @return the singleton instance of the global buffer pool
-	 */
-	public static synchronized GlobalBufferPool getInstance() {
-
-		if (instance == null) {
-			instance = new GlobalBufferPool();
-		}
-
-		return instance;
-	}
-
-	/**
-	 * Constructs the global buffer pool.
-	 */
-	private GlobalBufferPool() {
-
-		this.numberOfBuffers = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
-			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS);
-		this.bufferSizeInBytes = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY,
-			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_BUFFER_SIZE);
-
-		this.buffers = new ArrayBlockingQueue<MemorySegment>(this.numberOfBuffers);
-
-		// Initialize buffers
-		for (int i = 0; i < this.numberOfBuffers; i++) {
-			// allocate byteBuffer
-			final byte[] segMemory = new byte[this.bufferSizeInBytes];
-			final MemorySegment readBuffer = new MemorySegment(segMemory);
-			this.buffers.add(readBuffer);
-		}
-
-		LOG.info("Initialized global buffer pool with " + this.numberOfBuffers + " buffers with a size "
-			+ this.bufferSizeInBytes + " bytes each");
-	}
-
-	/**
-	 * Returns the maximum size of a buffer available at this pool in bytes.
-	 * 
-	 * @return the maximum size of a buffer available at this pool in bytes
-	 */
-	public int getMaximumBufferSize() {
-
-		return this.bufferSizeInBytes;
-	}
-
-	/**
-	 * Locks a buffer from the global buffer pool and returns it to the caller of this method.
-	 * 
-	 * @return the locked buffer from the pool or <code>null</code> if currently no global buffer is available
-	 */
-	public MemorySegment lockGlobalBuffer() {
-
-		return this.buffers.poll();
-	}
-
-	/**
-	 * Releases a lock on a previously locked buffer and returns the buffer to the global pool.
-	 * 
-	 * @param releasedBuffer
-	 *        the previously locked buffer to be released
-	 */
-	public void releaseGlobalBuffer(final MemorySegment releasedBuffer) {
-		this.buffers.add(releasedBuffer);
-	}
-
-	/**
-	 * Returns the total number of buffers managed by this pool.
-	 * 
-	 * @return the total number of buffers managed by this pool
-	 */
-	public int getTotalNumberOfBuffers() {
-
-		return this.numberOfBuffers;
-	}
-
-	/**
-	 * Returns the number of buffers which are currently available at this pool.
-	 * 
-	 * @return the number of buffers which are currently available at this pool
-	 */
-	public int getCurrentNumberOfBuffers() {
-
-		return this.buffers.size();
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPool.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPool.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPool.java
deleted file mode 100644
index f296003..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPool.java
+++ /dev/null
@@ -1,287 +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.taskmanager.bufferprovider;
-
-import java.io.IOException;
-import java.util.ArrayDeque;
-import java.util.Queue;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.core.memory.MemorySegment;
-import eu.stratosphere.nephele.io.channels.Buffer;
-import eu.stratosphere.nephele.io.channels.BufferFactory;
-import eu.stratosphere.nephele.io.channels.MemoryBufferPoolConnector;
-
-public final class LocalBufferPool implements BufferProvider {
-
-	private static final class LocalBufferPoolConnector implements MemoryBufferPoolConnector {
-
-		private final LocalBufferPool localBufferPool;
-
-		private LocalBufferPoolConnector(final LocalBufferPool localBufferPool) {
-			this.localBufferPool = localBufferPool;
-		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public void recycle(final MemorySegment byteBuffer) {
-
-			this.localBufferPool.recycleBuffer(byteBuffer);
-		}
-
-	}
-
-	private final static Log LOG = LogFactory.getLog(LocalBufferPool.class);
-
-	private final GlobalBufferPool globalBufferPool;
-
-	private final int maximumBufferSize;
-
-	private int designatedNumberOfBuffers;
-
-	private int requestedNumberOfBuffers = 0;
-
-	private final boolean isShared;
-
-	private boolean asynchronousEventOccurred = false;
-
-	private boolean isDestroyed = false;
-
-	private final Queue<MemorySegment> buffers = new ArrayDeque<MemorySegment>();
-
-	private final LocalBufferPoolConnector bufferPoolConnector;
-
-	private final Queue<BufferAvailabilityListener> bufferAvailabilityListenerQueue = new ArrayDeque<BufferAvailabilityListener>();
-
-	public LocalBufferPool(final int designatedNumberOfBuffers, final boolean isShared) {
-
-		this.globalBufferPool = GlobalBufferPool.getInstance();
-		this.maximumBufferSize = this.globalBufferPool.getMaximumBufferSize();
-		this.designatedNumberOfBuffers = designatedNumberOfBuffers;
-		this.isShared = isShared;
-		this.bufferPoolConnector = new LocalBufferPoolConnector(this);
-	}
-
-
-	@Override
-	public Buffer requestEmptyBuffer(final int minimumSizeOfBuffer) throws IOException {
-
-		try {
-			return requestBufferInternal(minimumSizeOfBuffer, false);
-		} catch (InterruptedException e) {
-			LOG.error("Caught unexpected InterruptedException");
-		}
-
-		return null;
-	}
-
-
-	@Override
-	public Buffer requestEmptyBufferBlocking(final int minimumSizeOfBuffer) throws IOException, InterruptedException {
-
-		return requestBufferInternal(minimumSizeOfBuffer, true);
-	}
-
-	private Buffer requestBufferInternal(final int minimumSizeOfBuffer, final boolean block) throws IOException,
-			InterruptedException {
-
-		if (minimumSizeOfBuffer > this.maximumBufferSize) {
-			throw new IllegalArgumentException("Buffer of " + minimumSizeOfBuffer
-				+ " bytes is requested, but maximum buffer size is " + this.maximumBufferSize);
-		}
-
-		while (true) {
-
-			boolean async = false;
-
-			synchronized (this.buffers) {
-
-				// Make sure we return excess buffers immediately
-				while (this.requestedNumberOfBuffers > this.designatedNumberOfBuffers) {
-
-					final MemorySegment seg = this.buffers.poll();
-					if (seg == null) {
-						break;
-					}
-
-					this.globalBufferPool.releaseGlobalBuffer(seg);
-					this.requestedNumberOfBuffers--;
-				}
-
-				while (this.buffers.isEmpty()) {
-
-					// Check if the number of cached buffers matches the number of designated buffers
-					if (this.requestedNumberOfBuffers < this.designatedNumberOfBuffers) {
-
-						final MemorySegment memSeg = this.globalBufferPool.lockGlobalBuffer();
-						if (memSeg != null) {
-							this.buffers.add(memSeg);
-							this.requestedNumberOfBuffers++;
-							continue;
-						}
-					}
-
-					if (this.asynchronousEventOccurred && block) {
-						this.asynchronousEventOccurred = false;
-						async = true;
-						break;
-					}
-
-					if (block) {
-						this.buffers.wait(100);
-					} else {
-						return null;
-					}
-				}
-
-				if (!async) {
-					final MemorySegment memSeg = this.buffers.poll();
-					return BufferFactory.createFromMemory(minimumSizeOfBuffer, memSeg, this.bufferPoolConnector);
-				}
-			}
-		}
-	}
-
-
-	@Override
-	public int getMaximumBufferSize() {
-
-		return this.maximumBufferSize;
-	}
-
-	/**
-	 * Sets the designated number of buffers for this local buffer cache.
-	 * 
-	 * @param designatedNumberOfBuffers
-	 *        the designated number of buffers for this local buffer cache
-	 */
-	public void setDesignatedNumberOfBuffers(final int designatedNumberOfBuffers) {
-
-		synchronized (this.buffers) {
-
-			this.designatedNumberOfBuffers = designatedNumberOfBuffers;
-
-			// Make sure we return excess buffers immediately
-			while (this.requestedNumberOfBuffers > this.designatedNumberOfBuffers) {
-
-				if (this.buffers.isEmpty()) {
-					break;
-				}
-
-				this.globalBufferPool.releaseGlobalBuffer(this.buffers.poll());
-				this.requestedNumberOfBuffers--;
-			}
-
-			this.buffers.notify();
-		}
-	}
-
-	public void destroy() {
-
-		synchronized (this.buffers) {
-
-			if (this.isDestroyed) {
-				LOG.error("destroy is called on LocalBufferPool multiple times");
-				return;
-			}
-
-			this.isDestroyed = true;
-
-			while (!this.buffers.isEmpty()) {
-				this.globalBufferPool.releaseGlobalBuffer(this.buffers.poll());
-			}
-
-			this.requestedNumberOfBuffers = 0;
-		}
-	}
-
-
-	@Override
-	public boolean isShared() {
-
-		return this.isShared;
-	}
-
-	public int getNumberOfAvailableBuffers() {
-
-		synchronized (this.buffers) {
-			return this.buffers.size();
-		}
-	}
-
-	public int getDesignatedNumberOfBuffers() {
-
-		synchronized (this.buffers) {
-			return this.designatedNumberOfBuffers;
-		}
-	}
-
-	public int getRequestedNumberOfBuffers() {
-
-		synchronized (this.buffers) {
-			return this.requestedNumberOfBuffers;
-		}
-	}
-
-	private void recycleBuffer(final MemorySegment memSeg) {
-
-		synchronized (this.buffers) {
-
-			if (this.isDestroyed) {
-				this.globalBufferPool.releaseGlobalBuffer(memSeg);
-				this.requestedNumberOfBuffers--;
-			} else {
-				this.buffers.add(memSeg);
-				this.buffers.notify();
-			}
-
-			while (!this.bufferAvailabilityListenerQueue.isEmpty()) {
-				this.bufferAvailabilityListenerQueue.poll().bufferAvailable();
-			}
-		}
-	}
-
-
-	@Override
-	public void reportAsynchronousEvent() {
-
-		synchronized (this.buffers) {
-			this.asynchronousEventOccurred = true;
-			this.buffers.notify();
-		}
-	}
-
-
-	@Override
-	public boolean registerBufferAvailabilityListener(final BufferAvailabilityListener bufferAvailabilityListener) {
-
-		synchronized (this.buffers) {
-			if (!this.buffers.isEmpty()) {
-				return false;
-			}
-
-			if (this.isDestroyed) {
-				return false;
-			}
-
-			this.bufferAvailabilityListenerQueue.add(bufferAvailabilityListener);
-		}
-
-		return true;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPoolOwner.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPoolOwner.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPoolOwner.java
deleted file mode 100644
index c56a73c..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPoolOwner.java
+++ /dev/null
@@ -1,54 +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.taskmanager.bufferprovider;
-
-/**
- * A local buffer pool owner is an object which initially retrieves its buffers from the {@link GlobalBufferPool} and
- * manages its fraction of the overall buffer pool locally by means of a {@link LocalBufferPool}.
- * 
- */
-public interface LocalBufferPoolOwner {
-
-	/**
-	 * Returns the number of byte-buffered channels that will retrieve their buffers from the local buffer pool.
-	 * 
-	 * @return the number of byte-buffered channels that will retrieve their buffers from the local buffer pool
-	 */
-	int getNumberOfChannels();
-
-	/**
-	 * Sets the designated number of buffers the local buffer pool owner is allowed to fetch from the global buffer pool
-	 * and manage locally by means of the {@link LocalBufferPool}.
-	 * 
-	 * @param numberOfBuffers
-	 *        the numberOfBuffers the local buffer pool owner is allowed to fetch from the global buffer pool
-	 */
-	void setDesignatedNumberOfBuffers(int numberOfBuffers);
-
-	/**
-	 * Clears the local buffer pool and returns all buffers to the global buffer pool.
-	 */
-	void clearLocalBufferPool();
-
-	/**
-	 * Logs the current status of the local buffer pool. This method is intended mainly for debugging purposes.
-	 */
-	void logBufferUtilization();
-
-	/**
-	 * Reports an asynchronous event. Calling this method interrupts each blocking method of the buffer pool owner and
-	 * allows the blocked thread to respond to the event.
-	 */
-	void reportAsynchronousEvent();
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelContext.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelContext.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelContext.java
deleted file mode 100644
index 066c268..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelContext.java
+++ /dev/null
@@ -1,84 +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.taskmanager.bytebuffered;
-
-import java.util.Iterator;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope;
-
-public abstract class AbstractOutputChannelContext implements OutputChannelContext {
-
-	/**
-	 * The forwarding chain used by this output channel context.
-	 */
-	private final OutputChannelForwardingChain forwardingChain;
-
-	public AbstractOutputChannelContext(final OutputChannelForwardingChain forwardingChain) {
-
-		this.forwardingChain = forwardingChain;
-	}
-
-
-	@Override
-	public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) {
-
-		if (transferEnvelope.getBuffer() != null) {
-			throw new IllegalStateException("Transfer envelope for output channel has buffer attached");
-		}
-
-		final Iterator<AbstractEvent> it = transferEnvelope.getEventList().iterator();
-		while (it.hasNext()) {
-
-			final AbstractEvent event = it.next();
-			if (event instanceof AbstractTaskEvent) {
-				processEventAsynchronously(event);
-			} else {
-				processEventSynchronously(event);
-			}
-		}
-	}
-
-	/**
-	 * Processes an event received from the framework in a synchronous fashion, i.e. the event processing is done by the
-	 * thread the event is destined for (usually the task thread).
-	 * 
-	 * @param event
-	 *        the event to be processed
-	 */
-	protected void processEventSynchronously(final AbstractEvent event) {
-
-		this.forwardingChain.offerEvent(event);
-	}
-
-	/**
-	 * Processes an event received from the framework in an asynchronous fashion, i.e. the event processing is done by
-	 * the thread which delivers the event.
-	 * 
-	 * @param event
-	 *        the event to be processed
-	 */
-	protected void processEventAsynchronously(final AbstractEvent event) {
-
-		// The default implementation does nothing
-	}
-
-
-	@Override
-	public void destroy() {
-
-		this.forwardingChain.destroy();
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelForwarder.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelForwarder.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelForwarder.java
deleted file mode 100644
index 4a59cdc..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelForwarder.java
+++ /dev/null
@@ -1,109 +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.taskmanager.bytebuffered;
-
-import java.io.IOException;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.io.channels.Buffer;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope;
-
-/**
- * An output channel forwarder is a component which processes a {@link TransferEnvelope} after it has been produced by
- * an {@link AbstractByteBufferedOutputChannel}. The component can decide based on the transfer envelope whether to
- * forward the envelope, discard it, or to store it.
- * 
- */
-public abstract class AbstractOutputChannelForwarder {
-
-	private final AbstractOutputChannelForwarder next;
-
-	private volatile AbstractOutputChannelForwarder prev = null;
-
-	protected AbstractOutputChannelForwarder(final AbstractOutputChannelForwarder next) {
-		this.next = next;
-		if (this.next != null) {
-			this.next.prev = this;
-		}
-	}
-
-	/**
-	 * Called by the framework to push a produced transfer envelope towards its receiver. This method will always be
-	 * called by the task thread itself.
-	 * 
-	 * @param transferEnvelope
-	 *        the transfer envelope to be processed
-	 * @throws IOException
-	 *         thrown if an I/O error occurs while processing the transfer envelope
-	 * @throws InterruptedException
-	 *         thrown if the task thread was interrupted while processing the transfer envelope
-	 */
-	public void push(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException {
-
-		if (this.next != null) {
-			this.next.push(transferEnvelope);
-		}
-	}
-
-	public TransferEnvelope pull() {
-
-		if (this.prev != null) {
-			return this.prev.pull();
-		}
-
-		return null;
-	}
-
-	public boolean hasDataLeft() throws IOException, InterruptedException {
-
-		if (this.next != null) {
-			this.next.hasDataLeft();
-		}
-
-		return false;
-	}
-
-	public void processEvent(final AbstractEvent event) {
-
-		if (this.next != null) {
-			this.next.processEvent(event);
-		}
-	}
-
-	public void destroy() {
-
-		if (this.next != null) {
-			this.next.destroy();
-		}
-	}
-
-	protected final void recycleTransferEnvelope(final TransferEnvelope transferEnvelope) {
-
-		final Buffer buffer = transferEnvelope.getBuffer();
-		if (buffer != null) {
-			buffer.recycleBuffer();
-
-		}
-	}
-
-	protected final AbstractOutputChannelForwarder getNext() {
-
-		return this.next;
-	}
-
-	protected final AbstractOutputChannelForwarder getPrev() {
-
-		return this.prev;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java
deleted file mode 100644
index 3bf351d..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java
+++ /dev/null
@@ -1,816 +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.taskmanager.bytebuffered;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-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.configuration.GlobalConfiguration;
-import eu.stratosphere.nephele.execution.Environment;
-import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
-import eu.stratosphere.nephele.instance.InstanceConnectionInfo;
-import eu.stratosphere.nephele.io.AbstractID;
-import eu.stratosphere.nephele.io.GateID;
-import eu.stratosphere.nephele.io.channels.Buffer;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.protocols.ChannelLookupProtocol;
-import eu.stratosphere.nephele.taskmanager.Task;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProviderBroker;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.GlobalBufferPool;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPool;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPoolOwner;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeReceiverList;
-
-public final class ByteBufferedChannelManager implements TransferEnvelopeDispatcher, BufferProviderBroker {
-
-	/**
-	 * The log object used to report problems and errors.
-	 */
-	private static final Log LOG = LogFactory.getLog(ByteBufferedChannelManager.class);
-
-	private static final boolean DEFAULT_ALLOW_SENDER_SIDE_SPILLING = false;
-
-	private static final boolean DEFAULT_MERGE_SPILLED_BUFFERS = true;
-
-	// TODO: Make this configurable
-	private static final int NUMBER_OF_CHANNELS_FOR_MULTICAST = 10;
-
-	private final Map<ChannelID, ChannelContext> registeredChannels = new ConcurrentHashMap<ChannelID, ChannelContext>();
-
-	private final Map<AbstractID, LocalBufferPoolOwner> localBufferPoolOwner = new ConcurrentHashMap<AbstractID, LocalBufferPoolOwner>();
-
-	private final NetworkConnectionManager networkConnectionManager;
-
-	private final ChannelLookupProtocol channelLookupService;
-
-	private final InstanceConnectionInfo localConnectionInfo;
-
-	private final LocalBufferPool transitBufferPool;
-
-	private final boolean allowSenderSideSpilling;
-
-	private final boolean mergeSpilledBuffers;
-
-	private final boolean multicastEnabled = true;
-
-	/**
-	 * This map caches transfer envelope receiver lists.
-	 */
-	private final Map<ChannelID, TransferEnvelopeReceiverList> receiverCache = new ConcurrentHashMap<ChannelID, TransferEnvelopeReceiverList>();
-
-	public ByteBufferedChannelManager(final ChannelLookupProtocol channelLookupService,
-			final InstanceConnectionInfo localInstanceConnectionInfo) throws IOException {
-
-		this.channelLookupService = channelLookupService;
-
-		this.localConnectionInfo = localInstanceConnectionInfo;
-
-		// Initialize the global buffer pool
-		GlobalBufferPool.getInstance();
-
-		// Initialize the transit buffer pool
-		this.transitBufferPool = new LocalBufferPool(128, true);
-
-		this.networkConnectionManager = new NetworkConnectionManager(this,
-			localInstanceConnectionInfo.getAddress(), localInstanceConnectionInfo.getDataPort());
-
-		this.allowSenderSideSpilling = GlobalConfiguration.getBoolean("channel.network.allowSenderSideSpilling",
-			DEFAULT_ALLOW_SENDER_SIDE_SPILLING);
-
-		this.mergeSpilledBuffers = GlobalConfiguration.getBoolean("channel.network.mergeSpilledBuffers",
-			DEFAULT_MERGE_SPILLED_BUFFERS);
-
-		LOG.info("Initialized byte buffered channel manager with sender-side spilling "
-			+ (this.allowSenderSideSpilling ? "enabled" : "disabled")
-			+ (this.mergeSpilledBuffers ? " and spilled buffer merging enabled" : ""));
-	}
-
-	/**
-	 * Registers the given task with the byte buffered channel manager.
-	 * 
-	 * @param task
-	 *        the task to be registered
-	 * @param the
-	 *        set of output channels which are initially active
-	 * @throws InsufficientResourcesException
-	 *         thrown if the channel manager does not have enough memory buffers to safely run this task
-	 */
-	public void register(final Task task, final Set<ChannelID> activeOutputChannels)
-			throws InsufficientResourcesException {
-
-		// Check if we can safely run this task with the given resources
-		checkBufferAvailability(task);
-
-		final Environment environment = task.getEnvironment();
-
-		final TaskContext taskContext = task.createTaskContext(this,
-			this.localBufferPoolOwner.remove(task.getVertexID()));
-
-		final Set<GateID> outputGateIDs = environment.getOutputGateIDs();
-		for (final Iterator<GateID> gateIt = outputGateIDs.iterator(); gateIt.hasNext();) {
-
-			final GateID gateID = gateIt.next();
-			final OutputGateContext outputGateContext = taskContext.createOutputGateContext(gateID);
-			final Set<ChannelID> outputChannelIDs = environment.getOutputChannelIDsOfGate(gateID);
-			for (final Iterator<ChannelID> channelIt = outputChannelIDs.iterator(); channelIt.hasNext();) {
-
-				final ChannelID channelID = channelIt.next();
-				final OutputChannelContext previousContext = (OutputChannelContext) this.registeredChannels
-					.get(channelID);
-
-				final boolean isActive = true;/* activeOutputChannels.contains(channelID); */
-
-				final OutputChannelContext outputChannelContext = outputGateContext.createOutputChannelContext(
-					channelID, previousContext, isActive, this.mergeSpilledBuffers);
-
-				// Add routing entry to receiver cache to reduce latency
-				if (outputChannelContext.getType() == ChannelType.INMEMORY) {
-					addReceiverListHint(outputChannelContext.getChannelID(),
-						outputChannelContext.getConnectedChannelID());
-				}
-
-				// Add routing entry to receiver cache to save lookup for data arriving at the output channel
-				if (outputChannelContext.getType() == ChannelType.NETWORK) {
-					addReceiverListHint(outputChannelContext.getConnectedChannelID(),
-						outputChannelContext.getChannelID());
-				}
-
-				if (LOG.isDebugEnabled()) {
-					LOG.debug("Registering byte buffered output channel " + outputChannelContext.getChannelID() + " ("
-							+ (isActive ? "active" : "inactive") + ")");
-				}
-
-				this.registeredChannels.put(outputChannelContext.getChannelID(), outputChannelContext);
-			}
-		}
-
-		final Set<GateID> inputGateIDs = environment.getInputGateIDs();
-		for (final Iterator<GateID> gateIt = inputGateIDs.iterator(); gateIt.hasNext();) {
-
-			final GateID gateID = gateIt.next();
-			final InputGateContext inputGateContext = taskContext.createInputGateContext(gateID);
-			final Set<ChannelID> inputChannelIDs = environment.getInputChannelIDsOfGate(gateID);
-			for (final Iterator<ChannelID> channelIt = inputChannelIDs.iterator(); channelIt.hasNext();) {
-
-				final ChannelID channelID = channelIt.next();
-				final InputChannelContext previousContext = (InputChannelContext) this.registeredChannels
-					.get(channelID);
-
-				final InputChannelContext inputChannelContext = inputGateContext.createInputChannelContext(
-					channelID, previousContext);
-
-				// Add routing entry to receiver cache to reduce latency
-				if (inputChannelContext.getType() == ChannelType.INMEMORY) {
-					addReceiverListHint(inputChannelContext.getChannelID(), inputChannelContext.getConnectedChannelID());
-				}
-
-				this.registeredChannels.put(inputChannelContext.getChannelID(), inputChannelContext);
-			}
-
-			// Add input gate context to set of local buffer pool owner
-			final LocalBufferPoolOwner bufferPoolOwner = inputGateContext.getLocalBufferPoolOwner();
-			if (bufferPoolOwner != null) {
-				this.localBufferPoolOwner.put(inputGateContext.getGateID(), bufferPoolOwner);
-			}
-
-		}
-
-		this.localBufferPoolOwner.put(task.getVertexID(), taskContext);
-
-		redistributeGlobalBuffers();
-	}
-
-	/**
-	 * Unregisters the given task from the byte buffered channel manager.
-	 * 
-	 * @param vertexID
-	 *        the ID of the task to be unregistered
-	 * @param task
-	 *        the task to be unregistered
-	 */
-	public void unregister(final ExecutionVertexID vertexID, final Task task) {
-
-		final Environment environment = task.getEnvironment();
-
-		Iterator<ChannelID> channelIterator = environment.getOutputChannelIDs().iterator();
-
-		while (channelIterator.hasNext()) {
-
-			final ChannelID outputChannelID = channelIterator.next();
-			final ChannelContext context = this.registeredChannels.remove(outputChannelID);
-			if (context != null) {
-				context.destroy();
-			}
-			this.receiverCache.remove(outputChannelID);
-		}
-
-		channelIterator = environment.getInputChannelIDs().iterator();
-
-		while (channelIterator.hasNext()) {
-
-			final ChannelID outputChannelID = channelIterator.next();
-			final ChannelContext context = this.registeredChannels.remove(outputChannelID);
-			if (context != null) {
-				context.destroy();
-			}
-			this.receiverCache.remove(outputChannelID);
-		}
-
-		final Iterator<GateID> inputGateIterator = environment.getInputGateIDs().iterator();
-
-		while (inputGateIterator.hasNext()) {
-
-			final GateID inputGateID = inputGateIterator.next();
-
-			final LocalBufferPoolOwner owner = this.localBufferPoolOwner.remove(inputGateID);
-			if (owner != null) {
-				owner.clearLocalBufferPool();
-			}
-		}
-
-		final LocalBufferPoolOwner owner = this.localBufferPoolOwner.remove(vertexID);
-		if (owner != null) {
-			owner.clearLocalBufferPool();
-		}
-
-		redistributeGlobalBuffers();
-	}
-
-	/**
-	 * Shuts down the byte buffered channel manager and stops all its internal processes.
-	 */
-	public void shutdown() {
-
-		this.networkConnectionManager.shutDown();
-	}
-
-	public NetworkConnectionManager getNetworkConnectionManager() {
-
-		return this.networkConnectionManager;
-	}
-
-	private void recycleBuffer(final TransferEnvelope envelope) {
-
-		final Buffer buffer = envelope.getBuffer();
-		if (buffer != null) {
-			buffer.recycleBuffer();
-		}
-	}
-
-	private void sendReceiverNotFoundEvent(final TransferEnvelope envelope, final ChannelID receiver)
-			throws IOException, InterruptedException {
-
-		if (ReceiverNotFoundEvent.isReceiverNotFoundEvent(envelope)) {
-
-			LOG.info("Dropping request to send ReceiverNotFoundEvent as response to ReceiverNotFoundEvent");
-			return;
-		}
-
-		final JobID jobID = envelope.getJobID();
-
-		final TransferEnvelope transferEnvelope = ReceiverNotFoundEvent.createEnvelopeWithEvent(jobID, receiver,
-			envelope.getSequenceNumber());
-
-		final TransferEnvelopeReceiverList receiverList = getReceiverList(jobID, receiver);
-		if (receiverList == null) {
-			return;
-		}
-
-		processEnvelopeEnvelopeWithoutBuffer(transferEnvelope, receiverList);
-	}
-
-	private void processEnvelope(final TransferEnvelope transferEnvelope, final boolean freeSourceBuffer)
-			throws IOException, InterruptedException {
-
-		TransferEnvelopeReceiverList receiverList = null;
-		try {
-			receiverList = getReceiverList(transferEnvelope.getJobID(),
-				transferEnvelope.getSource());
-		} catch (InterruptedException e) {
-			recycleBuffer(transferEnvelope);
-			throw e;
-		} catch (IOException e) {
-			recycleBuffer(transferEnvelope);
-			throw e;
-		}
-
-		if (receiverList == null) {
-			recycleBuffer(transferEnvelope);
-			return;
-		}
-
-		// This envelope is known to have either no buffer or an memory-based input buffer
-		if (transferEnvelope.getBuffer() == null) {
-			processEnvelopeEnvelopeWithoutBuffer(transferEnvelope, receiverList);
-		} else {
-			processEnvelopeWithBuffer(transferEnvelope, receiverList, freeSourceBuffer);
-		}
-	}
-
-	private void processEnvelopeWithBuffer(final TransferEnvelope transferEnvelope,
-			final TransferEnvelopeReceiverList receiverList, final boolean freeSourceBuffer)
-			throws IOException, InterruptedException {
-
-		// Handle the most common (unicast) case first
-		if (!freeSourceBuffer) {
-
-			final List<ChannelID> localReceivers = receiverList.getLocalReceivers();
-			if (localReceivers.size() != 1) {
-				LOG.error("Expected receiver list to have exactly one element");
-			}
-
-			final ChannelID localReceiver = localReceivers.get(0);
-
-			final ChannelContext cc = this.registeredChannels.get(localReceiver);
-			if (cc == null) {
-
-				try {
-					sendReceiverNotFoundEvent(transferEnvelope, localReceiver);
-				} finally {
-					recycleBuffer(transferEnvelope);
-				}
-				return;
-			}
-
-			if (!cc.isInputChannel()) {
-				LOG.error("Local receiver " + localReceiver
-					+ " is not an input channel, but is supposed to accept a buffer");
-			}
-
-			cc.queueTransferEnvelope(transferEnvelope);
-
-			return;
-		}
-
-		// This is the in-memory or multicast case
-		final Buffer srcBuffer = transferEnvelope.getBuffer();
-
-		try {
-
-			if (receiverList.hasLocalReceivers()) {
-
-				final List<ChannelID> localReceivers = receiverList.getLocalReceivers();
-
-				for (final ChannelID localReceiver : localReceivers) {
-
-					final ChannelContext cc = this.registeredChannels.get(localReceiver);
-					if (cc == null) {
-
-						sendReceiverNotFoundEvent(transferEnvelope, localReceiver);
-						continue;
-					}
-
-					if (!cc.isInputChannel()) {
-						LOG.error("Local receiver " + localReceiver
-							+ " is not an input channel, but is supposed to accept a buffer");
-						continue;
-					}
-
-					final InputChannelContext inputChannelContext = (InputChannelContext) cc;
-
-					Buffer destBuffer = null;
-					try {
-						destBuffer = inputChannelContext.requestEmptyBufferBlocking(srcBuffer.size());
-						srcBuffer.copyToBuffer(destBuffer);
-					} catch (IOException e) {
-						if (destBuffer != null) {
-							destBuffer.recycleBuffer();
-						}
-						throw e;
-					}
-					// TODO: See if we can save one duplicate step here
-					final TransferEnvelope dup = transferEnvelope.duplicateWithoutBuffer();
-					dup.setBuffer(destBuffer);
-					inputChannelContext.queueTransferEnvelope(dup);
-				}
-			}
-
-			if (receiverList.hasRemoteReceivers()) {
-
-				final List<RemoteReceiver> remoteReceivers = receiverList.getRemoteReceivers();
-
-				// Generate sender hint before sending the first envelope over the network
-				if (transferEnvelope.getSequenceNumber() == 0) {
-					generateSenderHint(transferEnvelope, remoteReceivers);
-				}
-
-				for (final RemoteReceiver remoteReceiver : remoteReceivers) {
-					TransferEnvelope dup = transferEnvelope.duplicate();
-					this.networkConnectionManager.queueEnvelopeForTransfer(remoteReceiver, dup);
-				}
-			}
-		} finally {
-			// Recycle the source buffer
-			srcBuffer.recycleBuffer();
-		}
-	}
-
-	private void processEnvelopeEnvelopeWithoutBuffer(final TransferEnvelope transferEnvelope,
-			final TransferEnvelopeReceiverList receiverList) throws IOException, InterruptedException {
-
-		// No need to copy anything
-		final Iterator<ChannelID> localIt = receiverList.getLocalReceivers().iterator();
-
-		while (localIt.hasNext()) {
-
-			final ChannelID localReceiver = localIt.next();
-
-			final ChannelContext channelContext = this.registeredChannels.get(localReceiver);
-			if (channelContext == null) {
-				sendReceiverNotFoundEvent(transferEnvelope, localReceiver);
-				continue;
-			}
-			channelContext.queueTransferEnvelope(transferEnvelope);
-		}
-
-		if (!receiverList.hasRemoteReceivers()) {
-			return;
-		}
-
-		// Generate sender hint before sending the first envelope over the network
-		final List<RemoteReceiver> remoteReceivers = receiverList.getRemoteReceivers();
-		if (transferEnvelope.getSequenceNumber() == 0) {
-			generateSenderHint(transferEnvelope, remoteReceivers);
-		}
-
-		final Iterator<RemoteReceiver> remoteIt = remoteReceivers.iterator();
-
-		while (remoteIt.hasNext()) {
-
-			final RemoteReceiver remoteReceiver = remoteIt.next();
-			this.networkConnectionManager.queueEnvelopeForTransfer(remoteReceiver, transferEnvelope);
-		}
-	}
-
-	private void addReceiverListHint(final ChannelID source, final ChannelID localReceiver) {
-
-		final TransferEnvelopeReceiverList receiverList = new TransferEnvelopeReceiverList(localReceiver);
-
-		if (this.receiverCache.put(source, receiverList) != null) {
-			LOG.warn("Receiver cache already contained entry for " + source);
-		}
-	}
-
-	private void addReceiverListHint(final ChannelID source, final RemoteReceiver remoteReceiver) {
-
-		final TransferEnvelopeReceiverList receiverList = new TransferEnvelopeReceiverList(remoteReceiver);
-
-		if (this.receiverCache.put(source, receiverList) != null) {
-			LOG.warn("Receiver cache already contained entry for " + source);
-		}
-	}
-
-	private void generateSenderHint(final TransferEnvelope transferEnvelope, final List<RemoteReceiver> remoteReceivers) {
-
-		final ChannelContext channelContext = this.registeredChannels.get(transferEnvelope.getSource());
-		if (channelContext == null) {
-			LOG.error("Cannot find channel context for channel ID " + transferEnvelope.getSource());
-			return;
-		}
-
-		// Only generate sender hints for output channels
-		if (channelContext.isInputChannel()) {
-			return;
-		}
-
-		final ChannelID remoteSourceID = channelContext.getConnectedChannelID();
-		final int connectionIndex = remoteReceivers.get(0).getConnectionIndex();
-		final InetSocketAddress isa = new InetSocketAddress(this.localConnectionInfo.getAddress(),
-			this.localConnectionInfo.getDataPort());
-
-		final RemoteReceiver remoteReceiver = new RemoteReceiver(isa, connectionIndex);
-		final TransferEnvelope senderHint = SenderHintEvent.createEnvelopeWithEvent(transferEnvelope, remoteSourceID,
-			remoteReceiver);
-
-		final Iterator<RemoteReceiver> remoteIt = remoteReceivers.iterator();
-
-		while (remoteIt.hasNext()) {
-
-			final RemoteReceiver rr = remoteIt.next();
-			this.networkConnectionManager.queueEnvelopeForTransfer(rr, senderHint);
-		}
-	}
-
-	/**
-	 * Returns the list of receivers for transfer envelopes produced by the channel with the given source channel ID.
-	 * 
-	 * @param jobID
-	 *        the ID of the job the given channel ID belongs to
-	 * @param sourceChannelID
-	 *        the source channel ID for which the receiver list shall be retrieved
-	 * @return the list of receivers or <code>null</code> if the receiver could not be determined
-	 * @throws IOException
-	 * @throws InterruptedExcption
-	 */
-	private TransferEnvelopeReceiverList getReceiverList(final JobID jobID, final ChannelID sourceChannelID)
-			throws IOException, InterruptedException {
-
-		TransferEnvelopeReceiverList receiverList = this.receiverCache.get(sourceChannelID);
-
-		if (receiverList != null) {
-			return receiverList;
-		}
-
-		while (true) {
-
-			if (Thread.currentThread().isInterrupted()) {
-				break;
-			}
-
-			ConnectionInfoLookupResponse lookupResponse;
-			synchronized (this.channelLookupService) {
-				lookupResponse = this.channelLookupService.lookupConnectionInfo(
-					this.localConnectionInfo, jobID, sourceChannelID);
-			}
-
-			if (lookupResponse.isJobAborting()) {
-				break;
-			}
-
-			if (lookupResponse.receiverNotFound()) {
-				LOG.error("Cannot find task(s) waiting for data from source channel with ID " + sourceChannelID);
-				break;
-			}
-
-			if (lookupResponse.receiverNotReady()) {
-				Thread.sleep(500);
-				continue;
-			}
-
-			if (lookupResponse.receiverReady()) {
-				receiverList = new TransferEnvelopeReceiverList(lookupResponse);
-				break;
-			}
-
-		}
-
-		if (receiverList != null) {
-
-			this.receiverCache.put(sourceChannelID, receiverList);
-
-			if (LOG.isDebugEnabled()) {
-
-				final StringBuilder sb = new StringBuilder();
-				sb.append("Receiver list for source channel ID " + sourceChannelID + " at task manager "
-					+ this.localConnectionInfo + "\n");
-
-				if (receiverList.hasLocalReceivers()) {
-					sb.append("\tLocal receivers:\n");
-					final Iterator<ChannelID> it = receiverList.getLocalReceivers().iterator();
-					while (it.hasNext()) {
-						sb.append("\t\t" + it.next() + "\n");
-					}
-				}
-
-				if (receiverList.hasRemoteReceivers()) {
-					sb.append("Remote receivers:\n");
-					final Iterator<RemoteReceiver> it = receiverList.getRemoteReceivers().iterator();
-					while (it.hasNext()) {
-						sb.append("\t\t" + it.next() + "\n");
-					}
-				}
-
-				LOG.debug(sb.toString());
-			}
-		}
-
-		return receiverList;
-	}
-
-
-	@Override
-	public void processEnvelopeFromOutputChannel(final TransferEnvelope transferEnvelope) throws IOException,
-			InterruptedException {
-
-		processEnvelope(transferEnvelope, true);
-	}
-
-
-	@Override
-	public void processEnvelopeFromInputChannel(final TransferEnvelope transferEnvelope) throws IOException,
-			InterruptedException {
-
-		processEnvelope(transferEnvelope, false);
-	}
-
-
-	@Override
-	public void processEnvelopeFromNetwork(final TransferEnvelope transferEnvelope, boolean freeSourceBuffer)
-			throws IOException, InterruptedException {
-
-		// Check if the envelope is the special envelope with the sender hint event
-		if (SenderHintEvent.isSenderHintEvent(transferEnvelope)) {
-
-			// Check if this is the final destination of the sender hint event before adding it
-			final SenderHintEvent seh = (SenderHintEvent) transferEnvelope.getEventList().get(0);
-			if (this.registeredChannels.get(seh.getSource()) != null) {
-
-				addReceiverListHint(seh.getSource(), seh.getRemoteReceiver());
-				return;
-			}
-		}
-
-		processEnvelope(transferEnvelope, freeSourceBuffer);
-	}
-
-	/**
-	 * Triggers the byte buffer channel manager write the current utilization of its read and write buffers to the logs.
-	 * This method is primarily for debugging purposes.
-	 */
-	public void logBufferUtilization() {
-
-		System.out.println("Buffer utilization at " + System.currentTimeMillis());
-
-		System.out.println("\tUnused global buffers: " + GlobalBufferPool.getInstance().getCurrentNumberOfBuffers());
-
-		System.out.println("\tLocal buffer pool status:");
-
-		final Iterator<LocalBufferPoolOwner> it = this.localBufferPoolOwner.values().iterator();
-		while (it.hasNext()) {
-			it.next().logBufferUtilization();
-		}
-
-		this.networkConnectionManager.logBufferUtilization();
-
-		System.out.println("\tIncoming connections:");
-
-		final Iterator<Map.Entry<ChannelID, ChannelContext>> it2 = this.registeredChannels.entrySet()
-				.iterator();
-
-		while (it2.hasNext()) {
-
-			final Map.Entry<ChannelID, ChannelContext> entry = it2.next();
-			final ChannelContext context = entry.getValue();
-			if (context.isInputChannel()) {
-
-				final InputChannelContext inputChannelContext = (InputChannelContext) context;
-				inputChannelContext.logQueuedEnvelopes();
-			}
-		}
-	}
-
-
-	@Override
-	public BufferProvider getBufferProvider(final JobID jobID, final ChannelID sourceChannelID) throws IOException,
-			InterruptedException {
-
-		final TransferEnvelopeReceiverList receiverList = getReceiverList(jobID, sourceChannelID);
-
-		// Receiver could not be determined, use transit buffer pool to read data from channel
-		if (receiverList == null) {
-			return this.transitBufferPool;
-		}
-
-		if (receiverList.hasLocalReceivers() && !receiverList.hasRemoteReceivers()) {
-
-			final List<ChannelID> localReceivers = receiverList.getLocalReceivers();
-			if (localReceivers.size() == 1) {
-				// Unicast case, get final buffer provider
-
-				final ChannelID localReceiver = localReceivers.get(0);
-				final ChannelContext cc = this.registeredChannels.get(localReceiver);
-				if (cc == null) {
-
-					// Use the transit buffer for this purpose, data will be discarded in most cases anyway.
-					return this.transitBufferPool;
-				}
-
-				if (!cc.isInputChannel()) {
-					throw new IOException("Channel context for local receiver " + localReceiver
-							+ " is not an input channel context");
-				}
-
-				final InputChannelContext icc = (InputChannelContext) cc;
-
-				return icc;
-			}
-		}
-
-		return this.transitBufferPool;
-	}
-
-	/**
-	 * Checks if the byte buffered channel manager has enough resources available to safely execute the given task.
-	 * 
-	 * @param task
-	 *        the task to be executed
-	 * @throws InsufficientResourcesException
-	 *         thrown if the byte buffered manager currently does not have enough resources available to execute the
-	 *         task
-	 */
-	private void checkBufferAvailability(final Task task) throws InsufficientResourcesException {
-
-		final int totalNumberOfBuffers = GlobalBufferPool.getInstance().getTotalNumberOfBuffers();
-		int numberOfAlreadyRegisteredChannels = this.registeredChannels.size();
-		if (this.multicastEnabled) {
-			numberOfAlreadyRegisteredChannels += NUMBER_OF_CHANNELS_FOR_MULTICAST;
-		}
-
-		final Environment env = task.getEnvironment();
-
-		final int numberOfNewChannels = env.getNumberOfOutputChannels() + env.getNumberOfInputChannels();
-		final int totalNumberOfChannels = numberOfAlreadyRegisteredChannels + numberOfNewChannels;
-
-		final double buffersPerChannel = (double) totalNumberOfBuffers
-			/ (double) totalNumberOfChannels;
-
-		if (buffersPerChannel < 1.0) {
-
-			// Construct error message
-			final StringBuilder sb = new StringBuilder(this.localConnectionInfo.getHostName());
-			sb.append(" has not enough buffers available to safely execute ");
-			sb.append(env.getTaskName());
-			sb.append(" (");
-			sb.append(totalNumberOfChannels - totalNumberOfBuffers);
-			sb.append(" buffers are currently missing)");
-
-			throw new InsufficientResourcesException(sb.toString());
-		}
-	}
-
-	/**
-	 * Redistributes the global buffers among the registered tasks.
-	 */
-	private void redistributeGlobalBuffers() {
-
-		final int totalNumberOfBuffers = GlobalBufferPool.getInstance().getTotalNumberOfBuffers();
-		int totalNumberOfChannels = this.registeredChannels.size();
-		if (this.multicastEnabled) {
-			totalNumberOfChannels += NUMBER_OF_CHANNELS_FOR_MULTICAST;
-		}
-		final double buffersPerChannel = (double) totalNumberOfBuffers / (double) totalNumberOfChannels;
-		if (buffersPerChannel < 1.0) {
-			LOG.warn("System is low on memory buffers. This may result in reduced performance.");
-		}
-
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("Total number of buffers is " + totalNumberOfBuffers);
-			LOG.debug("Total number of channels is " + totalNumberOfChannels);
-		}
-
-		if (this.localBufferPoolOwner.isEmpty()) {
-			return;
-		}
-
-		final Iterator<LocalBufferPoolOwner> it = this.localBufferPoolOwner.values().iterator();
-		while (it.hasNext()) {
-			final LocalBufferPoolOwner lbpo = it.next();
-			lbpo.setDesignatedNumberOfBuffers((int) Math.ceil(buffersPerChannel * lbpo.getNumberOfChannels()));
-		}
-
-		if (this.multicastEnabled) {
-			this.transitBufferPool.setDesignatedNumberOfBuffers((int) Math.ceil(buffersPerChannel
-				* NUMBER_OF_CHANNELS_FOR_MULTICAST));
-		}
-	}
-
-	/**
-	 * Invalidates the entries identified by the given channel IDs from the receiver lookup cache.
-	 * 
-	 * @param channelIDs
-	 *        the channel IDs identifying the cache entries to invalidate
-	 */
-	public void invalidateLookupCacheEntries(final Set<ChannelID> channelIDs) {
-
-		final Iterator<ChannelID> it = channelIDs.iterator();
-		while (it.hasNext()) {
-
-			this.receiverCache.remove(it.next());
-		}
-	}
-
-	public void reportAsynchronousEvent(final ExecutionVertexID vertexID) {
-
-		final LocalBufferPoolOwner lbpo = this.localBufferPoolOwner.get(vertexID);
-		if (lbpo == null) {
-			System.out.println("Cannot find local buffer pool owner for " + vertexID);
-			return;
-		}
-
-		lbpo.reportAsynchronousEvent();
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/CanceledChannelSet.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/CanceledChannelSet.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/CanceledChannelSet.java
deleted file mode 100644
index f1efdc7..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/CanceledChannelSet.java
+++ /dev/null
@@ -1,211 +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.taskmanager.bytebuffered;
-
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-
-import eu.stratosphere.nephele.io.channels.ChannelID;
-
-/**
- * This channel set stores the ID's of all channels whose tasks have been canceled recently. The set is cleaned up by
- * periodically calling the method <code>cleanup</code>.
- * <p>
- * This class is thread-safe.
- * 
- */
-public final class CanceledChannelSet implements Set<ChannelID> {
-
-	/**
-	 * The period of time the entries must at least remain in the map.
-	 */
-	private final static long CLEANUP_INTERVAL = 30000; // 30 sec.
-
-	/**
-	 * The map which stores the ID's of the channels whose tasks have been canceled.
-	 */
-	private final Map<ChannelID, Long> canceledChannels = new HashMap<ChannelID, Long>();
-
-
-	@Override
-	public boolean add(ChannelID arg0) {
-
-		final long now = System.currentTimeMillis();
-
-		synchronized (this.canceledChannels) {
-			if (this.canceledChannels.put(arg0, Long.valueOf(now)) == null) {
-				return true;
-			}
-		}
-
-		return false;
-	}
-
-
-	@Override
-	public boolean addAll(Collection<? extends ChannelID> arg0) {
-
-		final Long now = Long.valueOf(System.currentTimeMillis());
-		final Iterator<? extends ChannelID> it = arg0.iterator();
-		boolean retVal = false;
-
-		synchronized (this.canceledChannels) {
-
-			while (it.hasNext()) {
-
-				if (this.canceledChannels.put(it.next(), now) == null) {
-					retVal = true;
-				}
-			}
-		}
-
-		return retVal;
-	}
-
-
-	@Override
-	public void clear() {
-
-		synchronized (this.canceledChannels) {
-			this.canceledChannels.clear();
-		}
-
-	}
-
-
-	@Override
-	public boolean contains(Object arg0) {
-
-		synchronized (this.canceledChannels) {
-			return this.canceledChannels.containsKey(arg0);
-		}
-	}
-
-
-	@Override
-	public boolean containsAll(Collection<?> arg0) {
-
-		synchronized (this.canceledChannels) {
-			return this.canceledChannels.keySet().containsAll(arg0);
-		}
-	}
-
-
-	@Override
-	public boolean isEmpty() {
-
-		synchronized (this.canceledChannels) {
-			return this.canceledChannels.isEmpty();
-		}
-	}
-
-
-	@Override
-	public Iterator<ChannelID> iterator() {
-
-		synchronized (this.canceledChannels) {
-			return this.canceledChannels.keySet().iterator();
-		}
-	}
-
-
-	@Override
-	public boolean remove(Object arg0) {
-
-		synchronized (this.canceledChannels) {
-			if (this.canceledChannels.remove(arg0) == null) {
-				return false;
-			}
-		}
-
-		return true;
-	}
-
-
-	@Override
-	public boolean removeAll(Collection<?> arg0) {
-
-		final Iterator<?> it = arg0.iterator();
-		boolean retVal = false;
-
-		synchronized (this.canceledChannels) {
-
-			while (it.hasNext()) {
-				if (this.canceledChannels.remove(it.next()) != null) {
-					retVal = true;
-				}
-			}
-		}
-
-		return retVal;
-	}
-
-
-	@Override
-	public boolean retainAll(Collection<?> arg0) {
-
-		throw new RuntimeException("Method not implemented");
-	}
-
-
-	@Override
-	public int size() {
-
-		synchronized (this.canceledChannels) {
-			return this.canceledChannels.size();
-		}
-	}
-
-
-	@Override
-	public Object[] toArray() {
-
-		synchronized (this.canceledChannels) {
-			return this.canceledChannels.keySet().toArray();
-		}
-	}
-
-
-	@Override
-	public <T> T[] toArray(T[] arg0) {
-
-		synchronized (this.canceledChannels) {
-			return this.canceledChannels.keySet().toArray(arg0);
-		}
-	}
-
-	/**
-	 * Removes all entries from the set which have been added longer than <code>CLEANUP_INTERVAL</code> milliseconds
-	 * ago.
-	 */
-	public void cleanup() {
-
-		final long now = System.currentTimeMillis();
-
-		synchronized (this.canceledChannels) {
-
-			final Iterator<Map.Entry<ChannelID, Long>> it = this.canceledChannels.entrySet().iterator();
-			while (it.hasNext()) {
-
-				final Map.Entry<ChannelID, Long> entry = it.next();
-				if ((entry.getValue().longValue() + CLEANUP_INTERVAL) < now) {
-					it.remove();
-				}
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ChannelContext.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ChannelContext.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ChannelContext.java
deleted file mode 100644
index b3a9200..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ChannelContext.java
+++ /dev/null
@@ -1,36 +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.taskmanager.bytebuffered;
-
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope;
-
-public interface ChannelContext {
-
-	boolean isInputChannel();
-	
-	JobID getJobID();
-
-	ChannelID getChannelID();
-
-	ChannelID getConnectedChannelID();
-	
-	ChannelType getType();
-	
-	void queueTransferEnvelope(TransferEnvelope transferEnvelope);
-	
-	void destroy();
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ConnectionInfoLookupResponse.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ConnectionInfoLookupResponse.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ConnectionInfoLookupResponse.java
deleted file mode 100644
index 69daa0e..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ConnectionInfoLookupResponse.java
+++ /dev/null
@@ -1,176 +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.taskmanager.bytebuffered;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.List;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.util.EnumUtils;
-import eu.stratosphere.nephele.util.SerializableArrayList;
-
-public class ConnectionInfoLookupResponse implements IOReadableWritable {
-
-	private enum ReturnCode {
-		NOT_FOUND, FOUND_AND_RECEIVER_READY, FOUND_BUT_RECEIVER_NOT_READY, JOB_IS_ABORTING
-	};
-
-	// was request successful?
-	private ReturnCode returnCode;
-
-	/**
-	 * Contains next-hop instances, this instance must forward multicast transmissions to.
-	 */
-	private final SerializableArrayList<RemoteReceiver> remoteTargets = new SerializableArrayList<RemoteReceiver>();
-
-	/**
-	 * Contains local ChannelIDs, multicast packets must be forwarded to.
-	 */
-	private final SerializableArrayList<ChannelID> localTargets = new SerializableArrayList<ChannelID>();
-
-	public ConnectionInfoLookupResponse() {
-		this.returnCode = ReturnCode.NOT_FOUND;
-	}
-
-	public void addRemoteTarget(final RemoteReceiver remote) {
-		this.remoteTargets.add(remote);
-	}
-
-	public void addLocalTarget(ChannelID local) {
-		this.localTargets.add(local);
-	}
-
-	private void setReturnCode(ReturnCode code) {
-		this.returnCode = code;
-	}
-
-	public List<RemoteReceiver> getRemoteTargets() {
-		return this.remoteTargets;
-	}
-
-	public List<ChannelID> getLocalTargets() {
-		return this.localTargets;
-	}
-
-	@Override
-	public void read(DataInput in) throws IOException {
-
-		this.localTargets.read(in);
-		this.remoteTargets.read(in);
-
-		this.returnCode = EnumUtils.readEnum(in, ReturnCode.class);
-	}
-
-	@Override
-	public void write(DataOutput out) throws IOException {
-
-		this.localTargets.write(out);
-		this.remoteTargets.write(out);
-
-		EnumUtils.writeEnum(out, this.returnCode);
-
-	}
-
-	public boolean receiverNotFound() {
-
-		return (this.returnCode == ReturnCode.NOT_FOUND);
-	}
-
-	public boolean receiverNotReady() {
-
-		return (this.returnCode == ReturnCode.FOUND_BUT_RECEIVER_NOT_READY);
-	}
-
-	public boolean receiverReady() {
-
-		return (this.returnCode == ReturnCode.FOUND_AND_RECEIVER_READY);
-	}
-
-	public boolean isJobAborting() {
-
-		return (this.returnCode == ReturnCode.JOB_IS_ABORTING);
-	}
-
-	public static ConnectionInfoLookupResponse createReceiverFoundAndReady(final ChannelID targetChannelID) {
-
-		final ConnectionInfoLookupResponse response = new ConnectionInfoLookupResponse();
-		response.setReturnCode(ReturnCode.FOUND_AND_RECEIVER_READY);
-		response.addLocalTarget(targetChannelID);
-
-		return response;
-	}
-
-	public static ConnectionInfoLookupResponse createReceiverFoundAndReady(final RemoteReceiver remoteReceiver) {
-
-		final ConnectionInfoLookupResponse response = new ConnectionInfoLookupResponse();
-		response.setReturnCode(ReturnCode.FOUND_AND_RECEIVER_READY);
-		response.addRemoteTarget(remoteReceiver);
-
-		return response;
-	}
-
-	/**
-	 * Constructor used to generate a plain ConnectionInfoLookupResponse object to be filled with multicast targets.
-	 * 
-	 * @return
-	 */
-	public static ConnectionInfoLookupResponse createReceiverFoundAndReady() {
-
-		final ConnectionInfoLookupResponse response = new ConnectionInfoLookupResponse();
-		response.setReturnCode(ReturnCode.FOUND_AND_RECEIVER_READY);
-
-		return response;
-	}
-
-	public static ConnectionInfoLookupResponse createReceiverNotFound() {
-		final ConnectionInfoLookupResponse response = new ConnectionInfoLookupResponse();
-		response.setReturnCode(ReturnCode.NOT_FOUND);
-
-		return response;
-	}
-
-	public static ConnectionInfoLookupResponse createReceiverNotReady() {
-		final ConnectionInfoLookupResponse response = new ConnectionInfoLookupResponse();
-		response.setReturnCode(ReturnCode.FOUND_BUT_RECEIVER_NOT_READY);
-
-		return response;
-	}
-
-	public static ConnectionInfoLookupResponse createJobIsAborting() {
-		final ConnectionInfoLookupResponse response = new ConnectionInfoLookupResponse();
-		response.setReturnCode(ReturnCode.JOB_IS_ABORTING);
-
-		return response;
-	}
-
-	@Override
-	public String toString() {
-		StringBuilder returnstring = new StringBuilder();
-		returnstring.append("local targets (total: " + this.localTargets.size() + "):\n");
-		for (ChannelID i : this.localTargets) {
-			returnstring.append(i + "\n");
-		}
-		returnstring.append("remote targets: (total: " + this.remoteTargets.size() + "):\n");
-		for (final RemoteReceiver rr : this.remoteTargets) {
-			returnstring.append(rr + "\n");
-		}
-		return returnstring.toString();
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/GateContext.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/GateContext.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/GateContext.java
deleted file mode 100644
index 7b8ce36..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/GateContext.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.taskmanager.bytebuffered;
-
-import eu.stratosphere.nephele.io.GateID;
-
-public interface GateContext {
-
-	GateID getGateID();
-}


[18/30] Offer buffer-oriented API for I/O (#25)

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/IncomingConnection.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/IncomingConnection.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/IncomingConnection.java
deleted file mode 100644
index d0fa683..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/IncomingConnection.java
+++ /dev/null
@@ -1,125 +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.taskmanager.bytebuffered;
-
-import java.io.IOException;
-import java.nio.channels.ReadableByteChannel;
-import java.nio.channels.SelectionKey;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.DefaultDeserializer;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.NoBufferAvailableException;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope;
-import eu.stratosphere.util.StringUtils;
-
-/**
- * This class represents an incoming data connection through which data streams are read and transformed into
- * {@link TransferEnvelope} objects. The source of the data stream is a TCP connection.
- * 
- */
-public class IncomingConnection {
-
-	/**
-	 * The log object used to report debug information and possible errors.
-	 */
-	private static final Log LOG = LogFactory.getLog(IncomingConnection.class);
-
-	/**
-	 * The readable byte channel through which the input data is retrieved.
-	 */
-	private final ReadableByteChannel readableByteChannel;
-
-	/**
-	 * The {@link DefaultDeserializer} used to transform the read bytes into transfer envelopes which can be
-	 * passed on to the respective channels.
-	 */
-	private final DefaultDeserializer deserializer;
-
-	/**
-	 * The byte buffered channel manager which handles and dispatches the received transfer envelopes.
-	 */
-	private final ByteBufferedChannelManager byteBufferedChannelManager;
-
-	public IncomingConnection(ByteBufferedChannelManager byteBufferedChannelManager,
-			ReadableByteChannel readableByteChannel) {
-		this.byteBufferedChannelManager = byteBufferedChannelManager;
-		this.deserializer = new DefaultDeserializer(byteBufferedChannelManager);
-		this.readableByteChannel = readableByteChannel;
-	}
-
-	public void reportTransmissionProblem(SelectionKey key, IOException ioe) {
-
-		LOG.error(StringUtils.stringifyException(ioe));
-
-		try {
-			this.readableByteChannel.close();
-		} catch (IOException e) {
-			LOG.debug("An error occurred while closing the byte channel");
-		}
-
-		// Cancel key
-		if (key != null) {
-			key.cancel();
-		}
-
-		// Recycle read buffer
-		if (this.deserializer.getBuffer() != null) {
-			this.deserializer.getBuffer().recycleBuffer();
-		}
-
-		this.deserializer.reset();
-	}
-
-	public void read() throws IOException, InterruptedException, NoBufferAvailableException {
-
-		this.deserializer.read(this.readableByteChannel);
-
-		final TransferEnvelope transferEnvelope = this.deserializer.getFullyDeserializedTransferEnvelope();
-		if (transferEnvelope != null) {
-
-			final BufferProvider bufferProvider = this.deserializer.getBufferProvider();
-			if (bufferProvider == null) {
-				this.byteBufferedChannelManager.processEnvelopeFromNetwork(transferEnvelope, false);
-			} else {
-				this.byteBufferedChannelManager.processEnvelopeFromNetwork(transferEnvelope, bufferProvider.isShared());
-			}
-		}
-	}
-
-	public boolean isCloseUnexpected() {
-
-		return this.deserializer.hasUnfinishedData();
-	}
-
-	public ReadableByteChannel getReadableByteChannel() {
-		return this.readableByteChannel;
-	}
-
-	public void closeConnection(SelectionKey key) {
-
-		try {
-			this.readableByteChannel.close();
-		} catch (IOException ioe) {
-			LOG.error("On IOException occured while closing the socket: + " + StringUtils.stringifyException(ioe));
-		}
-
-		// Cancel key
-		if (key != null) {
-			key.cancel();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/IncomingConnectionThread.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/IncomingConnectionThread.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/IncomingConnectionThread.java
deleted file mode 100644
index 05c3326..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/IncomingConnectionThread.java
+++ /dev/null
@@ -1,223 +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.taskmanager.bytebuffered;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.nio.channels.ClosedChannelException;
-import java.nio.channels.SelectionKey;
-import java.nio.channels.Selector;
-import java.nio.channels.ServerSocketChannel;
-import java.nio.channels.SocketChannel;
-import java.util.ArrayDeque;
-import java.util.Iterator;
-import java.util.Queue;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferAvailabilityListener;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.NoBufferAvailableException;
-import eu.stratosphere.util.StringUtils;
-
-public class IncomingConnectionThread extends Thread {
-
-	private static final Log LOG = LogFactory.getLog(IncomingConnectionThread.class);
-
-	private final ByteBufferedChannelManager byteBufferedChannelManager;
-
-	private final Selector selector;
-
-	private final Queue<SelectionKey> pendingReadEventSubscribeRequests = new ArrayDeque<SelectionKey>();
-
-	private final ServerSocketChannel listeningSocket;
-
-	private static final class IncomingConnectionBufferAvailListener implements BufferAvailabilityListener {
-
-		private final Queue<SelectionKey> pendingReadEventSubscribeRequests;
-
-		private final SelectionKey key;
-
-		private IncomingConnectionBufferAvailListener(final Queue<SelectionKey> pendingReadEventSubscribeRequests,
-				final SelectionKey key) {
-
-			this.pendingReadEventSubscribeRequests = pendingReadEventSubscribeRequests;
-			this.key = key;
-		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public void bufferAvailable() {
-
-			synchronized (this.pendingReadEventSubscribeRequests) {
-				this.pendingReadEventSubscribeRequests.add(this.key);
-			}
-		}
-	}
-
-	public IncomingConnectionThread(ByteBufferedChannelManager byteBufferedChannelManager,
-			boolean isListeningThread, InetSocketAddress listeningAddress) throws IOException {
-		super("Incoming Connection Thread");
-
-		this.selector = Selector.open();
-		this.byteBufferedChannelManager = byteBufferedChannelManager;
-
-		if (isListeningThread) {
-			this.listeningSocket = ServerSocketChannel.open();
-			this.listeningSocket.configureBlocking(false);
-			listeningSocket.register(this.selector, SelectionKey.OP_ACCEPT);
-			this.listeningSocket.socket().bind(listeningAddress);
-			LOG.debug("Listening on " + this.listeningSocket.socket().getLocalSocketAddress());
-		} else {
-			this.listeningSocket = null;
-		}
-	}
-
-	@Override
-	public void run() {
-
-		while (!this.isInterrupted()) {
-
-			synchronized (this.pendingReadEventSubscribeRequests) {
-				while (!this.pendingReadEventSubscribeRequests.isEmpty()) {
-					final SelectionKey key = this.pendingReadEventSubscribeRequests.poll();
-					final IncomingConnection incomingConnection = (IncomingConnection) key.attachment();
-					final SocketChannel socketChannel = (SocketChannel) key.channel();
-
-					try {
-						final SelectionKey newKey = socketChannel.register(this.selector, SelectionKey.OP_READ);
-						newKey.attach(incomingConnection);
-					} catch (ClosedChannelException e) {
-						incomingConnection.reportTransmissionProblem(key, e);
-					}
-				}
-			}
-
-			try {
-				this.selector.select(500);
-			} catch (IOException e) {
-				LOG.error(e);
-			}
-
-			final Iterator<SelectionKey> iter = this.selector.selectedKeys().iterator();
-
-			while (iter.hasNext()) {
-				final SelectionKey key = iter.next();
-
-				iter.remove();
-				if (key.isValid()) {
-					if (key.isReadable()) {
-						doRead(key);
-					} else if (key.isAcceptable()) {
-						doAccept(key);
-					} else {
-						LOG.error("Unknown key: " + key);
-					}
-				} else {
-					LOG.error("Received invalid key: " + key);
-				}
-			}
-		}
-
-		// Do cleanup, if necessary
-		if (this.listeningSocket != null) {
-			try {
-				this.listeningSocket.close();
-			} catch (IOException ioe) {
-				// Actually, we can ignore this exception
-				LOG.debug(ioe);
-			}
-		}
-
-		// Finally, close the selector
-		try {
-			this.selector.close();
-		} catch (IOException ioe) {
-			LOG.debug(StringUtils.stringifyException(ioe));
-		}
-	}
-
-	private void doAccept(SelectionKey key) {
-
-		SocketChannel clientSocket = null;
-
-		try {
-			clientSocket = this.listeningSocket.accept();
-			if (clientSocket == null) {
-				LOG.error("Client socket is null");
-				return;
-			}
-		} catch (IOException ioe) {
-			LOG.error(ioe);
-			return;
-		}
-
-		final IncomingConnection incomingConnection = new IncomingConnection(this.byteBufferedChannelManager,
-			clientSocket);
-		SelectionKey clientKey = null;
-		try {
-			clientSocket.configureBlocking(false);
-			clientKey = clientSocket.register(this.selector, SelectionKey.OP_READ);
-			clientKey.attach(incomingConnection);
-		} catch (IOException ioe) {
-			incomingConnection.reportTransmissionProblem(clientKey, ioe);
-		}
-	}
-
-	private void doRead(SelectionKey key) {
-
-		final IncomingConnection incomingConnection = (IncomingConnection) key.attachment();
-		try {
-			incomingConnection.read();
-		} catch (EOFException eof) {
-			if (incomingConnection.isCloseUnexpected()) {
-				final SocketChannel socketChannel = (SocketChannel) key.channel();
-				LOG.error("Connection from " + socketChannel.socket().getRemoteSocketAddress()
-					+ " was closed unexpectedly");
-				incomingConnection.reportTransmissionProblem(key, eof);
-			} else {
-				incomingConnection.closeConnection(key);
-			}
-		} catch (IOException ioe) {
-			incomingConnection.reportTransmissionProblem(key, ioe);
-		} catch (InterruptedException e) {
-			// Nothing to do here
-		} catch (NoBufferAvailableException e) {
-			// There are no buffers available, unsubscribe from read event
-			final SocketChannel socketChannel = (SocketChannel) key.channel();
-			try {
-				final SelectionKey newKey = socketChannel.register(this.selector, 0);
-				newKey.attach(incomingConnection);
-			} catch (ClosedChannelException e1) {
-				incomingConnection.reportTransmissionProblem(key, e1);
-			}
-
-			final BufferAvailabilityListener bal = new IncomingConnectionBufferAvailListener(
-				this.pendingReadEventSubscribeRequests, key);
-			if (!e.getBufferProvider().registerBufferAvailabilityListener(bal)) {
-				// In the meantime, a buffer has become available again, subscribe to read event again
-
-				try {
-					final SelectionKey newKey = socketChannel.register(this.selector, SelectionKey.OP_READ);
-					newKey.attach(incomingConnection);
-				} catch (ClosedChannelException e1) {
-					incomingConnection.reportTransmissionProblem(key, e1);
-				}
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java
deleted file mode 100644
index 739ec3d..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.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.taskmanager.bytebuffered;
-
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider;
-
-public interface InputChannelContext extends ChannelContext, BufferProvider {
-
-	void logQueuedEnvelopes();
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputGateContext.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputGateContext.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputGateContext.java
deleted file mode 100644
index 0b05262..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputGateContext.java
+++ /dev/null
@@ -1,24 +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.taskmanager.bytebuffered;
-
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPoolOwner;
-
-public interface InputGateContext extends GateContext {
-
-	InputChannelContext createInputChannelContext(ChannelID channelID, InputChannelContext previousContext);
-	
-	LocalBufferPoolOwner getLocalBufferPoolOwner();
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InsufficientResourcesException.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InsufficientResourcesException.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InsufficientResourcesException.java
deleted file mode 100644
index 87543b8..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InsufficientResourcesException.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.taskmanager.bytebuffered;
-
-/**
- * This exception is thrown by the {@link ByteBufferedChannelManager} to indicate that a task cannot be accepted because
- * there are not enough resources available to safely execute it.
- * 
- */
-public final class InsufficientResourcesException extends Exception {
-
-	/**
-	 * The generated serial version UID.
-	 */
-	private static final long serialVersionUID = -8977049569413215169L;
-
-	/**
-	 * Constructs a new insufficient resources exception.
-	 * 
-	 * @param msg
-	 *        the message describing the exception
-	 */
-	InsufficientResourcesException(final String msg) {
-		super(msg);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/NetworkConnectionManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/NetworkConnectionManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/NetworkConnectionManager.java
deleted file mode 100644
index 6a5cc47..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/NetworkConnectionManager.java
+++ /dev/null
@@ -1,173 +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.taskmanager.bytebuffered;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.CopyOnWriteArrayList;
-
-import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.configuration.GlobalConfiguration;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope;
-
-/**
- * The network connection manager manages incoming and outgoing network connection from and to other hosts.
- * <p>
- * This class is thread-safe.
- * 
- */
-public final class NetworkConnectionManager {
-
-	/**
-	 * The default number of threads dealing with outgoing connections.
-	 */
-	private static final int DEFAULT_NUMBER_OF_OUTGOING_CONNECTION_THREADS = 1;
-
-	/**
-	 * The default number of connection retries before giving up.
-	 */
-	private static final int DEFAULT_NUMBER_OF_CONNECTION_RETRIES = 10;
-
-	/**
-	 * List of active threads dealing with outgoing connections.
-	 */
-	private final List<OutgoingConnectionThread> outgoingConnectionThreads = new CopyOnWriteArrayList<OutgoingConnectionThread>();
-
-	/**
-	 * Thread dealing with incoming connections.
-	 */
-	private final IncomingConnectionThread incomingConnectionThread;
-
-	/**
-	 * Map containing currently active outgoing connections.
-	 */
-	private final ConcurrentMap<RemoteReceiver, OutgoingConnection> outgoingConnections = new ConcurrentHashMap<RemoteReceiver, OutgoingConnection>();
-
-	/**
-	 * The number of connection retries before giving up.
-	 */
-	private final int numberOfConnectionRetries;
-
-	/**
-	 * A buffer provider for read buffers
-	 */
-	private final ByteBufferedChannelManager byteBufferedChannelManager;
-
-	public NetworkConnectionManager(final ByteBufferedChannelManager byteBufferedChannelManager,
-			final InetAddress bindAddress, final int dataPort) throws IOException {
-
-		final Configuration configuration = GlobalConfiguration.getConfiguration();
-
-		this.byteBufferedChannelManager = byteBufferedChannelManager;
-
-		// Start the connection threads
-		final int numberOfOutgoingConnectionThreads = configuration.getInteger(
-			"channel.network.numberOfOutgoingConnectionThreads", DEFAULT_NUMBER_OF_OUTGOING_CONNECTION_THREADS);
-
-		for (int i = 0; i < numberOfOutgoingConnectionThreads; i++) {
-			final OutgoingConnectionThread outgoingConnectionThread = new OutgoingConnectionThread();
-			outgoingConnectionThread.start();
-			this.outgoingConnectionThreads.add(outgoingConnectionThread);
-		}
-
-		this.incomingConnectionThread = new IncomingConnectionThread(
-			this.byteBufferedChannelManager, true, new InetSocketAddress(bindAddress, dataPort));
-		this.incomingConnectionThread.start();
-
-		this.numberOfConnectionRetries = configuration.getInteger("channel.network.numberOfConnectionRetries",
-			DEFAULT_NUMBER_OF_CONNECTION_RETRIES);
-	}
-
-	/**
-	 * Randomly selects one of the active threads dealing with outgoing connections.
-	 * 
-	 * @return one of the active threads dealing with outgoing connections
-	 */
-	private OutgoingConnectionThread getOutgoingConnectionThread() {
-
-		return this.outgoingConnectionThreads.get((int) (this.outgoingConnectionThreads.size() * Math.random()));
-	}
-
-	/**
-	 * Queues an envelope for transfer to a particular target host.
-	 * 
-	 * @param remoteReceiver
-	 *        the address of the remote receiver
-	 * @param transferEnvelope
-	 *        the envelope to be transfered
-	 */
-	public void queueEnvelopeForTransfer(final RemoteReceiver remoteReceiver, final TransferEnvelope transferEnvelope) {
-
-		getOutgoingConnection(remoteReceiver).queueEnvelope(transferEnvelope);
-	}
-
-	/**
-	 * Returns (and possibly creates) the outgoing connection for the given target address.
-	 * 
-	 * @param targetAddress
-	 *        the address of the connection target
-	 * @return the outgoing connection object
-	 */
-	private OutgoingConnection getOutgoingConnection(final RemoteReceiver remoteReceiver) {
-
-		OutgoingConnection outgoingConnection = this.outgoingConnections.get(remoteReceiver);
-
-		if (outgoingConnection == null) {
-
-			outgoingConnection = new OutgoingConnection(remoteReceiver, getOutgoingConnectionThread(),
-				this.numberOfConnectionRetries);
-
-			final OutgoingConnection oldEntry = this.outgoingConnections
-				.putIfAbsent(remoteReceiver, outgoingConnection);
-
-			// We had a race, use the old value
-			if (oldEntry != null) {
-				outgoingConnection = oldEntry;
-			}
-		}
-
-		return outgoingConnection;
-	}
-
-	public void shutDown() {
-
-		// Interrupt the threads we started
-		this.incomingConnectionThread.interrupt();
-
-		final Iterator<OutgoingConnectionThread> it = this.outgoingConnectionThreads.iterator();
-		while (it.hasNext()) {
-			it.next().interrupt();
-		}
-	}
-
-	public void logBufferUtilization() {
-
-		System.out.println("\tOutgoing connections:");
-
-		final Iterator<Map.Entry<RemoteReceiver, OutgoingConnection>> it = this.outgoingConnections.entrySet()
-			.iterator();
-
-		while (it.hasNext()) {
-
-			final Map.Entry<RemoteReceiver, OutgoingConnection> entry = it.next();
-			System.out.println("\t\tOC " + entry.getKey() + ": " + entry.getValue().getNumberOfQueuedWriteBuffers());
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnection.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnection.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnection.java
deleted file mode 100644
index 94463b6..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnection.java
+++ /dev/null
@@ -1,531 +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.taskmanager.bytebuffered;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.nio.channels.SelectionKey;
-import java.nio.channels.SocketChannel;
-import java.nio.channels.WritableByteChannel;
-import java.util.ArrayDeque;
-import java.util.Iterator;
-import java.util.Queue;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.DefaultSerializer;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope;
-
-/**
- * This class represents an outgoing TCP connection through which {@link TransferEnvelope} objects can be sent.
- * {@link TransferEnvelope} objects are received from the {@link ByteBufferedChannelManager} and added to a queue. An
- * additional network thread then takes the envelopes from the queue and transmits them to the respective destination
- * host.
- * 
- */
-public class OutgoingConnection {
-
-	/**
-	 * The log object used to report debug information and possible errors.
-	 */
-	private static final Log LOG = LogFactory.getLog(OutgoingConnection.class);
-
-	/**
-	 * The address this outgoing connection is connected to.
-	 */
-	private final RemoteReceiver remoteReceiver;
-
-	/**
-	 * The outgoing connection thread which actually transmits the queued transfer envelopes.
-	 */
-	private final OutgoingConnectionThread connectionThread;
-
-	/**
-	 * The queue of transfer envelopes to be transmitted.
-	 */
-	private final Queue<TransferEnvelope> queuedEnvelopes = new ArrayDeque<TransferEnvelope>();
-
-	/**
-	 * The {@link DefaultSerializer} object used to transform the envelopes into a byte stream.
-	 */
-	private final DefaultSerializer serializer = new DefaultSerializer();
-
-	/**
-	 * The {@link TransferEnvelope} that is currently processed.
-	 */
-	private TransferEnvelope currentEnvelope = null;
-
-	/**
-	 * Stores whether the underlying TCP connection is established. As this variable is accessed by the byte buffered
-	 * channel manager and the outgoing connection thread, it must be protected by a monitor.
-	 */
-	private boolean isConnected = false;
-
-	/**
-	 * Stores whether is underlying TCP connection is subscribed to the NIO write event. As this variable is accessed by
-	 * the byte buffered channel and the outgoing connection thread, it must be protected by a monitor.
-	 */
-	private boolean isSubscribedToWriteEvent = false;
-
-	/**
-	 * The overall number of connection retries which shall be performed before a connection error is reported.
-	 */
-	private final int numberOfConnectionRetries;
-
-	/**
-	 * The number of connection retries left before an I/O error is reported.
-	 */
-	private int retriesLeft = 0;
-
-	/**
-	 * The timestamp of the last connection retry.
-	 */
-	private long timstampOfLastRetry = 0;
-
-	/**
-	 * The current selection key representing the interest set of the underlying TCP NIO connection. This variable may
-	 * only be accessed the the outgoing connection thread.
-	 */
-	private SelectionKey selectionKey = null;
-
-	/**
-	 * The period of time in milliseconds that shall be waited before a connection attempt is considered to be failed.
-	 */
-	private static long RETRYINTERVAL = 1000L; // 1 second
-
-	/**
-	 * Constructs a new outgoing connection object.
-	 * 
-	 * @param remoteReceiver
-	 *        the address of the destination host this outgoing connection object is supposed to connect to
-	 * @param connectionThread
-	 *        the connection thread which actually handles the network transfer
-	 * @param numberOfConnectionRetries
-	 *        the number of connection retries allowed before an I/O error is reported
-	 */
-	public OutgoingConnection(RemoteReceiver remoteReceiver, OutgoingConnectionThread connectionThread,
-			int numberOfConnectionRetries) {
-
-		this.remoteReceiver = remoteReceiver;
-		this.connectionThread = connectionThread;
-		this.numberOfConnectionRetries = numberOfConnectionRetries;
-	}
-
-	/**
-	 * Adds a new {@link TransferEnvelope} to the queue of envelopes to be transmitted to the destination host of this
-	 * connection.
-	 * <p>
-	 * This method should only be called by the {@link ByteBufferedChannelManager} object.
-	 * 
-	 * @param transferEnvelope
-	 *        the envelope to be added to the transfer queue
-	 */
-	public void queueEnvelope(TransferEnvelope transferEnvelope) {
-
-		synchronized (this.queuedEnvelopes) {
-
-			checkConnection();
-			this.queuedEnvelopes.add(transferEnvelope);
-		}
-	}
-
-	private void checkConnection() {
-
-		synchronized (this.queuedEnvelopes) {
-
-			if (!this.isConnected) {
-
-				this.retriesLeft = this.numberOfConnectionRetries;
-				this.timstampOfLastRetry = System.currentTimeMillis();
-				this.connectionThread.triggerConnect(this);
-				this.isConnected = true;
-				this.isSubscribedToWriteEvent = true;
-			} else {
-
-				if (!this.isSubscribedToWriteEvent) {
-					this.connectionThread.subscribeToWriteEvent(this.selectionKey);
-					this.isSubscribedToWriteEvent = true;
-				}
-			}
-
-		}
-	}
-
-	/**
-	 * Returns the {@link InetSocketAddress} to the destination host this outgoing connection is supposed to be
-	 * connected to.
-	 * <p>
-	 * This method should be called by the {@link OutgoingConnectionThread} object only.
-	 * 
-	 * @return the {@link InetSocketAddress} to the destination host this outgoing connection is supposed to be
-	 *         connected to
-	 */
-	public InetSocketAddress getConnectionAddress() {
-
-		return this.remoteReceiver.getConnectionAddress();
-	}
-
-	/**
-	 * Reports a problem which occurred while establishing the underlying TCP connection to this outgoing connection
-	 * object. Depending on the number of connection retries left, this method will either try to reestablish the TCP
-	 * connection or report an I/O error to all tasks which have queued envelopes for this connection. In the latter
-	 * case all queued envelopes will be dropped and all included buffers will be freed.
-	 * <p>
-	 * This method should only be called by the {@link OutgoingConnectionThread} object.
-	 * 
-	 * @param ioe
-	 *        thrown if an error occurs while reseting the underlying TCP connection
-	 */
-	public void reportConnectionProblem(IOException ioe) {
-
-		// First, write exception to log
-		final long currentTime = System.currentTimeMillis();
-		if (currentTime - this.timstampOfLastRetry >= RETRYINTERVAL) {
-			LOG.error("Cannot connect to " + this.remoteReceiver + ", " + this.retriesLeft + " retries left");
-		}
-
-		synchronized (this.queuedEnvelopes) {
-
-			if (this.selectionKey != null) {
-
-				final SocketChannel socketChannel = (SocketChannel) this.selectionKey.channel();
-				if (socketChannel != null) {
-					try {
-						socketChannel.close();
-					} catch (IOException e) {
-						LOG.debug("Error while trying to close the socket channel to " + this.remoteReceiver);
-					}
-				}
-
-				this.selectionKey.cancel();
-				this.selectionKey = null;
-				this.isConnected = false;
-				this.isSubscribedToWriteEvent = false;
-			}
-
-			if (hasRetriesLeft(currentTime)) {
-				this.connectionThread.triggerConnect(this);
-				this.isConnected = true;
-				this.isSubscribedToWriteEvent = true;
-				return;
-			}
-
-			// Error is fatal
-			LOG.error(ioe);
-
-			// Notify source of current envelope and release buffer
-			if (this.currentEnvelope != null) {
-				if (this.currentEnvelope.getBuffer() != null) {
-					this.currentEnvelope.getBuffer().recycleBuffer();
-					this.currentEnvelope = null;
-				}
-			}
-
-			// Notify all other tasks which are waiting for data to be transmitted
-			final Iterator<TransferEnvelope> iter = this.queuedEnvelopes.iterator();
-			while (iter.hasNext()) {
-				final TransferEnvelope envelope = iter.next();
-				iter.remove();
-				// Recycle the buffer inside the envelope
-				if (envelope.getBuffer() != null) {
-					envelope.getBuffer().recycleBuffer();
-				}
-			}
-
-			this.queuedEnvelopes.clear();
-		}
-	}
-
-	/**
-	 * Reports an I/O error which occurred while writing data to the TCP connection. As a result of the I/O error the
-	 * connection is closed and the interest keys are canceled. Moreover, the task which queued the currently
-	 * transmitted transfer envelope is notified about the error and the current envelope is dropped. If the current
-	 * envelope contains a buffer, the buffer is freed.
-	 * <p>
-	 * This method should only be called by the {@link OutgoingConnectionThread} object.
-	 * 
-	 * @param ioe
-	 *        thrown if an error occurs while reseting the connection
-	 */
-	public void reportTransmissionProblem(IOException ioe) {
-
-		final SocketChannel socketChannel = (SocketChannel) this.selectionKey.channel();
-
-		// First, write exception to log
-		if (this.currentEnvelope != null) {
-			LOG.error("The connection between " + socketChannel.socket().getLocalAddress() + " and "
-				+ socketChannel.socket().getRemoteSocketAddress()
-				+ " experienced an IOException for transfer envelope " + this.currentEnvelope.getSequenceNumber());
-		} else {
-			LOG.error("The connection between " + socketChannel.socket().getLocalAddress() + " and "
-				+ socketChannel.socket().getRemoteSocketAddress() + " experienced an IOException");
-		}
-
-		// Close the connection and cancel the interest key
-		synchronized (this.queuedEnvelopes) {
-			try {
-				LOG.debug("Closing connection to " + socketChannel.socket().getRemoteSocketAddress());
-				socketChannel.close();
-			} catch (IOException e) {
-				LOG.debug("An error occurred while responding to an IOException");
-				LOG.debug(e);
-			}
-
-			this.selectionKey.cancel();
-
-			// Error is fatal
-			LOG.error(ioe);
-
-			// Trigger new connection if there are more envelopes to be transmitted
-			if (this.queuedEnvelopes.isEmpty()) {
-				this.isConnected = false;
-				this.isSubscribedToWriteEvent = false;
-			} else {
-				this.connectionThread.triggerConnect(this);
-				this.isConnected = true;
-				this.isSubscribedToWriteEvent = true;
-			}
-
-			// We must assume the current envelope is corrupted so we notify the task which created it.
-			if (this.currentEnvelope != null) {
-				if (this.currentEnvelope.getBuffer() != null) {
-					this.currentEnvelope.getBuffer().recycleBuffer();
-					this.currentEnvelope = null;
-				}
-			}
-		}
-	}
-
-	/**
-	 * Checks whether further retries are left for establishing the underlying TCP connection.
-	 * 
-	 * @param currentTime
-	 *        the current system time in milliseconds since January 1st, 1970
-	 * @return <code>true</code> if there are retries left, <code>false</code> otherwise
-	 */
-	private boolean hasRetriesLeft(long currentTime) {
-
-		if (currentTime - this.timstampOfLastRetry >= RETRYINTERVAL) {
-			this.retriesLeft--;
-			this.timstampOfLastRetry = currentTime;
-			if (this.retriesLeft == 0) {
-				return false;
-			}
-		}
-
-		return true;
-	}
-
-	/**
-	 * Writes the content of the current {@link TransferEnvelope} object to the underlying TCP connection.
-	 * <p>
-	 * This method should only be called by the {@link OutgoingConnectionThread} object.
-	 * 
-	 * @return <code>true</code> if there is more data from this/other queued envelopes to be written to this channel
-	 * @throws IOException
-	 *         thrown if an error occurs while writing the data to the channel
-	 */
-	public boolean write() throws IOException {
-
-		final WritableByteChannel writableByteChannel = (WritableByteChannel) this.selectionKey.channel();
-
-		if (this.currentEnvelope == null) {
-			synchronized (this.queuedEnvelopes) {
-				if (this.queuedEnvelopes.isEmpty()) {
-					return false;
-				} else {
-					this.currentEnvelope = this.queuedEnvelopes.peek();
-					this.serializer.setTransferEnvelope(this.currentEnvelope);
-				}
-			}
-		}
-
-		if (!this.serializer.write(writableByteChannel)) {
-
-			// Make sure we recycle the attached memory or file buffers correctly
-			if (this.currentEnvelope.getBuffer() != null) {
-				this.currentEnvelope.getBuffer().recycleBuffer();
-			}
-
-			synchronized (this.queuedEnvelopes) {
-				this.queuedEnvelopes.poll();
-				this.currentEnvelope = null;
-			}
-		}
-
-		return true;
-	}
-
-	/**
-	 * Requests to close the underlying TCP connection. The request is ignored if at least one {@link TransferEnvelope}
-	 * is queued.
-	 * <p>
-	 * This method should only be called by the {@link OutgoingConnectionThread} object.
-	 * 
-	 * @throws IOException
-	 *         thrown if an error occurs while closing the TCP connection
-	 */
-	public void requestClose() throws IOException {
-
-		synchronized (this.queuedEnvelopes) {
-
-			if (this.queuedEnvelopes.isEmpty()) {
-
-				if (this.isSubscribedToWriteEvent) {
-
-					this.connectionThread.unsubscribeFromWriteEvent(this.selectionKey);
-					this.isSubscribedToWriteEvent = false;
-				}
-			}
-		}
-	}
-
-	/**
-	 * Closes the underlying TCP connection if no more {@link TransferEnvelope} objects are in the transmission queue.
-	 * <p>
-	 * This method should only be called by the {@link OutgoingConnectionThread} object.
-	 * 
-	 * @throws IOException
-	 */
-	public void closeConnection() throws IOException {
-
-		synchronized (this.queuedEnvelopes) {
-
-			if (!this.queuedEnvelopes.isEmpty()) {
-				return;
-			}
-
-			if (this.selectionKey != null) {
-
-				final SocketChannel socketChannel = (SocketChannel) this.selectionKey.channel();
-				socketChannel.close();
-				this.selectionKey.cancel();
-				this.selectionKey = null;
-			}
-
-			this.isConnected = false;
-			this.isSubscribedToWriteEvent = false;
-		}
-	}
-
-	/**
-	 * Returns the number of queued {@link TransferEnvelope} objects with the given source channel ID.
-	 * 
-	 * @param sourceChannelID
-	 *        the source channel ID to count the queued envelopes for
-	 * @return the number of queued transfer envelopes with the given source channel ID
-	 */
-	public int getNumberOfQueuedEnvelopesFromChannel(final ChannelID sourceChannelID) {
-
-		synchronized (this.queuedEnvelopes) {
-
-			int number = 0;
-
-			final Iterator<TransferEnvelope> it = this.queuedEnvelopes.iterator();
-			while (it.hasNext()) {
-				final TransferEnvelope te = it.next();
-				if (sourceChannelID.equals(te.getSource())) {
-					number++;
-				}
-			}
-
-			return number;
-		}
-	}
-
-	/**
-	 * Removes all queued {@link TransferEnvelope} objects from the transmission which match the given source channel
-	 * ID.
-	 * 
-	 * @param sourceChannelID
-	 *        the source channel ID of the transfered transfer envelopes to be dropped
-	 */
-	public void dropAllQueuedEnvelopesFromChannel(final ChannelID sourceChannelID) {
-
-		synchronized (this.queuedEnvelopes) {
-
-			final Iterator<TransferEnvelope> it = this.queuedEnvelopes.iterator();
-			while (it.hasNext()) {
-				final TransferEnvelope te = it.next();
-				if (sourceChannelID.equals(te.getSource())) {
-					it.remove();
-					if (te.getBuffer() != null) {
-						te.getBuffer().recycleBuffer();
-					}
-				}
-			}
-		}
-	}
-
-	/**
-	 * Checks whether this outgoing connection object manages an active connection or can be removed by the
-	 * {@link ByteBufferedChannelManager} object.
-	 * <p>
-	 * This method should only be called by the byte buffered channel manager.
-	 * 
-	 * @return <code>true</code> if this object is no longer manages an active connection and can be removed,
-	 *         <code>false</code> otherwise.
-	 */
-	public boolean canBeRemoved() {
-
-		synchronized (this.queuedEnvelopes) {
-
-			if (this.isConnected) {
-				return false;
-			}
-
-			if (this.currentEnvelope != null) {
-				return false;
-			}
-
-			return this.queuedEnvelopes.isEmpty();
-		}
-	}
-
-	/**
-	 * Sets the selection key representing the interest set of the underlying TCP NIO connection.
-	 * 
-	 * @param selectionKey
-	 *        the selection of the underlying TCP connection
-	 */
-	public void setSelectionKey(SelectionKey selectionKey) {
-		this.selectionKey = selectionKey;
-	}
-
-	/**
-	 * Returns the number of currently queued envelopes which contain a write buffer.
-	 * 
-	 * @return the number of currently queued envelopes which contain a write buffer
-	 */
-	public int getNumberOfQueuedWriteBuffers() {
-
-		int retVal = 0;
-
-		synchronized (this.queuedEnvelopes) {
-
-			final Iterator<TransferEnvelope> it = this.queuedEnvelopes.iterator();
-			while (it.hasNext()) {
-
-				final TransferEnvelope envelope = it.next();
-				if (envelope.getBuffer() != null) {
-					++retVal;
-				}
-			}
-		}
-
-		return retVal;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnectionThread.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnectionThread.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnectionThread.java
deleted file mode 100644
index ef03a9c..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnectionThread.java
+++ /dev/null
@@ -1,270 +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.taskmanager.bytebuffered;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.SelectionKey;
-import java.nio.channels.Selector;
-import java.nio.channels.SocketChannel;
-import java.util.ArrayDeque;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Queue;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.util.StringUtils;
-
-public class OutgoingConnectionThread extends Thread {
-
-	/**
-	 * The minimum time a TCP connection must be idle it is closed.
-	 */
-	private static final long MIN_IDLE_TIME_BEFORE_CLOSE = 80000L; // 80 seconds
-
-	private static final Log LOG = LogFactory.getLog(OutgoingConnectionThread.class);
-
-	private final Selector selector;
-
-	private final Queue<OutgoingConnection> pendingConnectionRequests = new ArrayDeque<OutgoingConnection>();
-
-	private final Queue<SelectionKey> pendingWriteEventSubscribeRequests = new ArrayDeque<SelectionKey>();
-
-	private final Map<OutgoingConnection, Long> connectionsToClose = new HashMap<OutgoingConnection, Long>();
-
-	public OutgoingConnectionThread() throws IOException {
-		super("Outgoing Connection Thread");
-
-		this.selector = Selector.open();
-	}
-
-
-	@Override
-	public void run() {
-
-		while (!isInterrupted()) {
-
-			synchronized (this.pendingConnectionRequests) {
-
-				if (!this.pendingConnectionRequests.isEmpty()) {
-
-					final OutgoingConnection outgoingConnection = this.pendingConnectionRequests.poll();
-					try {
-						final SocketChannel socketChannel = SocketChannel.open();
-						socketChannel.configureBlocking(false);
-						final SelectionKey key = socketChannel.register(this.selector, SelectionKey.OP_CONNECT);
-						socketChannel.connect(outgoingConnection.getConnectionAddress());
-						key.attach(outgoingConnection);
-					} catch (final IOException ioe) {
-						// IOException is reported by separate thread to avoid deadlocks
-						final Runnable reporterThread = new Runnable() {
-
-							@Override
-							public void run() {
-								outgoingConnection.reportConnectionProblem(ioe);
-							}
-						};
-						new Thread(reporterThread).start();
-					}
-				}
-			}
-
-			synchronized (this.pendingWriteEventSubscribeRequests) {
-
-				if (!this.pendingWriteEventSubscribeRequests.isEmpty()) {
-					final SelectionKey oldSelectionKey = this.pendingWriteEventSubscribeRequests.poll();
-					final OutgoingConnection outgoingConnection = (OutgoingConnection) oldSelectionKey.attachment();
-					final SocketChannel socketChannel = (SocketChannel) oldSelectionKey.channel();
-
-					try {
-						final SelectionKey newSelectionKey = socketChannel.register(this.selector, SelectionKey.OP_READ
-							| SelectionKey.OP_WRITE);
-						newSelectionKey.attach(outgoingConnection);
-						outgoingConnection.setSelectionKey(newSelectionKey);
-					} catch (final IOException ioe) {
-						// IOException is reported by separate thread to avoid deadlocks
-						final Runnable reporterThread = new Runnable() {
-
-							@Override
-							public void run() {
-								outgoingConnection.reportTransmissionProblem(ioe);
-							}
-						};
-						new Thread(reporterThread).start();
-					}
-				}
-			}
-
-			synchronized (this.connectionsToClose) {
-
-				final Iterator<Map.Entry<OutgoingConnection, Long>> closeIt = this.connectionsToClose.entrySet()
-					.iterator();
-				final long now = System.currentTimeMillis();
-				while (closeIt.hasNext()) {
-
-					final Map.Entry<OutgoingConnection, Long> entry = closeIt.next();
-					if ((entry.getValue().longValue() + MIN_IDLE_TIME_BEFORE_CLOSE) < now) {
-						final OutgoingConnection outgoingConnection = entry.getKey();
-						closeIt.remove();
-						// Create new thread to close connection to avoid deadlocks
-						final Runnable closeThread = new Runnable() {
-
-							@Override
-							public void run() {
-								try {
-									outgoingConnection.closeConnection();
-								} catch (IOException ioe) {
-									outgoingConnection.reportTransmissionProblem(ioe);
-								}
-							}
-						};
-
-						new Thread(closeThread).start();
-					}
-
-				}
-			}
-
-			try {
-				this.selector.select(10);
-			} catch (IOException e) {
-				LOG.error(e);
-			}
-
-			final Iterator<SelectionKey> iter = this.selector.selectedKeys().iterator();
-
-			while (iter.hasNext()) {
-				final SelectionKey key = iter.next();
-
-				iter.remove();
-				if (key.isValid()) {
-					if (key.isConnectable()) {
-						doConnect(key);
-					} else {
-						if (key.isReadable()) {
-							doRead(key);
-							// A read will always result in an exception, so the write key will not be valid anymore
-							continue;
-						}
-						if (key.isWritable()) {
-							doWrite(key);
-						}
-					}
-				} else {
-					LOG.error("Received invalid key: " + key);
-				}
-			}
-		}
-
-		// Finally, try to close the selector
-		try {
-			this.selector.close();
-		} catch (IOException ioe) {
-			LOG.debug(StringUtils.stringifyException(ioe));
-		}
-	}
-
-	private void doConnect(SelectionKey key) {
-
-		final OutgoingConnection outgoingConnection = (OutgoingConnection) key.attachment();
-		final SocketChannel socketChannel = (SocketChannel) key.channel();
-		try {
-			while (!socketChannel.finishConnect()) {
-				try {
-					Thread.sleep(100);
-				} catch (InterruptedException e1) {
-					LOG.error(e1);
-				}
-			}
-
-			final SelectionKey channelKey = socketChannel.register(selector, SelectionKey.OP_WRITE
-				| SelectionKey.OP_READ);
-			outgoingConnection.setSelectionKey(channelKey);
-			channelKey.attach(outgoingConnection);
-
-		} catch (IOException ioe) {
-			outgoingConnection.reportConnectionProblem(ioe);
-		}
-	}
-
-	private void doWrite(SelectionKey key) {
-
-		final OutgoingConnection outgoingConnection = (OutgoingConnection) key.attachment();
-
-		try {
-
-			if (!outgoingConnection.write()) {
-				// Try to close the connection
-				outgoingConnection.requestClose();
-			}
-
-		} catch (IOException ioe) {
-			outgoingConnection.reportTransmissionProblem(ioe);
-		}
-	}
-
-	private void doRead(SelectionKey key) {
-
-		final SocketChannel socketChannel = (SocketChannel) key.channel();
-		final OutgoingConnection outgoingConnection = (OutgoingConnection) key.attachment();
-		final ByteBuffer buf = ByteBuffer.allocate(8);
-
-		try {
-
-			if (socketChannel.read(buf) == -1) {
-				outgoingConnection.reportTransmissionProblem(new IOException(
-					"Read unexpected EOF from channel"));
-			} else {
-				LOG.error("Outgoing connection read real data from channel");
-			}
-		} catch (IOException ioe) {
-			outgoingConnection.reportTransmissionProblem(ioe);
-		}
-	}
-
-	public void triggerConnect(OutgoingConnection outgoingConnection) {
-
-		synchronized (this.pendingConnectionRequests) {
-			this.pendingConnectionRequests.add(outgoingConnection);
-		}
-	}
-
-	public void unsubscribeFromWriteEvent(SelectionKey selectionKey) throws IOException {
-
-		final SocketChannel socketChannel = (SocketChannel) selectionKey.channel();
-		final OutgoingConnection outgoingConnection = (OutgoingConnection) selectionKey.attachment();
-
-		final SelectionKey newSelectionKey = socketChannel.register(this.selector, SelectionKey.OP_READ);
-		newSelectionKey.attach(outgoingConnection);
-		outgoingConnection.setSelectionKey(newSelectionKey);
-
-		synchronized (this.connectionsToClose) {
-			this.connectionsToClose.put(outgoingConnection, Long.valueOf(System.currentTimeMillis()));
-		}
-	}
-
-	public void subscribeToWriteEvent(SelectionKey selectionKey) {
-
-		synchronized (this.pendingWriteEventSubscribeRequests) {
-			this.pendingWriteEventSubscribeRequests.add(selectionKey);
-		}
-		synchronized (this.connectionsToClose) {
-			this.connectionsToClose.remove((OutgoingConnection) selectionKey.attachment());
-		}
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelContext.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelContext.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelContext.java
deleted file mode 100644
index 7d8a571..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelContext.java
+++ /dev/null
@@ -1,17 +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.taskmanager.bytebuffered;
-
-public interface OutputChannelContext extends ChannelContext {
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwardingChain.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwardingChain.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwardingChain.java
deleted file mode 100644
index d9a16b7..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwardingChain.java
+++ /dev/null
@@ -1,84 +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.taskmanager.bytebuffered;
-
-import java.io.IOException;
-import java.util.Queue;
-import java.util.concurrent.LinkedBlockingDeque;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope;
-
-public final class OutputChannelForwardingChain {
-
-	private final Queue<AbstractEvent> incomingEventQueue = new LinkedBlockingDeque<AbstractEvent>();
-
-	private final AbstractOutputChannelForwarder first;
-
-	private final AbstractOutputChannelForwarder last;
-
-	public OutputChannelForwardingChain(final AbstractOutputChannelForwarder first,
-			final AbstractOutputChannelForwarder last) {
-
-		if (first == null) {
-			throw new IllegalArgumentException("Argument first must not be null");
-		}
-
-		if (last == null) {
-			throw new IllegalArgumentException("Argument last must not be null");
-		}
-
-		this.first = first;
-		this.last = last;
-	}
-
-	public void pushEnvelope(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException {
-
-		this.first.push(transferEnvelope);
-	}
-
-	public TransferEnvelope pullEnvelope() {
-
-		return this.last.pull();
-	}
-
-	public void processEvent(final AbstractEvent event) {
-
-		this.first.processEvent(event);
-	}
-
-	public boolean anyForwarderHasDataLeft() throws IOException, InterruptedException {
-
-		return this.first.hasDataLeft();
-	}
-
-	public void destroy() {
-
-		this.first.destroy();
-	}
-
-	public void processQueuedEvents() {
-
-		AbstractEvent event = this.incomingEventQueue.poll();
-		while (event != null) {
-
-			this.first.processEvent(event);
-			event = this.incomingEventQueue.poll();
-		}
-	}
-
-	void offerEvent(final AbstractEvent event) {
-		this.incomingEventQueue.offer(event);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputGateContext.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputGateContext.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputGateContext.java
deleted file mode 100644
index fb2022a..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputGateContext.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.taskmanager.bytebuffered;
-
-import eu.stratosphere.nephele.io.channels.ChannelID;
-
-public interface OutputGateContext extends GateContext {
-
-	OutputChannelContext createOutputChannelContext(ChannelID channelID, OutputChannelContext previousContext,
-			boolean isReceiverRunning, boolean mergeSpillBuffers);
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ReceiverNotFoundEvent.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ReceiverNotFoundEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ReceiverNotFoundEvent.java
deleted file mode 100644
index 0b8f351..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ReceiverNotFoundEvent.java
+++ /dev/null
@@ -1,169 +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.taskmanager.bytebuffered;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.event.task.EventList;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope;
-
-/**
- * An unknown receiver event can be used by the framework to inform a sender task that the delivery of a
- * {@link TransferEnvelope} has failed since the receiver could not be found.
- * 
- */
-public final class ReceiverNotFoundEvent extends AbstractEvent {
-
-	/**
-	 * The sequence number that will be set for transfer envelopes which contain the receiver not found event.
-	 */
-	private static final int RECEIVER_NOT_FOUND_SEQUENCE_NUMBER = 0;
-
-	/**
-	 * The ID of the receiver which could not be found
-	 */
-	private ChannelID receiverID;
-
-	/**
-	 * The sequence number of the envelope this event refers to
-	 */
-	private int sequenceNumber;
-
-	/**
-	 * Constructs a new unknown receiver event.
-	 * 
-	 * @param receiverID
-	 *        the ID of the receiver which could not be found
-	 * @param sequenceNumber
-	 *        the sequence number of the envelope this event refers to
-	 */
-	public ReceiverNotFoundEvent(final ChannelID receiverID, final int sequenceNumber) {
-
-		if (receiverID == null) {
-			throw new IllegalArgumentException("Argument unknownReceiverID must not be null");
-		}
-
-		if (sequenceNumber < 0) {
-			throw new IllegalArgumentException("Argument sequenceNumber must be non-negative");
-		}
-
-		this.receiverID = receiverID;
-		this.sequenceNumber = sequenceNumber;
-	}
-
-	/**
-	 * Default constructor for serialization/deserialization.
-	 */
-	public ReceiverNotFoundEvent() {
-
-		this.receiverID = new ChannelID();
-	}
-
-	/**
-	 * Returns the ID of the receiver which could not be found.
-	 * 
-	 * @return the ID of the receiver which could not be found
-	 */
-	public ChannelID getReceiverID() {
-
-		return this.receiverID;
-	}
-
-	/**
-	 * Returns the sequence number of the envelope this event refers to.
-	 * 
-	 * @return the sequence number of the envelope this event refers to
-	 */
-	public int getSequenceNumber() {
-
-		return this.sequenceNumber;
-	}
-
-
-	@Override
-	public void write(final DataOutput out) throws IOException {
-
-		this.receiverID.write(out);
-		out.writeInt(this.sequenceNumber);
-	}
-
-
-	@Override
-	public void read(final DataInput in) throws IOException {
-
-		this.receiverID.read(in);
-		this.sequenceNumber = in.readInt();
-	}
-
-	/**
-	 * Creates a transfer envelope which only contains a ReceiverNotFoundEvent.
-	 * 
-	 * @param jobID
-	 *        the ID of the job the event relates to.
-	 * @param receiver
-	 *        the channel ID of the receiver that could not be found
-	 * @param sequenceNumber
-	 *        the sequence number of the transfer envelope which caused the creation of this event
-	 * @return a transfer envelope which only contains a ReceiverNotFoundEvent
-	 */
-	public static TransferEnvelope createEnvelopeWithEvent(final JobID jobID, final ChannelID receiver,
-			final int sequenceNumber) {
-
-		final TransferEnvelope transferEnvelope = new TransferEnvelope(RECEIVER_NOT_FOUND_SEQUENCE_NUMBER, jobID,
-			receiver);
-
-		final ReceiverNotFoundEvent unknownReceiverEvent = new ReceiverNotFoundEvent(receiver, sequenceNumber);
-		transferEnvelope.addEvent(unknownReceiverEvent);
-
-		return transferEnvelope;
-	}
-
-	/**
-	 * Checks if the given envelope only contains a ReceiverNotFoundEvent.
-	 * 
-	 * @param transferEnvelope
-	 *        the envelope to be checked
-	 * @return <code>true</code> if the envelope only contains a ReceiverNotFoundEvent, <code>false</code> otherwise
-	 */
-	public static boolean isReceiverNotFoundEvent(final TransferEnvelope transferEnvelope) {
-
-		if (transferEnvelope.getSequenceNumber() != RECEIVER_NOT_FOUND_SEQUENCE_NUMBER) {
-			return false;
-		}
-
-		if (transferEnvelope.getBuffer() != null) {
-			return false;
-		}
-
-		final EventList eventList = transferEnvelope.getEventList();
-		if (eventList == null) {
-			return false;
-		}
-
-		if (eventList.size() != 1) {
-			return false;
-		}
-
-		if (!(eventList.get(0) instanceof ReceiverNotFoundEvent)) {
-			return false;
-		}
-
-		return true;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/RemoteReceiver.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/RemoteReceiver.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/RemoteReceiver.java
deleted file mode 100644
index db4e412..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/RemoteReceiver.java
+++ /dev/null
@@ -1,157 +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.taskmanager.bytebuffered;
-
-import java.io.DataInput;
-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}.
- * 
- */
-public final class RemoteReceiver implements IOReadableWritable {
-
-	/**
-	 * The address of the connection to the remote {@link TaskManager}.
-	 */
-	private InetSocketAddress connectionAddress;
-
-	/**
-	 * The index of the connection to the remote {@link TaskManager}.
-	 */
-	private int connectionIndex;
-
-	/**
-	 * Constructs a new remote receiver object.
-	 * 
-	 * @param connectionAddress
-	 *        the address of the connection to the remote {@link TaskManager}
-	 * @param connectionIndex
-	 *        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");
-		}
-
-		this.connectionAddress = connectionAddress;
-		this.connectionIndex = connectionIndex;
-	}
-
-	/**
-	 * Default constructor for serialization/deserialization.
-	 */
-	public RemoteReceiver() {
-		this.connectionAddress = null;
-		this.connectionIndex = -1;
-	}
-
-	/**
-	 * Returns the address of the connection to the remote {@link TaskManager}.
-	 * 
-	 * @return the address of the connection to the remote {@link TaskManager}
-	 */
-	public InetSocketAddress getConnectionAddress() {
-
-		return this.connectionAddress;
-	}
-
-	/**
-	 * Returns the index of the connection to the remote {@link TaskManager}.
-	 * 
-	 * @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);
-	}
-
-
-	@Override
-	public boolean equals(final Object obj) {
-
-		if (!(obj instanceof RemoteReceiver)) {
-			return false;
-		}
-
-		final RemoteReceiver rr = (RemoteReceiver) obj;
-		if (!this.connectionAddress.equals(rr.connectionAddress)) {
-			return false;
-		}
-
-		if (this.connectionIndex != rr.connectionIndex) {
-			return false;
-		}
-
-		return true;
-	}
-
-
-	@Override
-	public void write(final DataOutput out) throws IOException {
-
-		final InetAddress ia = this.connectionAddress.getAddress();
-		out.writeInt(ia.getAddress().length);
-		out.write(ia.getAddress());
-		out.writeInt(this.connectionAddress.getPort());
-
-		out.writeInt(this.connectionIndex);
-	}
-
-
-	@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();
-		this.connectionAddress = new InetSocketAddress(ia, port);
-
-		this.connectionIndex = in.readInt();
-	}
-
-
-	@Override
-	public String toString() {
-
-		return this.connectionAddress + " (" + this.connectionIndex + ")";
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/SenderHintEvent.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/SenderHintEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/SenderHintEvent.java
deleted file mode 100644
index a6aebb1..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/SenderHintEvent.java
+++ /dev/null
@@ -1,119 +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.taskmanager.bytebuffered;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.event.task.EventList;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope;
-
-public final class SenderHintEvent extends AbstractEvent {
-
-	/**
-	 * The sequence number that will be set for transfer envelopes which contain the sender hint event.
-	 */
-	private static final int SENDER_HINT_SEQUENCE_NUMBER = 0;
-
-	private final ChannelID source;
-
-	private final RemoteReceiver remoteReceiver;
-
-	SenderHintEvent(final ChannelID source, final RemoteReceiver remoteReceiver) {
-
-		if (source == null) {
-			throw new IllegalArgumentException("Argument source must not be null");
-		}
-
-		if (remoteReceiver == null) {
-			throw new IllegalArgumentException("Argument remoteReceiver must not be null");
-		}
-
-		this.source = source;
-		this.remoteReceiver = remoteReceiver;
-	}
-
-	public SenderHintEvent() {
-
-		this.source = new ChannelID();
-		this.remoteReceiver = new RemoteReceiver();
-	}
-
-	public ChannelID getSource() {
-
-		return this.source;
-	}
-
-	public RemoteReceiver getRemoteReceiver() {
-
-		return this.remoteReceiver;
-	}
-
-
-	@Override
-	public void write(final DataOutput out) throws IOException {
-
-		this.source.write(out);
-		this.remoteReceiver.write(out);
-	}
-
-
-	@Override
-	public void read(final DataInput in) throws IOException {
-
-		this.source.read(in);
-		this.remoteReceiver.read(in);
-	}
-
-	public static TransferEnvelope createEnvelopeWithEvent(final TransferEnvelope originalEnvelope,
-			final ChannelID source, final RemoteReceiver remoteReceiver) {
-
-		final TransferEnvelope transferEnvelope = new TransferEnvelope(SENDER_HINT_SEQUENCE_NUMBER,
-			originalEnvelope.getJobID(), originalEnvelope.getSource());
-
-		final SenderHintEvent senderEvent = new SenderHintEvent(source, remoteReceiver);
-		transferEnvelope.addEvent(senderEvent);
-
-		return transferEnvelope;
-	}
-
-	static boolean isSenderHintEvent(final TransferEnvelope transferEnvelope) {
-
-		if (transferEnvelope.getSequenceNumber() != SENDER_HINT_SEQUENCE_NUMBER) {
-			return false;
-		}
-
-		if (transferEnvelope.getBuffer() != null) {
-			return false;
-		}
-
-		final EventList eventList = transferEnvelope.getEventList();
-		if (eventList == null) {
-			return false;
-		}
-
-		if (eventList.size() != 1) {
-			return false;
-		}
-
-		if (!(eventList.get(0) instanceof SenderHintEvent)) {
-			return false;
-		}
-
-		return true;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/TaskContext.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/TaskContext.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/TaskContext.java
deleted file mode 100644
index 6c41a4f..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/TaskContext.java
+++ /dev/null
@@ -1,24 +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.taskmanager.bytebuffered;
-
-import eu.stratosphere.nephele.io.GateID;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPoolOwner;
-
-public interface TaskContext extends LocalBufferPoolOwner {
-
-	OutputGateContext createOutputGateContext(GateID gateID);
-
-	InputGateContext createInputGateContext(GateID gateID);
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/UnexpectedEnvelopeEvent.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/UnexpectedEnvelopeEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/UnexpectedEnvelopeEvent.java
deleted file mode 100644
index b7d59c6..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/UnexpectedEnvelopeEvent.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.taskmanager.bytebuffered;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-
-/**
- * This event is sent by an {@link InputChannelContext}. It indicates that the input channel context has received a
- * {@link TransferEnvelope} with a lower sequence number than expected. The typical reason for this is that data is
- * being replayed from a checkpoint. With the help of this event it is possible to request the sender to skip sending
- * transfer envelopes up to the given expected sequence number.
- * 
- */
-public final class UnexpectedEnvelopeEvent extends AbstractEvent {
-
-	/**
-	 * The expected sequence number.
-	 */
-	private int expectedSequenceNumber;
-
-	/**
-	 * Constructs a new unexpected envelope event.
-	 * 
-	 * @param expectedSequenceNumber
-	 *        the expected sequence number
-	 */
-	public UnexpectedEnvelopeEvent(final int expectedSequenceNumber) {
-
-		if (expectedSequenceNumber < 0) {
-			throw new IllegalArgumentException("Argument expectedSequenceNumber must be non-negative.");
-		}
-
-		this.expectedSequenceNumber = expectedSequenceNumber;
-	}
-
-	/**
-	 * Default constructor for serialization/deserialization.
-	 */
-	public UnexpectedEnvelopeEvent() {
-	}
-
-	/**
-	 * Returns the expected sequence number.
-	 * 
-	 * @return the expected sequence number
-	 */
-	public int getExpectedSequenceNumber() {
-
-		return this.expectedSequenceNumber;
-	}
-
-
-	@Override
-	public void write(final DataOutput out) throws IOException {
-
-		out.writeInt(this.expectedSequenceNumber);
-	}
-
-
-	@Override
-	public void read(final DataInput in) throws IOException {
-
-		this.expectedSequenceNumber = in.readInt();
-	}
-
-}

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

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java
deleted file mode 100644
index 4a104d7..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.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.taskmanager.runtime;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.AbstractOutputChannelForwarder;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.UnexpectedEnvelopeEvent;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope;
-
-public final class ForwardingBarrier extends AbstractOutputChannelForwarder {
-
-	private static final Log LOG = LogFactory.getLog(ForwardingBarrier.class);
-
-	private final ChannelID outputChannelID;
-
-	private int forwardingBarrier = -1;
-
-	public ForwardingBarrier(final ChannelID outputChannelID, final AbstractOutputChannelForwarder next) {
-		super(next);
-
-		if (next == null) {
-			throw new IllegalArgumentException("Argument next must not be null");
-		}
-
-		this.outputChannelID = outputChannelID;
-	}
-
-
-	@Override
-	public void push(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException {
-
-		if (transferEnvelope.getSequenceNumber() < this.forwardingBarrier) {
-			recycleTransferEnvelope(transferEnvelope);
-			return;
-		}
-
-		getNext().push(transferEnvelope);
-	}
-
-
-	@Override
-	public void processEvent(final AbstractEvent event) {
-
-		if (event instanceof UnexpectedEnvelopeEvent) {
-
-			final UnexpectedEnvelopeEvent uee = (UnexpectedEnvelopeEvent) event;
-			if (uee.getExpectedSequenceNumber() > this.forwardingBarrier) {
-				this.forwardingBarrier = uee.getExpectedSequenceNumber();
-				if (LOG.isDebugEnabled()) {
-					LOG.debug("Setting forwarding barrier to sequence number " + this.forwardingBarrier
-						+ " for output channel " + this.outputChannelID);
-				}
-			}
-		}
-
-		getNext().processEvent(event);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeDispatcher.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeDispatcher.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeDispatcher.java
deleted file mode 100644
index 5f14743..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeDispatcher.java
+++ /dev/null
@@ -1,38 +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.taskmanager.runtime;
-
-import java.io.IOException;
-
-import eu.stratosphere.nephele.taskmanager.bytebuffered.AbstractOutputChannelForwarder;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher;
-
-public final class RuntimeDispatcher extends AbstractOutputChannelForwarder {
-
-	private final TransferEnvelopeDispatcher dispatcher;
-
-	public RuntimeDispatcher(final TransferEnvelopeDispatcher dispatcher) {
-		super(null);
-
-		this.dispatcher = dispatcher;
-	}
-
-
-	@Override
-	public void push(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException {
-
-		this.dispatcher.processEnvelopeFromOutputChannel(transferEnvelope);
-	}
-}


[08/30] Offer buffer-oriented API for I/O (#25)

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 56c2a8e..c9323a6 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
@@ -42,6 +42,7 @@ import eu.stratosphere.pact.runtime.task.GroupReduceDriver;
 import eu.stratosphere.pact.runtime.task.chaining.ChainedCollectorMapDriver;
 import eu.stratosphere.pact.runtime.task.util.LocalStrategy;
 import eu.stratosphere.pact.runtime.task.util.TaskConfig;
+import eu.stratosphere.runtime.io.channels.ChannelType;
 import eu.stratosphere.test.iterative.nephele.JobGraphUtils;
 import eu.stratosphere.test.recordJobs.kmeans.KMeansBroadcast.PointBuilder;
 import eu.stratosphere.test.recordJobs.kmeans.KMeansBroadcast.RecomputeClusterCenter;
@@ -303,8 +304,6 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 		JobGraphUtils.connect(head, output, ChannelType.NETWORK, DistributionPattern.POINTWISE);
 		
 		JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
-		
-		
 
 		// -- instance sharing -------------------------------------------------------------------------------------
 		points.setVertexToShareInstancesWith(output);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 bac36bd..a229086 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
@@ -22,8 +22,8 @@ import eu.stratosphere.api.common.io.InputFormat;
 import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.nephele.client.JobClient;
 import eu.stratosphere.nephele.client.JobExecutionException;
-import eu.stratosphere.nephele.io.DistributionPattern;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.nephele.jobgraph.DistributionPattern;
+import eu.stratosphere.runtime.io.channels.ChannelType;
 import eu.stratosphere.nephele.jobgraph.AbstractJobVertex;
 import eu.stratosphere.nephele.jobgraph.JobGraph;
 import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 906ccc0..06badd9 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
@@ -20,8 +20,8 @@ import eu.stratosphere.api.common.typeutils.TypeSerializerFactory;
 import eu.stratosphere.api.java.record.io.FileOutputFormat;
 import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.core.fs.Path;
-import eu.stratosphere.nephele.io.DistributionPattern;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.nephele.jobgraph.DistributionPattern;
+import eu.stratosphere.runtime.io.channels.ChannelType;
 import eu.stratosphere.nephele.jobgraph.JobGraph;
 import eu.stratosphere.nephele.jobgraph.JobInputVertex;
 import eu.stratosphere.nephele.jobgraph.JobOutputVertex;
@@ -285,7 +285,7 @@ public class CustomCompensatableDanglingPageRank {
 
 		JobGraphUtils.connect(pageWithRankInput, head, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
 
-		JobGraphUtils.connect(head, intermediate, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+		JobGraphUtils.connect(head, intermediate, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
 		intermediateConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
 		
 		JobGraphUtils.connect(adjacencyListInput, intermediate, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
@@ -295,8 +295,8 @@ public class CustomCompensatableDanglingPageRank {
 		tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
 		tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(1, degreeOfParallelism);
 
-		JobGraphUtils.connect(head, output, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
-		JobGraphUtils.connect(tail, fakeTailOutput, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+		JobGraphUtils.connect(head, output, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
+		JobGraphUtils.connect(tail, fakeTailOutput, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
 
 		JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE);
 		

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 6edcef3..3e51808 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
@@ -20,8 +20,8 @@ import eu.stratosphere.api.common.typeutils.TypeSerializerFactory;
 import eu.stratosphere.api.java.record.io.FileOutputFormat;
 import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.core.fs.Path;
-import eu.stratosphere.nephele.io.DistributionPattern;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.nephele.jobgraph.DistributionPattern;
+import eu.stratosphere.runtime.io.channels.ChannelType;
 import eu.stratosphere.nephele.jobgraph.JobGraph;
 import eu.stratosphere.nephele.jobgraph.JobInputVertex;
 import eu.stratosphere.nephele.jobgraph.JobOutputVertex;
@@ -297,7 +297,7 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 
 		JobGraphUtils.connect(pageWithRankInput, head, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
 
-		JobGraphUtils.connect(head, intermediate, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+		JobGraphUtils.connect(head, intermediate, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
 		intermediateConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
 		
 		JobGraphUtils.connect(adjacencyListInput, intermediate, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
@@ -307,8 +307,8 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 		tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
 		tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(1, degreeOfParallelism);
 
-		JobGraphUtils.connect(head, output, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
-		JobGraphUtils.connect(tail, fakeTailOutput, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+		JobGraphUtils.connect(head, output, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
+		JobGraphUtils.connect(tail, fakeTailOutput, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
 
 		JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE);
 		

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 ccf529a..b50c33e 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
@@ -19,8 +19,8 @@ import eu.stratosphere.api.common.typeutils.TypePairComparatorFactory;
 import eu.stratosphere.api.common.typeutils.TypeSerializerFactory;
 import eu.stratosphere.api.java.record.io.FileOutputFormat;
 import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.nephele.io.DistributionPattern;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.nephele.jobgraph.DistributionPattern;
+import eu.stratosphere.runtime.io.channels.ChannelType;
 import eu.stratosphere.nephele.jobgraph.JobGraph;
 import eu.stratosphere.nephele.jobgraph.JobInputVertex;
 import eu.stratosphere.nephele.jobgraph.JobOutputVertex;
@@ -265,7 +265,7 @@ public class CompensatableDanglingPageRank {
 
 		JobGraphUtils.connect(pageWithRankInput, head, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
 
-		JobGraphUtils.connect(head, intermediate, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+		JobGraphUtils.connect(head, intermediate, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
 		intermediateConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
 		
 		JobGraphUtils.connect(adjacencyListInput, intermediate, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
@@ -275,8 +275,8 @@ public class CompensatableDanglingPageRank {
 		tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
 		tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(1, degreeOfParallelism);
 
-		JobGraphUtils.connect(head, output, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
-		JobGraphUtils.connect(tail, fakeTailOutput, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+		JobGraphUtils.connect(head, output, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
+		JobGraphUtils.connect(tail, fakeTailOutput, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
 
 		JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE);
 		

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query1Util/LineItemFilterTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query1Util/LineItemFilterTest.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query1Util/LineItemFilterTest.java
index 03e442e..04767c6 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query1Util/LineItemFilterTest.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query1Util/LineItemFilterTest.java
@@ -21,12 +21,12 @@ import static org.mockito.MockitoAnnotations.initMocks;
 import java.util.ArrayList;
 import java.util.List;
 
+import eu.stratosphere.runtime.io.api.RecordWriter;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 import org.mockito.Mock;
 
-import eu.stratosphere.nephele.io.AbstractRecordWriter;
 import eu.stratosphere.pact.runtime.shipping.RecordOutputCollector;
 import eu.stratosphere.test.recordJobs.util.Tuple;
 import eu.stratosphere.types.IntValue;
@@ -39,9 +39,9 @@ public class LineItemFilterTest {
 	private static final String RETURN_FLAG = "N";
 	
 	@Mock
-	AbstractRecordWriter<Record> recordWriterMock; 
+	RecordWriter<Record> recordWriterMock;
 	
-	private List<AbstractRecordWriter<Record>> writerList = new ArrayList<AbstractRecordWriter<Record>>();
+	private List<RecordWriter<Record>> writerList = new ArrayList<RecordWriter<Record>>();
 	
 	@Before
 	public void setUp()


[23/30] Offer buffer-oriented API for I/O (#25)

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractRecordWriter.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractRecordWriter.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractRecordWriter.java
deleted file mode 100644
index 9d50e9c..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractRecordWriter.java
+++ /dev/null
@@ -1,140 +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.io;
-
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
-import eu.stratosphere.nephele.event.task.EventListener;
-import eu.stratosphere.nephele.execution.Environment;
-import eu.stratosphere.nephele.io.channels.bytebuffered.EndOfSuperstepEvent;
-import eu.stratosphere.nephele.template.AbstractInvokable;
-
-/**
- * Abstract base class for a regular record writer and broadcast record writer.
- * 
- * @param <T> The type of the record that can be emitted with this record writer.
- */
-public abstract class AbstractRecordWriter<T extends IOReadableWritable> implements Writer<T> {
-
-	/**
-	 * The output gate assigned to this record writer.
-	 */
-	private OutputGate<T> outputGate;
-
-	/**
-	 * The environment associated to this record writer.
-	 */
-	private Environment environment;
-
-	/**
-	 * Constructs a new record writer and registers a new output gate with the application's environment.
-	 * 
-	 * @param invokable
-	 *        the application that instantiated the record writer
-	 * @param outputClass
-	 *        the class of records that can be emitted with this record writer
-	 * @param selector
-	 *        the channel selector to be used to determine the output channel to be used for a record
-	 * @param isBroadcast
-	 *        <code>true</code> if this record writer shall broadcast the records to all connected channels,
-	 *        <code>false/<code> otherwise
-	 */
-	public AbstractRecordWriter(AbstractInvokable invokable, Class<T> outputClass, ChannelSelector<T> selector, boolean isBroadcast) {
-		this.environment = invokable.getEnvironment();
-		connectOutputGate(outputClass, selector, isBroadcast);
-	}
-
-	/**
-	 * Connects a record writer to an output gate.
-	 * 
-	 * @param outputClass
-	 *        the class of the record that can be emitted with this record writer
-	 * @param selector
-	 *        the channel selector to be used to determine the output channel to be used for a record
-	 * @param isBroadcast
-	 *        <code>true</code> if this record writer shall broadcast the records to all connected channels,
-	 *        <code>false/<code> otherwise
-	 */
-	private void connectOutputGate(Class<T> outputClass, ChannelSelector<T> selector, boolean isBroadcast)
-	{
-		GateID gateID = this.environment.getNextUnboundOutputGateID();
-		if (gateID == null) {
-			gateID = new GateID();
-		}
-
-		this.outputGate = this.environment.createOutputGate(gateID, outputClass, selector, isBroadcast);
-		this.environment.registerOutputGate(this.outputGate);
-	}
-
-	/**
-	 * This method emits a record to the corresponding output gate. The method may block
-	 * until the record was transfered via any of the connected channels.
-	 * 
-	 * @param record
-	 *        The record to be emitted.
-	 * @throws IOException
-	 *         Thrown on an error that may happen during the transfer of the given record or a previous record.
-	 */
-	public void emit(final T record) throws IOException, InterruptedException {
-		this.outputGate.writeRecord(record);
-	}
-
-	/**
-	 * Subscribes the listener object to receive events of the given type.
-	 * 
-	 * @param eventListener
-	 *        the listener object to register
-	 * @param eventType
-	 *        the type of event to register the listener for
-	 */
-	public void subscribeToEvent(EventListener eventListener, Class<? extends AbstractTaskEvent> eventType) {
-		this.outputGate.subscribeToEvent(eventListener, eventType);
-	}
-
-	/**
-	 * Removes the subscription for events of the given type for the listener object.
-	 * 
-	 * @param eventListener
-	 *        the listener object to cancel the subscription for
-	 * @param eventType
-	 *        the type of the event to cancel the subscription for
-	 */
-	public void unsubscribeFromEvent(EventListener eventListener, Class<? extends AbstractTaskEvent> eventType) {
-		this.outputGate.unsubscribeFromEvent(eventListener, eventType);
-	}
-
-	/**
-	 * Publishes an event.
-	 * 
-	 * @param event
-	 *        the event to be published
-	 * @throws IOException
-	 *         thrown if an error occurs while transmitting the event
-	 * @throws InterruptedException
-	 *         thrown if the thread is interrupted while waiting for the event to be published
-	 */
-	public void publishEvent(AbstractTaskEvent event) throws IOException, InterruptedException {
-		this.outputGate.publishEvent(event);
-	}
-
-	public void flush() throws IOException, InterruptedException {
-		this.outputGate.flush();
-	}
-	
-	public void sendEndOfSuperstep() throws IOException, InterruptedException {
-		this.outputGate.publishEvent(EndOfSuperstepEvent.INSTANCE);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractSingleGateRecordReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractSingleGateRecordReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractSingleGateRecordReader.java
deleted file mode 100644
index 8def532..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractSingleGateRecordReader.java
+++ /dev/null
@@ -1,77 +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.io;
-
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
-import eu.stratosphere.nephele.execution.Environment;
-import eu.stratosphere.nephele.template.AbstractInvokable;
-
-/**
- * This is an abstract base class for a record reader, either dealing with mutable or immutable records.
- * 
- * @param <T> The type of the record that can be read from this record reader.
- */
-public abstract class AbstractSingleGateRecordReader<T extends IOReadableWritable> extends AbstractRecordReader {
-	
-	/**
-	 * The input gate associated with the record reader.
-	 */
-	protected final InputGate<T> inputGate;
-	
-	// --------------------------------------------------------------------------------------------
-
-	protected AbstractSingleGateRecordReader(AbstractInvokable invokable, RecordDeserializerFactory<T> deserializerFactory, int inputGateID) {
-		Environment environment = invokable.getEnvironment();
-		GateID gateID = environment.getNextUnboundInputGateID();
-		if (gateID == null) {
-			gateID = new GateID();
-		}
-
-		this.inputGate = environment.createInputGate(gateID, deserializerFactory);
-		environment.registerInputGate(this.inputGate);
-	}
-
-	/**
-	 * Returns the number of input channels wired to this reader's input gate.
-	 * 
-	 * @return the number of input channels wired to this reader's input gate
-	 */
-	public int getNumberOfInputChannels() {
-		return this.inputGate.getNumberOfInputChannels();
-	}
-
-	/**
-	 * Publishes an event.
-	 * 
-	 * @param event
-	 *        the event to be published
-	 * @throws IOException
-	 *         thrown if an error occurs while transmitting the event
-	 * @throws InterruptedException
-	 *         thrown if the thread is interrupted while waiting for the event to be published
-	 */
-	@Override
-	public void publishEvent(AbstractTaskEvent event) throws IOException, InterruptedException {
-		// Delegate call to input gate to send events
-		this.inputGate.publishEvent(event);
-	}
-	
-	
-	InputGate<T> getInputGate() {
-		return this.inputGate;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractUnionRecordReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractUnionRecordReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractUnionRecordReader.java
deleted file mode 100644
index 5de3f67..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractUnionRecordReader.java
+++ /dev/null
@@ -1,152 +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.io;
-
-import java.io.IOException;
-import java.util.ArrayDeque;
-import java.util.HashSet;
-import java.util.Set;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
-
-public abstract class AbstractUnionRecordReader<T extends IOReadableWritable> extends AbstractRecordReader implements RecordAvailabilityListener<T> {
-
-	/**
-	 * The set of all input gates.
-	 */
-	private final InputGate<T>[] allInputGates;
-	
-	/**
-	 * The set of unclosed input gates.
-	 */
-	private final Set<InputGate<T>> remainingInputGates;
-
-	/**
-	 * Queue with indices of channels that store at least one available record.
-	 */
-	private final ArrayDeque<InputGate<T>> availableInputGates = new ArrayDeque<InputGate<T>>();
-	
-	/**
-	 * The next input gate to read a record from.
-	 */
-	private InputGate<T> nextInputGateToReadFrom;
-
-	
-	@Override
-	public boolean isInputClosed() {
-		return this.remainingInputGates.isEmpty();
-	}
-	
-	/**
-	 * Constructs a new mutable union record reader.
-	 * 
-	 * @param recordReaders
-	 *        the individual mutable record readers whose input is used to construct the union
-	 */
-	@SuppressWarnings("unchecked")
-	protected AbstractUnionRecordReader(MutableRecordReader<T>[] recordReaders) {
-
-		if (recordReaders == null) {
-			throw new IllegalArgumentException("Provided argument recordReaders is null");
-		}
-
-		if (recordReaders.length < 2) {
-			throw new IllegalArgumentException(
-				"The mutable union record reader must at least be initialized with two individual mutable record readers");
-		}
-		
-		this.allInputGates = new InputGate[recordReaders.length];
-		this.remainingInputGates = new HashSet<InputGate<T>>((int) (recordReaders.length * 1.6f));
-		
-		for (int i = 0; i < recordReaders.length; i++) {
-			InputGate<T> inputGate = recordReaders[i].getInputGate();
-			inputGate.registerRecordAvailabilityListener(this);
-			this.allInputGates[i] = inputGate;
-			this.remainingInputGates.add(inputGate);
-		}
-	}
-	
-	
-	@Override
-	public void publishEvent(AbstractTaskEvent event) throws IOException, InterruptedException {
-		for (InputGate<T> gate : this.allInputGates) {
-			gate.publishEvent(event);
-		}
-	}
-	
-	@Override
-	public void reportRecordAvailability(InputGate<T> inputGate) {
-		synchronized (this.availableInputGates) {
-			this.availableInputGates.add(inputGate);
-			this.availableInputGates.notifyAll();
-		}
-	}
-	
-	protected boolean getNextRecord(T target) throws IOException, InterruptedException {
-
-		while (true) {
-			// has the current input gate more data?
-			if (this.nextInputGateToReadFrom == null) {
-				if (this.remainingInputGates.isEmpty()) {
-					return false;
-				}
-				
-				this.nextInputGateToReadFrom = getNextAvailableInputGate();
-			}
-
-			InputChannelResult result = this.nextInputGateToReadFrom.readRecord(target);
-			switch (result) {
-				case INTERMEDIATE_RECORD_FROM_BUFFER: // record is available and we can stay on the same channel
-					return true;
-					
-				case LAST_RECORD_FROM_BUFFER: // record is available, but we need to re-check the channels
-					this.nextInputGateToReadFrom = null;
-					return true;
-					
-				case END_OF_SUPERSTEP:
-					this.nextInputGateToReadFrom = null;
-					if (incrementEndOfSuperstepEventAndCheck()) {
-						return false; // end of the superstep
-					}
-					else {
-						break; // fall through and wait for next record/event
-					}
-					
-				case TASK_EVENT:	// event for the subscribers is available
-					handleEvent(this.nextInputGateToReadFrom.getCurrentEvent());
-					this.nextInputGateToReadFrom = null;
-					break;
-					
-				case END_OF_STREAM: // one gate is empty
-					this.remainingInputGates.remove(this.nextInputGateToReadFrom);
-					this.nextInputGateToReadFrom = null;
-					break;
-					
-				case NONE: // gate processed an internal event and could not return a record on this call
-					this.nextInputGateToReadFrom = null;
-					break;
-			}
-		}
-	}
-	
-	private InputGate<T> getNextAvailableInputGate() throws InterruptedException {
-		synchronized (this.availableInputGates) {
-			while (this.availableInputGates.isEmpty()) {
-				this.availableInputGates.wait();
-			}
-			return this.availableInputGates.pop();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/BroadcastRecordWriter.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/BroadcastRecordWriter.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/BroadcastRecordWriter.java
deleted file mode 100644
index f2b1141..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/BroadcastRecordWriter.java
+++ /dev/null
@@ -1,53 +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.io;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.template.AbstractInputTask;
-import eu.stratosphere.nephele.template.AbstractTask;
-
-/**
- * A record writer connects the application to an output gate. It allows the application
- * of emit (send out) to the output gate. The broadcast record writer will make sure that each emitted record will be
- * transfered via all connected output channels.
- * 
- * @param <T>
- *        the type of the record that can be emitted with this record writer
- */
-public class BroadcastRecordWriter<T extends IOReadableWritable> extends AbstractRecordWriter<T> {
-
-	/**
-	 * Constructs a new broadcast record writer and registers a new output gate with the application's environment.
-	 * 
-	 * @param taskBase
-	 *        the application that instantiated the record writer
-	 * @param outputClass
-	 *        the class of records that can be emitted with this record writer
-	 */
-	public BroadcastRecordWriter(AbstractTask taskBase, Class<T> outputClass) {
-		super(taskBase, outputClass, null, true);
-	}
-
-	/**
-	 * Constructs a new broadcast record writer and registers a new output gate with the application's environment.
-	 * 
-	 * @param inputBase
-	 *        the application that instantiated the record writer
-	 * @param outputClass
-	 *        the class of records that can be emitted with this record writer
-	 */
-	public BroadcastRecordWriter(AbstractInputTask<?> inputBase, Class<T> outputClass) {
-		super(inputBase, outputClass, null, true);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/ChannelSelector.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/ChannelSelector.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/ChannelSelector.java
deleted file mode 100644
index f7a7e9b..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/ChannelSelector.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.io;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-
-/**
- * Objects implementing this interface are passed to an {@link OutputGate}. When a record is sent through the output
- * gate, the channel selector object is called to determine to which {@link AbstractOutputChannel} objects the record
- * shall be passed on.
- * 
- * @param <T>
- *        the type of record which is sent through the attached output gate
- */
-public interface ChannelSelector<T extends IOReadableWritable> {
-
-	/**
-	 * Called to determine to which attached {@link AbstractOutputChannel} objects the given record shall be forwarded.
-	 * 
-	 * @param record
-	 *        the record to the determine the output channels for
-	 * @param numberOfOutputChannels
-	 *        the total number of output channels which are attached to respective output gate
-	 * @return a (possibly empty) array of integer numbers which indicate the indices of the output channels through
-	 *         which the record shall be forwarded
-	 */
-	int[] selectChannels(T record, int numberOfOutputChannels);
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/DataOutputBuffer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/DataOutputBuffer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/DataOutputBuffer.java
deleted file mode 100644
index 171e985..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/DataOutputBuffer.java
+++ /dev/null
@@ -1,165 +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.
- **********************************************************************************************************************/
-
-/**
- * This file is based on source code from the Hadoop Project (http://hadoop.apache.org/), licensed by the Apache
- * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for
- * additional information regarding copyright ownership. 
- */
-
-package eu.stratosphere.nephele.io;
-
-import java.io.DataInput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-
-/**
- * A reusable {@link DataOutput} implementation that writes to an in-memory
- * buffer.
- * <p>
- * This saves memory over creating a new DataOutputStream and ByteArrayOutputStream each time data is written.
- * <p>
- * Typical usage is something like the following:
- * 
- * <pre>
- * 
- * DataOutputBuffer buffer = new DataOutputBuffer();
- * while (... loop condition ...) {
- *   buffer.reset();
- *   ... write buffer using DataOutput methods ...
- *   byte[] data = buffer.getData();
- *   int dataLength = buffer.getLength();
- *   ... write data to its ultimate destination ...
- * }
- * </pre>
- */
-public class DataOutputBuffer extends DataOutputStream {
-
-	private static class ByteBufferedOutputStream extends OutputStream {
-
-		private ByteBuffer buf;
-
-		public ByteBuffer getData() {
-			return this.buf;
-		}
-
-		public int getLength() {
-			return this.buf.limit();
-		}
-
-		public ByteBufferedOutputStream() {
-			this(1024);
-		}
-
-		public ByteBufferedOutputStream(int size) {
-			this.buf = ByteBuffer.allocate(size);
-			this.buf.position(0);
-			this.buf.limit(0);
-		}
-
-		public void reset() {
-			this.buf.position(0);
-			this.buf.limit(0);
-		}
-
-		public void write(DataInput in, int len) throws IOException {
-
-			final int newcount = this.buf.limit() + len;
-			if (newcount > this.buf.capacity()) {
-				final ByteBuffer newBuf = ByteBuffer.allocate(Math.max(this.buf.capacity() << 1, newcount));
-				newBuf.position(0);
-				System.arraycopy(this.buf.array(), 0, newBuf.array(), 0, this.buf.limit());
-				newBuf.limit(this.buf.limit());
-				this.buf = newBuf;
-			}
-
-			in.readFully(this.buf.array(), this.buf.limit(), len);
-			this.buf.limit(newcount);
-		}
-
-		@Override
-		public void write(byte[] b, int off, int len) throws IOException {
-
-			final int newcount = this.buf.limit() + len;
-			if (newcount > this.buf.capacity()) {
-				increaseInternalBuffer(newcount);
-			}
-
-			System.arraycopy(b, off, this.buf.array(), this.buf.limit(), len);
-			this.buf.limit(newcount);
-		}
-
-		@Override
-		public void write(byte[] b) throws IOException {
-			write(b, 0, b.length);
-		}
-
-		@Override
-		public void write(int arg0) throws IOException {
-
-			final int oldLimit = this.buf.limit();
-			final int newLimit = oldLimit + 1;
-
-			if (newLimit > this.buf.capacity()) {
-				increaseInternalBuffer(newLimit);
-			}
-
-			this.buf.limit(newLimit);
-			this.buf.put(oldLimit, (byte) arg0);
-		}
-
-		private void increaseInternalBuffer(int minimumRequiredSize) {
-			final ByteBuffer newBuf = ByteBuffer.allocate(Math.max(this.buf.capacity() << 1, minimumRequiredSize));
-			newBuf.position(0);
-			System.arraycopy(this.buf.array(), 0, newBuf.array(), 0, this.buf.limit());
-			newBuf.limit(this.buf.limit());
-			this.buf = newBuf;
-		}
-	}
-
-	private final ByteBufferedOutputStream byteBufferedOutputStream;
-
-	/** Constructs a new empty buffer. */
-	public DataOutputBuffer() {
-		this(new ByteBufferedOutputStream());
-	}
-
-	public DataOutputBuffer(int size) {
-		this(new ByteBufferedOutputStream(size));
-	}
-
-	private DataOutputBuffer(ByteBufferedOutputStream byteBufferedOutputStream) {
-		super(byteBufferedOutputStream);
-		this.byteBufferedOutputStream = byteBufferedOutputStream;
-	}
-
-	public ByteBuffer getData() {
-		return this.byteBufferedOutputStream.getData();
-	}
-
-	public int getLength() {
-		return this.byteBufferedOutputStream.getLength();
-	}
-
-	/** Resets the buffer to empty. */
-	public DataOutputBuffer reset() {
-		this.byteBufferedOutputStream.reset();
-		return this;
-	}
-
-	public void write(DataInput in, int length) throws IOException {
-		this.byteBufferedOutputStream.write(in, length);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/DefaultChannelSelector.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/DefaultChannelSelector.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/DefaultChannelSelector.java
deleted file mode 100644
index cabc208..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/DefaultChannelSelector.java
+++ /dev/null
@@ -1,47 +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.io;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-
-/**
- * This is the default implementation of the {@link ChannelSelector} interface. It represents a simple round-robin
- * strategy, i.e. regardless of the record every attached exactly one output channel is selected at a time.
-
- * @param <T>
- *        the type of record which is sent through the attached output gate
- */
-public class DefaultChannelSelector<T extends IOReadableWritable> implements ChannelSelector<T> {
-
-	/**
-	 * Stores the index of the channel to send the next record to.
-	 */
-	private final int[] nextChannelToSendTo = new int[1];
-
-	/**
-	 * Constructs a new default channel selector.
-	 */
-	public DefaultChannelSelector() {
-		this.nextChannelToSendTo[0] = 0;
-	}
-
-
-	@Override
-	public int[] selectChannels(final T record, final int numberOfOutpuChannels) {
-
-		this.nextChannelToSendTo[0] = (this.nextChannelToSendTo[0] + 1) % numberOfOutpuChannels;
-
-		return this.nextChannelToSendTo;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/DistributionPattern.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/DistributionPattern.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/DistributionPattern.java
deleted file mode 100644
index 634fbcc..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/DistributionPattern.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.io;
-
-/**
- * A distribution pattern determines which subtasks of a producing Nephele task a wired to which
- * subtasks of a consuming subtask.
- * 
- */
-
-public enum DistributionPattern {
-
-	/**
-	 * Each subtask of the producing Nephele task is wired to each subtask of the consuming Nephele task.
-	 */
-	BIPARTITE,
-
-	/**
-	 * The i-th subtask of the producing Nephele task is wired to the i-th subtask of the consuming Nephele task.
-	 */
-	POINTWISE
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/Gate.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/Gate.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/Gate.java
deleted file mode 100644
index 26773ca..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/Gate.java
+++ /dev/null
@@ -1,126 +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.io;
-
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
-import eu.stratosphere.nephele.event.task.EventListener;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-import eu.stratosphere.nephele.jobgraph.JobID;
-
-public interface Gate<T extends IOReadableWritable> {
-
-	/**
-	 * Returns the index that has been assigned to the gate upon initialization.
-	 * 
-	 * @return the index that has been assigned to the gate upon initialization.
-	 */
-	int getIndex();
-
-	/**
-	 * Subscribes the listener object to receive events of the given type.
-	 * 
-	 * @param eventListener
-	 *        the listener object to register
-	 * @param eventType
-	 *        the type of event to register the listener for
-	 */
-	void subscribeToEvent(EventListener eventListener, Class<? extends AbstractTaskEvent> eventType);
-
-	/**
-	 * Removes the subscription for events of the given type for the listener object.
-	 * 
-	 * @param eventListener
-	 *        the listener object to cancel the subscription for
-	 * @param eventType
-	 *        the type of the event to cancel the subscription for
-	 */
-	void unsubscribeFromEvent(EventListener eventListener, Class<? extends AbstractTaskEvent> eventType);
-
-	/**
-	 * Publishes an event.
-	 * 
-	 * @param event
-	 *        the event to be published
-	 * @throws IOException
-	 *         thrown if an error occurs while transmitting the event
-	 * @throws InterruptedException
-	 *         thrown if the thread is interrupted while waiting for the event to be published
-	 */
-	void publishEvent(AbstractEvent event) throws IOException, InterruptedException;
-
-	/**
-	 * Passes a received event on to the event notification manager so it cam ne dispatched.
-	 * 
-	 * @param event
-	 *        the event to pass on to the notification manager
-	 */
-	void deliverEvent(AbstractTaskEvent event);
-
-	/**
-	 * Returns the ID of the job this gate belongs to.
-	 * 
-	 * @return the ID of the job this gate belongs to
-	 */
-	JobID getJobID();
-
-	/**
-	 * Returns the type of the input/output channels which are connected to this gate.
-	 * 
-	 * @return the type of input/output channels which are connected to this gate
-	 */
-	ChannelType getChannelType();
-
-	/**
-	 * Returns the ID of the gate.
-	 * 
-	 * @return the ID of the gate
-	 */
-	GateID getGateID();
-
-	/**
-	 * Releases the allocated resources (particularly buffer) of all channels attached to this gate. This method
-	 * should only be called after the respected task has stopped running.
-	 */
-	void releaseAllChannelResources();
-
-	/**
-	 * Checks if the gate is closed. The gate is closed if all this associated channels are closed.
-	 * 
-	 * @return <code>true</code> if the gate is closed, <code>false</code> otherwise
-	 * @throws IOException
-	 *         thrown if any error occurred while closing the gate
-	 * @throws InterruptedException
-	 *         thrown if the gate is interrupted while waiting for this operation to complete
-	 */
-	boolean isClosed() throws IOException, InterruptedException;
-
-	/**
-	 * Checks if the considered gate is an input gate.
-	 * 
-	 * @return <code>true</code> if the considered gate is an input gate, <code>false</code> if it is an output gate
-	 */
-	boolean isInputGate();
-
-	/**
-	 * Sets the type of the input/output channels which are connected to this gate.
-	 * 
-	 * @param channelType
-	 *        the type of input/output channels which are connected to this gate
-	 */
-	void setChannelType(ChannelType channelType);
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/GateID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/GateID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/GateID.java
deleted file mode 100644
index 9998916..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/GateID.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.io;
-
-/**
- * A class for statistically unique gate IDs.
- * 
- */
-public final class GateID extends AbstractID {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/ImmutableRecordDeserializerFactory.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/ImmutableRecordDeserializerFactory.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/ImmutableRecordDeserializerFactory.java
deleted file mode 100644
index be0a60f..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/ImmutableRecordDeserializerFactory.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.io;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.io.channels.DefaultDeserializer;
-
-/**
- * As simple factory implementation that instantiates deserializers for immutable records. For
- * each deserialization, a new record is instantiated from the given class.
- */
-public class ImmutableRecordDeserializerFactory<T extends IOReadableWritable> implements RecordDeserializerFactory<T> {
-	
-	private final Class<? extends T> recordType;			// the type of the record to be deserialized
-	
-	
-	/**
-	 * Creates a new factory that instantiates deserializers for immutable records.
-	 * 
-	 * @param recordType The type of the record to be deserialized.
-	 */
-	public ImmutableRecordDeserializerFactory(final Class<? extends T> recordType) {
-		this.recordType = recordType;
-	}
-
-	@Override
-	public RecordDeserializer<T> createDeserializer() {
-		return new DefaultDeserializer<T>(this.recordType);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/InputChannelResult.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/InputChannelResult.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/InputChannelResult.java
deleted file mode 100644
index f154a3f..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/InputChannelResult.java
+++ /dev/null
@@ -1,23 +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.io;
-
-public enum InputChannelResult {
-
-	NONE,
-	INTERMEDIATE_RECORD_FROM_BUFFER,
-	LAST_RECORD_FROM_BUFFER,
-	END_OF_SUPERSTEP,
-	TASK_EVENT,
-	END_OF_STREAM;
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/InputGate.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/InputGate.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/InputGate.java
deleted file mode 100644
index 6a57756..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/InputGate.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.nephele.io;
-
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
-import eu.stratosphere.nephele.io.channels.AbstractInputChannel;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.bytebuffered.InMemoryInputChannel;
-import eu.stratosphere.nephele.io.channels.bytebuffered.NetworkInputChannel;
-
-/**
- * @param <T> The type of record that can be transported through this gate.
- */
-public interface InputGate<T extends IOReadableWritable> extends Gate<T> {
-
-	/**
-	 * Reads a record from one of the associated input channels. Channels are read such that one buffer from a channel is 
-	 * consecutively consumed. The buffers in turn are consumed in the order in which they arrive.
-	 * Note that this method is not guaranteed to return a record, because the currently available channel data may not always
-	 * constitute an entire record, when events or partial records are part of the data.
-	 * 
-	 * When called even though no data is available, this call will block until data is available, so this method should be called
-	 * when waiting is desired (such as when synchronously consuming a single gate) or only when it is known that data is available
-	 * (such as when reading a union of multiple input gates).
-	 * 
-	 * @param target The record object into which to construct the complete record.
-	 * @return The result indicating whether a complete record is available, a event is available, only incomplete data
-	 *         is available (NONE), or the gate is exhausted.
-	 * @throws IOException Thrown when an error occurred in the network stack relating to this channel.
-	 * @throws InterruptedException Thrown, when the thread working on this channel is interrupted.
-	 */
-	InputChannelResult readRecord(T target) throws IOException, InterruptedException;
-
-	/**
-	 * Returns the number of input channels associated with this input gate.
-	 * 
-	 * @return the number of input channels associated with this input gate
-	 */
-	int getNumberOfInputChannels();
-
-	/**
-	 * Returns the input channel from position <code>pos</code> of the gate's internal channel list.
-	 * 
-	 * @param pos
-	 *        the position to retrieve the channel from
-	 * @return the channel from the given position or <code>null</code> if such position does not exist.
-	 */
-	AbstractInputChannel<T> getInputChannel(int pos);
-
-	/**
-	 * Notify the gate that the channel with the given index has
-	 * at least one record available.
-	 * 
-	 * @param channelIndex
-	 *        the index of the channel which has at least one record available
-	 */
-	void notifyRecordIsAvailable(int channelIndex);
-
-	/**
-	 * Notify the gate that is has consumed a data unit from the channel with the given index
-	 * 
-	 * @param channelIndex
-	 *        the index of the channel from which a data unit has been consumed
-	 */
-	void notifyDataUnitConsumed(int channelIndex);
-
-	/**
-	 * Immediately closes the input gate and all its input channels. The corresponding
-	 * output channels are notified. Any remaining records in any buffers or queue is considered
-	 * irrelevant and is discarded.
-	 * 
-	 * @throws IOException
-	 *         thrown if an I/O error occurs while closing the gate
-	 * @throws InterruptedException
-	 *         thrown if the thread is interrupted while waiting for the gate to be closed
-	 */
-	void close() throws IOException, InterruptedException;
-
-	/**
-	 * Creates a new network input channel and assigns it to the given input gate.
-	 * 
-	 * @param inputGate
-	 *        the input gate the channel shall be assigned to
-	 * @param channelID
-	 *        the ID of the channel
-	 * @param connectedChannelID
-	 *        the ID of the channel this channel is connected to
-	 * @param compressionLevel
-	 *        the level of compression to be used for this channel
-	 * @return the new network input channel
-	 */
-	NetworkInputChannel<T> createNetworkInputChannel(InputGate<T> inputGate, ChannelID channelID,
-			ChannelID connectedChannelID);
-
-
-	/**
-	 * Creates a new in-memory input channel and assigns it to the given input gate.
-	 * 
-	 * @param inputGate
-	 *        the input gate the channel shall be assigned to
-	 * @param channelID
-	 *        the ID of the channel
-	 * @param connectedChannelID
-	 *        the ID of the channel this channel is connected to
-	 * @param compressionLevel
-	 *        the level of compression to be used for this channel
-	 * @return the new in-memory input channel
-	 */
-	InMemoryInputChannel<T> createInMemoryInputChannel(InputGate<T> inputGate, ChannelID channelID,
-			ChannelID connectedChannelID);
-
-	/**
-	 * Registers a {@link RecordAvailabilityListener} with this input gate.
-	 * 
-	 * @param listener
-	 *        the listener object to be registered
-	 */
-	void registerRecordAvailabilityListener(RecordAvailabilityListener<T> listener);
-	
-	
-	AbstractTaskEvent getCurrentEvent();
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableReader.java
deleted file mode 100644
index ef1080c..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableReader.java
+++ /dev/null
@@ -1,32 +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.io;
-
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-
-/**
- * 
- */
-public interface MutableReader<T extends IOReadableWritable> extends ReaderBase {
-	
-	/**
-	 * @param target
-	 * @return
-	 * @throws IOException
-	 * @throws InterruptedException
-	 */
-	boolean next(T target) throws IOException, InterruptedException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableRecordDeserializerFactory.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableRecordDeserializerFactory.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableRecordDeserializerFactory.java
deleted file mode 100644
index 5890562..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableRecordDeserializerFactory.java
+++ /dev/null
@@ -1,52 +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.io;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.io.channels.DefaultDeserializer;
-
-/**
- * As simple factory implementation that instantiates deserializers for mutable records.
- */
-public class MutableRecordDeserializerFactory<T extends IOReadableWritable> implements RecordDeserializerFactory<T> {
-	
-	/**
-	 * Creates a new factory that instantiates deserializers for immutable records.
-	 * 
-	 * @param recordType The type of the record to be deserialized.
-	 */
-	public MutableRecordDeserializerFactory() {}
-
-	@Override
-	public RecordDeserializer<T> createDeserializer() {
-		return new DefaultDeserializer<T>(null);
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	
-	private static final RecordDeserializerFactory<IOReadableWritable> INSTANCE = 
-									new MutableRecordDeserializerFactory<IOReadableWritable>();
-	
-	/**
-	 * Gets the singleton instance of the {@code MutableRecordDeserializerFactory}.
-	 * 
-	 * @param <E> The generic type of the record to be deserialized.
-	 * @return An instance of the factory.
-	 */
-	public static final <E extends IOReadableWritable> RecordDeserializerFactory<E> get() {
-		@SuppressWarnings("unchecked")
-		RecordDeserializerFactory<E> toReturn = (RecordDeserializerFactory<E>) INSTANCE;
-		return toReturn;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableRecordReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableRecordReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableRecordReader.java
deleted file mode 100644
index 23c26c4..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableRecordReader.java
+++ /dev/null
@@ -1,119 +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.io;
-
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.template.AbstractOutputTask;
-import eu.stratosphere.nephele.template.AbstractTask;
-
-public class MutableRecordReader<T extends IOReadableWritable> extends AbstractSingleGateRecordReader<T> implements MutableReader<T> {
-	
-	private boolean endOfStream;
-	
-	
-	/**
-	 * Constructs a new mutable record reader and registers a new input gate with the application's environment.
-	 * 
-	 * @param taskBase The application that instantiated the record reader.
-	 */
-	public MutableRecordReader(final AbstractTask taskBase) {
-		super(taskBase, MutableRecordDeserializerFactory.<T>get(), 0);
-	}
-
-	/**
-	 * 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, MutableRecordDeserializerFactory.<T>get(), 0);
-	}
-
-	/**
-	 * 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, MutableRecordDeserializerFactory.<T>get(), inputGateID);
-	}
-
-	/**
-	 * 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, MutableRecordDeserializerFactory.<T>get(), inputGateID);
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	
-	@Override
-	public boolean next(final T target) throws IOException, InterruptedException {
-		if (this.endOfStream) {
-			return false;
-			
-		}
-		while (true) {
-			InputChannelResult result = this.inputGate.readRecord(target);
-			switch (result) {
-				case INTERMEDIATE_RECORD_FROM_BUFFER:
-				case LAST_RECORD_FROM_BUFFER:
-					return true;
-					
-				case END_OF_SUPERSTEP:
-					if (incrementEndOfSuperstepEventAndCheck()) {
-						return false; // end of the superstep
-					}
-					else {
-						break; // fall through and wait for next record/event
-					}
-					
-				case TASK_EVENT:
-					handleEvent(this.inputGate.getCurrentEvent());
-					break;	// fall through to get next record
-				
-				case END_OF_STREAM:
-					this.endOfStream = true;
-					return false;
-					
-				default:
-					; // fall through to get next record
-			}
-		}
-	}
-	
-	@Override
-	public boolean isInputClosed() {
-		return this.endOfStream;
-	}
-
-	@Override
-	public void setIterative(int numEventsUntilEndOfSuperstep) {
-		// sanity check for debug purposes
-		if (numEventsUntilEndOfSuperstep != getNumberOfInputChannels()) {
-			throw new IllegalArgumentException("Number of events till end of superstep is different from the number of input channels.");
-		}
-		super.setIterative(numEventsUntilEndOfSuperstep);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableUnionRecordReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableUnionRecordReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableUnionRecordReader.java
deleted file mode 100644
index 1fc8dd2..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableUnionRecordReader.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.io;
-
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-
-public class MutableUnionRecordReader<T extends IOReadableWritable> extends AbstractUnionRecordReader<T> implements MutableReader<T> {
-
-	
-	/**
-	 * Constructs a new mutable union record reader.
-	 * 
-	 * @param recordReaders
-	 *        the individual mutable record readers whose input is used to construct the union
-	 */
-	public MutableUnionRecordReader(MutableRecordReader<T>[] recordReaders) {
-		super(recordReaders);
-	}
-
-	@Override
-	public boolean next(T target) throws IOException, InterruptedException {
-		return getNextRecord(target);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/OutputGate.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/OutputGate.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/OutputGate.java
deleted file mode 100644
index 0bbe5e0..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/OutputGate.java
+++ /dev/null
@@ -1,149 +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.io;
-
-import java.io.IOException;
-import java.util.List;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.io.channels.AbstractOutputChannel;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.bytebuffered.InMemoryOutputChannel;
-import eu.stratosphere.nephele.io.channels.bytebuffered.NetworkOutputChannel;
-
-/**
- * In Nephele output gates are a specialization of general gates and connect
- * record writers and output channels. As channels, output gates are always
- * parameterized to a specific type of record which they can transport.
- * 
- * @param <T>
- *        the type of record that can be transported through this gate
- */
-public interface OutputGate<T extends IOReadableWritable> extends Gate<T> {
-
-	/**
-	 * Returns the type of record that can be transported through this gate.
-	 * 
-	 * @return the type of record that can be transported through this gate
-	 */
-	Class<T> getType();
-
-	/**
-	 * Writes a record to one of the associated output channels. Currently, the
-	 * channels are chosen in a simple round-robin fashion. This operation may
-	 * block until the respective channel has received the data.
-	 * 
-	 * @param record
-	 *        the record to be written
-	 * @throws IOException
-	 *         thrown if any error occurs during channel I/O
-	 */
-	void writeRecord(T record) throws IOException, InterruptedException;
-
-	/**
-	 * Returns all the OutputChannels connected to this gate
-	 * 
-	 * @return the list of OutputChannels connected to this RecordWriter
-	 */
-	List<AbstractOutputChannel<T>> getOutputChannels();
-
-	/**
-	 * Flushes all connected output channels.
-	 * 
-	 * @throws IOException
-	 *         thrown if an error occurs while flushing an output channel
-	 * @throws InterruptedException
-	 *         thrown if the thread is interrupted while waiting for the data to be flushed
-	 */
-	void flush() throws IOException, InterruptedException;
-
-	/**
-	 * Checks if this output gate operates in broadcast mode, i.e. all records passed to it are transferred through all
-	 * connected output channels.
-	 * 
-	 * @return <code>true</code> if this output gate operates in broadcast mode, <code>false</code> otherwise
-	 */
-	boolean isBroadcast();
-
-	/**
-	 * Returns the number of output channels associated with this output gate.
-	 * 
-	 * @return the number of output channels associated with this output gate
-	 */
-	int getNumberOfOutputChannels();
-
-	/**
-	 * Returns the output channel from position <code>pos</code> of the gate's
-	 * internal channel list.
-	 * 
-	 * @param pos
-	 *        the position to retrieve the channel from
-	 * @return the channel from the given position or <code>null</code> if such
-	 *         position does not exist.
-	 */
-	AbstractOutputChannel<T> getOutputChannel(int pos);
-
-	/**
-	 * Returns the output gate's channel selector.
-	 * 
-	 * @return the output gate's channel selector or <code>null</code> if the gate operates in broadcast mode
-	 */
-	ChannelSelector<T> getChannelSelector();
-
-	/**
-	 * Requests the output gate to closed. This means the application will send
-	 * no records through this gate anymore.
-	 * 
-	 * @throws IOException
-	 * @throws InterruptedException
-	 */
-	void requestClose() throws IOException, InterruptedException;
-
-	/**
-	 * Removes all output channels from the output gate.
-	 */
-	void removeAllOutputChannels();
-
-	/**
-	 * Creates a new network output channel and assigns it to the given output gate.
-	 * 
-	 * @param outputGate
-	 *        the output gate the channel shall be assigned to
-	 * @param channelID
-	 *        the ID of the channel
-	 * @param connectedChannelID
-	 *        the ID of the channel this channel is connected to
-	 * @param compressionLevel
-	 *        the level of compression to be used for this channel
-	 * @return the new network output channel
-	 */
-	NetworkOutputChannel<T> createNetworkOutputChannel(OutputGate<T> outputGate, ChannelID channelID,
-			ChannelID connectedChannelID);
-
-	/**
-	 * Creates a new in-memory output channel and assigns it to the given output gate.
-	 * 
-	 * @param outputGate
-	 *        the output gate the channel shall be assigned to
-	 * @param channelID
-	 *        the ID of the channel
-	 * @param connectedChannelID
-	 *        the ID of the channel this channel is connected to
-	 * @param compressionLevel
-	 *        the level of compression to be used for this channel
-	 * @return the new in-memory output channel
-	 */
-	InMemoryOutputChannel<T> createInMemoryOutputChannel(OutputGate<T> outputGate, ChannelID channelID,
-			ChannelID connectedChannelID);
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/Reader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/Reader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/Reader.java
deleted file mode 100644
index 80d2010..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/Reader.java
+++ /dev/null
@@ -1,30 +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.io;
-
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-
-/**
- * A reader interface to read records from an input.
- * 
- * @param <T> The type of the record that can be emitted with this record writer
- */
-public interface Reader<T extends IOReadableWritable> extends ReaderBase {
-
-	boolean hasNext() throws IOException, InterruptedException;
-
-	T next() throws IOException, InterruptedException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/ReaderBase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/ReaderBase.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/ReaderBase.java
deleted file mode 100644
index ae69ad0..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/ReaderBase.java
+++ /dev/null
@@ -1,67 +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.io;
-
-import java.io.IOException;
-
-import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
-import eu.stratosphere.nephele.event.task.EventListener;
-
-
-/**
- *
- */
-public interface ReaderBase {
-
-	boolean isInputClosed();
-	
-	/**
-	 * Subscribes the listener object to receive events of the given type.
-	 * 
-	 * @param eventListener
-	 *        the listener object to register
-	 * @param eventType
-	 *        the type of event to register the listener for
-	 */
-	void subscribeToEvent(EventListener eventListener, Class<? extends AbstractTaskEvent> eventType);
-	
-	/**
-	 * Removes the subscription for events of the given type for the listener object.
-	 * 
-	 * @param eventListener
-	 *        the listener object to cancel the subscription for
-	 * @param eventType
-	 *        the type of the event to cancel the subscription for
-	 */
-	void unsubscribeFromEvent(final EventListener eventListener, final Class<? extends AbstractTaskEvent> eventType);
-
-	/**
-	 * Publishes an event.
-	 * 
-	 * @param event
-	 *        the event to be published
-	 * @throws IOException
-	 *         thrown if an error occurs while transmitting the event
-	 * @throws InterruptedException
-	 *         thrown if the thread is interrupted while waiting for the event to be published
-	 */
-	void publishEvent(AbstractTaskEvent event) throws IOException, InterruptedException;
-	
-	
-	void setIterative(int numEventsUntilEndOfSuperstep);
-
-	
-	void startNextSuperstep();
-	
-	boolean hasReachedEndOfSuperstep();
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordAvailabilityListener.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordAvailabilityListener.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordAvailabilityListener.java
deleted file mode 100644
index d22f048..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordAvailabilityListener.java
+++ /dev/null
@@ -1,35 +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.io;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-
-/**
- * This interface can be implemented by a class which shall be notified by an input gate when one of the its connected
- * input channels has at least one record available for reading.
- * 
- * @param <T>
- *        the type of record transported through the corresponding input gate
- */
-public interface RecordAvailabilityListener<T extends IOReadableWritable> {
-
-	/**
-	 * This method is called by an input gate when one of its connected input channels has at least one record available
-	 * for reading.
-	 * 
-	 * @param inputGate
-	 *        the input gate which has at least one record available
-	 */
-	void reportRecordAvailability(InputGate<T> inputGate);
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordDeserializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordDeserializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordDeserializer.java
deleted file mode 100644
index e8ccb77..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordDeserializer.java
+++ /dev/null
@@ -1,55 +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.io;
-
-import java.io.IOException;
-import java.nio.channels.ReadableByteChannel;
-
-/**
- * This interface must be implemented by classes which transfer bytes streams back into {@link Record} objects.
- * 
- * @param <T> The type of record this record deserializer works with.
- */
-public interface RecordDeserializer<T>
-{
-	/**
-	 * Transforms a record back from a readable byte channel. The deserialization might not complete, because the channel
-	 * has not all required data available. In that case, this method must return {@code null}. Furthermore, it may
-	 * not retain a reference to the given target object in that case, but must manage to put the data aside.
-	 * 
-	 * @param target The record object into which to deserialize the data. May be null for deserializers
-	 *               that operate on immutable objects, in which case the deserializer has to instantiate an
-	 *               object. In the case where this object is non-null, but the deserialization does not complete,
-	 *               the object must not be used to cache the partial state, as it is not guaranteed that the object
-	 *               will remain unchanged until the next attempt to continue the deserialization.
-	 * @param in The byte stream which contains the record's data.
-	 * @return The record deserialized from <code>in</code>, or null, if the record .
-	 * @throws IOException Thrown if an I/O error occurs while deserializing the record from the stream
-	 */
-	T readData(final T target, final ReadableByteChannel readableByteChannel) throws IOException;
-
-	/**
-	 * Clears the internal buffers of the deserializer and resets its state.
-	 */
-	void clear();
-	
-	/**
-	 * Checks whether the deserializer has data from a previous deserialization attempt stored in its internal buffers which
-	 * is not yet finished.
-	 * 
-	 * @return <code>true</code>, if the deserializer's internal buffers contain data from a previous deserialization
-	 *         attempt, <code>false</code> otherwise.
-	 */
-	boolean hasUnfinishedData();
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordDeserializerFactory.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordDeserializerFactory.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordDeserializerFactory.java
deleted file mode 100644
index eadd326..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordDeserializerFactory.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/***********************************************************************************************************************
- * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
- *
- * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- **********************************************************************************************************************/
-
-package eu.stratosphere.nephele.io;
-
-/**
- * A simple factory to instantiate record deserializer objects. Since a deserializer might be stateful, the system
- * must be able to instantiate an arbitrary number of them, equal to the number of data channels.
- * 
- * If the created deserializers are in fact not stateful, the factory should return a shared object.
- */
-public interface RecordDeserializerFactory<T>
-{
-	/**
-	 * Creates a new instance of the deserializer. The returned instance may not share any state with
-	 * any previously returned instance.
-	 * 
-	 * @return An instance of the deserializer.
-	 */
-	RecordDeserializer<T> createDeserializer();
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordReader.java
deleted file mode 100644
index 5010013..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordReader.java
+++ /dev/null
@@ -1,153 +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.io;
-
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.template.AbstractOutputTask;
-import eu.stratosphere.nephele.template.AbstractTask;
-
-/**
- * A record writer connects an input gate to an application. It allows the application
- * query for incoming records and read them from input gate.
- * 
- * @param <T> The type of the record that can be read from this record reader.
- */
-public class RecordReader<T extends IOReadableWritable> extends AbstractSingleGateRecordReader<T> implements Reader<T> {
-	
-	private final Class<T> recordType;
-	
-	/**
-	 * Stores the last read record.
-	 */
-	private T lookahead;
-
-	/**
-	 * Stores if more no more records will be received from the assigned input gate.
-	 */
-	private boolean noMoreRecordsWillFollow;
-
-	// --------------------------------------------------------------------------------------------
-	
-	/**
-	 * 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 recordType
-	 *        The class of records that can be read from the record reader.
-	 */
-	public RecordReader(AbstractTask taskBase, Class<T> recordType) {
-		super(taskBase, MutableRecordDeserializerFactory.<T>get(), 0);
-		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, MutableRecordDeserializerFactory.<T>get(), 0);
-		this.recordType = recordType;
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	
-	/**
-	 * Checks if at least one more record can be read from the associated input gate. This method may block
-	 * until the associated input gate is able to read the record from one of its input channels.
-	 * 
-	 * @return <code>true</code>it at least one more record can be read from the associated input gate, otherwise
-	 *         <code>false</code>
-	 */
-	@Override
-	public boolean hasNext() throws IOException, InterruptedException{
-		if (this.lookahead != null) {
-			return true;
-		} else {
-			if (this.noMoreRecordsWillFollow) {
-				return false;
-			}
-			
-			T record = instantiateRecordType();
-			
-			while (true) {
-				InputChannelResult result = this.inputGate.readRecord(record);
-				switch (result) {
-					case INTERMEDIATE_RECORD_FROM_BUFFER:
-					case LAST_RECORD_FROM_BUFFER:
-						this.lookahead = record;
-						return true;
-						
-					case END_OF_SUPERSTEP:
-						if (incrementEndOfSuperstepEventAndCheck()) {
-							return false;
-						}
-						else {
-							break; // fall through and wait for next record/event
-						}
-						
-					case TASK_EVENT:
-						handleEvent(this.inputGate.getCurrentEvent());
-						break;
-						
-					case END_OF_STREAM:
-						this.noMoreRecordsWillFollow = true;
-						return false;
-				
-					default:
-						; // fall through the loop
-				}
-			}
-		}
-	}
-
-	/**
-	 * Reads the current record from the associated input gate.
-	 * 
-	 * @return the current record from the associated input gate.
-	 * @throws IOException
-	 *         thrown if any error occurs while reading the record from the input gate
-	 */
-	@Override
-	public T next() throws IOException, InterruptedException {
-		if (hasNext()) {
-			T tmp = this.lookahead;
-			this.lookahead = null;
-			return tmp;
-		} else {
-			return null;
-		}
-	}
-	
-	@Override
-	public boolean isInputClosed() {
-		return this.noMoreRecordsWillFollow;
-	}
-	
-	private T instantiateRecordType() {
-		try {
-			return this.recordType.newInstance();
-		} catch (InstantiationException e) {
-			throw new RuntimeException("Cannot instantiate class '" + this.recordType.getName() + "'.", e);
-		} catch (IllegalAccessException e) {
-			throw new RuntimeException("Cannot instantiate class '" + this.recordType.getName() + "'.", e);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordWriter.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordWriter.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordWriter.java
deleted file mode 100644
index 2f4c21b..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordWriter.java
+++ /dev/null
@@ -1,82 +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.io;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.template.AbstractInputTask;
-import eu.stratosphere.nephele.template.AbstractTask;
-
-/**
- * A record writer connects the application to an output gate. It allows the application
- * of emit (send out) to the output gate. The output gate will then take care of distributing
- * the emitted records among the output channels.
- * 
- * @param <T>
- *        the type of the record that can be emitted with this record writer
- */
-public class RecordWriter<T extends IOReadableWritable> extends AbstractRecordWriter<T> {
-
-	/**
-	 * Constructs a new record writer and registers a new output gate with the application's environment.
-	 * 
-	 * @param taskBase
-	 *        the application that instantiated the record writer
-	 * @param outputClass
-	 *        the class of records that can be emitted with this record writer
-	 * @param selector
-	 *        the channel selector to be used to determine the output channel to be used for a record
-	 */
-	public RecordWriter(AbstractTask taskBase, Class<T> outputClass, ChannelSelector<T> selector) {
-		super(taskBase, outputClass, selector, false);
-	}
-
-	/**
-	 * Constructs a new record writer and registers a new output gate with the application's environment.
-	 * 
-	 * @param taskBase
-	 *        the application that instantiated the record writer
-	 * @param outputClass
-	 *        the class of records that can be emitted with this record writer
-	 */
-	public RecordWriter(AbstractTask taskBase, Class<T> outputClass) {
-		super(taskBase, outputClass, null, false);
-	}
-
-	/**
-	 * This method emits a record to the corresponding output gate. The method may block
-	 * until the record was transfered via any of the connected channels.
-	 * 
-	 * @param inputBase
-	 *        the application that instantiated the record writer
-	 * @param outputClass
-	 *        the class of records that can be emitted with this record writer
-	 */
-	public RecordWriter(AbstractInputTask<?> inputBase, Class<T> outputClass) {
-		super(inputBase, outputClass, null, false);
-	}
-
-	/**
-	 * Constructs a new record writer and registers a new output gate with the application's environment.
-	 * 
-	 * @param inputBase
-	 *        the application that instantiated the record writer
-	 * @param outputClass
-	 *        the class of records that can be emitted with this record writer
-	 * @param selector
-	 *        the channel selector to be used to determine the output channel to be used for a record
-	 */
-	public RecordWriter(AbstractInputTask<?> inputBase, Class<T> outputClass, ChannelSelector<T> selector) {
-		super(inputBase, outputClass, selector, false);
-	}
-}


[13/30] Offer buffer-oriented API for I/O (#25)

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProviderBroker.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProviderBroker.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProviderBroker.java
new file mode 100644
index 0000000..518fe47
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProviderBroker.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.runtime.io.network.bufferprovider;
+
+import java.io.IOException;
+
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.nephele.jobgraph.JobID;
+
+public interface BufferProviderBroker {
+
+	BufferProvider getBufferProvider(JobID jobID, ChannelID sourceChannelID) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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
new file mode 100644
index 0000000..2141017
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/GlobalBufferPool.java
@@ -0,0 +1,123 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.network.bufferprovider;
+
+import java.util.Queue;
+import java.util.concurrent.ArrayBlockingQueue;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import eu.stratosphere.core.memory.MemorySegment;
+
+/**
+ * A global buffer pool for the network stack.
+ * <p>
+ * All buffers used by the network stack come from this pool. Requests to this pool are mediated by instances of
+ * {@link LocalBufferPool}.
+ * <p>
+ * The size and number of buffers can be configured via the global system config.
+ */
+public final class GlobalBufferPool {
+
+	private final static Log LOG = LogFactory.getLog(GlobalBufferPool.class);
+
+	// -----------------------------------------------------------------------------------------------------------------
+
+	/** Total number of buffers */
+	private final int numBuffers;
+
+	/** Size of each buffer (in bytes) */
+	private final int bufferSize;
+
+	/** The available buffers */
+	private final Queue<MemorySegment> buffers;
+
+	private boolean isDestroyed;
+
+	// -----------------------------------------------------------------------------------------------------------------
+
+	public GlobalBufferPool(int numBuffers, int bufferSize) {
+		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]));
+		}
+
+		LOG.info(String.format("Initialized global buffer pool with %d buffers (%d bytes each).",
+				this.numBuffers, this.bufferSize));
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+
+	/**
+	 * Requests a buffer <strong>from</strong> the pool.
+	 *
+	 * @return buffer from pool or <code>null</code>, if no buffer available
+	 */
+	public MemorySegment requestBuffer() {
+		return this.buffers.poll();
+	}
+
+	/**
+	 * Returns a buffer <em>to</em> the pool.
+	 *
+	 * @param buffer the buffer to be returned
+	 */
+	public void returnBuffer(MemorySegment buffer) {
+		this.buffers.add(buffer);
+	}
+
+	/**
+	 * Returns the size of buffers (in bytes).
+	 *
+	 * @return size of buffers (in bytes)
+	 */
+	public int getBufferSize() {
+		return this.bufferSize;
+	}
+
+	/**
+	 * Returns the total number of managed buffers.
+	 * 
+	 * @return total number of managed buffers
+	 */
+	public int numBuffers() {
+		return this.numBuffers;
+	}
+
+	/**
+	 * Returns the number of currently available buffers.
+	 * 
+	 * @return currently available number of buffers
+	 */
+	public int numAvailableBuffers() {
+		return this.buffers.size();
+	}
+
+	public synchronized void destroy() {
+		if (!this.isDestroyed) {
+			// mark as shutdown and release memory
+			this.isDestroyed = true;
+
+			for (MemorySegment buffer : this.buffers) {
+				buffer.free();
+			}
+
+			this.buffers.clear();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java
new file mode 100644
index 0000000..e8aeb11
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java
@@ -0,0 +1,306 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.network.bufferprovider;
+
+import eu.stratosphere.core.memory.MemorySegment;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.BufferRecycler;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.Queue;
+
+/**
+ * A buffer pool used to manage a designated number of buffers from a {@link GlobalBufferPool}.
+ * <p>
+ * A local buffer pool mediates buffer requests to the global buffer pool to ensure dead-lock free operation of the
+ * network stack by limiting the number of designated buffers per local buffer pool. It also implements the default
+ * mechanism for buffer recycling, which ensures that every buffer is ultimately returned to the global buffer pool.
+ */
+public final class LocalBufferPool implements BufferProvider {
+
+	private static final class LocalBufferPoolRecycler implements BufferRecycler {
+
+		private final LocalBufferPool bufferPool;
+
+		private LocalBufferPoolRecycler(LocalBufferPool bufferPool) {
+			this.bufferPool = bufferPool;
+		}
+
+		@Override
+		public void recycle(MemorySegment buffer) {
+			this.bufferPool.recycleBuffer(buffer);
+		}
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+
+	/** Time (ms) to wait before retry for blocking buffer requests */
+	private static final int WAIT_TIME = 100;
+
+	/** Global buffer pool to request buffers from */
+	private final GlobalBufferPool globalBufferPool;
+
+	/** Buffers managed by this local buffer pool */
+	private final Queue<MemorySegment> buffers = new ArrayDeque<MemorySegment>();
+
+	/** The recycler via which to return buffers to this local buffer pool */
+	private final LocalBufferPoolRecycler recycler;
+
+	/** Queue of buffer availability listeners */
+	private final Queue<BufferAvailabilityListener> listeners = new ArrayDeque<BufferAvailabilityListener>();
+
+	/** Size of each buffer in this pool (in bytes) */
+	private final int bufferSize;
+
+	/** Number of buffers assigned to this local buffer pool */
+	private int numDesignatedBuffers;
+
+	/** Number of buffers requested from the global buffer pool */
+	private int numRequestedBuffers;
+
+	/** Flag to indicate whether an asynchronous event has been reported */
+	private boolean hasAsyncEventOccurred;
+
+	/** Flag to indicate whether this local buffer pool has been destroyed */
+	private boolean isDestroyed;
+
+	// -----------------------------------------------------------------------------------------------------------------
+
+	public LocalBufferPool(GlobalBufferPool globalBufferPool, int numDesignatedBuffers) {
+		this.globalBufferPool = globalBufferPool;
+		this.bufferSize = globalBufferPool.getBufferSize();
+		this.numDesignatedBuffers = numDesignatedBuffers;
+
+		this.recycler = new LocalBufferPoolRecycler(this);
+	}
+	// -----------------------------------------------------------------------------------------------------------------
+
+	@Override
+	public Buffer requestBuffer(int minBufferSize) throws IOException {
+		try {
+			return requestBuffer(minBufferSize, false);
+		} catch (InterruptedException e) {
+			throw new IOException("Unexpected InterruptedException while non-blocking buffer request.");
+		}
+	}
+
+	@Override
+	public Buffer requestBufferBlocking(int minBufferSize) throws IOException, InterruptedException {
+		return requestBuffer(minBufferSize, true);
+	}
+
+	/**
+	 * Requests a buffer from this local buffer pool.
+	 * <p>
+	 * A non-blocking call to this method will only return a buffer, if one is available in the local pool after
+	 * having returned excess buffers. Otherwise, it will return <code>null</code>.
+	 * <p>
+	 * A blocking call will request a new buffer from the global buffer and block until one is available or an
+	 * asynchronous event has been reported via {@link #reportAsynchronousEvent()}.
+	 *
+	 * @param minBufferSize minimum size of requested buffer (in bytes)
+	 * @param isBlocking flag to indicate whether to block until buffer is available
+	 * @return buffer from the global buffer pool or <code>null</code>, if no buffer available
+	 * @throws IOException
+	 * @throws InterruptedException
+	 */
+	private Buffer requestBuffer(int minBufferSize, boolean isBlocking) throws IOException, InterruptedException {
+		if (minBufferSize > this.bufferSize) {
+			throw new IllegalArgumentException(String.format("Too large buffer requested (requested %d, maximum %d).",
+					minBufferSize, this.bufferSize));
+		}
+
+		while (true) {
+			boolean isAsyncRequest = false;
+
+			synchronized (this.buffers) {
+				// Return excess buffers to global buffer pool
+				while (this.numRequestedBuffers > this.numDesignatedBuffers) {
+					final MemorySegment buffer = this.buffers.poll();
+					if (buffer == null) {
+						break;
+					}
+
+					this.globalBufferPool.returnBuffer(buffer);
+					this.numRequestedBuffers--;
+				}
+
+				// Request buffers from global buffer pool
+				while (this.buffers.isEmpty()) {
+					if (this.numRequestedBuffers < this.numDesignatedBuffers) {
+						final MemorySegment buffer = this.globalBufferPool.requestBuffer();
+
+						if (buffer != null) {
+							this.buffers.add(buffer);
+
+							this.numRequestedBuffers++;
+							continue;
+						}
+					}
+
+					if (this.hasAsyncEventOccurred && isBlocking) {
+						this.hasAsyncEventOccurred = false;
+						isAsyncRequest = true;
+						break;
+					}
+
+					if (isBlocking) {
+						this.buffers.wait(WAIT_TIME);
+					} else {
+						return null;
+					}
+				}
+
+				if (!isAsyncRequest) {
+					return new Buffer(this.buffers.poll(), minBufferSize, this.recycler);
+				}
+			}
+		}
+	}
+
+	@Override
+	public int getBufferSize() {
+		return this.bufferSize;
+	}
+
+	@Override
+	public void reportAsynchronousEvent() {
+		synchronized (this.buffers) {
+			this.hasAsyncEventOccurred = true;
+			this.buffers.notify();
+		}
+	}
+
+	@Override
+	public boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
+		synchronized (this.buffers) {
+			if (!this.buffers.isEmpty()) {
+				return false;
+			}
+
+			if (this.isDestroyed) {
+				return false;
+			}
+
+			this.listeners.add(listener);
+		}
+
+		return true;
+	}
+
+	/**
+	 * Sets the designated number of buffers for this local buffer pool and returns excess buffers to the global buffer
+	 * pool.
+	 * <p>
+	 * The designated number of buffers determines how many buffers this buffer pool is allowed to manage. New buffers
+	 * can only be requested, if the requested number of buffers is less than the designated number. If possible, excess
+	 * buffers will be returned to the global buffer pool.
+	 *
+	 * @param numDesignatedBuffers number of buffers designated for this local buffer pool
+	 */
+	public void setNumDesignatedBuffers(int numDesignatedBuffers) {
+		synchronized (this.buffers) {
+			this.numDesignatedBuffers = numDesignatedBuffers;
+
+			// Return excess buffers to global buffer pool
+			while (this.numRequestedBuffers > this.numDesignatedBuffers) {
+				if (this.buffers.isEmpty()) {
+					break;
+				}
+
+				this.globalBufferPool.returnBuffer(this.buffers.poll());
+				this.numRequestedBuffers --;
+			}
+
+			this.buffers.notify();
+		}
+	}
+
+	/**
+	 * Returns the number of buffers available in the local buffer pool.
+	 *
+	 * @return number of available buffers
+	 */
+	public int numAvailableBuffers() {
+		synchronized (this.buffers) {
+			return this.buffers.size();
+		}
+	}
+
+	/**
+	 * Returns the number of buffers, which have been requested from the global buffer pool.
+	 *
+	 * @return number of buffers requested from the global buffer pool
+	 */
+	public int numRequestedBuffers() {
+		synchronized (this.buffers) {
+			return this.numRequestedBuffers;
+		}
+	}
+
+	/**
+	 * Returns the designated number of buffers for this local buffer pool.
+	 *
+	 * @return number of designated buffers for this buffer pool
+	 */
+	public int numDesignatedBuffers() {
+		synchronized (this.buffers) {
+			return this.numDesignatedBuffers;
+		}
+	}
+
+	/**
+	 * Destroys this local buffer pool and immediately returns all available buffers to the global buffer pool.
+	 * <p>
+	 * Buffers, which have been requested from this local buffer pool via <code>requestBuffer</code> cannot be returned
+	 * immediately and will be returned when the respective buffer is recycled (see {@link #recycleBuffer(MemorySegment)}).
+	 */
+	public void destroy() {
+		synchronized (this.buffers) {
+			if (this.isDestroyed) {
+				return;
+			}
+
+			this.isDestroyed = true;
+
+			// return all buffers
+			while (!this.buffers.isEmpty()) {
+				this.globalBufferPool.returnBuffer(this.buffers.poll());
+				this.numRequestedBuffers--;
+			}
+		}
+	}
+
+	/**
+	 * Returns a buffer to the buffer pool and notifies listeners about the availability of a new buffer.
+	 *
+	 * @param buffer buffer to return to the buffer pool
+	 */
+	private void recycleBuffer(MemorySegment buffer) {
+		synchronized (this.buffers) {
+			if (this.isDestroyed) {
+				this.globalBufferPool.returnBuffer(buffer);
+				this.numRequestedBuffers--;
+			} else {
+				this.buffers.add(buffer);
+				this.buffers.notify();
+
+				while (!this.listeners.isEmpty()) {
+					this.listeners.poll().bufferAvailable();
+				}
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPoolOwner.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPoolOwner.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPoolOwner.java
new file mode 100644
index 0000000..ccab2ca
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPoolOwner.java
@@ -0,0 +1,56 @@
+/***********************************************************************************************************************
+ * 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.network.bufferprovider;
+
+/**
+ * A local buffer pool owner is an object which initially retrieves its buffers from the {@link GlobalBufferPool} and
+ * manages its fraction of the overall buffer pool locally by means of a {@link LocalBufferPool}.
+ * 
+ */
+public interface LocalBufferPoolOwner {
+
+	/**
+	 * Returns the number of byte-buffered channels that will retrieve their buffers from the local buffer pool.
+	 * 
+	 * @return the number of byte-buffered channels that will retrieve their buffers from the local buffer pool
+	 */
+	int getNumberOfChannels();
+
+	/**
+	 * Sets the designated number of buffers the local buffer pool owner is allowed to fetch from the global buffer pool
+	 * and manage locally by means of the {@link LocalBufferPool}.
+	 * 
+	 * @param numBuffers
+	 *        the numBuffers the local buffer pool owner is allowed to fetch from the global buffer pool
+	 */
+	void setDesignatedNumberOfBuffers(int numBuffers);
+
+	/**
+	 * Clears the local buffer pool and returns all buffers to the global buffer pool.
+	 */
+	void clearLocalBufferPool();
+
+	void registerGlobalBufferPool(GlobalBufferPool globalBufferPool);
+
+	/**
+	 * Logs the current status of the local buffer pool. This method is intended mainly for debugging purposes.
+	 */
+	void logBufferUtilization();
+
+	/**
+	 * Reports an asynchronous event. Calling this method interrupts each blocking method of the buffer pool owner and
+	 * allows the blocked thread to respond to the event.
+	 */
+	void reportAsynchronousEvent();
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/SerialSingleBufferPool.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/SerialSingleBufferPool.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/SerialSingleBufferPool.java
new file mode 100644
index 0000000..217a5ce
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/SerialSingleBufferPool.java
@@ -0,0 +1,77 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.network.bufferprovider;
+
+import eu.stratosphere.core.memory.MemorySegment;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.BufferRecycler;
+
+/**
+ * 
+ */
+public final class SerialSingleBufferPool implements BufferProvider, BufferRecycler {
+	
+	private final Buffer buffer;
+
+	/** Size of the buffer in this pool */
+	private final int bufferSize;
+
+
+	// -----------------------------------------------------------------------------------------------------------------
+
+	public SerialSingleBufferPool(int bufferSize) {
+		this.buffer = new Buffer(new MemorySegment(new byte[bufferSize]), bufferSize, this);
+		this.bufferSize = bufferSize;
+	}
+	
+	// -----------------------------------------------------------------------------------------------------------------
+
+	@Override
+	public Buffer requestBuffer(int minBufferSize) {
+		if (minBufferSize <= this.bufferSize) {
+			return this.buffer.duplicate();
+		}
+		else {
+			throw new IllegalArgumentException("Requesting buffer with size " + minBufferSize + ". Pool's buffer size is " + this.bufferSize);
+		}
+	}
+
+	@Override
+	public Buffer requestBufferBlocking(int minBufferSize) {
+		if (minBufferSize <= this.bufferSize) {
+			return this.buffer.duplicate();
+		}
+		else {
+			throw new IllegalArgumentException("Requesting buffer with size " + minBufferSize + ". Pool's buffer size is " + this.bufferSize);
+		}
+	}
+
+	@Override
+	public int getBufferSize() {
+		return this.bufferSize;
+	}
+
+	@Override
+	public void reportAsynchronousEvent() {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public void recycle(MemorySegment buffer) {}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/Envelope.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/Envelope.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/Envelope.java
new file mode 100644
index 0000000..a692aec
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/Envelope.java
@@ -0,0 +1,169 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.network.envelope;
+
+import eu.stratosphere.nephele.event.task.AbstractEvent;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.nephele.jobgraph.JobID;
+import eu.stratosphere.runtime.io.serialization.DataInputDeserializer;
+import eu.stratosphere.runtime.io.serialization.DataOutputSerializer;
+import eu.stratosphere.util.InstantiationUtil;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public final class Envelope {
+
+	private final JobID jobID;
+
+	private final ChannelID source;
+
+	private final int sequenceNumber;
+
+	private ByteBuffer serializedEventList;
+
+	private Buffer buffer;
+
+	public Envelope(int sequenceNumber, JobID jobID, ChannelID source) {
+		this.sequenceNumber = sequenceNumber;
+		this.jobID = jobID;
+		this.source = source;
+	}
+
+	private Envelope(Envelope toDuplicate) {
+		this.jobID = toDuplicate.jobID;
+		this.source = toDuplicate.source;
+		this.sequenceNumber = toDuplicate.sequenceNumber;
+		this.serializedEventList = null;
+		this.buffer = null;
+	}
+
+	public Envelope duplicate() {
+		Envelope duplicate = new Envelope(this);
+		if (hasBuffer()) {
+			duplicate.setBuffer(this.buffer.duplicate());
+		}
+
+		return duplicate;
+	}
+
+	public Envelope duplicateWithoutBuffer() {
+		return new Envelope(this);
+	}
+
+	public JobID getJobID() {
+		return this.jobID;
+	}
+
+	public ChannelID getSource() {
+		return this.source;
+	}
+
+	public int getSequenceNumber() {
+		return this.sequenceNumber;
+	}
+
+	public void setEventsSerialized(ByteBuffer serializedEventList) {
+		if (this.serializedEventList != null)
+			throw new IllegalStateException("Event list has already been set.");
+
+		this.serializedEventList = serializedEventList;
+	}
+
+	public void serializeEventList(List<? extends AbstractEvent> eventList) {
+		if (this.serializedEventList != null)
+			throw new IllegalStateException("Event list has already been set.");
+
+		this.serializedEventList = serializeEvents(eventList);
+	}
+
+	public ByteBuffer getEventsSerialized() {
+		return this.serializedEventList;
+	}
+
+	public List<? extends AbstractEvent> deserializeEvents() {
+		return deserializeEvents(getClass().getClassLoader());
+	}
+
+	public List<? extends AbstractEvent> deserializeEvents(ClassLoader classloader) {
+		if (this.serializedEventList == null) {
+			return Collections.emptyList();
+		}
+
+		try {
+			DataInputDeserializer deserializer = new DataInputDeserializer(this.serializedEventList);
+
+			int numEvents = deserializer.readInt();
+			ArrayList<AbstractEvent> events = new ArrayList<AbstractEvent>(numEvents);
+
+			for (int i = 0; i < numEvents; i++) {
+				String className = deserializer.readUTF();
+				Class<? extends AbstractEvent> clazz;
+				try {
+					clazz = Class.forName(className).asSubclass(AbstractEvent.class);
+				} catch (ClassNotFoundException e) {
+					throw new RuntimeException("Could not load event class '" + className + "'.", e);
+				} catch (ClassCastException e) {
+					throw new RuntimeException("The class '" + className + "' is no valid subclass of '" + AbstractEvent.class.getName() + "'.", e);
+				}
+
+				AbstractEvent evt = InstantiationUtil.instantiate(clazz, AbstractEvent.class);
+				evt.read(deserializer);
+
+				events.add(evt);
+			}
+
+			return events;
+		}
+		catch (IOException e) {
+			throw new RuntimeException("Error while deserializing the events.", e);
+		}
+	}
+
+	public void setBuffer(Buffer buffer) {
+		this.buffer = buffer;
+	}
+
+	public Buffer getBuffer() {
+		return this.buffer;
+	}
+
+	private ByteBuffer serializeEvents(List<? extends AbstractEvent> events) {
+		try {
+			// create the serialized event list
+			DataOutputSerializer serializer = events.size() == 0
+				? new DataOutputSerializer(4)
+				: new DataOutputSerializer(events.size() * 32);
+			serializer.writeInt(events.size());
+
+			for (AbstractEvent evt : events) {
+				serializer.writeUTF(evt.getClass().getName());
+				evt.write(serializer);
+			}
+
+			return serializer.wrapAsByteBuffer();
+		}
+		catch (IOException e) {
+			throw new RuntimeException("Error while serializing the task events.", e);
+		}
+	}
+
+	public boolean hasBuffer() {
+		return this.buffer != null;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeDispatcher.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeDispatcher.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeDispatcher.java
new file mode 100644
index 0000000..2b69c0d
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeDispatcher.java
@@ -0,0 +1,46 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.network.envelope;
+
+import java.io.IOException;
+
+/**
+ * A envelope dispatcher receives {@link Envelope}s and sends them to all of its destinations.
+ */
+public interface EnvelopeDispatcher {
+
+	/**
+	 * Dispatches an envelope from an output channel to the receiving input channels (forward flow).
+	 *
+	 * @param envelope envelope to be sent
+	 */
+	void dispatchFromOutputChannel(Envelope envelope) throws IOException, InterruptedException;
+
+	/**
+	 * Dispatches an envelope from an input channel to the receiving output channels (backwards flow).
+	 *
+	 * @param envelope envelope to be sent
+	 */
+	void dispatchFromInputChannel(Envelope envelope) throws IOException, InterruptedException;
+
+	/**
+	 * Dispatches an envelope from an incoming TCP connection.
+	 * <p>
+	 * After an envelope has been constructed from a TCP socket, this method is called to send the envelope to the
+	 * receiving input channel.
+	 *
+	 * @param envelope envelope to be sent
+	 */
+	void dispatchFromNetwork(Envelope envelope) throws IOException, InterruptedException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReader.java
new file mode 100644
index 0000000..7b7e178
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReader.java
@@ -0,0 +1,212 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.network.envelope;
+
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.nephele.jobgraph.JobID;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferProviderBroker;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.channels.ReadableByteChannel;
+
+public class EnvelopeReader {
+
+	public enum DeserializationState {
+		COMPLETE,
+		PENDING,
+		NO_BUFFER_AVAILABLE;
+	}
+
+	private final BufferProviderBroker bufferProviderBroker;
+
+	private final ByteBuffer headerBuffer;
+
+	private ByteBuffer currentHeaderBuffer;
+
+	private ByteBuffer currentEventsList;
+
+	private ByteBuffer currentDataBuffer;
+
+	private int bufferRequestPendingWithSize;
+
+
+	private Envelope pendingEnvelope;
+
+	private Envelope constructedEnvelope;
+
+
+	public BufferProvider bufferProvider;
+
+	private JobID lastDeserializedJobID;
+
+	private ChannelID lastDeserializedSourceID;
+
+
+	public EnvelopeReader(BufferProviderBroker bufferProviderBroker) {
+		this.bufferProviderBroker = bufferProviderBroker;
+
+		this.headerBuffer = ByteBuffer.allocateDirect(EnvelopeWriter.HEADER_SIZE);
+		this.headerBuffer.order(ByteOrder.LITTLE_ENDIAN);
+
+		this.currentHeaderBuffer = this.headerBuffer;
+	}
+
+	public DeserializationState readNextChunk(ReadableByteChannel channel) throws IOException {
+
+		// 1) check if the header is pending
+		if (this.currentHeaderBuffer != null) {
+			ByteBuffer header = this.currentHeaderBuffer;
+
+			channel.read(header);
+			if (header.hasRemaining()) {
+				// not finished with the header
+				return DeserializationState.PENDING;
+			} else {
+				// header done, construct the envelope
+				this.currentHeaderBuffer = null;
+
+				Envelope env = constructEnvelopeFromHeader(header);
+				this.pendingEnvelope = env;
+
+				// check for events and data
+				int eventsSize = getEventListSize(header);
+				int bufferSize = getBufferSize(header);
+
+				// make the events list the next buffer to be read
+				if (eventsSize > 0) {
+					this.currentEventsList = ByteBuffer.allocate(eventsSize);
+				}
+
+				// if we have a data buffer, we need memory segment for it
+				// we may not immediately get the memory segment, though, so we first record
+				// that we need it
+				if (bufferSize > 0) {
+					this.bufferRequestPendingWithSize = bufferSize;
+				}
+			}
+		}
+
+		// 2) read the eventList, if it should have one
+		if (this.currentEventsList != null) {
+			channel.read(this.currentEventsList);
+			if (this.currentEventsList.hasRemaining()) {
+				// events list still incomplete
+				return DeserializationState.PENDING;
+			} else {
+				this.currentEventsList.flip();
+				this.pendingEnvelope.setEventsSerialized(this.currentEventsList);
+				this.currentEventsList = null;
+			}
+		}
+
+		// 3) check if we need to get a buffer
+		if (this.bufferRequestPendingWithSize > 0) {
+			Buffer b = getBufferForTarget(this.pendingEnvelope.getJobID(), this.pendingEnvelope.getSource(), this.bufferRequestPendingWithSize);
+			if (b == null) {
+				// no buffer available at this time. come back later
+				return DeserializationState.NO_BUFFER_AVAILABLE;
+			} else {
+				// buffer is available. set the field so the buffer will be filled
+				this.pendingEnvelope.setBuffer(b);
+				this.currentDataBuffer = b.getMemorySegment().wrap(0, this.bufferRequestPendingWithSize);
+				this.bufferRequestPendingWithSize = 0;
+			}
+		}
+
+		// 4) fill the buffer
+		if (this.currentDataBuffer != null) {
+			channel.read(this.currentDataBuffer);
+			if (this.currentDataBuffer.hasRemaining()) {
+				// data buffer incomplete
+				return DeserializationState.PENDING;
+			} else {
+				this.currentDataBuffer = null;
+			}
+		}
+
+		// if we get here, we completed our job, or did nothing, if the deserializer was not
+		// reset after the previous envelope
+		if (this.pendingEnvelope != null) {
+			this.constructedEnvelope = this.pendingEnvelope;
+			this.pendingEnvelope = null;
+			return DeserializationState.COMPLETE;
+		} else {
+			throw new IllegalStateException("Error: read() was called before reserializer was reset after the last envelope.");
+		}
+	}
+
+	private Envelope constructEnvelopeFromHeader(ByteBuffer header) throws IOException {
+		int magicNumber = header.getInt(EnvelopeWriter.MAGIC_NUMBER_OFFSET);
+
+		if (magicNumber != EnvelopeWriter.MAGIC_NUMBER) {
+			throw new IOException("Network stream corrupted: invalid magic number in envelope header.");
+		}
+
+		int seqNum = header.getInt(EnvelopeWriter.SEQUENCE_NUMBER_OFFSET);
+		JobID jid = JobID.fromByteBuffer(header, EnvelopeWriter.JOB_ID_OFFSET);
+		ChannelID cid = ChannelID.fromByteBuffer(header, EnvelopeWriter.CHANNEL_ID_OFFSET);
+		return new Envelope(seqNum, jid, cid);
+	}
+
+	private int getBufferSize(ByteBuffer header) {
+		return header.getInt(EnvelopeWriter.BUFFER_SIZE_OFFSET);
+	}
+
+	private int getEventListSize(ByteBuffer header) {
+		return header.getInt(EnvelopeWriter.EVENTS_SIZE_OFFSET);
+	}
+
+	private Buffer getBufferForTarget(JobID jid, ChannelID cid, int size) throws IOException {
+		if (!(jid.equals(this.lastDeserializedJobID) && cid.equals(this.lastDeserializedSourceID))) {
+			this.bufferProvider = this.bufferProviderBroker.getBufferProvider(jid, cid);
+			this.lastDeserializedJobID = jid;
+			this.lastDeserializedSourceID = cid;
+		}
+
+		return this.bufferProvider.requestBuffer(size);
+	}
+
+
+	public Envelope getFullyDeserializedTransferEnvelope() {
+		Envelope t = this.constructedEnvelope;
+		if (t == null) {
+			throw new IllegalStateException("Envelope has not yet been fully constructed.");
+		}
+
+		this.constructedEnvelope = null;
+		return t;
+	}
+
+	public void reset() {
+		this.headerBuffer.clear();
+		this.currentHeaderBuffer = this.headerBuffer;
+		this.constructedEnvelope = null;
+	}
+
+	public boolean hasUnfinishedData() {
+		return this.pendingEnvelope != null || this.currentHeaderBuffer != null;
+	}
+
+	public BufferProvider getBufferProvider() {
+		return bufferProvider;
+	}
+
+	public Envelope getPendingEnvelope() {
+		return pendingEnvelope;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReceiverList.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReceiverList.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReceiverList.java
new file mode 100644
index 0000000..f99e1f2
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReceiverList.java
@@ -0,0 +1,75 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES 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.network.envelope;
+
+import java.net.InetAddress;
+
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.network.ConnectionInfoLookupResponse;
+import eu.stratosphere.runtime.io.network.RemoteReceiver;
+
+/**
+ * A transfer envelope receiver list contains all recipients of a transfer envelope. Their are three d ifferent types of
+ * receivers: Local receivers identified by {@link ChannelID} objects, remote receivers identified by
+ * {@link InetAddress} objects and finally checkpoints which are identified by
+ * <p>
+ * This class is thread-safe.
+ * 
+ */
+public class EnvelopeReceiverList {
+
+	private final ChannelID localReceiver;
+
+	private final RemoteReceiver remoteReceiver;
+
+	public EnvelopeReceiverList(ConnectionInfoLookupResponse cilr) {
+		this.localReceiver = cilr.getLocalTarget();
+		this.remoteReceiver = cilr.getRemoteTarget();
+	}
+
+	public EnvelopeReceiverList(ChannelID localReceiver) {
+		this.localReceiver = localReceiver;
+		this.remoteReceiver = null;
+	}
+
+	public EnvelopeReceiverList(RemoteReceiver remoteReceiver) {
+		this.localReceiver = null;
+		this.remoteReceiver = remoteReceiver;
+	}
+
+	public boolean hasLocalReceiver() {
+		return this.localReceiver != null;
+	}
+
+	public boolean hasRemoteReceiver() {
+		return this.remoteReceiver != null;
+	}
+
+	public int getTotalNumberOfReceivers() {
+		return (this.localReceiver == null ? 0 : 1) + (this.remoteReceiver == null ? 0 : 1);
+	}
+
+	public RemoteReceiver getRemoteReceiver() {
+		return this.remoteReceiver;
+	}
+
+	public ChannelID getLocalReceiver() {
+		return this.localReceiver;
+	}
+	
+	@Override
+	public String toString() {
+		return "local receiver: " + this.localReceiver + ", remote receiver: " + this.remoteReceiver;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeWriter.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeWriter.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeWriter.java
new file mode 100644
index 0000000..c00e61b
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeWriter.java
@@ -0,0 +1,134 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.network.envelope;
+
+import eu.stratosphere.nephele.AbstractID;
+import eu.stratosphere.runtime.io.Buffer;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.channels.WritableByteChannel;
+
+public class EnvelopeWriter {
+
+	protected static final int MAGIC_NUMBER = 0xBADC0FFE;
+
+	/**
+	 * Size of the envelope header: 48 bytes = 4 bytes magic number, 4 bytes sequence number, 16 bytes job id,
+	 * 16 bytes sender id, 4 bytes bufferSize, 4 bytes event list length
+	 */
+	public static final int HEADER_SIZE = 4 + 4 + 2 * AbstractID.SIZE + 4 + 4;
+
+	public static final int MAGIC_NUMBER_OFFSET = 0;
+
+	public static final int SEQUENCE_NUMBER_OFFSET = 4;
+
+	public static final int JOB_ID_OFFSET = 8;
+
+	public static final int CHANNEL_ID_OFFSET = 24;
+
+	public static final int BUFFER_SIZE_OFFSET = 40;
+
+	public static final int EVENTS_SIZE_OFFSET = 44;
+
+	private ByteBuffer currentHeader;
+
+	private ByteBuffer currentEvents;
+
+	private ByteBuffer currentDataBuffer;
+
+	private final ByteBuffer headerBuffer;
+
+	public EnvelopeWriter() {
+		this.headerBuffer = ByteBuffer.allocateDirect(HEADER_SIZE);
+		this.headerBuffer.order(ByteOrder.LITTLE_ENDIAN);
+	}
+
+	/**
+	 * @param channel
+	 * @return True, if the writer has more pending data for the current envelope, false if not.
+	 *
+	 * @throws java.io.IOException
+	 */
+	public boolean writeNextChunk(WritableByteChannel channel) throws IOException {
+		// 1) check if the the header is still pending
+		if (this.currentHeader != null) {
+			channel.write(this.currentHeader);
+
+			if (this.currentHeader.hasRemaining()) {
+				// header was not fully written, so we can leave this method
+				return true;
+			} else {
+				this.currentHeader = null;
+			}
+		}
+
+		// 2) check if there are events pending
+		if (this.currentEvents != null) {
+			channel.write(this.currentEvents);
+			if (this.currentEvents.hasRemaining()) {
+				// events were not fully written, so leave this method
+				return true;
+			} else {
+				this.currentEvents = null;
+			}
+		}
+
+		// 3) write the data buffer
+		if (this.currentDataBuffer != null) {
+			channel.write(this.currentDataBuffer);
+			if (this.currentDataBuffer.hasRemaining()) {
+				return true;
+			} else {
+				this.currentDataBuffer = null;
+			}
+		}
+
+		return false;
+	}
+
+	public void setEnvelopeForWriting(Envelope env) {
+		// header
+		constructHeader(env);
+		this.currentHeader = this.headerBuffer;
+
+		// events (possibly null)
+		this.currentEvents = env.getEventsSerialized();
+
+		// data buffer (possibly null)
+		Buffer buf = env.getBuffer();
+		if (buf != null && buf.size() > 0) {
+			this.currentDataBuffer = buf.getMemorySegment().wrap(0, buf.size());
+		}
+	}
+
+	private void constructHeader(Envelope env) {
+		final ByteBuffer buf = this.headerBuffer;
+
+		buf.clear();							// reset
+		buf.putInt(MAGIC_NUMBER);
+		buf.putInt(env.getSequenceNumber());	// sequence number (4 bytes)
+		env.getJobID().write(buf);				// job Id (16 bytes)
+		env.getSource().write(buf);				// producerId (16 bytes)
+
+		// buffer size
+		buf.putInt(env.getBuffer() == null ? 0 : env.getBuffer().size());
+
+		// size of event list
+		buf.putInt(env.getEventsSerialized() == null ? 0 : env.getEventsSerialized().remaining());
+
+		buf.flip();
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/NoBufferAvailableException.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/NoBufferAvailableException.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/NoBufferAvailableException.java
new file mode 100644
index 0000000..f7d49bf
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/NoBufferAvailableException.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.runtime.io.network.envelope;
+
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
+
+/**
+ * This exception is thrown to indicate that the deserialization process of a {@link Envelope} could not be
+ * continued because a {@link Buffer} to store the envelope's content is currently not available.
+ * 
+ */
+public final class NoBufferAvailableException extends Exception {
+
+	/**
+	 * Generated serial UID.
+	 */
+	private static final long serialVersionUID = -9164212953646457026L;
+
+	/**
+	 * The buffer provider which could not deliver a buffer.
+	 */
+	private final BufferProvider bufferProvider;
+
+	/**
+	 * Constructs a new exception.
+	 * 
+	 * @param bufferProvider
+	 *        the buffer provider which could not deliver a buffer
+	 */
+	public NoBufferAvailableException(final BufferProvider bufferProvider) {
+		this.bufferProvider = bufferProvider;
+	}
+
+	/**
+	 * Returns the buffer provider which could not deliver a buffer.
+	 * 
+	 * @return the buffer provider which could not deliver a buffer
+	 */
+	public BufferProvider getBufferProvider() {
+		return this.bufferProvider;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnection.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnection.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnection.java
new file mode 100644
index 0000000..f22e6f7
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnection.java
@@ -0,0 +1,115 @@
+/***********************************************************************************************************************
+ * 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.network.tcp;
+
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.network.ChannelManager;
+import eu.stratosphere.runtime.io.network.envelope.Envelope;
+import eu.stratosphere.runtime.io.network.envelope.EnvelopeReader;
+import eu.stratosphere.runtime.io.network.envelope.EnvelopeReader.DeserializationState;
+import eu.stratosphere.runtime.io.network.envelope.NoBufferAvailableException;
+import eu.stratosphere.util.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.io.IOException;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.SelectionKey;
+
+/**
+ * An incoming TCP connection through which data is read and transformed into {@link Envelope} objects.
+ */
+public class IncomingConnection {
+
+	private static final Log LOG = LogFactory.getLog(IncomingConnection.class);
+
+	/** Readable byte channel (TCP socket) to read data from */
+	private final ReadableByteChannel channel;
+
+	/** Channel manager to dispatch complete envelopes */
+	private final ChannelManager channelManager;
+
+	/** Envelope reader to turn the channel data into envelopes */
+	private final EnvelopeReader reader;
+
+	// -----------------------------------------------------------------------------------------------------------------
+
+	public IncomingConnection(ReadableByteChannel channel, ChannelManager channelManager) {
+		this.channel = channel;
+		this.channelManager = channelManager;
+		this.reader = new EnvelopeReader(channelManager);
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+
+	public void read() throws IOException, InterruptedException, NoBufferAvailableException {
+		DeserializationState deserializationState = this.reader.readNextChunk(this.channel);
+
+		switch (deserializationState) {
+			case COMPLETE:
+				Envelope envelope = this.reader.getFullyDeserializedTransferEnvelope();
+				this.channelManager.dispatchFromNetwork(envelope);
+				this.reader.reset();
+				break;
+
+			case NO_BUFFER_AVAILABLE:
+				throw new NoBufferAvailableException(this.reader.getBufferProvider());
+
+			case PENDING:
+				break;
+		}
+	}
+
+	public void reportTransmissionProblem(SelectionKey key, IOException ioe) {
+		LOG.error(StringUtils.stringifyException(ioe));
+
+		try {
+			this.channel.close();
+		} catch (IOException e) {
+			LOG.debug("An error occurred while closing the byte channel");
+		}
+
+		if (key != null) {
+			key.cancel();
+		}
+
+		Envelope pendingEnvelope = this.reader.getPendingEnvelope();
+		if (pendingEnvelope != null) {
+			if (pendingEnvelope.hasBuffer()) {
+				Buffer buffer = pendingEnvelope.getBuffer();
+				if (buffer != null) {
+					buffer.recycleBuffer();
+				}
+			}
+		}
+
+		this.reader.reset();
+	}
+
+	public boolean isCloseUnexpected() {
+		return this.reader.hasUnfinishedData();
+	}
+
+	public void closeConnection(SelectionKey key) {
+		try {
+			this.channel.close();
+		} catch (IOException ioe) {
+			LOG.error("An IOException occurred while closing the socket: + " + StringUtils.stringifyException(ioe));
+		}
+
+		if (key != null) {
+			key.cancel();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnectionThread.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnectionThread.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnectionThread.java
new file mode 100644
index 0000000..774ad4e
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnectionThread.java
@@ -0,0 +1,226 @@
+/***********************************************************************************************************************
+ * 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.network.tcp;
+
+import eu.stratosphere.runtime.io.network.ChannelManager;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferAvailabilityListener;
+import eu.stratosphere.runtime.io.network.envelope.NoBufferAvailableException;
+import eu.stratosphere.util.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.channels.ClosedChannelException;
+import java.nio.channels.SelectionKey;
+import java.nio.channels.Selector;
+import java.nio.channels.ServerSocketChannel;
+import java.nio.channels.SocketChannel;
+import java.util.ArrayDeque;
+import java.util.Iterator;
+import java.util.Queue;
+
+public class IncomingConnectionThread extends Thread {
+
+	private static final Log LOG = LogFactory.getLog(IncomingConnectionThread.class);
+
+	private final ChannelManager channelManager;
+
+	private final Selector selector;
+
+	private final Queue<SelectionKey> pendingReadEventSubscribeRequests = new ArrayDeque<SelectionKey>();
+
+	private final ServerSocketChannel listeningSocket;
+
+	private static final class IncomingConnectionBufferAvailListener implements BufferAvailabilityListener {
+
+		private final Queue<SelectionKey> pendingReadEventSubscribeRequests;
+
+		private final SelectionKey key;
+
+		private IncomingConnectionBufferAvailListener(final Queue<SelectionKey> pendingReadEventSubscribeRequests,
+				final SelectionKey key) {
+
+			this.pendingReadEventSubscribeRequests = pendingReadEventSubscribeRequests;
+			this.key = key;
+		}
+
+		@Override
+		public void bufferAvailable() {
+
+			synchronized (this.pendingReadEventSubscribeRequests) {
+				this.pendingReadEventSubscribeRequests.add(this.key);
+			}
+		}
+	}
+
+	public IncomingConnectionThread(ChannelManager channelManager,
+			boolean isListeningThread, InetSocketAddress listeningAddress) throws IOException {
+		super("Incoming Connection Thread");
+
+		this.selector = Selector.open();
+		this.channelManager = channelManager;
+
+		if (isListeningThread) {
+			this.listeningSocket = ServerSocketChannel.open();
+			this.listeningSocket.configureBlocking(false);
+			listeningSocket.register(this.selector, SelectionKey.OP_ACCEPT);
+			this.listeningSocket.socket().bind(listeningAddress);
+			LOG.debug("Listening on " + this.listeningSocket.socket().getLocalSocketAddress());
+		} else {
+			this.listeningSocket = null;
+		}
+	}
+
+	@Override
+	public void run() {
+		try {
+			while (!this.isInterrupted()) {
+	
+				synchronized (this.pendingReadEventSubscribeRequests) {
+					while (!this.pendingReadEventSubscribeRequests.isEmpty()) {
+						final SelectionKey key = this.pendingReadEventSubscribeRequests.poll();
+						final IncomingConnection incomingConnection = (IncomingConnection) key.attachment();
+						final SocketChannel socketChannel = (SocketChannel) key.channel();
+	
+						try {
+							final SelectionKey newKey = socketChannel.register(this.selector, SelectionKey.OP_READ);
+							newKey.attach(incomingConnection);
+						} catch (ClosedChannelException e) {
+							incomingConnection.reportTransmissionProblem(key, e);
+						}
+					}
+				}
+	
+				try {
+					this.selector.select(500);
+				} catch (IOException e) {
+					LOG.error(e);
+				}
+	
+				final Iterator<SelectionKey> iter = this.selector.selectedKeys().iterator();
+	
+				while (iter.hasNext()) {
+					final SelectionKey key = iter.next();
+	
+					iter.remove();
+					if (key.isValid()) {
+						if (key.isReadable()) {
+							doRead(key);
+						} else if (key.isAcceptable()) {
+							doAccept(key);
+						} else {
+							LOG.error("Unknown key: " + key);
+						}
+					} else {
+						LOG.error("Received invalid key: " + key);
+					}
+				}
+			}
+	
+			// Do cleanup, if necessary
+			if (this.listeningSocket != null) {
+				try {
+					this.listeningSocket.close();
+				} catch (IOException ioe) {
+					// Actually, we can ignore this exception
+					LOG.debug(ioe);
+				}
+			}
+	
+			// Finally, close the selector
+			try {
+				this.selector.close();
+			} catch (IOException ioe) {
+				LOG.debug(StringUtils.stringifyException(ioe));
+			}
+		}
+		catch (Throwable t) {
+			// this is a disaster, this task manager cannot go on!
+			LOG.fatal("Incoming connection thread died with an exception: " + t.getMessage(), t);
+			System.exit(1);
+		}
+	}
+
+	private void doAccept(SelectionKey key) {
+
+		SocketChannel clientSocket = null;
+
+		try {
+			clientSocket = this.listeningSocket.accept();
+			if (clientSocket == null) {
+				LOG.error("Client socket is null");
+				return;
+			}
+		} catch (IOException ioe) {
+			LOG.error(ioe);
+			return;
+		}
+
+		final IncomingConnection incomingConnection = new IncomingConnection(
+			clientSocket, this.channelManager);
+		SelectionKey clientKey = null;
+		try {
+			clientSocket.configureBlocking(false);
+			clientKey = clientSocket.register(this.selector, SelectionKey.OP_READ);
+			clientKey.attach(incomingConnection);
+		} catch (IOException ioe) {
+			incomingConnection.reportTransmissionProblem(clientKey, ioe);
+		}
+	}
+
+	private void doRead(SelectionKey key) {
+
+		final IncomingConnection incomingConnection = (IncomingConnection) key.attachment();
+		try {
+			incomingConnection.read();
+		} catch (EOFException eof) {
+			if (incomingConnection.isCloseUnexpected()) {
+				final SocketChannel socketChannel = (SocketChannel) key.channel();
+				LOG.error("Connection from " + socketChannel.socket().getRemoteSocketAddress()
+					+ " was closed unexpectedly");
+				incomingConnection.reportTransmissionProblem(key, eof);
+			} else {
+				incomingConnection.closeConnection(key);
+			}
+		} catch (IOException ioe) {
+			incomingConnection.reportTransmissionProblem(key, ioe);
+		} catch (InterruptedException e) {
+			// Nothing to do here
+		} catch (NoBufferAvailableException e) {
+			// There are no buffers available, unsubscribe from read event
+			final SocketChannel socketChannel = (SocketChannel) key.channel();
+			try {
+				final SelectionKey newKey = socketChannel.register(this.selector, 0);
+				newKey.attach(incomingConnection);
+			} catch (ClosedChannelException e1) {
+				incomingConnection.reportTransmissionProblem(key, e1);
+			}
+
+			final BufferAvailabilityListener bal = new IncomingConnectionBufferAvailListener(
+				this.pendingReadEventSubscribeRequests, key);
+			if (!e.getBufferProvider().registerBufferAvailabilityListener(bal)) {
+				// In the meantime, a buffer has become available again, subscribe to read event again
+
+				try {
+					final SelectionKey newKey = socketChannel.register(this.selector, SelectionKey.OP_READ);
+					newKey.attach(incomingConnection);
+				} catch (ClosedChannelException e1) {
+					incomingConnection.reportTransmissionProblem(key, e1);
+				}
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnection.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnection.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnection.java
new file mode 100644
index 0000000..7df1901
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnection.java
@@ -0,0 +1,529 @@
+/***********************************************************************************************************************
+ * 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.network.tcp;
+
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.network.RemoteReceiver;
+import eu.stratosphere.runtime.io.network.envelope.Envelope;
+import eu.stratosphere.runtime.io.network.envelope.EnvelopeWriter;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.channels.SelectionKey;
+import java.nio.channels.SocketChannel;
+import java.nio.channels.WritableByteChannel;
+import java.util.ArrayDeque;
+import java.util.Iterator;
+import java.util.Queue;
+
+/**
+ * This class represents an outgoing TCP connection through which {@link eu.stratosphere.runtime.io.network.envelope.Envelope} objects can be sent.
+ * {@link eu.stratosphere.runtime.io.network.envelope.Envelope} objects are received from the {@link eu.stratosphere.nephele.taskmanager.io.bytebuffered.ChannelManager} and added to a queue. An
+ * additional network thread then takes the envelopes from the queue and transmits them to the respective destination
+ * host.
+ * 
+ */
+public class OutgoingConnection {
+
+	/**
+	 * The log object used to report debug information and possible errors.
+	 */
+	private static final Log LOG = LogFactory.getLog(OutgoingConnection.class);
+
+	/**
+	 * The address this outgoing connection is connected to.
+	 */
+	private final RemoteReceiver remoteReceiver;
+
+	/**
+	 * The outgoing connection thread which actually transmits the queued transfer envelopes.
+	 */
+	private final OutgoingConnectionThread connectionThread;
+
+	/**
+	 * The queue of transfer envelopes to be transmitted.
+	 */
+	private final Queue<Envelope> queuedEnvelopes = new ArrayDeque<Envelope>();
+
+	/**
+	 * The {@link eu.stratosphere.runtime.io.network.envelope.Envelope} that is currently processed.
+	 */
+	private Envelope currentEnvelope = null;
+
+	/**
+	 * Stores whether the underlying TCP connection is established. As this variable is accessed by the byte buffered
+	 * channel manager and the outgoing connection thread, it must be protected by a monitor.
+	 */
+	private boolean isConnected = false;
+
+	/**
+	 * Stores whether is underlying TCP connection is subscribed to the NIO write event. As this variable is accessed by
+	 * the byte buffered channel and the outgoing connection thread, it must be protected by a monitor.
+	 */
+	private boolean isSubscribedToWriteEvent = false;
+
+	/**
+	 * The overall number of connection retries which shall be performed before a connection error is reported.
+	 */
+	private final int numberOfConnectionRetries;
+
+	/**
+	 * The number of connection retries left before an I/O error is reported.
+	 */
+	private int retriesLeft = 0;
+
+	/**
+	 * The timestamp of the last connection retry.
+	 */
+	private long timstampOfLastRetry = 0;
+
+	/**
+	 * The current selection key representing the interest set of the underlying TCP NIO connection. This variable may
+	 * only be accessed the the outgoing connection thread.
+	 */
+	private SelectionKey selectionKey = null;
+
+	/**
+	 * The period of time in milliseconds that shall be waited before a connection attempt is considered to be failed.
+	 */
+	private static long RETRYINTERVAL = 1000L; // 1 second
+
+	private EnvelopeWriter writer;
+
+	/**
+	 * Constructs a new outgoing connection object.
+	 * 
+	 * @param remoteReceiver
+	 *        the address of the destination host this outgoing connection object is supposed to connect to
+	 * @param connectionThread
+	 *        the connection thread which actually handles the network transfer
+	 * @param numberOfConnectionRetries
+	 *        the number of connection retries allowed before an I/O error is reported
+	 */
+	public OutgoingConnection(RemoteReceiver remoteReceiver, OutgoingConnectionThread connectionThread,
+			int numberOfConnectionRetries) {
+
+		this.remoteReceiver = remoteReceiver;
+		this.connectionThread = connectionThread;
+		this.numberOfConnectionRetries = numberOfConnectionRetries;
+		this.writer = new EnvelopeWriter();
+	}
+
+	/**
+	 * Adds a new {@link eu.stratosphere.runtime.io.network.envelope.Envelope} to the queue of envelopes to be transmitted to the destination host of this
+	 * connection.
+	 * <p>
+	 * This method should only be called by the {@link eu.stratosphere.nephele.taskmanager.io.bytebuffered.ChannelManager} object.
+	 * 
+	 * @param envelope
+	 *        the envelope to be added to the transfer queue
+	 */
+	public void queueEnvelope(Envelope envelope) {
+
+		synchronized (this.queuedEnvelopes) {
+
+			checkConnection();
+			this.queuedEnvelopes.add(envelope);
+		}
+	}
+
+	private void checkConnection() {
+
+		synchronized (this.queuedEnvelopes) {
+
+			if (!this.isConnected) {
+
+				this.retriesLeft = this.numberOfConnectionRetries;
+				this.timstampOfLastRetry = System.currentTimeMillis();
+				this.connectionThread.triggerConnect(this);
+				this.isConnected = true;
+				this.isSubscribedToWriteEvent = true;
+			} else {
+
+				if (!this.isSubscribedToWriteEvent) {
+					this.connectionThread.subscribeToWriteEvent(this.selectionKey);
+					this.isSubscribedToWriteEvent = true;
+				}
+			}
+
+		}
+	}
+
+	/**
+	 * Returns the {@link InetSocketAddress} to the destination host this outgoing connection is supposed to be
+	 * connected to.
+	 * <p>
+	 * This method should be called by the {@link OutgoingConnectionThread} object only.
+	 * 
+	 * @return the {@link InetSocketAddress} to the destination host this outgoing connection is supposed to be
+	 *         connected to
+	 */
+	public InetSocketAddress getConnectionAddress() {
+
+		return this.remoteReceiver.getConnectionAddress();
+	}
+
+	/**
+	 * Reports a problem which occurred while establishing the underlying TCP connection to this outgoing connection
+	 * object. Depending on the number of connection retries left, this method will either try to reestablish the TCP
+	 * connection or report an I/O error to all tasks which have queued envelopes for this connection. In the latter
+	 * case all queued envelopes will be dropped and all included buffers will be freed.
+	 * <p>
+	 * This method should only be called by the {@link OutgoingConnectionThread} object.
+	 * 
+	 * @param ioe
+	 *        thrown if an error occurs while reseting the underlying TCP connection
+	 */
+	public void reportConnectionProblem(IOException ioe) {
+
+		// First, write exception to log
+		final long currentTime = System.currentTimeMillis();
+		if (currentTime - this.timstampOfLastRetry >= RETRYINTERVAL) {
+			LOG.error("Cannot connect to " + this.remoteReceiver + ", " + this.retriesLeft + " retries left");
+		}
+
+		synchronized (this.queuedEnvelopes) {
+
+			if (this.selectionKey != null) {
+
+				final SocketChannel socketChannel = (SocketChannel) this.selectionKey.channel();
+				if (socketChannel != null) {
+					try {
+						socketChannel.close();
+					} catch (IOException e) {
+						LOG.debug("Error while trying to close the socket channel to " + this.remoteReceiver);
+					}
+				}
+
+				this.selectionKey.cancel();
+				this.selectionKey = null;
+				this.isConnected = false;
+				this.isSubscribedToWriteEvent = false;
+			}
+
+			if (hasRetriesLeft(currentTime)) {
+				this.connectionThread.triggerConnect(this);
+				this.isConnected = true;
+				this.isSubscribedToWriteEvent = true;
+				return;
+			}
+
+			// Error is fatal
+			LOG.error(ioe);
+
+			// Notify source of current envelope and release buffer
+			if (this.currentEnvelope != null) {
+				if (this.currentEnvelope.getBuffer() != null) {
+					this.currentEnvelope.getBuffer().recycleBuffer();
+					this.currentEnvelope = null;
+				}
+			}
+
+			// Notify all other tasks which are waiting for data to be transmitted
+			final Iterator<Envelope> iter = this.queuedEnvelopes.iterator();
+			while (iter.hasNext()) {
+				final Envelope envelope = iter.next();
+				iter.remove();
+				// Recycle the buffer inside the envelope
+				if (envelope.getBuffer() != null) {
+					envelope.getBuffer().recycleBuffer();
+				}
+			}
+
+			this.queuedEnvelopes.clear();
+		}
+	}
+
+	/**
+	 * Reports an I/O error which occurred while writing data to the TCP connection. As a result of the I/O error the
+	 * connection is closed and the interest keys are canceled. Moreover, the task which queued the currently
+	 * transmitted transfer envelope is notified about the error and the current envelope is dropped. If the current
+	 * envelope contains a buffer, the buffer is freed.
+	 * <p>
+	 * This method should only be called by the {@link OutgoingConnectionThread} object.
+	 * 
+	 * @param ioe
+	 *        thrown if an error occurs while reseting the connection
+	 */
+	public void reportTransmissionProblem(IOException ioe) {
+
+		final SocketChannel socketChannel = (SocketChannel) this.selectionKey.channel();
+
+		// First, write exception to log
+		if (this.currentEnvelope != null) {
+			LOG.error("The connection between " + socketChannel.socket().getLocalAddress() + " and "
+				+ socketChannel.socket().getRemoteSocketAddress()
+				+ " experienced an IOException for transfer envelope " + this.currentEnvelope.getSequenceNumber());
+		} else {
+			LOG.error("The connection between " + socketChannel.socket().getLocalAddress() + " and "
+				+ socketChannel.socket().getRemoteSocketAddress() + " experienced an IOException");
+		}
+
+		// Close the connection and cancel the interest key
+		synchronized (this.queuedEnvelopes) {
+			try {
+				LOG.debug("Closing connection to " + socketChannel.socket().getRemoteSocketAddress());
+				socketChannel.close();
+			} catch (IOException e) {
+				LOG.debug("An error occurred while responding to an IOException");
+				LOG.debug(e);
+			}
+
+			this.selectionKey.cancel();
+
+			// Error is fatal
+			LOG.error(ioe);
+
+			// Trigger new connection if there are more envelopes to be transmitted
+			if (this.queuedEnvelopes.isEmpty()) {
+				this.isConnected = false;
+				this.isSubscribedToWriteEvent = false;
+			} else {
+				this.connectionThread.triggerConnect(this);
+				this.isConnected = true;
+				this.isSubscribedToWriteEvent = true;
+			}
+
+			// We must assume the current envelope is corrupted so we notify the task which created it.
+			if (this.currentEnvelope != null) {
+				if (this.currentEnvelope.getBuffer() != null) {
+					this.currentEnvelope.getBuffer().recycleBuffer();
+					this.currentEnvelope = null;
+				}
+			}
+		}
+	}
+
+	/**
+	 * Checks whether further retries are left for establishing the underlying TCP connection.
+	 * 
+	 * @param currentTime
+	 *        the current system time in milliseconds since January 1st, 1970
+	 * @return <code>true</code> if there are retries left, <code>false</code> otherwise
+	 */
+	private boolean hasRetriesLeft(long currentTime) {
+
+		if (currentTime - this.timstampOfLastRetry >= RETRYINTERVAL) {
+			this.retriesLeft--;
+			this.timstampOfLastRetry = currentTime;
+			if (this.retriesLeft == 0) {
+				return false;
+			}
+		}
+
+		return true;
+	}
+
+	/**
+	 * Writes the content of the current {@link eu.stratosphere.runtime.io.network.envelope.Envelope} object to the underlying TCP connection.
+	 * <p>
+	 * This method should only be called by the {@link OutgoingConnectionThread} object.
+	 * 
+	 * @return <code>true</code> if there is more data from this/other queued envelopes to be written to this channel
+	 * @throws IOException
+	 *         thrown if an error occurs while writing the data to the channel
+	 */
+	public boolean write() throws IOException {
+
+		final WritableByteChannel writableByteChannel = (WritableByteChannel) this.selectionKey.channel();
+
+		if (this.currentEnvelope == null) {
+			synchronized (this.queuedEnvelopes) {
+				if (this.queuedEnvelopes.isEmpty()) {
+					return false;
+				} else {
+					this.currentEnvelope = this.queuedEnvelopes.peek();
+
+					this.writer.setEnvelopeForWriting(this.currentEnvelope);
+				}
+			}
+		}
+
+		if (!this.writer.writeNextChunk(writableByteChannel)) {
+			// Make sure we recycle the attached memory or file buffers correctly
+			if (this.currentEnvelope.getBuffer() != null) {
+				this.currentEnvelope.getBuffer().recycleBuffer();
+			}
+
+			synchronized (this.queuedEnvelopes) {
+				this.queuedEnvelopes.poll();
+				this.currentEnvelope = null;
+			}
+		}
+
+		return true;
+	}
+
+	/**
+	 * Requests to close the underlying TCP connection. The request is ignored if at least one {@link eu.stratosphere.runtime.io.network.envelope.Envelope}
+	 * is queued.
+	 * <p>
+	 * This method should only be called by the {@link OutgoingConnectionThread} object.
+	 * 
+	 * @throws IOException
+	 *         thrown if an error occurs while closing the TCP connection
+	 */
+	public void requestClose() throws IOException {
+
+		synchronized (this.queuedEnvelopes) {
+
+			if (this.queuedEnvelopes.isEmpty()) {
+
+				if (this.isSubscribedToWriteEvent) {
+
+					this.connectionThread.unsubscribeFromWriteEvent(this.selectionKey);
+					this.isSubscribedToWriteEvent = false;
+				}
+			}
+		}
+	}
+
+	/**
+	 * Closes the underlying TCP connection if no more {@link eu.stratosphere.runtime.io.network.envelope.Envelope} objects are in the transmission queue.
+	 * <p>
+	 * This method should only be called by the {@link OutgoingConnectionThread} object.
+	 * 
+	 * @throws IOException
+	 */
+	public void closeConnection() throws IOException {
+
+		synchronized (this.queuedEnvelopes) {
+
+			if (!this.queuedEnvelopes.isEmpty()) {
+				return;
+			}
+
+			if (this.selectionKey != null) {
+
+				final SocketChannel socketChannel = (SocketChannel) this.selectionKey.channel();
+				socketChannel.close();
+				this.selectionKey.cancel();
+				this.selectionKey = null;
+			}
+
+			this.isConnected = false;
+			this.isSubscribedToWriteEvent = false;
+		}
+	}
+
+	/**
+	 * Returns the number of queued {@link eu.stratosphere.runtime.io.network.envelope.Envelope} objects with the given source channel ID.
+	 * 
+	 * @param sourceChannelID
+	 *        the source channel ID to count the queued envelopes for
+	 * @return the number of queued transfer envelopes with the given source channel ID
+	 */
+	public int getNumberOfQueuedEnvelopesFromChannel(final ChannelID sourceChannelID) {
+
+		synchronized (this.queuedEnvelopes) {
+
+			int number = 0;
+
+			final Iterator<Envelope> it = this.queuedEnvelopes.iterator();
+			while (it.hasNext()) {
+				final Envelope te = it.next();
+				if (sourceChannelID.equals(te.getSource())) {
+					number++;
+				}
+			}
+
+			return number;
+		}
+	}
+
+	/**
+	 * Removes all queued {@link eu.stratosphere.runtime.io.network.envelope.Envelope} objects from the transmission which match the given source channel
+	 * ID.
+	 * 
+	 * @param sourceChannelID
+	 *        the source channel ID of the transfered transfer envelopes to be dropped
+	 */
+	public void dropAllQueuedEnvelopesFromChannel(final ChannelID sourceChannelID) {
+
+		synchronized (this.queuedEnvelopes) {
+
+			final Iterator<Envelope> it = this.queuedEnvelopes.iterator();
+			while (it.hasNext()) {
+				final Envelope te = it.next();
+				if (sourceChannelID.equals(te.getSource())) {
+					it.remove();
+					if (te.getBuffer() != null) {
+						te.getBuffer().recycleBuffer();
+					}
+				}
+			}
+		}
+	}
+
+	/**
+	 * Checks whether this outgoing connection object manages an active connection or can be removed by the
+	 * {@link eu.stratosphere.nephele.taskmanager.io.bytebuffered.ChannelManager} object.
+	 * <p>
+	 * This method should only be called by the byte buffered channel manager.
+	 * 
+	 * @return <code>true</code> if this object is no longer manages an active connection and can be removed,
+	 *         <code>false</code> otherwise.
+	 */
+	public boolean canBeRemoved() {
+
+		synchronized (this.queuedEnvelopes) {
+
+			if (this.isConnected) {
+				return false;
+			}
+
+			if (this.currentEnvelope != null) {
+				return false;
+			}
+
+			return this.queuedEnvelopes.isEmpty();
+		}
+	}
+
+	/**
+	 * Sets the selection key representing the interest set of the underlying TCP NIO connection.
+	 * 
+	 * @param selectionKey
+	 *        the selection of the underlying TCP connection
+	 */
+	public void setSelectionKey(SelectionKey selectionKey) {
+		this.selectionKey = selectionKey;
+	}
+
+	/**
+	 * Returns the number of currently queued envelopes which contain a write buffer.
+	 * 
+	 * @return the number of currently queued envelopes which contain a write buffer
+	 */
+	public int getNumberOfQueuedWriteBuffers() {
+
+		int retVal = 0;
+
+		synchronized (this.queuedEnvelopes) {
+
+			final Iterator<Envelope> it = this.queuedEnvelopes.iterator();
+			while (it.hasNext()) {
+
+				final Envelope envelope = it.next();
+				if (envelope.getBuffer() != null) {
+					++retVal;
+				}
+			}
+		}
+
+		return retVal;
+	}
+}


[12/30] Offer buffer-oriented API for I/O (#25)

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnectionThread.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnectionThread.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnectionThread.java
new file mode 100644
index 0000000..dde26e3
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnectionThread.java
@@ -0,0 +1,276 @@
+/***********************************************************************************************************************
+ * 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.network.tcp;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.SelectionKey;
+import java.nio.channels.Selector;
+import java.nio.channels.SocketChannel;
+import java.util.ArrayDeque;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Queue;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import eu.stratosphere.util.StringUtils;
+
+public class OutgoingConnectionThread extends Thread {
+
+	/**
+	 * The minimum time a TCP connection must be idle it is closed.
+	 */
+	private static final long MIN_IDLE_TIME_BEFORE_CLOSE = 80000L; // 80 seconds
+
+	private static final Log LOG = LogFactory.getLog(OutgoingConnectionThread.class);
+
+	private final Selector selector;
+
+	private final Queue<OutgoingConnection> pendingConnectionRequests = new ArrayDeque<OutgoingConnection>();
+
+	private final Queue<SelectionKey> pendingWriteEventSubscribeRequests = new ArrayDeque<SelectionKey>();
+
+	private final Map<OutgoingConnection, Long> connectionsToClose = new HashMap<OutgoingConnection, Long>();
+
+	public OutgoingConnectionThread() throws IOException {
+		super("Outgoing Connection Thread");
+
+		this.selector = Selector.open();
+	}
+
+
+	@Override
+	public void run() {
+		try {
+			while (!isInterrupted()) {
+	
+				synchronized (this.pendingConnectionRequests) {
+	
+					if (!this.pendingConnectionRequests.isEmpty()) {
+	
+						final OutgoingConnection outgoingConnection = this.pendingConnectionRequests.poll();
+						try {
+							final SocketChannel socketChannel = SocketChannel.open();
+							socketChannel.configureBlocking(false);
+							final SelectionKey key = socketChannel.register(this.selector, SelectionKey.OP_CONNECT);
+							socketChannel.connect(outgoingConnection.getConnectionAddress());
+							key.attach(outgoingConnection);
+						} catch (final IOException ioe) {
+							// IOException is reported by separate thread to avoid deadlocks
+							final Runnable reporterThread = new Runnable() {
+	
+								@Override
+								public void run() {
+									outgoingConnection.reportConnectionProblem(ioe);
+								}
+							};
+							new Thread(reporterThread).start();
+						}
+					}
+				}
+	
+				synchronized (this.pendingWriteEventSubscribeRequests) {
+	
+					if (!this.pendingWriteEventSubscribeRequests.isEmpty()) {
+						final SelectionKey oldSelectionKey = this.pendingWriteEventSubscribeRequests.poll();
+						final OutgoingConnection outgoingConnection = (OutgoingConnection) oldSelectionKey.attachment();
+						final SocketChannel socketChannel = (SocketChannel) oldSelectionKey.channel();
+	
+						try {
+							final SelectionKey newSelectionKey = socketChannel.register(this.selector, SelectionKey.OP_READ
+								| SelectionKey.OP_WRITE);
+							newSelectionKey.attach(outgoingConnection);
+							outgoingConnection.setSelectionKey(newSelectionKey);
+						} catch (final IOException ioe) {
+							// IOException is reported by separate thread to avoid deadlocks
+							final Runnable reporterThread = new Runnable() {
+	
+								@Override
+								public void run() {
+									outgoingConnection.reportTransmissionProblem(ioe);
+								}
+							};
+							new Thread(reporterThread).start();
+						}
+					}
+				}
+	
+				synchronized (this.connectionsToClose) {
+	
+					final Iterator<Map.Entry<OutgoingConnection, Long>> closeIt = this.connectionsToClose.entrySet()
+						.iterator();
+					final long now = System.currentTimeMillis();
+					while (closeIt.hasNext()) {
+	
+						final Map.Entry<OutgoingConnection, Long> entry = closeIt.next();
+						if ((entry.getValue().longValue() + MIN_IDLE_TIME_BEFORE_CLOSE) < now) {
+							final OutgoingConnection outgoingConnection = entry.getKey();
+							closeIt.remove();
+							// Create new thread to close connection to avoid deadlocks
+							final Runnable closeThread = new Runnable() {
+	
+								@Override
+								public void run() {
+									try {
+										outgoingConnection.closeConnection();
+									} catch (IOException ioe) {
+										outgoingConnection.reportTransmissionProblem(ioe);
+									}
+								}
+							};
+	
+							new Thread(closeThread).start();
+						}
+	
+					}
+				}
+	
+				try {
+					this.selector.select(10);
+				} catch (IOException e) {
+					LOG.error(e);
+				}
+	
+				final Iterator<SelectionKey> iter = this.selector.selectedKeys().iterator();
+	
+				while (iter.hasNext()) {
+					final SelectionKey key = iter.next();
+	
+					iter.remove();
+					if (key.isValid()) {
+						if (key.isConnectable()) {
+							doConnect(key);
+						} else {
+							if (key.isReadable()) {
+								doRead(key);
+								// A read will always result in an exception, so the write key will not be valid anymore
+								continue;
+							}
+							if (key.isWritable()) {
+								doWrite(key);
+							}
+						}
+					} else {
+						LOG.error("Received invalid key: " + key);
+					}
+				}
+			}
+	
+			// Finally, try to close the selector
+			try {
+				this.selector.close();
+			} catch (IOException ioe) {
+				LOG.debug(StringUtils.stringifyException(ioe));
+			}
+		}
+		catch (Throwable t) {
+			// this is a disaster, this task manager cannot go on!
+			LOG.fatal("Outgoing connection thread died with an exception: " + t.getMessage(), t);
+			System.exit(1);
+		}
+	}
+
+	private void doConnect(SelectionKey key) {
+
+		final OutgoingConnection outgoingConnection = (OutgoingConnection) key.attachment();
+		final SocketChannel socketChannel = (SocketChannel) key.channel();
+		try {
+			while (!socketChannel.finishConnect()) {
+				try {
+					Thread.sleep(100);
+				} catch (InterruptedException e1) {
+					LOG.error(e1);
+				}
+			}
+
+			final SelectionKey channelKey = socketChannel.register(selector, SelectionKey.OP_WRITE
+				| SelectionKey.OP_READ);
+			outgoingConnection.setSelectionKey(channelKey);
+			channelKey.attach(outgoingConnection);
+
+		} catch (IOException ioe) {
+			outgoingConnection.reportConnectionProblem(ioe);
+		}
+	}
+
+	private void doWrite(SelectionKey key) {
+
+		final OutgoingConnection outgoingConnection = (OutgoingConnection) key.attachment();
+
+		try {
+
+			if (!outgoingConnection.write()) {
+				// Try to close the connection
+				outgoingConnection.requestClose();
+			}
+
+		} catch (IOException ioe) {
+			outgoingConnection.reportTransmissionProblem(ioe);
+		}
+	}
+
+	private void doRead(SelectionKey key) {
+
+		final SocketChannel socketChannel = (SocketChannel) key.channel();
+		final OutgoingConnection outgoingConnection = (OutgoingConnection) key.attachment();
+		final ByteBuffer buf = ByteBuffer.allocate(8);
+
+		try {
+
+			if (socketChannel.read(buf) == -1) {
+				outgoingConnection.reportTransmissionProblem(new IOException(
+					"Read unexpected EOF from channel"));
+			} else {
+				LOG.error("Outgoing connection read real data from channel");
+			}
+		} catch (IOException ioe) {
+			outgoingConnection.reportTransmissionProblem(ioe);
+		}
+	}
+
+	public void triggerConnect(OutgoingConnection outgoingConnection) {
+
+		synchronized (this.pendingConnectionRequests) {
+			this.pendingConnectionRequests.add(outgoingConnection);
+		}
+	}
+
+	public void unsubscribeFromWriteEvent(SelectionKey selectionKey) throws IOException {
+
+		final SocketChannel socketChannel = (SocketChannel) selectionKey.channel();
+		final OutgoingConnection outgoingConnection = (OutgoingConnection) selectionKey.attachment();
+
+		final SelectionKey newSelectionKey = socketChannel.register(this.selector, SelectionKey.OP_READ);
+		newSelectionKey.attach(outgoingConnection);
+		outgoingConnection.setSelectionKey(newSelectionKey);
+
+		synchronized (this.connectionsToClose) {
+			this.connectionsToClose.put(outgoingConnection, Long.valueOf(System.currentTimeMillis()));
+		}
+	}
+
+	public void subscribeToWriteEvent(SelectionKey selectionKey) {
+
+		synchronized (this.pendingWriteEventSubscribeRequests) {
+			this.pendingWriteEventSubscribeRequests.add(selectionKey);
+		}
+		synchronized (this.connectionsToClose) {
+			this.connectionsToClose.remove((OutgoingConnection) selectionKey.attachment());
+		}
+
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/AdaptiveSpanningRecordDeserializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/AdaptiveSpanningRecordDeserializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/AdaptiveSpanningRecordDeserializer.java
new file mode 100644
index 0000000..f4c8aec
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/AdaptiveSpanningRecordDeserializer.java
@@ -0,0 +1,521 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.serialization;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.core.memory.MemorySegment;
+import eu.stratosphere.runtime.io.serialization.DataInputDeserializer;
+import eu.stratosphere.runtime.io.serialization.DataOutputSerializer;
+import eu.stratosphere.runtime.io.serialization.RecordDeserializer;
+
+import java.io.DataInput;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.UTFDataFormatException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+
+/**
+ * @param <T> The type of the record to be deserialized.
+ */
+public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> implements RecordDeserializer<T> {
+	
+	private final NonSpanningWrapper nonSpanningWrapper;
+	
+	private final SpanningWrapper spanningWrapper;
+
+	public AdaptiveSpanningRecordDeserializer() {
+		this.nonSpanningWrapper = new NonSpanningWrapper();
+		this.spanningWrapper = new SpanningWrapper();
+	}
+	
+	@Override
+	public void setNextMemorySegment(MemorySegment segment, int numBytes) throws IOException {
+		// check if some spanning record deserialization is pending
+		if (this.spanningWrapper.getNumGatheredBytes() > 0) {
+			this.spanningWrapper.addNextChunkFromMemorySegment(segment, numBytes);
+		}
+		else {
+			this.nonSpanningWrapper.initializeFromMemorySegment(segment, 0, numBytes);
+		}
+	}
+	
+	@Override
+	public DeserializationResult getNextRecord(T target) throws IOException {
+		// always check the non-spanning wrapper first.
+		// this should be the majority of the cases for small records
+		// for large records, this portion of the work is very small in comparison anyways
+		
+		int nonSpanningRemaining = this.nonSpanningWrapper.remaining();
+		
+		// check if we can get a full length;
+		if (nonSpanningRemaining >= 4) {
+			int len = this.nonSpanningWrapper.readInt();
+			if (len <= nonSpanningRemaining - 4) {
+				// we can get a full record from here
+				target.read(this.nonSpanningWrapper);
+				
+				return (this.nonSpanningWrapper.remaining() == 0) ?
+					DeserializationResult.LAST_RECORD_FROM_BUFFER :
+					DeserializationResult.INTERMEDIATE_RECORD_FROM_BUFFER;
+			} else {
+				// we got the length, but we need the rest from the spanning deserializer
+				// and need to wait for more buffers
+				this.spanningWrapper.initializeWithPartialRecord(this.nonSpanningWrapper, len);
+				this.nonSpanningWrapper.clear();
+				return DeserializationResult.PARTIAL_RECORD;
+			}
+		} else if (nonSpanningRemaining > 0) {
+			// we have an incomplete length
+			// add our part of the length to the length buffer
+			this.spanningWrapper.initializeWithPartialLength(this.nonSpanningWrapper);
+			this.nonSpanningWrapper.clear();
+			return DeserializationResult.PARTIAL_RECORD;
+		}
+		
+		// spanning record case
+		if (this.spanningWrapper.hasFullRecord()) {
+			// get the full record
+			target.read(this.spanningWrapper);
+			
+			// move the remainder to the non-spanning wrapper
+			// this does not copy it, only sets the memory segment
+			this.spanningWrapper.moveRemainderToNonSpanningDeserializer(this.nonSpanningWrapper);
+			this.spanningWrapper.clear();
+			
+			return (this.nonSpanningWrapper.remaining() == 0) ?
+				DeserializationResult.LAST_RECORD_FROM_BUFFER :
+				DeserializationResult.INTERMEDIATE_RECORD_FROM_BUFFER;
+//		} else if (this.spanningWrapper.getNumGatheredBytes() == 0) {
+//			// error case. we are in the spanning deserializer, but it has no bytes, yet
+//			throw new IllegalStateException();
+		} else {
+			return DeserializationResult.PARTIAL_RECORD;
+		}
+	}
+
+	@Override
+	public void clear() {
+		this.nonSpanningWrapper.clear();
+		this.spanningWrapper.clear();
+	}
+
+	@Override
+	public boolean hasUnfinishedData() {
+		return this.nonSpanningWrapper.remaining() > 0 || this.spanningWrapper.getNumGatheredBytes() > 0;
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+	
+	private static final class NonSpanningWrapper implements DataInput {
+		
+		private MemorySegment segment;
+		
+		private int limit;
+		
+		private int position;
+		
+		private byte[] utfByteBuffer; // reusable byte buffer for utf-8 decoding
+		private char[] utfCharBuffer; // reusable char buffer for utf-8 decoding
+		
+		int remaining() {
+			return this.limit - this.position;
+		}
+		
+		void clear() {
+			this.segment = null;
+			this.limit = 0;
+			this.position = 0;
+		}
+		
+		void initializeFromMemorySegment(MemorySegment seg, int position, int leftOverLimit) {
+			this.segment = seg;
+			this.position = position;
+			this.limit = leftOverLimit;
+		}
+		
+		// -------------------------------------------------------------------------------------------------------------
+		//                                       DataInput specific methods
+		// -------------------------------------------------------------------------------------------------------------
+		
+		@Override
+		public final void readFully(byte[] b) throws IOException {
+			readFully(b, 0, b.length);
+		}
+
+		@Override
+		public final void readFully(byte[] b, int off, int len) throws IOException {
+			if (off < 0 || len < 0 || off + len > b.length)
+				throw new IndexOutOfBoundsException();
+			
+			this.segment.get(this.position, b, off, len);
+			this.position += len;
+		}
+
+		@Override
+		public final boolean readBoolean() throws IOException {
+			return readByte() == 1;
+		}
+
+		@Override
+		public final byte readByte() throws IOException {
+			return this.segment.get(this.position++);
+		}
+
+		@Override
+		public final int readUnsignedByte() throws IOException {
+			return readByte() & 0xff;
+		}
+
+		@Override
+		public final short readShort() throws IOException {
+			final short v = this.segment.getShort(this.position);
+			this.position += 2;
+			return v;
+		}
+
+		@Override
+		public final int readUnsignedShort() throws IOException {
+			final int v = this.segment.getShort(this.position) & 0xffff;
+			this.position += 2;
+			return v;
+		}
+
+		@Override
+		public final char readChar() throws IOException  {
+			final char v = this.segment.getChar(this.position);
+			this.position += 2;
+			return v;
+		}
+
+		@Override
+		public final int readInt() throws IOException {
+			final int v = this.segment.getIntBigEndian(this.position);
+			this.position += 4;
+			return v;
+		}
+
+		@Override
+		public final long readLong() throws IOException {
+			final long v = this.segment.getLongBigEndian(this.position);
+			this.position += 8;
+			return v;
+		}
+
+		@Override
+		public final float readFloat() throws IOException {
+			return Float.intBitsToFloat(readInt());
+		}
+
+		@Override
+		public final double readDouble() throws IOException {
+			return Double.longBitsToDouble(readLong());
+		}
+
+		@Override
+		public final String readLine() throws IOException {
+			final StringBuilder bld = new StringBuilder(32);
+			
+			try {
+				int b;
+				while ((b = readUnsignedByte()) != '\n') {
+					if (b != '\r')
+						bld.append((char) b);
+				}
+			}
+			catch (EOFException eofex) {}
+
+			if (bld.length() == 0)
+				return null;
+			
+			// trim a trailing carriage return
+			int len = bld.length();
+			if (len > 0 && bld.charAt(len - 1) == '\r') {
+				bld.setLength(len - 1);
+			}
+			return bld.toString();
+		}
+
+		@Override
+		public final String readUTF() throws IOException {
+			final int utflen = readUnsignedShort();
+			
+			final byte[] bytearr;
+			final char[] chararr;
+			
+			if (this.utfByteBuffer == null || this.utfByteBuffer.length < utflen) {
+				bytearr = new byte[utflen];
+				this.utfByteBuffer = bytearr;
+			} else {
+				bytearr = this.utfByteBuffer;
+			}
+			if (this.utfCharBuffer == null || this.utfCharBuffer.length < utflen) {
+				chararr = new char[utflen];
+				this.utfCharBuffer = chararr;
+			} else {
+				chararr = this.utfCharBuffer;
+			}
+
+			int c, char2, char3;
+			int count = 0;
+			int chararr_count = 0;
+
+			readFully(bytearr, 0, utflen);
+
+			while (count < utflen) {
+				c = (int) bytearr[count] & 0xff;
+				if (c > 127)
+					break;
+				count++;
+				chararr[chararr_count++] = (char) c;
+			}
+
+			while (count < utflen) {
+				c = (int) bytearr[count] & 0xff;
+				switch (c >> 4) {
+				case 0:
+				case 1:
+				case 2:
+				case 3:
+				case 4:
+				case 5:
+				case 6:
+				case 7:
+					count++;
+					chararr[chararr_count++] = (char) c;
+					break;
+				case 12:
+				case 13:
+					count += 2;
+					if (count > utflen)
+						throw new UTFDataFormatException("malformed input: partial character at end");
+					char2 = (int) bytearr[count - 1];
+					if ((char2 & 0xC0) != 0x80)
+						throw new UTFDataFormatException("malformed input around byte " + count);
+					chararr[chararr_count++] = (char) (((c & 0x1F) << 6) | (char2 & 0x3F));
+					break;
+				case 14:
+					count += 3;
+					if (count > utflen)
+						throw new UTFDataFormatException("malformed input: partial character at end");
+					char2 = (int) bytearr[count - 2];
+					char3 = (int) bytearr[count - 1];
+					if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80))
+						throw new UTFDataFormatException("malformed input around byte " + (count - 1));
+					chararr[chararr_count++] = (char) (((c & 0x0F) << 12) | ((char2 & 0x3F) << 6) | ((char3 & 0x3F) << 0));
+					break;
+				default:
+					throw new UTFDataFormatException("malformed input around byte " + count);
+				}
+			}
+			// The number of chars produced may be less than utflen
+			return new String(chararr, 0, chararr_count);
+		}
+		
+		@Override
+		public final int skipBytes(int n) throws IOException {
+			if (n < 0)
+				throw new IllegalArgumentException();
+			
+			int toSkip = Math.min(n, remaining());
+			this.position += toSkip;
+			return toSkip;
+		}
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+	
+	private static final class SpanningWrapper implements DataInput {
+
+		private final DataOutputSerializer serializationBuffer;
+
+		private final DataInputDeserializer serializationReadBuffer;
+
+		private final ByteBuffer lengthBuffer;
+
+		private int recordLength;
+
+		private MemorySegment leftOverData;
+
+		private int leftOverStart;
+
+		private int leftOverLimit;
+
+		private int recordLimit;
+
+		public SpanningWrapper() {
+			this.lengthBuffer = ByteBuffer.allocate(4);
+			this.lengthBuffer.order(ByteOrder.BIG_ENDIAN);
+
+			this.recordLength = -1;
+
+			this.serializationBuffer = new DataOutputSerializer(1024);
+			this.serializationReadBuffer = new DataInputDeserializer();
+		}
+		
+		private void initializeWithPartialRecord(NonSpanningWrapper partial, int nextRecordLength) throws IOException {
+			// set the length and copy what is available to the buffer
+			this.recordLength = nextRecordLength;
+			this.recordLimit = partial.remaining();
+			partial.segment.get(this.serializationBuffer, partial.position, partial.remaining());
+			this.serializationReadBuffer.setBuffer(this.serializationBuffer.wrapAsByteBuffer());
+		}
+		
+		private void initializeWithPartialLength(NonSpanningWrapper partial) throws IOException {
+			// copy what we have to the length buffer
+			partial.segment.get(partial.position, this.lengthBuffer, partial.remaining());
+		}
+		
+		private void addNextChunkFromMemorySegment(MemorySegment segment, int numBytesInSegment) throws IOException {
+			int segmentPosition = 0;
+			
+			// check where to go. if we have a partial length, we need to complete it first
+			if (this.lengthBuffer.position() > 0) {
+				int toPut = Math.min(this.lengthBuffer.remaining(), numBytesInSegment);
+				segment.get(0, this.lengthBuffer, toPut);
+				
+				// did we complete the length?
+				if (this.lengthBuffer.hasRemaining()) {
+					return;
+				} else {
+					this.recordLength = this.lengthBuffer.getInt(0);
+					this.lengthBuffer.clear();
+					segmentPosition = toPut;
+				}
+			}
+
+			// copy as much as we need or can for this next spanning record
+			int needed = this.recordLength - this.recordLimit;
+			int available = numBytesInSegment - segmentPosition;
+			int toCopy = Math.min(needed, available);
+
+			segment.get(this.serializationBuffer, segmentPosition, toCopy);
+			this.recordLimit += toCopy;
+			
+			if (toCopy < available) {
+				// there is more data in the segment
+				this.leftOverData = segment;
+				this.leftOverStart = segmentPosition + toCopy;
+				this.leftOverLimit = numBytesInSegment;
+			}
+
+			// update read view
+			this.serializationReadBuffer.setBuffer(this.serializationBuffer.wrapAsByteBuffer());
+		}
+		
+		private void moveRemainderToNonSpanningDeserializer(NonSpanningWrapper deserializer) {
+			deserializer.clear();
+			
+			if (leftOverData != null) {
+				deserializer.initializeFromMemorySegment(leftOverData, leftOverStart, leftOverLimit);
+			}
+		}
+		
+		private boolean hasFullRecord() {
+			return this.recordLength >= 0 && this.recordLimit >= this.recordLength;
+		}
+		
+		private int getNumGatheredBytes() {
+			return this.recordLimit + (this.recordLength >= 0 ? 4 : lengthBuffer.position()) + this.serializationBuffer.length();
+		}
+
+		public void clear() {
+			this.serializationBuffer.clear();
+
+			this.recordLength = -1;
+			this.lengthBuffer.clear();
+			this.leftOverData = null;
+			this.recordLimit = 0;
+		}
+
+		// -------------------------------------------------------------------------------------------------------------
+		//                                       DataInput specific methods
+		// -------------------------------------------------------------------------------------------------------------
+
+		@Override
+		public void readFully(byte[] b) throws IOException {
+			this.serializationReadBuffer.readFully(b);
+		}
+
+		@Override
+		public void readFully(byte[] b, int off, int len) throws IOException {
+			this.serializationReadBuffer.readFully(b, off, len);
+		}
+
+		@Override
+		public int skipBytes(int n) throws IOException {
+			return this.serializationReadBuffer.skipBytes(n);
+		}
+
+		@Override
+		public boolean readBoolean() throws IOException {
+			return this.serializationReadBuffer.readBoolean();
+		}
+
+		@Override
+		public byte readByte() throws IOException {
+			return this.serializationReadBuffer.readByte();
+		}
+
+		@Override
+		public int readUnsignedByte() throws IOException {
+			return this.serializationReadBuffer.readUnsignedByte();
+		}
+
+		@Override
+		public short readShort() throws IOException {
+			return this.serializationReadBuffer.readShort();
+		}
+
+		@Override
+		public int readUnsignedShort() throws IOException {
+			return this.serializationReadBuffer.readUnsignedShort();
+		}
+
+		@Override
+		public char readChar() throws IOException {
+			return this.serializationReadBuffer.readChar();
+		}
+
+		@Override
+		public int readInt() throws IOException {
+			return this.serializationReadBuffer.readInt();
+		}
+
+		@Override
+		public long readLong() throws IOException {
+			return this.serializationReadBuffer.readLong();
+		}
+
+		@Override
+		public float readFloat() throws IOException {
+			return this.serializationReadBuffer.readFloat();
+		}
+
+		@Override
+		public double readDouble() throws IOException {
+			return this.serializationReadBuffer.readDouble();
+		}
+
+		@Override
+		public String readLine() throws IOException {
+			return this.serializationReadBuffer.readLine();
+		}
+
+		@Override
+		public String readUTF() throws IOException {
+			return this.serializationReadBuffer.readUTF();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataInputDeserializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataInputDeserializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataInputDeserializer.java
new file mode 100644
index 0000000..e6479fe
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataInputDeserializer.java
@@ -0,0 +1,307 @@
+/***********************************************************************************************************************
+ *
+ * Copyright (C) 2010-2014 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.serialization;
+
+import eu.stratosphere.core.memory.MemoryUtils;
+
+import java.io.DataInput;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.UTFDataFormatException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+
+/**
+ * A simple and efficient deserializer for the {@link java.io.DataInput} interface.
+ */
+public class DataInputDeserializer implements DataInput {
+	
+	private byte[] buffer;
+	
+	private int end;
+
+	private int position;
+
+	public DataInputDeserializer() {
+	}
+	
+	public DataInputDeserializer(byte[] buffer, int start, int len) {
+		setBuffer(buffer, start, len);
+	}
+	
+	public DataInputDeserializer(ByteBuffer buffer) {
+		setBuffer(buffer);
+	}
+
+	public void setBuffer(ByteBuffer buffer) {
+		if (buffer.hasArray()) {
+			this.buffer = buffer.array();
+			this.position = buffer.arrayOffset() + buffer.position();
+			this.end = this.position + buffer.remaining();
+		} else if (buffer.isDirect()) {
+			this.buffer = new byte[buffer.remaining()];
+			this.position = 0;
+			this.end = this.buffer.length;
+
+			buffer.get(this.buffer);
+		} else {
+			throw new IllegalArgumentException("The given buffer is neither an array-backed heap ByteBuffer, nor a direct ByteBuffer.");
+		}
+	}
+
+	public void setBuffer(byte[] buffer, int start, int len) {
+		if (buffer == null)
+			throw new NullPointerException();
+
+		if (start < 0 || len < 0 || start + len >= buffer.length)
+			throw new IllegalArgumentException();
+
+		this.buffer = buffer;
+		this.position = start;
+		this.end = start * len;
+	}
+
+	// ----------------------------------------------------------------------------------------
+	//                               Data Input
+	// ----------------------------------------------------------------------------------------
+	
+	@Override
+	public boolean readBoolean() throws IOException {
+		if (this.position < this.end) {
+			return this.buffer[this.position++] != 0;
+		} else {
+			throw new EOFException();
+		}
+	}
+
+	@Override
+	public byte readByte() throws IOException {
+		if (this.position < this.end) {
+			return this.buffer[this.position++];
+		} else {
+			throw new EOFException();
+		}
+	}
+
+	@Override
+	public char readChar() throws IOException {
+		if (this.position < this.end - 1) {
+			return (char) (((this.buffer[this.position++] & 0xff) << 8) | ((this.buffer[this.position++] & 0xff) << 0));
+		} else {
+			throw new EOFException();
+		}
+	}
+
+	@Override
+	public double readDouble() throws IOException {
+		return Double.longBitsToDouble(readLong());
+	}
+
+	@Override
+	public float readFloat() throws IOException {
+		return Float.intBitsToFloat(readInt());
+	}
+
+	@Override
+	public void readFully(byte[] b) throws IOException {
+		readFully(b, 0, b.length);
+	}
+
+	@Override
+	public void readFully(byte[] b, int off, int len) throws IOException {
+		if (len >= 0) {
+			if (off <= b.length - len) {
+				if (this.position <= this.end - len) {
+					System.arraycopy(this.buffer, position, b, off, len);
+					position += len;
+				} else {
+					throw new EOFException();
+				}
+			} else {
+				throw new ArrayIndexOutOfBoundsException();
+			}
+		} else if (len < 0) {
+			throw new IllegalArgumentException("Length may not be negative.");
+		}
+	}
+
+	@Override
+	public int readInt() throws IOException {
+		if (this.position >= 0 && this.position < this.end - 3) {
+			@SuppressWarnings("restriction")
+			int value = UNSAFE.getInt(this.buffer, BASE_OFFSET + this.position);
+			if (LITTLE_ENDIAN) {
+				 value = Integer.reverseBytes(value);
+			}
+			
+			this.position += 4;
+			return value;
+		} else {
+			throw new EOFException();
+		}
+	}
+
+	@Override
+	public String readLine() throws IOException {
+		if (this.position < this.end) {
+			// read until a newline is found
+			StringBuilder bld = new StringBuilder();
+			char curr = (char) readUnsignedByte();
+			while (position < this.end && curr != '\n') {
+				bld.append(curr);
+				curr = (char) readUnsignedByte();
+			}
+			// trim a trailing carriage return
+			int len = bld.length();
+			if (len > 0 && bld.charAt(len - 1) == '\r') {
+				bld.setLength(len - 1);
+			}
+			String s = bld.toString();
+			bld.setLength(0);
+			return s;
+		} else {
+			return null;
+		}
+	}
+
+	@Override
+	public long readLong() throws IOException {
+		if (position >= 0 && position < this.end - 7) {
+			@SuppressWarnings("restriction")
+			long value = UNSAFE.getLong(this.buffer, BASE_OFFSET + this.position);
+			if (LITTLE_ENDIAN) {
+				 value = Long.reverseBytes(value);
+			}
+			this.position += 8;
+			return value;
+		} else {
+			throw new EOFException();
+		}
+	}
+
+	@Override
+	public short readShort() throws IOException {
+		if (position >= 0 && position < this.end - 1) {
+			return (short) ((((this.buffer[position++]) & 0xff) << 8) | (((this.buffer[position++]) & 0xff) << 0));
+		} else {
+			throw new EOFException();
+		}
+	}
+
+	@Override
+	public String readUTF() throws IOException {
+		int utflen = readUnsignedShort();
+		byte[] bytearr = new byte[utflen];
+		char[] chararr = new char[utflen];
+
+		int c, char2, char3;
+		int count = 0;
+		int chararr_count = 0;
+
+		readFully(bytearr, 0, utflen);
+
+		while (count < utflen) {
+			c = (int) bytearr[count] & 0xff;
+			if (c > 127)
+				break;
+			count++;
+			chararr[chararr_count++] = (char) c;
+		}
+
+		while (count < utflen) {
+			c = (int) bytearr[count] & 0xff;
+			switch (c >> 4) {
+			case 0:
+			case 1:
+			case 2:
+			case 3:
+			case 4:
+			case 5:
+			case 6:
+			case 7:
+				/* 0xxxxxxx */
+				count++;
+				chararr[chararr_count++] = (char) c;
+				break;
+			case 12:
+			case 13:
+				/* 110x xxxx 10xx xxxx */
+				count += 2;
+				if (count > utflen)
+					throw new UTFDataFormatException("malformed input: partial character at end");
+				char2 = (int) bytearr[count - 1];
+				if ((char2 & 0xC0) != 0x80)
+					throw new UTFDataFormatException("malformed input around byte " + count);
+				chararr[chararr_count++] = (char) (((c & 0x1F) << 6) | (char2 & 0x3F));
+				break;
+			case 14:
+				/* 1110 xxxx 10xx xxxx 10xx xxxx */
+				count += 3;
+				if (count > utflen)
+					throw new UTFDataFormatException("malformed input: partial character at end");
+				char2 = (int) bytearr[count - 2];
+				char3 = (int) bytearr[count - 1];
+				if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80))
+					throw new UTFDataFormatException("malformed input around byte " + (count - 1));
+				chararr[chararr_count++] = (char) (((c & 0x0F) << 12) | ((char2 & 0x3F) << 6) | ((char3 & 0x3F) << 0));
+				break;
+			default:
+				/* 10xx xxxx, 1111 xxxx */
+				throw new UTFDataFormatException("malformed input around byte " + count);
+			}
+		}
+		// The number of chars produced may be less than utflen
+		return new String(chararr, 0, chararr_count);
+	}
+
+	@Override
+	public int readUnsignedByte() throws IOException {
+		if (this.position < this.end) {
+			return (this.buffer[this.position++] & 0xff);
+		} else {
+			throw new EOFException();
+		}
+	}
+
+	@Override
+	public int readUnsignedShort() throws IOException {
+		if (this.position < this.end - 1) {
+			return ((this.buffer[this.position++] & 0xff) << 8) | ((this.buffer[this.position++] & 0xff) << 0);
+		} else {
+			throw new EOFException();
+		}
+	}
+	
+	@Override
+	public int skipBytes(int n) throws IOException {
+		if (this.position <= this.end - n) {
+			this.position += n;
+			return n;
+		} else {
+			n = this.end - this.position;
+			this.position = this.end;
+			return n;
+		}
+	}
+	
+	@SuppressWarnings("restriction")
+	private static final sun.misc.Unsafe UNSAFE = MemoryUtils.UNSAFE;
+	
+	@SuppressWarnings("restriction")
+	private static final long BASE_OFFSET = UNSAFE.arrayBaseOffset(byte[].class);
+	
+	private static final boolean LITTLE_ENDIAN = (MemoryUtils.NATIVE_BYTE_ORDER == ByteOrder.LITTLE_ENDIAN);
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataOutputSerializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataOutputSerializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataOutputSerializer.java
new file mode 100644
index 0000000..b5171b9
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataOutputSerializer.java
@@ -0,0 +1,259 @@
+/***********************************************************************************************************************
+ *
+ * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES 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.serialization;
+
+import eu.stratosphere.core.memory.MemoryUtils;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.UTFDataFormatException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+
+/**
+ * A simple and efficient serializer for the {@link java.io.DataOutput} interface.
+ */
+public class DataOutputSerializer implements DataOutput {
+	
+	private byte[] buffer;
+	
+	private int position;
+
+	private ByteBuffer wrapper;
+	
+	public DataOutputSerializer(int startSize) {
+		if (startSize < 1) {
+			throw new IllegalArgumentException();
+		}
+
+		this.buffer = new byte[startSize];
+		this.wrapper = ByteBuffer.wrap(buffer);
+	}
+	
+	public ByteBuffer wrapAsByteBuffer() {
+		this.wrapper.position(0);
+		this.wrapper.limit(this.position);
+		return this.wrapper;
+	}
+
+	public void clear() {
+		this.position = 0;
+	}
+
+	public int length() {
+		return this.position;
+	}
+
+	@Override
+	public String toString() {
+		return String.format("[pos=%d cap=%d]", this.position, this.buffer.length);
+	}
+
+	// ----------------------------------------------------------------------------------------
+	//                               Data Output
+	// ----------------------------------------------------------------------------------------
+	
+	@Override
+	public void write(int b) throws IOException {
+		if (this.position >= this.buffer.length) {
+			resize(1);
+		}
+		this.buffer[this.position++] = (byte) (b & 0xff);
+	}
+
+	@Override
+	public void write(byte[] b) throws IOException {
+		write(b, 0, b.length);
+	}
+
+	@Override
+	public void write(byte[] b, int off, int len) throws IOException {
+		if (len < 0 || off > b.length - len) {
+			throw new ArrayIndexOutOfBoundsException();
+		}
+		if (this.position > this.buffer.length - len) {
+			resize(len);
+		}
+		System.arraycopy(b, off, this.buffer, this.position, len);
+		this.position += len;
+	}
+
+	@Override
+	public void writeBoolean(boolean v) throws IOException {
+		write(v ? 1 : 0);
+	}
+
+	@Override
+	public void writeByte(int v) throws IOException {
+		write(v);
+	}
+
+	@Override
+	public void writeBytes(String s) throws IOException {
+		final int sLen = s.length();
+		if (this.position >= this.buffer.length - sLen) {
+			resize(sLen);
+		}
+		
+		for (int i = 0; i < sLen; i++) {
+			writeByte(s.charAt(i));
+		}
+		this.position += sLen;
+	}
+
+	@Override
+	public void writeChar(int v) throws IOException {
+		if (this.position >= this.buffer.length - 1) {
+			resize(2);
+		}
+		this.buffer[this.position++] = (byte) (v >> 8);
+		this.buffer[this.position++] = (byte) v;
+	}
+
+	@Override
+	public void writeChars(String s) throws IOException {
+		final int sLen = s.length();
+		if (this.position >= this.buffer.length - 2*sLen) {
+			resize(2*sLen);
+		} 
+		for (int i = 0; i < sLen; i++) {
+			writeChar(s.charAt(i));
+		}
+	}
+
+	@Override
+	public void writeDouble(double v) throws IOException {
+		writeLong(Double.doubleToLongBits(v));
+	}
+
+	@Override
+	public void writeFloat(float v) throws IOException {
+		writeInt(Float.floatToIntBits(v));
+	}
+
+	@SuppressWarnings("restriction")
+	@Override
+	public void writeInt(int v) throws IOException {
+		if (this.position >= this.buffer.length - 3) {
+			resize(4);
+		}
+		if (LITTLE_ENDIAN) {
+			v = Integer.reverseBytes(v);
+		}			
+		UNSAFE.putInt(this.buffer, BASE_OFFSET + this.position, v);
+		this.position += 4;
+	}
+
+	@SuppressWarnings("restriction")
+	@Override
+	public void writeLong(long v) throws IOException {
+		if (this.position >= this.buffer.length - 7) {
+			resize(8);
+		}
+		if (LITTLE_ENDIAN) {
+			v = Long.reverseBytes(v);
+		}
+		UNSAFE.putLong(this.buffer, BASE_OFFSET + this.position, v);
+		this.position += 8;
+	}
+
+	@Override
+	public void writeShort(int v) throws IOException {
+		if (this.position >= this.buffer.length - 1) {
+			resize(2);
+		}
+		this.buffer[this.position++] = (byte) ((v >>> 8) & 0xff);
+		this.buffer[this.position++] = (byte) ((v >>> 0) & 0xff);
+	}
+
+	@Override
+	public void writeUTF(String str) throws IOException {
+		int strlen = str.length();
+		int utflen = 0;
+		int c;
+
+		/* use charAt instead of copying String to char array */
+		for (int i = 0; i < strlen; i++) {
+			c = str.charAt(i);
+			if ((c >= 0x0001) && (c <= 0x007F)) {
+				utflen++;
+			} else if (c > 0x07FF) {
+				utflen += 3;
+			} else {
+				utflen += 2;
+			}
+		}
+
+		if (utflen > 65535)
+			throw new UTFDataFormatException("Encoded string is too long: " + utflen);
+		
+		else if (this.position > this.buffer.length - utflen - 2) {
+			resize(utflen + 2);
+		}
+		
+		byte[] bytearr = this.buffer;
+		int count = this.position;
+
+		bytearr[count++] = (byte) ((utflen >>> 8) & 0xFF);
+		bytearr[count++] = (byte) ((utflen >>> 0) & 0xFF);
+
+		int i = 0;
+		for (i = 0; i < strlen; i++) {
+			c = str.charAt(i);
+			if (!((c >= 0x0001) && (c <= 0x007F)))
+				break;
+			bytearr[count++] = (byte) c;
+		}
+
+		for (; i < strlen; i++) {
+			c = str.charAt(i);
+			if ((c >= 0x0001) && (c <= 0x007F)) {
+				bytearr[count++] = (byte) c;
+
+			} else if (c > 0x07FF) {
+				bytearr[count++] = (byte) (0xE0 | ((c >> 12) & 0x0F));
+				bytearr[count++] = (byte) (0x80 | ((c >> 6) & 0x3F));
+				bytearr[count++] = (byte) (0x80 | ((c >> 0) & 0x3F));
+			} else {
+				bytearr[count++] = (byte) (0xC0 | ((c >> 6) & 0x1F));
+				bytearr[count++] = (byte) (0x80 | ((c >> 0) & 0x3F));
+			}
+		}
+
+		this.position = count;
+	}
+	
+	
+	private final void resize(int minCapacityAdd) throws IOException {
+		try {
+			final int newLen = Math.max(this.buffer.length * 2, this.buffer.length + minCapacityAdd);
+			final byte[] nb = new byte[newLen];
+			System.arraycopy(this.buffer, 0, nb, 0, this.position);
+			this.buffer = nb;
+			this.wrapper = ByteBuffer.wrap(this.buffer);
+		}
+		catch (NegativeArraySizeException nasex) {
+			throw new IOException("Serialization failed because the record length would exceed 2GB (max addressable array size in Java).");
+		}
+	}
+	
+	@SuppressWarnings("restriction")
+	private static final sun.misc.Unsafe UNSAFE = MemoryUtils.UNSAFE;
+	
+	@SuppressWarnings("restriction")
+	private static final long BASE_OFFSET = UNSAFE.arrayBaseOffset(byte[].class);
+	
+	private static final boolean LITTLE_ENDIAN = (MemoryUtils.NATIVE_BYTE_ORDER == ByteOrder.LITTLE_ENDIAN);
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/RecordDeserializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/RecordDeserializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/RecordDeserializer.java
new file mode 100644
index 0000000..708c693
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/RecordDeserializer.java
@@ -0,0 +1,56 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.serialization;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.core.memory.MemorySegment;
+
+import java.io.IOException;
+
+/**
+ * Interface for turning sequences of memory segments into records.
+ */
+public interface RecordDeserializer<T extends IOReadableWritable> {
+
+	public static enum DeserializationResult {
+		PARTIAL_RECORD(false, true),
+		INTERMEDIATE_RECORD_FROM_BUFFER(true, false),
+		LAST_RECORD_FROM_BUFFER(true, true);
+
+		private final boolean isFullRecord;
+
+		private final boolean isBufferConsumed;
+
+		private DeserializationResult(boolean isFullRecord, boolean isBufferConsumed) {
+			this.isFullRecord = isFullRecord;
+			this.isBufferConsumed = isBufferConsumed;
+		}
+
+		public boolean isFullRecord () {
+			return this.isFullRecord;
+		}
+
+		public boolean isBufferConsumed() {
+			return this.isBufferConsumed;
+		}
+	}
+	
+	DeserializationResult getNextRecord(T target) throws IOException;
+
+	void setNextMemorySegment(MemorySegment segment, int numBytes) throws IOException;
+
+	void clear();
+	
+	boolean hasUnfinishedData();
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/RecordSerializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/RecordSerializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/RecordSerializer.java
new file mode 100644
index 0000000..b540e27
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/RecordSerializer.java
@@ -0,0 +1,60 @@
+/***********************************************************************************************************************
+ *
+ * Copyright (C) 2010-2014 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.serialization;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.runtime.io.Buffer;
+
+import java.io.IOException;
+
+/**
+ * Interface for turning records into sequences of memory segments.
+ */
+public interface RecordSerializer<T extends IOReadableWritable> {
+
+	public static enum SerializationResult {
+		PARTIAL_RECORD_MEMORY_SEGMENT_FULL(false, true),
+		FULL_RECORD_MEMORY_SEGMENT_FULL(true, true),
+		FULL_RECORD(true, false);
+		
+		private final boolean isFullRecord;
+
+		private final boolean isFullBuffer;
+		
+		private SerializationResult(boolean isFullRecord, boolean isFullBuffer) {
+			this.isFullRecord = isFullRecord;
+			this.isFullBuffer = isFullBuffer;
+		}
+		
+		public boolean isFullRecord() {
+			return this.isFullRecord;
+		}
+		
+		public boolean isFullBuffer() {
+			return this.isFullBuffer;
+		}
+	}
+	
+	SerializationResult addRecord(T record) throws IOException;
+
+	SerializationResult setNextBuffer(Buffer buffer) throws IOException;
+
+	Buffer getCurrentBuffer();
+	
+	void clear();
+	
+	boolean hasData();
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializer.java
new file mode 100644
index 0000000..443f8d8
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializer.java
@@ -0,0 +1,153 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.serialization;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.core.memory.MemorySegment;
+import eu.stratosphere.runtime.io.Buffer;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+
+public class SpanningRecordSerializer<T extends IOReadableWritable> implements RecordSerializer<T> {
+
+	/** Flag to enable/disable checks, if buffer not set/full or pending serialization */
+	private static final boolean CHECKED = true;
+
+	/** Intermediate data serialization */
+	private final DataOutputSerializer serializationBuffer;
+
+	/** Intermediate buffer for data serialization */
+	private ByteBuffer dataBuffer;
+
+	/** Intermediate buffer for length serialization */
+	private final ByteBuffer lengthBuffer;
+
+	/** Current target {@link eu.stratosphere.runtime.io.Buffer} of the serializer */
+	private Buffer targetBuffer;
+
+	/** Position in current {@link MemorySegment} of target buffer */
+	private int position;
+
+	/** Limit of current {@link MemorySegment} of target buffer */
+	private int limit;
+
+	public SpanningRecordSerializer() {
+		this.serializationBuffer = new DataOutputSerializer(128);
+
+		this.lengthBuffer = ByteBuffer.allocate(4);
+		this.lengthBuffer.order(ByteOrder.BIG_ENDIAN);
+
+		// ensure initial state with hasRemaining false (for correct setNextBuffer logic)
+		this.dataBuffer = this.serializationBuffer.wrapAsByteBuffer();
+		this.lengthBuffer.position(4);
+	}
+
+	@Override
+	public SerializationResult addRecord(T record) throws IOException {
+		if (CHECKED) {
+			if (this.dataBuffer.hasRemaining()) {
+				throw new IllegalStateException("Pending serialization of previous record.");
+			}
+		}
+
+		this.serializationBuffer.clear();
+		this.lengthBuffer.clear();
+
+		// write data and length
+		record.write(this.serializationBuffer);
+		this.lengthBuffer.putInt(0, this.serializationBuffer.length());
+
+		this.dataBuffer = this.serializationBuffer.wrapAsByteBuffer();
+
+		// Copy from intermediate buffers to current target memory segment
+		copyToTargetBufferFrom(this.lengthBuffer);
+		copyToTargetBufferFrom(this.dataBuffer);
+
+		return getSerializationResult();
+	}
+
+	@Override
+	public SerializationResult setNextBuffer(Buffer buffer) throws IOException {
+		this.targetBuffer = buffer;
+		this.position = 0;
+		this.limit = buffer.size();
+
+		if (this.lengthBuffer.hasRemaining()) {
+			copyToTargetBufferFrom(this.lengthBuffer);
+		}
+
+		if (this.dataBuffer.hasRemaining()) {
+			copyToTargetBufferFrom(this.dataBuffer);
+		}
+
+		return getSerializationResult();
+	}
+
+	/**
+	 * Copies as many bytes as possible from the given {@link ByteBuffer} to the {@link MemorySegment} of the target
+	 * {@link Buffer} and advances the current position by the number of written bytes.
+	 *
+	 * @param source the {@link ByteBuffer} to copy data from
+	 */
+	private void copyToTargetBufferFrom(ByteBuffer source) {
+		if (this.targetBuffer == null)
+			return;
+
+		int needed = source.remaining();
+		int available = this.limit - this.position;
+		int toCopy = Math.min(needed, available);
+
+		this.targetBuffer.getMemorySegment().put(this.position, source, toCopy);
+
+		this.position += toCopy;
+	}
+
+	private SerializationResult getSerializationResult() {
+		if (!this.dataBuffer.hasRemaining() && !this.lengthBuffer.hasRemaining()) {
+			return (this.position < this.limit)
+					? SerializationResult.FULL_RECORD
+					: SerializationResult.FULL_RECORD_MEMORY_SEGMENT_FULL;
+		}
+
+		return SerializationResult.PARTIAL_RECORD_MEMORY_SEGMENT_FULL;
+	}
+
+	@Override
+	public Buffer getCurrentBuffer() {
+		if (targetBuffer == null)
+			return null;
+
+		this.targetBuffer.limitSize(this.position);
+		return this.targetBuffer;
+	}
+
+	@Override
+	public void clear() {
+		this.targetBuffer = null;
+		this.position = 0;
+		this.limit = 0;
+
+		// ensure clear state with hasRemaining false (for correct setNextBuffer logic)
+		this.dataBuffer.position(this.dataBuffer.limit());
+		this.lengthBuffer.position(4);
+	}
+
+	@Override
+	public boolean hasData() {
+		// either data in current target buffer or intermediate buffers
+		return this.position > 0 || (this.lengthBuffer.hasRemaining() || this.dataBuffer.hasRemaining());
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/AbstractIDTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/AbstractIDTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/AbstractIDTest.java
new file mode 100644
index 0000000..65a7b19
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/AbstractIDTest.java
@@ -0,0 +1,62 @@
+/***********************************************************************************************************************
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.nephele.util.CommonTestUtils;
+
+/**
+ * This class contains tests for the {@link eu.stratosphere.nephele.AbstractID} class.
+ * 
+ */
+public class AbstractIDTest {
+
+	/**
+	 * Tests the setID method of an abstract ID.
+	 */
+	@Test
+	public void testSetID() {
+
+		final ChannelID id1 = new ChannelID();
+		final ChannelID id2 = new ChannelID();
+		id1.setID(id2);
+
+		assertEquals(id1.hashCode(), id2.hashCode());
+		assertEquals(id1, id2);
+	}
+
+	/**
+	 * Tests the serialization/deserialization of an abstract ID.
+	 */
+	@Test
+	public void testSerialization() {
+
+		final ChannelID origID = new ChannelID();
+		try {
+			final ChannelID copyID = (ChannelID) CommonTestUtils.createCopy(origID);
+
+			assertEquals(origID.hashCode(), copyID.hashCode());
+			assertEquals(origID, copyID);
+
+		} catch (IOException e) {
+			e.printStackTrace();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/deployment/ChannelDeploymentDescriptorTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/deployment/ChannelDeploymentDescriptorTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/deployment/ChannelDeploymentDescriptorTest.java
index 7cf83d7..b3ff279 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/deployment/ChannelDeploymentDescriptorTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/deployment/ChannelDeploymentDescriptorTest.java
@@ -21,7 +21,7 @@ import java.io.IOException;
 
 import org.junit.Test;
 
-import eu.stratosphere.nephele.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.channels.ChannelID;
 import eu.stratosphere.nephele.util.ServerTestUtils;
 import eu.stratosphere.util.StringUtils;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/deployment/GateDeploymentDescriptorTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/deployment/GateDeploymentDescriptorTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/deployment/GateDeploymentDescriptorTest.java
index 66ccad8..bc7034f 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/deployment/GateDeploymentDescriptorTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/deployment/GateDeploymentDescriptorTest.java
@@ -23,9 +23,9 @@ import java.util.List;
 
 import org.junit.Test;
 
-import eu.stratosphere.nephele.io.GateID;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.gates.GateID;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.channels.ChannelType;
 import eu.stratosphere.nephele.util.ServerTestUtils;
 import eu.stratosphere.util.StringUtils;
 
@@ -42,7 +42,7 @@ public class GateDeploymentDescriptorTest {
 	public void testConstructorWithValidArguments() {
 
 		final GateID gateID = new GateID();
-		final ChannelType channelType = ChannelType.INMEMORY;
+		final ChannelType channelType = ChannelType.IN_MEMORY;
 		final List<ChannelDeploymentDescriptor> channels = new ArrayList<ChannelDeploymentDescriptor>(0);
 
 		final GateDeploymentDescriptor gdd = new GateDeploymentDescriptor(gateID, channelType, channels);
@@ -59,7 +59,7 @@ public class GateDeploymentDescriptorTest {
 	public void testConstructorWithInvalidArguments() {
 
 		final GateID gateID = new GateID();
-		final ChannelType channelType = ChannelType.INMEMORY;
+		final ChannelType channelType = ChannelType.IN_MEMORY;
 		final List<ChannelDeploymentDescriptor> channels = new ArrayList<ChannelDeploymentDescriptor>(0);
 
 		boolean firstExceptionCaught = false;
@@ -105,7 +105,7 @@ public class GateDeploymentDescriptorTest {
 	public void testSerialization() {
 
 		final GateID gateID = new GateID();
-		final ChannelType channelType = ChannelType.INMEMORY;
+		final ChannelType channelType = ChannelType.IN_MEMORY;
 		final List<ChannelDeploymentDescriptor> channels = new ArrayList<ChannelDeploymentDescriptor>(0);
 		final ChannelDeploymentDescriptor cdd = new ChannelDeploymentDescriptor(new ChannelID(), new ChannelID());
 		channels.add(cdd);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 e2581f8..7000667 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
@@ -24,7 +24,7 @@ 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.io.library.FileLineReader;
+import eu.stratosphere.nephele.util.FileLineReader;
 import eu.stratosphere.nephele.jobgraph.JobID;
 import eu.stratosphere.nephele.template.AbstractInvokable;
 import eu.stratosphere.nephele.util.SerializableArrayList;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 25e538d..5ff5f1c 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
@@ -45,10 +45,10 @@ 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.io.DistributionPattern;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-import eu.stratosphere.nephele.io.library.FileLineReader;
-import eu.stratosphere.nephele.io.library.FileLineWriter;
+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;
@@ -448,7 +448,7 @@ public class ExecutionGraphTest {
 	 * input1 -> task1 -> output1
 	 * no subtasks defined
 	 * input1 is default, task1 is m1.large, output1 is m1.xlarge
-	 * all channels are INMEMORY
+	 * all channels are IN_MEMORY
 	 */
 	@Test
 	public void testConvertJobGraphToExecutionGraph2() {
@@ -478,8 +478,8 @@ public class ExecutionGraphTest {
 			o1.setFilePath(new Path(new File(ServerTestUtils.getRandomFilename()).toURI()));
 
 			// connect vertices
-			i1.connectTo(t1, ChannelType.INMEMORY);
-			t1.connectTo(o1, ChannelType.INMEMORY);
+			i1.connectTo(t1, ChannelType.IN_MEMORY);
+			t1.connectTo(o1, ChannelType.IN_MEMORY);
 
 			LibraryCacheManager.register(jobID, new String[0]);
 
@@ -865,11 +865,11 @@ public class ExecutionGraphTest {
 			o1.setVertexToShareInstancesWith(o2);
 
 			// connect vertices
-			i1.connectTo(t1, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
-			i2.connectTo(t2, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+			i1.connectTo(t1, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
+			i2.connectTo(t2, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
 			t1.connectTo(t3, ChannelType.NETWORK);
 			t2.connectTo(t3, ChannelType.NETWORK);
-			t3.connectTo(t4, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+			t3.connectTo(t4, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
 			t4.connectTo(o1, ChannelType.NETWORK);
 			t4.connectTo(o2, ChannelType.NETWORK);
 
@@ -973,11 +973,11 @@ public class ExecutionGraphTest {
 			output.setNumberOfSubtasks(degreeOfParallelism);
 
 			// connect vertices
-			input.connectTo(cross, ChannelType.INMEMORY, 0, 0,
+			input.connectTo(cross, ChannelType.IN_MEMORY, 0, 0,
 				DistributionPattern.POINTWISE);
 			input.connectTo(cross, ChannelType.NETWORK, 1, 1,
 				DistributionPattern.BIPARTITE);
-			cross.connectTo(output, ChannelType.INMEMORY, 0, 0,
+			cross.connectTo(output, ChannelType.IN_MEMORY, 0, 0,
 				DistributionPattern.POINTWISE);
 
 			LibraryCacheManager.register(jobID, new String[0]);
@@ -1113,13 +1113,13 @@ public class ExecutionGraphTest {
 			output1.setNumberOfSubtasks(degreeOfParallelism);
 
 			// connect vertices
-			input1.connectTo(forward1, ChannelType.INMEMORY,
+			input1.connectTo(forward1, ChannelType.IN_MEMORY,
 				DistributionPattern.POINTWISE);
-			forward1.connectTo(forward2, ChannelType.INMEMORY,
+			forward1.connectTo(forward2, ChannelType.IN_MEMORY,
 				DistributionPattern.POINTWISE);
 			forward2.connectTo(forward3, ChannelType.NETWORK,
 				DistributionPattern.POINTWISE);
-			forward3.connectTo(output1, ChannelType.INMEMORY);
+			forward3.connectTo(output1, ChannelType.IN_MEMORY);
 
 			// setup instance sharing
 			input1.setVertexToShareInstancesWith(forward1);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 21341fa..0a2f52b 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,8 +14,8 @@
 package eu.stratosphere.nephele.executiongraph;
 
 import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.RecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
+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 {
@@ -26,18 +26,21 @@ public class ForwardTask1Input1Output extends AbstractTask {
 
 	@Override
 	public void invoke() throws Exception {
+		this.output.initializeSerializers();
 
 		while (this.input.hasNext()) {
 
 			StringRecord s = input.next();
 			this.output.emit(s);
 		}
+
+		this.output.flush();
 	}
 
 	@Override
 	public void registerInputOutput() {
 		this.input = new RecordReader<StringRecord>(this, StringRecord.class);
-		this.output = new RecordWriter<StringRecord>(this, StringRecord.class);
+		this.output = new RecordWriter<StringRecord>(this);
 	}
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 6582ea7..5a5c325 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
@@ -14,8 +14,8 @@
 package eu.stratosphere.nephele.executiongraph;
 
 import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.RecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
+import eu.stratosphere.runtime.io.api.RecordReader;
+import eu.stratosphere.runtime.io.api.RecordWriter;
 import eu.stratosphere.nephele.template.AbstractTask;
 
 public class ForwardTask1Input2Outputs extends AbstractTask {
@@ -29,18 +29,24 @@ public class ForwardTask1Input2Outputs extends AbstractTask {
 	@Override
 	public void invoke() throws Exception {
 
+		this.output1.initializeSerializers();
+		this.output2.initializeSerializers();
+
 		while (this.input.hasNext()) {
 
 			StringRecord s = input.next();
 			this.output1.emit(s);
 			this.output2.emit(s);
 		}
+
+		this.output1.flush();
+		this.output2.flush();
 	}
 
 	@Override
 	public void registerInputOutput() {
 		this.input = new RecordReader<StringRecord>(this, StringRecord.class);
-		this.output1 = new RecordWriter<StringRecord>(this, StringRecord.class);
-		this.output2 = new RecordWriter<StringRecord>(this, StringRecord.class);
+		this.output1 = new RecordWriter<StringRecord>(this);
+		this.output2 = new RecordWriter<StringRecord>(this);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 7e8af0b..c87d093 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,8 +14,8 @@
 package eu.stratosphere.nephele.executiongraph;
 
 import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.RecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
+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 {
@@ -28,6 +28,7 @@ public class ForwardTask2Inputs1Output extends AbstractTask {
 
 	@Override
 	public void invoke() throws Exception {
+		this.output.initializeSerializers();
 
 		while (this.input1.hasNext()) {
 
@@ -44,12 +45,14 @@ public class ForwardTask2Inputs1Output extends AbstractTask {
 				e.printStackTrace();
 			}
 		}
+
+		this.output.flush();
 	}
 
 	@Override
 	public void registerInputOutput() {
 		this.input1 = new RecordReader<StringRecord>(this, StringRecord.class);
 		this.input2 = new RecordReader<StringRecord>(this, StringRecord.class);
-		this.output = new RecordWriter<StringRecord>(this, StringRecord.class);
+		this.output = new RecordWriter<StringRecord>(this);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 c375fd4..05f181c 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
@@ -15,8 +15,8 @@ package eu.stratosphere.nephele.executiongraph;
 
 
 import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.RecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
+import eu.stratosphere.runtime.io.api.RecordReader;
+import eu.stratosphere.runtime.io.api.RecordWriter;
 import eu.stratosphere.nephele.template.AbstractTask;
 
 /**
@@ -31,7 +31,7 @@ public class SelfCrossForwardTask extends AbstractTask {
 		
 		new RecordReader<StringRecord>(this, StringRecord.class);
 		new RecordReader<StringRecord>(this, StringRecord.class);
-		new RecordWriter<StringRecord>(this, StringRecord.class);
+		new RecordWriter<StringRecord>(this);
 	}
 
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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
index a2af3a9..1ce23e6 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/SelfCrossInputTask.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/SelfCrossInputTask.java
@@ -14,7 +14,7 @@
 package eu.stratosphere.nephele.executiongraph;
 
 import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.RecordWriter;
+import eu.stratosphere.runtime.io.api.RecordWriter;
 import eu.stratosphere.nephele.template.AbstractFileInputTask;
 
 /**
@@ -27,8 +27,8 @@ public class SelfCrossInputTask extends AbstractFileInputTask {
 	@Override
 	public void registerInputOutput() {
 
-		new RecordWriter<StringRecord>(this, StringRecord.class);
-		new RecordWriter<StringRecord>(this, StringRecord.class);
+		new RecordWriter<StringRecord>(this);
+		new RecordWriter<StringRecord>(this);
 	}
 
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/fs/LineReaderTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/fs/LineReaderTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/fs/LineReaderTest.java
deleted file mode 100644
index 00dd645..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/fs/LineReaderTest.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.fs;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import java.io.File;
-import java.io.PrintWriter;
-
-import org.junit.Test;
-
-import eu.stratosphere.core.fs.FSDataInputStream;
-import eu.stratosphere.core.fs.Path;
-import eu.stratosphere.core.fs.local.LocalFileSystem;
-import eu.stratosphere.nephele.util.CommonTestUtils;
-import eu.stratosphere.runtime.fs.LineReader;
-
-/**
- * This class tests the functionality of the LineReader class using a local filesystem.
- * 
- */
-
-public class LineReaderTest {
-
-	/**
-	 * This test tests the LineReader. So far only under usual conditions.
-	 */
-	@Test
-	public void testLineReader() {
-		final File testfile = new File(CommonTestUtils.getTempDir() + File.separator
-			+ CommonTestUtils.getRandomFilename());
-		final Path pathtotestfile = new Path(testfile.toURI().getPath());
-
-		try {
-			PrintWriter pw = new PrintWriter(testfile, "UTF8");
-
-			for (int i = 0; i < 100; i++) {
-				pw.append("line\n");
-			}
-			pw.close();
-
-			LocalFileSystem lfs = new LocalFileSystem();
-			FSDataInputStream fis = lfs.open(pathtotestfile);
-
-			// first, we test under "usual" conditions
-			final LineReader lr = new LineReader(fis, 0, testfile.length(), 256);
-
-			byte[] buffer;
-			int linecount = 0;
-			while ((buffer = lr.readLine()) != null) {
-				assertEquals(new String(buffer, "UTF8"), "line");
-				linecount++;
-			}
-			assertEquals(linecount, 100);
-
-			// the linereader can not handle situations with larger length than the total file...
-
-		} catch (Exception e) {
-			fail(e.toString());
-			e.printStackTrace();
-		} finally {
-			testfile.delete();
-		}
-
-	}
-
-}


[29/30] git commit: Remove @Test from NettyConnectionManagerTest and run tests via main method

Posted by rm...@apache.org.
Remove @Test from NettyConnectionManagerTest and run tests via main method


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

Branch: refs/heads/master
Commit: b0f8ba06953f978516044db220a89cacbcbb2d5c
Parents: 6a40d41
Author: uce <u....@fu-berlin.de>
Authored: Tue Jun 3 15:14:22 2014 +0200
Committer: StephanEwen <st...@tu-berlin.de>
Committed: Sat Jun 7 09:41:22 2014 +0200

----------------------------------------------------------------------
 .../netty/NettyConnectionManagerTest.java       | 26 +++++++++++++++++---
 1 file changed, 22 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b0f8ba06/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManagerTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManagerTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManagerTest.java
index c424a1f..c380431 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManagerTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManagerTest.java
@@ -19,7 +19,6 @@ import eu.stratosphere.runtime.io.network.ChannelManager;
 import eu.stratosphere.runtime.io.network.Envelope;
 import eu.stratosphere.runtime.io.network.RemoteReceiver;
 import junit.framework.Assert;
-import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.mockito.Matchers;
@@ -69,9 +68,9 @@ public class NettyConnectionManagerTest {
 	}
 
 	@Parameterized.Parameters
-	public static Collection configure() {
+	public static Collection<Integer[]> configure() {
 		return Arrays.asList(
-				new Object[][]{
+				new Integer[][]{
 						{64, 4096, 1, 1, 1},
 						{128, 2048, 1, 1, 1},
 						{256, 1024, 1, 1, 1},
@@ -88,7 +87,6 @@ public class NettyConnectionManagerTest {
 		);
 	}
 
-	@Test
 	public void testEnqueueRaceAndDeadlockFreeMultipleChannels() throws Exception {
 		final InetAddress localhost = InetAddress.getLocalHost();
 		final CountDownLatch latch = new CountDownLatch(this.numSubtasks);
@@ -193,4 +191,24 @@ public class NettyConnectionManagerTest {
 			}
 		}
 	}
+
+	private void runAllTests() throws Exception {
+		System.out.println(String.format("Running tests with config: %d sub tasks, %d envelopes to send per subtasks, "
+				+ "%d num channels, %d num in threads, %d num out threads.",
+				this.numSubtasks, this.numToSendPerSubtask, this.numChannels, this.numInThreads, this.numOutThreads));
+
+		testEnqueueRaceAndDeadlockFreeMultipleChannels();
+
+		System.out.println("Done.");
+	}
+
+	public static void main(String[] args) throws Exception {
+		Collection<Integer[]> configs = configure();
+
+		for (Integer[] params : configs) {
+
+			NettyConnectionManagerTest test = new NettyConnectionManagerTest(params[0], params[1], params[2], params[3], params[4]);
+			test.runAllTests();
+		}
+	}
 }


[11/30] Offer buffer-oriented API for I/O (#25)

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/fs/s3/S3FileSystemTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/fs/s3/S3FileSystemTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/fs/s3/S3FileSystemTest.java
deleted file mode 100644
index 784178d..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/fs/s3/S3FileSystemTest.java
+++ /dev/null
@@ -1,461 +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.fs.s3;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.URI;
-
-import org.junit.Before;
-import org.junit.Test;
-
-import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.configuration.GlobalConfiguration;
-import eu.stratosphere.core.fs.BlockLocation;
-import eu.stratosphere.core.fs.FSDataInputStream;
-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.runtime.fs.s3.S3FileSystem;
-
-/**
- * This test checks the S3 implementation of the {@link FileSystem} interface.
- * 
- */
-public class S3FileSystemTest {
-
-	/**
-	 * The length of the bucket/object names used in this test.
-	 */
-	private static final int NAME_LENGTH = 32;
-
-	/**
-	 * The alphabet to generate the random bucket/object names from.
-	 */
-	private static final char[] ALPHABET = { 'a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k', 'l', 'm', 'n', 'o',
-		'p', 'q', 'r', 's', 't', 'u', 'v', 'w', 'x', 'y', 'z', '0', '1', '2', '3', '4', '5', '6', '7', '8', '9' };
-
-	/**
-	 * The size of the byte buffer used during the tests in bytes.
-	 */
-	private static final int TEST_BUFFER_SIZE = 128;
-
-	/**
-	 * The size of the small test file in bytes.
-	 */
-	private static final int SMALL_FILE_SIZE = 512;
-
-	/**
-	 * The size of the large test file in bytes.
-	 */
-	private static final int LARGE_FILE_SIZE = 1024 * 1024 * 12; // 12 MB
-
-	/**
-	 * The modulus to be used when generating the test data. Must not be larger than 128.
-	 */
-	private static final int MODULUS = 128;
-
-	private static final String S3_BASE_URI = "s3:///";
-
-	/**
-	 * Tries to read the AWS access key and the AWS secret key from the environments variables. If accessing these keys
-	 * fails, all tests will be skipped and marked as successful.
-	 */
-	@Before
-	public void initKeys() {
-		final String accessKey = System.getenv("AK");
-		final String secretKey = System.getenv("SK");
-		
-		if (accessKey != null || secretKey != null) {
-			Configuration conf = new Configuration();
-			if (accessKey != null) {
-				conf.setString(S3FileSystem.S3_ACCESS_KEY_KEY, accessKey);
-			}
-			if (secretKey != null) {
-				conf.setString(S3FileSystem.S3_SECRET_KEY_KEY, secretKey);
-			}
-			GlobalConfiguration.includeConfiguration(conf);
-		}
-	}
-
-	/**
-	 * This test creates and deletes a bucket inside S3 and checks it is correctly displayed inside the directory
-	 * listing.
-	 */
-	@Test
-	public void createAndDeleteBucketTest() {
-
-		if (!testActivated()) {
-			return;
-		}
-
-		final String bucketName = getRandomName();
-		final Path bucketPath = new Path(S3_BASE_URI + bucketName + Path.SEPARATOR);
-
-		try {
-
-			final FileSystem fs = bucketPath.getFileSystem();
-
-			// Create directory
-			fs.mkdirs(bucketPath);
-
-			// Check if directory is correctly displayed in file system hierarchy
-			final FileStatus[] content = fs.listStatus(new Path(S3_BASE_URI));
-			boolean entryFound = false;
-			for (final FileStatus entry : content) {
-				if (bucketPath.equals(entry.getPath())) {
-					entryFound = true;
-					break;
-				}
-			}
-
-			if (!entryFound) {
-				fail("Cannot find entry " + bucketName + " in directory " + S3_BASE_URI);
-			}
-
-			// Check the concrete directory file status
-			try {
-				final FileStatus directoryFileStatus = fs.getFileStatus(bucketPath);
-				assertTrue(directoryFileStatus.isDir());
-				assertEquals(0L, directoryFileStatus.getAccessTime());
-				assertTrue(directoryFileStatus.getModificationTime() > 0L);
-
-			} catch (FileNotFoundException e) {
-				fail(e.getMessage());
-			}
-
-			// Delete the bucket
-			fs.delete(bucketPath, true);
-
-			// Make sure the bucket no longer exists
-			try {
-				fs.getFileStatus(bucketPath);
-				fail("Expected FileNotFoundException for " + bucketPath.toUri());
-			} catch (FileNotFoundException e) {
-				// This is an expected exception
-			}
-
-		} catch (IOException ioe) {
-			fail(ioe.getMessage());
-		}
-	}
-
-	/**
-	 * Creates and reads the a larger test file in S3. The test file is generated according to a specific pattern.
-	 * During the read phase the incoming data stream is also checked against this pattern.
-	 */
-	@Test
-	public void createAndReadLargeFileTest() {
-
-		try {
-			createAndReadFileTest(LARGE_FILE_SIZE);
-		} catch (IOException ioe) {
-			fail(ioe.getMessage());
-		}
-	}
-
-	/**
-	 * Creates and reads the a small test file in S3. The test file is generated according to a specific pattern.
-	 * During the read phase the incoming data stream is also checked against this pattern.
-	 */
-	@Test
-	public void createAndReadSmallFileTest() {
-
-		try {
-			createAndReadFileTest(SMALL_FILE_SIZE);
-		} catch (IOException ioe) {
-			fail(ioe.getMessage());
-		}
-	}
-
-	/**
-	 * The tests checks the mapping of the file system directory structure to the underlying bucket/object model of
-	 * Amazon S3.
-	 */
-	@Test
-	public void multiLevelDirectoryTest() {
-
-		if (!testActivated()) {
-			return;
-		}
-
-		final String dirName = getRandomName();
-		final String subdirName = getRandomName();
-		final String subsubdirName = getRandomName();
-		final String fileName = getRandomName();
-		final Path dir = new Path(S3_BASE_URI + dirName + Path.SEPARATOR);
-		final Path subdir = new Path(S3_BASE_URI + dirName + Path.SEPARATOR + subdirName + Path.SEPARATOR);
-		final Path subsubdir = new Path(S3_BASE_URI + dirName + Path.SEPARATOR + subdirName + Path.SEPARATOR
-			+ subsubdirName + Path.SEPARATOR);
-		final Path file = new Path(S3_BASE_URI + dirName + Path.SEPARATOR + subdirName + Path.SEPARATOR + fileName);
-
-		try {
-
-			final FileSystem fs = dir.getFileSystem();
-
-			fs.mkdirs(subsubdir);
-
-			final OutputStream os = fs.create(file, true);
-			generateTestData(os, SMALL_FILE_SIZE);
-			os.close();
-
-			// On this directory levels there should only be one subdirectory
-			FileStatus[] list = fs.listStatus(dir);
-			int numberOfDirs = 0;
-			int numberOfFiles = 0;
-			for (final FileStatus entry : list) {
-
-				if (entry.isDir()) {
-					++numberOfDirs;
-					assertEquals(subdir, entry.getPath());
-				} else {
-					fail(entry.getPath() + " is a file which must not appear on this directory level");
-				}
-			}
-
-			assertEquals(1, numberOfDirs);
-			assertEquals(0, numberOfFiles);
-
-			list = fs.listStatus(subdir);
-			numberOfDirs = 0;
-
-			for (final FileStatus entry : list) {
-				if (entry.isDir()) {
-					assertEquals(subsubdir, entry.getPath());
-					++numberOfDirs;
-				} else {
-					assertEquals(file, entry.getPath());
-					++numberOfFiles;
-				}
-			}
-
-			assertEquals(1, numberOfDirs);
-			assertEquals(1, numberOfFiles);
-
-			fs.delete(dir, true);
-
-		} catch (IOException ioe) {
-			fail(ioe.getMessage());
-		}
-	}
-
-	/**
-	 * This test checks the S3 implementation of the file system method to retrieve the block locations of a file.
-	 */
-	@Test
-	public void blockLocationTest() {
-
-		if (!testActivated()) {
-			return;
-		}
-
-		final String dirName = getRandomName();
-		final String fileName = getRandomName();
-		final Path dir = new Path(S3_BASE_URI + dirName + Path.SEPARATOR);
-		final Path file = new Path(S3_BASE_URI + dirName + Path.SEPARATOR + fileName);
-
-		try {
-
-			final FileSystem fs = dir.getFileSystem();
-
-			fs.mkdirs(dir);
-
-			final OutputStream os = fs.create(file, true);
-			generateTestData(os, SMALL_FILE_SIZE);
-			os.close();
-
-			final FileStatus fileStatus = fs.getFileStatus(file);
-			assertNotNull(fileStatus);
-
-			BlockLocation[] blockLocations = fs.getFileBlockLocations(fileStatus, 0, SMALL_FILE_SIZE + 1);
-			assertNull(blockLocations);
-
-			blockLocations = fs.getFileBlockLocations(fileStatus, 0, SMALL_FILE_SIZE);
-			assertEquals(1, blockLocations.length);
-
-			final BlockLocation bl = blockLocations[0];
-			assertNotNull(bl.getHosts());
-			assertEquals(1, bl.getHosts().length);
-			assertEquals(SMALL_FILE_SIZE, bl.getLength());
-			assertEquals(0, bl.getOffset());
-			final URI s3Uri = fs.getUri();
-			assertNotNull(s3Uri);
-			assertEquals(s3Uri.getHost(), bl.getHosts()[0]);
-
-			fs.delete(dir, true);
-
-		} catch (IOException ioe) {
-			fail(ioe.getMessage());
-		}
-	}
-
-	/**
-	 * Creates and reads a file with the given size in S3. The test file is generated according to a specific pattern.
-	 * During the read phase the incoming data stream is also checked against this pattern.
-	 * 
-	 * @param fileSize
-	 *        the size of the file to be generated in bytes
-	 * @throws IOException
-	 *         thrown if an I/O error occurs while writing or reading the test file
-	 */
-	private void createAndReadFileTest(final int fileSize) throws IOException {
-
-		if (!testActivated()) {
-			return;
-		}
-
-		final String bucketName = getRandomName();
-		final String objectName = getRandomName();
-		final Path bucketPath = new Path(S3_BASE_URI + bucketName + Path.SEPARATOR);
-		final Path objectPath = new Path(S3_BASE_URI + bucketName + Path.SEPARATOR + objectName);
-
-		FileSystem fs = bucketPath.getFileSystem();
-
-		// Create test bucket
-		fs.mkdirs(bucketPath);
-
-		// Write test file to S3
-		final FSDataOutputStream outputStream = fs.create(objectPath, false);
-		generateTestData(outputStream, fileSize);
-		outputStream.close();
-
-		// Now read the same file back from S3
-		final FSDataInputStream inputStream = fs.open(objectPath);
-		testReceivedData(inputStream, fileSize);
-		inputStream.close();
-
-		// Delete test bucket
-		fs.delete(bucketPath, true);
-	}
-
-	/**
-	 * Receives test data from the given input stream and checks the size of the data as well as the pattern inside the
-	 * received data.
-	 * 
-	 * @param inputStream
-	 *        the input stream to read the test data from
-	 * @param expectedSize
-	 *        the expected size of the data to be read from the input stream in bytes
-	 * @throws IOException
-	 *         thrown if an error occurs while reading the data
-	 */
-	private void testReceivedData(final InputStream inputStream, final int expectedSize) throws IOException {
-
-		final byte[] testBuffer = new byte[TEST_BUFFER_SIZE];
-
-		int totalBytesRead = 0;
-		int nextExpectedNumber = 0;
-		while (true) {
-
-			final int bytesRead = inputStream.read(testBuffer);
-			if (bytesRead < 0) {
-				break;
-			}
-
-			totalBytesRead += bytesRead;
-
-			for (int i = 0; i < bytesRead; ++i) {
-				if (testBuffer[i] != nextExpectedNumber) {
-					throw new IOException("Read number " + testBuffer[i] + " but expected " + nextExpectedNumber);
-				}
-
-				++nextExpectedNumber;
-
-				if (nextExpectedNumber == MODULUS) {
-					nextExpectedNumber = 0;
-				}
-			}
-		}
-
-		if (totalBytesRead != expectedSize) {
-			throw new IOException("Expected to read " + expectedSize + " bytes but only received " + totalBytesRead);
-		}
-	}
-
-	/**
-	 * Generates test data of the given size according to some specific pattern and writes it to the provided output
-	 * stream.
-	 * 
-	 * @param outputStream
-	 *        the output stream to write the data to
-	 * @param size
-	 *        the size of the test data to be generated in bytes
-	 * @throws IOException
-	 *         thrown if an error occurs while writing the data
-	 */
-	private void generateTestData(final OutputStream outputStream, final int size) throws IOException {
-
-		final byte[] testBuffer = new byte[TEST_BUFFER_SIZE];
-		for (int i = 0; i < testBuffer.length; ++i) {
-			testBuffer[i] = (byte) (i % MODULUS);
-		}
-
-		int bytesWritten = 0;
-		while (bytesWritten < size) {
-
-			final int diff = size - bytesWritten;
-			if (diff < testBuffer.length) {
-				outputStream.write(testBuffer, 0, diff);
-				bytesWritten += diff;
-			} else {
-				outputStream.write(testBuffer);
-				bytesWritten += testBuffer.length;
-			}
-		}
-	}
-
-	/**
-	 * Generates a random name.
-	 * 
-	 * @return a random name
-	 */
-	private String getRandomName() {
-
-		final StringBuilder stringBuilder = new StringBuilder();
-		for (int i = 0; i < NAME_LENGTH; ++i) {
-			final char c = ALPHABET[(int) (Math.random() * (double) ALPHABET.length)];
-			stringBuilder.append(c);
-		}
-
-		return stringBuilder.toString();
-	}
-
-	/**
-	 * Checks whether the AWS access key and the AWS secret keys have been successfully loaded from the configuration
-	 * and whether the S3 tests shall be performed.
-	 * 
-	 * @return <code>true</code> if the tests shall be performed, <code>false</code> if the tests shall be skipped
-	 *         because at least one AWS key is missing
-	 */
-	private boolean testActivated() {
-
-		final String accessKey = GlobalConfiguration.getString(S3FileSystem.S3_ACCESS_KEY_KEY, null);
-		final String secretKey = GlobalConfiguration.getString(S3FileSystem.S3_SECRET_KEY_KEY, null);
-
-		if (accessKey != null && secretKey != null) {
-			return true;
-		}
-
-		return false;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/AbstractIDTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/AbstractIDTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/AbstractIDTest.java
deleted file mode 100644
index 2130f5b..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/AbstractIDTest.java
+++ /dev/null
@@ -1,62 +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.io;
-
-import static org.junit.Assert.assertEquals;
-
-import java.io.IOException;
-
-import org.junit.Test;
-
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.util.CommonTestUtils;
-
-/**
- * This class contains tests for the {@link AbstractID} class.
- * 
- */
-public class AbstractIDTest {
-
-	/**
-	 * Tests the setID method of an abstract ID.
-	 */
-	@Test
-	public void testSetID() {
-
-		final ChannelID id1 = new ChannelID();
-		final ChannelID id2 = new ChannelID();
-		id1.setID(id2);
-
-		assertEquals(id1.hashCode(), id2.hashCode());
-		assertEquals(id1, id2);
-	}
-
-	/**
-	 * Tests the serialization/deserialization of an abstract ID.
-	 */
-	@Test
-	public void testSerialization() {
-
-		final ChannelID origID = new ChannelID();
-		try {
-			final ChannelID copyID = (ChannelID) CommonTestUtils.createCopy(origID);
-
-			assertEquals(origID.hashCode(), copyID.hashCode());
-			assertEquals(origID, copyID);
-
-		} catch (IOException e) {
-			e.printStackTrace();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/DefaultChannelSelectorTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/DefaultChannelSelectorTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/DefaultChannelSelectorTest.java
deleted file mode 100644
index 387e49f..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/DefaultChannelSelectorTest.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.io;
-
-import static org.junit.Assert.assertEquals;
-
-import org.junit.Test;
-
-import eu.stratosphere.core.io.StringRecord;
-
-/**
- * This class checks the functionality of the {@link DefaultChannelSelector} class.
- * 
- */
-public class DefaultChannelSelectorTest {
-
-	/**
-	 * This test checks the channel selection
-	 */
-	@Test
-	public void channelSelect() {
-
-		final StringRecord dummyRecord = new StringRecord("abc");
-		final DefaultChannelSelector<StringRecord> selector = new DefaultChannelSelector<StringRecord>();
-		// Test with two channels
-		final int numberOfOutputChannels = 2;
-		int[] selectedChannels = selector.selectChannels(dummyRecord, numberOfOutputChannels);
-		assertEquals(1, selectedChannels.length);
-		assertEquals(1, selectedChannels[0]);
-		selectedChannels = selector.selectChannels(dummyRecord, numberOfOutputChannels);
-		assertEquals(1, selectedChannels.length);
-		assertEquals(0, selectedChannels[0]);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/BufferTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/BufferTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/BufferTest.java
deleted file mode 100644
index 13cc30e..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/BufferTest.java
+++ /dev/null
@@ -1,192 +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.io.channels;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.nio.channels.FileChannel;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.types.IntegerRecord;
-
-/**
- * This class checks the functionality of the {@link SerializationBuffer} class and the {@link DefaultDeserializer}
- * class
- * 
- */
-public class BufferTest
-{
-	private File file = new File("./tmp");
-
-	private FileInputStream fileinstream;
-
-	private FileOutputStream filestream;
-
-	private FileChannel writeable;
-
-	private FileChannel readable;
-
-	/**
-	 * Set up files and stream for testing
-	 * 
-	 * @throws IOException
-	 */
-	@Before
-	public void before() throws IOException {
-		file.createNewFile();
-		filestream = new FileOutputStream(file);
-		fileinstream = new FileInputStream(file);
-		writeable = filestream.getChannel();
-		readable = fileinstream.getChannel();
-
-	}
-
-	/**
-	 * clean up. Remove file close streams and channels
-	 * 
-	 * @throws IOException
-	 */
-	@After
-	public void after() throws IOException {
-		fileinstream.close();
-		writeable.close();
-		readable.close();
-		filestream.close();
-		file.delete();
-	}
-
-	/**
-	 * Tests serialization and deserialization of an {@link IntegerRecord}
-	 */
-	@Test
-	public void testIntSerialize()
-	{
-		final SerializationBuffer<IntegerRecord> intSerializationBuffer = new SerializationBuffer<IntegerRecord>();
-		final int NUM = 0xab627ef;
-		
-		IntegerRecord intRecord = new IntegerRecord(NUM);
-		// Serialize a record.
-		try {
-			intSerializationBuffer.serialize(intRecord);
-		} catch (IOException e) {
-			e.printStackTrace();
-			fail();
-		}
-		// Last record is still in buffer, serializing another should throw IOException
-		try {
-			intSerializationBuffer.serialize(intRecord);
-			fail();
-		} catch (IOException e) {
-		}
-		
-		// Read from buffer (written in file)
-		try {
-			intSerializationBuffer.read(writeable);
-		} catch (IOException e) {
-			e.printStackTrace();
-			fail();
-		}
-		// Now a new Record can be serialized
-		try {
-			intSerializationBuffer.serialize(intRecord);
-		} catch (IOException e) {
-			e.printStackTrace();
-			fail();
-		}
-
-		DefaultDeserializer<IntegerRecord> intDeserialitionBuffer = new DefaultDeserializer<IntegerRecord>(IntegerRecord.class, true);
-		IntegerRecord record = new IntegerRecord();
-		// Deserialize a Record
-		try {
-			record = intDeserialitionBuffer.readData(record, readable);
-		} catch (IOException e) {
-			e.printStackTrace();
-			fail();
-		}
-		// Check it contains the right value
-		assertEquals(NUM, record.getValue());
-		// File empty, another read should throw IOException
-		try {
-			record = intDeserialitionBuffer.readData(record, readable);
-			fail();
-		} catch (IOException e) {
-		}
-
-	}
-
-	/**
-	 * Tests serialization and deserialization of an {@link StringRecord}
-	 */
-	@Test
-	public void testStringSerialize()
-	{
-		final SerializationBuffer<StringRecord> stringSerializationBuffer = new SerializationBuffer<StringRecord>();
-		final String str = "abc";
-		
-		StringRecord stringrecord = new StringRecord(str);
-		
-		// Serialize a record.
-		try {
-			stringSerializationBuffer.serialize(stringrecord);
-		} catch (IOException e) {
-			e.printStackTrace();
-			fail();
-		}
-		// Read from buffer (write in file)
-		try {
-			stringSerializationBuffer.read(writeable);
-		} catch (IOException e) {
-			e.printStackTrace();
-			fail();
-		}
-		// Serialize next Record.
-		// Read from buffer (write in file)
-		final String str2 = "abcdef";
-		stringrecord = new StringRecord(str2);
-		try {
-			stringSerializationBuffer.serialize(stringrecord);
-			stringSerializationBuffer.read(writeable);
-		} catch (IOException e) {
-			e.printStackTrace();
-			fail();
-		}
-
-		final DefaultDeserializer<StringRecord> stringDeserialitionBuffer = new DefaultDeserializer<StringRecord>(StringRecord.class, true);
-		StringRecord record = new StringRecord();
-		// Deserialize and check record are correct
-		try {
-			record = stringDeserialitionBuffer.readData(record, readable);
-		} catch (IOException e) {
-			e.printStackTrace();
-			fail();
-		}
-		assertEquals(str, record.toString());
-		try {
-			record = stringDeserialitionBuffer.readData(record, readable);
-		} catch (IOException e) {
-			e.printStackTrace();
-			fail();
-		}
-		assertEquals(str2, record.toString());
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/MemoryBufferTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/MemoryBufferTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/MemoryBufferTest.java
deleted file mode 100644
index d8ffc27..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/MemoryBufferTest.java
+++ /dev/null
@@ -1,155 +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.io.channels;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Queue;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import eu.stratosphere.core.memory.MemorySegment;
-import eu.stratosphere.nephele.util.BufferPoolConnector;
-
-
-public class MemoryBufferTest {
-
-	private MemoryBufferPoolConnector bufferPoolConnector;
-	private Queue<MemorySegment> bufferPool;
-	
-	private final static int INT_COUNT = 512;
-	private final static int INT_SIZE = Integer.SIZE / Byte.SIZE;
-
-	@Before
-	public void setUp() throws Exception {
-		bufferPool = new LinkedBlockingQueue<MemorySegment>();
-		bufferPoolConnector = new BufferPoolConnector(bufferPool);
-	}
-
-	@After
-	public void tearDown() throws Exception {
-	}
-
-	@Test
-	public void readToSmallByteBuffer() throws IOException {
-		MemoryBuffer buf = new MemoryBuffer(INT_COUNT*INT_SIZE, new MemorySegment(new byte[INT_COUNT*INT_SIZE]), bufferPoolConnector);
-		fillBuffer(buf);
-		
-		ByteBuffer target = ByteBuffer.allocate(INT_SIZE);
-		ByteBuffer largeTarget = ByteBuffer.allocate(INT_COUNT*INT_SIZE);
-		int i = 0;
-		while(buf.hasRemaining()) {
-			buf.read(target);
-			target.rewind();
-			largeTarget.put(target);
-			target.rewind();
-			if( i++ >= INT_COUNT) {
-				fail("There were too many elements in the buffer");
-			}
-		}
-		assertEquals(-1, buf.read(target));
-		
-		target.rewind();
-		validateByteBuffer(largeTarget);
-	}
-		
-	
-	/**
-	 * CopyToBuffer uses system.arraycopy()
-	 * 
-	 * @throws IOException
-	 */
-	@Test
-	public void copyToBufferTest() throws IOException {
-
-		MemoryBuffer buf = new MemoryBuffer(INT_COUNT*INT_SIZE, new MemorySegment(new byte[INT_COUNT*INT_SIZE]), bufferPoolConnector);
-		fillBuffer(buf);
-		
-		
-		// the target buffer is larger to check if the limit is set appropriately
-		MemoryBuffer destination = new MemoryBuffer(INT_COUNT*INT_SIZE*2, 
-					new MemorySegment(new byte[INT_COUNT*INT_SIZE*2]), 
-					bufferPoolConnector);
-		assertEquals(INT_COUNT*INT_SIZE*2, destination.limit());
-		// copy buf contents to double sized MemBuffer
-		buf.copyToBuffer(destination);
-		assertEquals(INT_COUNT*INT_SIZE, destination.limit());
-		
-		// copy contents of destination to byteBuffer
-		ByteBuffer test = ByteBuffer.allocate(INT_COUNT*INT_SIZE);
-		int written = destination.read(test);
-		assertEquals(INT_COUNT*INT_SIZE, written);
-		// validate byteBuffer contents
-		validateByteBuffer(test);
-		
-		destination.position(written);
-		destination.limit(destination.getTotalSize());
-		// allocate another byte buffer to write the rest of destination into a byteBuffer
-		ByteBuffer testRemainder = ByteBuffer.allocate(INT_COUNT*INT_SIZE);
-		written = destination.read(testRemainder);
-		assertEquals(INT_COUNT*INT_SIZE, written);
-		expectAllNullByteBuffer(testRemainder);
-		
-		buf.close(); // make eclipse happy
-	}
-	
-	@Test
-	public void testDuplicate() throws Exception {
-		MemoryBuffer buf = new MemoryBuffer(INT_COUNT*INT_SIZE, new MemorySegment(new byte[INT_COUNT*INT_SIZE]), bufferPoolConnector);
-		MemoryBuffer buf2 = buf.duplicate();
-		
-		buf2.close();
-		buf.close();
-	}
-
-	private void fillBuffer(Buffer buf) throws IOException {
-		ByteBuffer src = ByteBuffer.allocate(INT_SIZE);
-		// write some data into buf:
-		for(int i = 0; i < INT_COUNT; ++i) {
-			src.putInt(0,i);
-			src.rewind();
-			buf.write(src);
-		}
-		buf.flip();
-	}
-	
-	
-	/**
-	 * Validates if the ByteBuffer contains the what fillMemoryBuffer has written!
-	 * 
-	 * @param target
-	 */
-	private void validateByteBuffer(ByteBuffer target) {
-		ByteBuffer ref = ByteBuffer.allocate(INT_SIZE);
-		
-		for(int i = 0; i < INT_SIZE*INT_COUNT; ++i) {
-			ref.putInt(0,i / INT_SIZE);
-			assertEquals("Byte at position "+i+" is different", ref.get(i%INT_SIZE), target.get(i));
-		}
-	}
-	
-	private void expectAllNullByteBuffer(ByteBuffer target) {
-		ByteBuffer ref = ByteBuffer.allocate(INT_SIZE);
-		ref.putInt(0,0);
-		for(int i = 0; i < INT_COUNT; ++i) {
-			assertEquals("Byte at position "+i+" is different", ref.getInt(0), target.getInt(i));
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/BooleanType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/BooleanType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/BooleanType.java
deleted file mode 100644
index 749a702..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/BooleanType.java
+++ /dev/null
@@ -1,77 +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.io.channels.serialization;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Random;
-
-/**
- */
-public class BooleanType implements SerializationTestType
-{
-	private boolean value;
-	
-
-	public BooleanType()
-	{
-		this.value = false;
-	}
-	
-	private BooleanType(boolean value)
-	{
-		this.value = value;
-	}
-	
-
-	@Override
-	public BooleanType getRandom(Random rnd)
-	{
-		return new BooleanType(rnd.nextBoolean());
-	}
-	
-
-	@Override
-	public void write(DataOutput out) throws IOException
-	{
-		out.writeBoolean(this.value);
-	}
-
-
-	@Override
-	public void read(DataInput in) throws IOException
-	{
-		this.value = in.readBoolean();
-	}
-
-
-	@Override
-	public int hashCode()
-	{
-		return this.value ? 1 : 0;
-	}
-
-
-	@Override
-	public boolean equals(Object obj)
-	{
-		if (obj instanceof BooleanType) {
-			BooleanType other = (BooleanType) obj;
-			return this.value == other.value;
-		} else {
-			return false;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ByteArrayType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ByteArrayType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ByteArrayType.java
deleted file mode 100644
index 9c5bdca..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ByteArrayType.java
+++ /dev/null
@@ -1,88 +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.io.channels.serialization;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Random;
-
-/**
- */
-public class ByteArrayType implements SerializationTestType
-{
-	private static final int MAX_LEN = 512 * 15;
-	
-	private byte[] data;
-	
-
-	public ByteArrayType()
-	{
-		this.data = new byte[0];
-	}
-	
-	private ByteArrayType(byte[] data)
-	{
-		this.data = data;
-	}
-	
-
-	@Override
-	public ByteArrayType getRandom(Random rnd)
-	{
-		final int len = rnd.nextInt(MAX_LEN) + 1;
-		final byte[] data = new byte[len];
-		rnd.nextBytes(data);
-		return new ByteArrayType(data);
-	}
-	
-	
-	
-
-	@Override
-	public void write(DataOutput out) throws IOException
-	{
-		out.writeInt(this.data.length);
-		out.write(this.data);
-	}
-
-
-	@Override
-	public void read(DataInput in) throws IOException
-	{
-		final int len = in.readInt();
-		this.data = new byte[len];
-		in.readFully(this.data);
-	}
-
-
-	@Override
-	public int hashCode()
-	{
-		return Arrays.hashCode(this.data);
-	}
-
-
-	@Override
-	public boolean equals(Object obj)
-	{
-		if (obj instanceof ByteArrayType) {
-			ByteArrayType other = (ByteArrayType) obj;
-			return Arrays.equals(this.data, other.data);
-		} else {
-			return false;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ByteSubArrayType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ByteSubArrayType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ByteSubArrayType.java
deleted file mode 100644
index 719bd0d..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ByteSubArrayType.java
+++ /dev/null
@@ -1,98 +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.io.channels.serialization;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Random;
-
-/**
- */
-public class ByteSubArrayType implements SerializationTestType
-{
-	private static final int MAX_LEN = 512;
-
-	private final byte[] data;
-	private int len;
-	
-	public ByteSubArrayType()
-	{
-		this.data = new byte[MAX_LEN];
-		this.len = 0;
-	}
-	
-
-	@Override
-	public ByteSubArrayType getRandom(Random rnd)
-	{
-		final int len = rnd.nextInt(MAX_LEN) + 1;
-		final ByteSubArrayType t = new ByteSubArrayType();
-		t.len = len;
-		
-		final byte[] data = t.data;
-		for (int i = 0; i < len; i++) {
-			data[i] = (byte) rnd.nextInt(256);
-		}
-		
-		return t;
-	}
-	
-
-	@Override
-	public void write(DataOutput out) throws IOException
-	{
-		out.writeInt(this.len);
-		out.write(this.data, 0, this.len);
-	}
-
-
-	@Override
-	public void read(DataInput in) throws IOException
-	{
-		this.len = in.readInt();
-		in.readFully(this.data, 0, this.len);
-	}
-	
-
-	@Override
-	public int hashCode()
-	{
-		final byte[] copy = new byte[this.len];
-		System.arraycopy(this.data, 0, copy, 0, this.len);
-		return Arrays.hashCode(copy);
-	}
-
-
-	@Override
-	public boolean equals(Object obj)
-	{
-		if (obj instanceof ByteSubArrayType) {
-			ByteSubArrayType other = (ByteSubArrayType) obj;
-			if (this.len == other.len) {
-				for (int i = 0; i < this.len; i++) {
-					if (this.data[i] != other.data[i]) {
-						return false;
-					}
-				}
-				return true;
-			} else {
-				return false;
-			}
-		} else {
-			return false;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ByteType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ByteType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ByteType.java
deleted file mode 100644
index 141f7b3..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ByteType.java
+++ /dev/null
@@ -1,77 +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.io.channels.serialization;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Random;
-
-/**
- */
-public class ByteType implements SerializationTestType
-{
-	private byte value;
-	
-
-	public ByteType()
-	{
-		this.value = (byte) 0;
-	}
-	
-	private ByteType(byte value)
-	{
-		this.value = value;
-	}
-	
-
-	@Override
-	public ByteType getRandom(Random rnd)
-	{
-		return new ByteType((byte) rnd.nextInt(256));
-	}
-	
-
-	@Override
-	public void write(DataOutput out) throws IOException
-	{
-		out.writeByte(this.value);
-	}
-
-
-	@Override
-	public void read(DataInput in) throws IOException
-	{
-		this.value = in.readByte();
-	}
-
-
-	@Override
-	public int hashCode()
-	{
-		return this.value;
-	}
-
-
-	@Override
-	public boolean equals(Object obj)
-	{
-		if (obj instanceof ByteType) {
-			ByteType other = (ByteType) obj;
-			return this.value == other.value;
-		} else {
-			return false;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/CharType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/CharType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/CharType.java
deleted file mode 100644
index e51aeb8..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/CharType.java
+++ /dev/null
@@ -1,78 +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.io.channels.serialization;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Random;
-
-/**
- */
-public class CharType implements SerializationTestType
-{
-	private char value;
-	
-
-	public CharType()
-	{
-		this.value = 0;
-	}
-	
-	private CharType(char value)
-	{
-		this.value = value;
-	}
-	
-
-	@Override
-	public CharType getRandom(Random rnd)
-	{
-		return new CharType((char) rnd.nextInt(10000));
-	}
-	
-
-	@Override
-	public void write(DataOutput out) throws IOException
-	{
-		out.writeChar(this.value);
-	}
-
-
-	@Override
-	public void read(DataInput in) throws IOException
-	{
-		this.value = in.readChar();
-	}
-
-
-	@Override
-	public int hashCode()
-	{
-		return this.value;
-	}
-
-
-	@Override
-	public boolean equals(Object obj)
-	{
-		if (obj instanceof CharType) {
-			CharType other = (CharType) obj;
-			return this.value == other.value;
-		} else {
-			return false;
-		}
-	}
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/DeSerializerTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/DeSerializerTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/DeSerializerTest.java
deleted file mode 100644
index 1c42277..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/DeSerializerTest.java
+++ /dev/null
@@ -1,299 +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.io.channels.serialization;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.ReadableByteChannel;
-import java.nio.channels.WritableByteChannel;
-import java.util.ArrayDeque;
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-import java.util.Random;
-
-import org.junit.Before;
-import org.junit.Test;
-
-import eu.stratosphere.nephele.io.channels.DefaultDeserializer;
-import eu.stratosphere.nephele.io.channels.SerializationBuffer;
-
-
-/**
- */
-public class DeSerializerTest
-{
-	private static final SerializationTestType[] TYPE_FACTORIES = new SerializationTestType[] {
-		new BooleanType(),
-		new ByteArrayType(),
-		new ByteSubArrayType(),
-		new ByteType(),
-		new CharType(),
-		new DoubleType(),
-		new FloatType(),
-		new IntType(),
-		new LongType(),
-		new ShortType(),
-		new UnsignedByteType(),
-		new UnsignedShortType(),
-		new UTFStringType()
-	};
-	
-	private static final long SEED = 64871654635745873L;
-	
-	private Random rnd;
-	
-	// --------------------------------------------------------------------------------------------
-	
-	@Before
-	public void setup()
-	{
-		this.rnd = new Random(SEED);
-	}
-
-	// --------------------------------------------------------------------------------------------
-	
-	@Test
-	public void testSequenceOfIntegersWithAlignedBuffers()
-	{
-		try {
-			final Random rnd = this.rnd;
-			final int NUM_INTS = 1000000;
-			
-			final Iterator<SerializationTestType> intSource = new Iterator<SerializationTestType>()
-			{
-				private final Random random = rnd;
-				private final int limit = NUM_INTS;
-				private int pos = 0;
-				
-				@Override
-				public boolean hasNext() {
-					return this.pos < this.limit;
-				}
-				@Override
-				public IntType next() {
-					if (hasNext()) {
-						this.pos++;
-						return new IntType(this.random.nextInt());
-					} else {
-						throw new NoSuchElementException();
-					}
-				}
-				@Override
-				public void remove() {
-					throw new UnsupportedOperationException();
-				}
-			};
-			
-			testSequenceOfTypes(intSource, 2048);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail("Test encountered an unexpected exception.");
-		}
-	}
-	
-	@Test
-	public void testSequenceOfIntegersWithUnalignedBuffers()
-	{
-		try {
-			final Random rnd = this.rnd;
-			final int NUM_INTS = 1000000;
-			
-			final Iterator<SerializationTestType> intSource = new Iterator<SerializationTestType>()
-			{
-				private final Random random = rnd;
-				private final int limit = NUM_INTS;
-				private int pos = 0;
-				
-				@Override
-				public boolean hasNext() {
-					return this.pos < this.limit;
-				}
-				@Override
-				public IntType next() {
-					if (hasNext()) {
-						this.pos++;
-						return new IntType(this.random.nextInt());
-					} else {
-						throw new NoSuchElementException();
-					}
-				}
-				@Override
-				public void remove() {
-					throw new UnsupportedOperationException();
-				}
-			};
-			
-			testSequenceOfTypes(intSource, 2047);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail("Test encountered an unexpected exception.");
-		}
-	}
-	
-	@Test
-	public void testRandomTypes()
-	{
-		try {
-			final Random rnd = this.rnd;
-			final int NUM_TYPES = 1000000;
-			
-			final Iterator<SerializationTestType> randomSource = new Iterator<SerializationTestType>()
-			{
-				private final Random random = rnd;
-				private final int limit = NUM_TYPES;
-				private int pos = 0;
-				
-				@Override
-				public boolean hasNext() {
-					return this.pos < this.limit;
-				}
-				@Override
-				public SerializationTestType next() {
-					if (hasNext()) {
-						this.pos++;
-						return TYPE_FACTORIES[this.random.nextInt(TYPE_FACTORIES.length)].getRandom(this.random);
-					} else {
-						throw new NoSuchElementException();
-					}
-				}
-				@Override
-				public void remove() {
-					throw new UnsupportedOperationException();
-				}
-			};
-			
-			// test with an odd buffer size to force many unaligned cases
-			testSequenceOfTypes(randomSource, 512 * 7);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail("Test encountered an unexpected exception.");
-		}
-	}
-	
-	private static final void testSequenceOfTypes(Iterator<SerializationTestType> sequence, int bufferSize) throws Exception
-	{
-		final ArrayDeque<SerializationTestType> elements = new ArrayDeque<SerializationTestType>(512);
-		
-		final PipeChannel channel = new PipeChannel(bufferSize);
-		final SerializationBuffer<SerializationTestType> serBuffer = new SerializationBuffer<SerializationTestType>();
-		final DefaultDeserializer<SerializationTestType> deserBuffer = new DefaultDeserializer<SerializationTestType>(null);
-		
-		while (sequence.hasNext()) {
-			final SerializationTestType type = sequence.next();
-			
-			// serialize the record
-			serBuffer.serialize(type);
-			elements.addLast(type);
-			
-			// write the serialized record
-			while (true) {
-				serBuffer.read(channel);
-				if (serBuffer.dataLeftFromPreviousSerialization()) {
-					// current buffer is full, we need to start de-serializing to make space
-					channel.flip();
-					
-					while (!elements.isEmpty()) {
-						final SerializationTestType reference = elements.pollFirst();
-						final SerializationTestType result = deserBuffer.readData(reference.getClass().newInstance(), channel);
-						if (result == null) {
-							// not yet complete, we need to break
-							elements.addFirst(reference);
-							break;
-						} else {
-							// validate that we deserialized correctly
-							assertEquals("The deserialized element is not equal to the serialized element.", reference, result);
-						}
-					}
-					
-					channel.clear();
-				} else {
-					break;
-				}
-			}
-		}
-		
-		// check the remaining records in the buffers...
-		channel.flip();
-		while (!elements.isEmpty()) {
-			final SerializationTestType reference = elements.pollFirst();
-			final SerializationTestType result = deserBuffer.readData(reference.getClass().newInstance(), channel);
-			
-			assertNotNull(result);
-			assertEquals("The deserialized element is not equal to the serialized element.", reference, result);
-		}
-	}
-	
-	// ============================================================================================
-	
-	private static final class PipeChannel implements WritableByteChannel, ReadableByteChannel
-	{
-		private final byte[] buffer;
-		
-		private int position;
-		private int limit;
-		
-		
-		PipeChannel(int capacity) {
-			this.buffer = new byte[capacity];
-			this.limit = capacity;
-		}
-		
-		public void flip() {
-			this.limit = this.position;
-			this.position = 0;
-		}
-		
-		public void clear() {
-			this.position = 0;
-			this.limit = this.buffer.length;
-		}
-		
-		@Override
-		public boolean isOpen() {
-			return true;
-		}
-
-		@Override
-		public void close()
-		{}
-
-
-		@Override
-		public int write(ByteBuffer src)
-		{
-			final int toGet = Math.min(this.limit - this.position, src.remaining());
-			src.get(this.buffer, this.position, toGet);
-			this.position += toGet;
-			return toGet;
-		}
-
-
-		@Override
-		public int read(ByteBuffer dst) throws IOException
-		{
-			final int toPut = Math.min(this.limit - this.position, dst.remaining());
-			dst.put(this.buffer, this.position, toPut);
-			this.position += toPut;
-			return toPut;
-		}
-		
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/DoubleType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/DoubleType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/DoubleType.java
deleted file mode 100644
index 3f3e958..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/DoubleType.java
+++ /dev/null
@@ -1,78 +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.io.channels.serialization;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Random;
-
-/**
- */
-public class DoubleType implements SerializationTestType
-{
-	private double value;
-	
-
-	public DoubleType()
-	{
-		this.value = 0;
-	}
-	
-	private DoubleType(double value)
-	{
-		this.value = value;
-	}
-	
-
-	@Override
-	public DoubleType getRandom(Random rnd)
-	{
-		return new DoubleType(rnd.nextDouble());
-	}
-	
-
-	@Override
-	public void write(DataOutput out) throws IOException
-	{
-		out.writeDouble(this.value);
-	}
-
-
-	@Override
-	public void read(DataInput in) throws IOException
-	{
-		this.value = in.readDouble();
-	}
-
-
-	@Override
-	public int hashCode()
-	{
-		final long l = Double.doubleToLongBits(this.value);
-		return (int) (l ^ l >>> 32);
-	}
-
-
-	@Override
-	public boolean equals(Object obj)
-	{
-		if (obj instanceof DoubleType) {
-			DoubleType other = (DoubleType) obj;
-			return Double.doubleToLongBits(this.value) == Double.doubleToLongBits(other.value);
-		} else {
-			return false;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/FloatType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/FloatType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/FloatType.java
deleted file mode 100644
index 7bc37a0..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/FloatType.java
+++ /dev/null
@@ -1,77 +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.io.channels.serialization;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Random;
-
-/**
- */
-public class FloatType implements SerializationTestType
-{
-	private float value;
-	
-
-	public FloatType()
-	{
-		this.value = 0;
-	}
-	
-	private FloatType(float value)
-	{
-		this.value = value;
-	}
-	
-
-	@Override
-	public FloatType getRandom(Random rnd)
-	{
-		return new FloatType(rnd.nextFloat());
-	}
-	
-
-	@Override
-	public void write(DataOutput out) throws IOException
-	{
-		out.writeFloat(this.value);
-	}
-
-
-	@Override
-	public void read(DataInput in) throws IOException
-	{
-		this.value = in.readFloat();
-	}
-
-
-	@Override
-	public int hashCode()
-	{
-		return Float.floatToIntBits(this.value);
-	}
-
-
-	@Override
-	public boolean equals(Object obj)
-	{
-		if (obj instanceof FloatType) {
-			FloatType other = (FloatType) obj;
-			return Float.floatToIntBits(this.value) == Float.floatToIntBits(other.value);
-		} else {
-			return false;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/IntType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/IntType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/IntType.java
deleted file mode 100644
index 1180884..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/IntType.java
+++ /dev/null
@@ -1,77 +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.io.channels.serialization;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Random;
-
-/**
- */
-public class IntType implements SerializationTestType
-{
-	private int value;
-	
-
-	public IntType()
-	{
-		this.value = 0;
-	}
-	
-	public IntType(int value)
-	{
-		this.value = value;
-	}
-	
-
-	@Override
-	public IntType getRandom(Random rnd)
-	{
-		return new IntType(rnd.nextInt());
-	}
-	
-
-	@Override
-	public void write(DataOutput out) throws IOException
-	{
-		out.writeInt(this.value);
-	}
-
-
-	@Override
-	public void read(DataInput in) throws IOException
-	{
-		this.value = in.readInt();
-	}
-
-
-	@Override
-	public int hashCode()
-	{
-		return this.value;
-	}
-
-
-	@Override
-	public boolean equals(Object obj)
-	{
-		if (obj instanceof IntType) {
-			IntType other = (IntType) obj;
-			return this.value == other.value;
-		} else {
-			return false;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/LongType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/LongType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/LongType.java
deleted file mode 100644
index 8c752f6..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/LongType.java
+++ /dev/null
@@ -1,77 +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.io.channels.serialization;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Random;
-
-/**
- */
-public class LongType implements SerializationTestType
-{
-	private long value;
-	
-
-	public LongType()
-	{
-		this.value = 0;
-	}
-	
-	private LongType(long value)
-	{
-		this.value = value;
-	}
-	
-
-	@Override
-	public LongType getRandom(Random rnd)
-	{
-		return new LongType(rnd.nextLong());
-	}
-	
-
-	@Override
-	public void write(DataOutput out) throws IOException
-	{
-		out.writeLong(this.value);
-	}
-
-
-	@Override
-	public void read(DataInput in) throws IOException
-	{
-		this.value = in.readLong();
-	}
-
-
-	@Override
-	public int hashCode()
-	{
-		return (int) (this.value ^ this.value >>> 32);
-	}
-
-
-	@Override
-	public boolean equals(Object obj)
-	{
-		if (obj instanceof LongType) {
-			LongType other = (LongType) obj;
-			return this.value == other.value;
-		} else {
-			return false;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/SerializationTestType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/SerializationTestType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/SerializationTestType.java
deleted file mode 100644
index 77630b0..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/SerializationTestType.java
+++ /dev/null
@@ -1,52 +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.io.channels.serialization;
-
-import java.util.Random;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-
-/**
- */
-public interface SerializationTestType extends IOReadableWritable
-{
-	public SerializationTestType getRandom(Random rnd);
-
-	// public static final String REST1_PATH = "src/test/resources/clustering/rest1.json";
-	// public static final String SAMPLE1_PATH = "src/test/resources/clustering/sample1.json";
-	//
-	// public static IJsonNode asJson(final String... values) {
-	// return new Point(String.valueOf(pointCount++), values).write(null);
-	// }
-	//
-	// public static List<IJsonNode> loadPoints(final String filePath)
-	// throws IOException {
-	// BufferedReader reader = null;
-	// try {
-	// final File pointFile = new File(filePath);
-	// reader = new BufferedReader(new FileReader(pointFile));
-	// final JsonParser parser = new JsonParser(reader);
-	// final List<IJsonNode> pointNodes = new LinkedList<IJsonNode>();
-	// while (!parser.checkEnd())
-	// pointNodes.add(parser.readValueAsTree());
-	// return pointNodes;
-	// } finally {
-	// try {
-	// reader.close();
-	// } catch (final Exception e) {
-	// e.printStackTrace();
-	// }
-	// }
-	// }
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ShortType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ShortType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ShortType.java
deleted file mode 100644
index 09d0009..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ShortType.java
+++ /dev/null
@@ -1,77 +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.io.channels.serialization;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Random;
-
-/**
- */
-public class ShortType implements SerializationTestType
-{
-	private short value;
-	
-
-	public ShortType()
-	{
-		this.value = (short) 0;
-	}
-	
-	private ShortType(short value)
-	{
-		this.value = value;
-	}
-	
-
-	@Override
-	public ShortType getRandom(Random rnd)
-	{
-		return new ShortType((short) rnd.nextInt(65536));
-	}
-	
-
-	@Override
-	public void write(DataOutput out) throws IOException
-	{
-		out.writeShort(this.value);
-	}
-
-
-	@Override
-	public void read(DataInput in) throws IOException
-	{
-		this.value = in.readShort();
-	}
-
-
-	@Override
-	public int hashCode()
-	{
-		return this.value;
-	}
-
-
-	@Override
-	public boolean equals(Object obj)
-	{
-		if (obj instanceof ShortType) {
-			ShortType other = (ShortType) obj;
-			return this.value == other.value;
-		} else {
-			return false;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/UTFStringType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/UTFStringType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/UTFStringType.java
deleted file mode 100644
index c4481fa..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/UTFStringType.java
+++ /dev/null
@@ -1,86 +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.io.channels.serialization;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Random;
-
-/**
- */
-public class UTFStringType implements SerializationTestType
-{
-	private static final int MAX_LEN = 1500;
-	
-	private String value;
-	
-
-	public UTFStringType()
-	{
-		this.value = "";
-	}
-	
-	private UTFStringType(String value)
-	{
-		this.value = value;
-	}
-	
-
-	@Override
-	public UTFStringType getRandom(Random rnd)
-	{
-		final StringBuilder bld = new StringBuilder();
-		final int len = rnd.nextInt(MAX_LEN + 1);
-		
-		for (int i = 0; i < len; i++) {
-			bld.append((char) rnd.nextInt(Character.MAX_VALUE));
-		}
-		
-		return new UTFStringType(bld.toString());
-	}
-	
-
-	@Override
-	public void write(DataOutput out) throws IOException
-	{
-		out.writeUTF(this.value);
-	}
-
-
-	@Override
-	public void read(DataInput in) throws IOException
-	{
-		this.value = in.readUTF();
-	}
-
-
-	@Override
-	public int hashCode()
-	{
-		return this.value.hashCode();
-	}
-
-
-	@Override
-	public boolean equals(Object obj)
-	{
-		if (obj instanceof UTFStringType) {
-			UTFStringType other = (UTFStringType) obj;
-			return this.value.equals(other.value);
-		} else {
-			return false;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/UnsignedByteType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/UnsignedByteType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/UnsignedByteType.java
deleted file mode 100644
index 4edcd05..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/UnsignedByteType.java
+++ /dev/null
@@ -1,77 +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.io.channels.serialization;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Random;
-
-/**
- */
-public class UnsignedByteType implements SerializationTestType
-{
-	private int value;
-	
-
-	public UnsignedByteType()
-	{
-		this.value = 0;
-	}
-	
-	private UnsignedByteType(int value)
-	{
-		this.value = value;
-	}
-	
-
-	@Override
-	public UnsignedByteType getRandom(Random rnd)
-	{
-		return new UnsignedByteType(rnd.nextInt(128) + 128);
-	}
-	
-
-	@Override
-	public void write(DataOutput out) throws IOException
-	{
-		out.writeByte(this.value);
-	}
-
-
-	@Override
-	public void read(DataInput in) throws IOException
-	{
-		this.value = in.readUnsignedByte();
-	}
-
-
-	@Override
-	public int hashCode()
-	{
-		return this.value;
-	}
-
-
-	@Override
-	public boolean equals(Object obj)
-	{
-		if (obj instanceof UnsignedByteType) {
-			UnsignedByteType other = (UnsignedByteType) obj;
-			return this.value == other.value;
-		} else {
-			return false;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/UnsignedShortType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/UnsignedShortType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/UnsignedShortType.java
deleted file mode 100644
index 898b343..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/UnsignedShortType.java
+++ /dev/null
@@ -1,77 +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.io.channels.serialization;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Random;
-
-/**
- */
-public class UnsignedShortType implements SerializationTestType
-{
-	private int value;
-	
-
-	public UnsignedShortType()
-	{
-		this.value = 0;
-	}
-	
-	private UnsignedShortType(int value)
-	{
-		this.value = value;
-	}
-	
-
-	@Override
-	public UnsignedShortType getRandom(Random rnd)
-	{
-		return new UnsignedShortType(rnd.nextInt(32768) + 32768);
-	}
-	
-
-	@Override
-	public void write(DataOutput out) throws IOException
-	{
-		out.writeShort(this.value);
-	}
-
-
-	@Override
-	public void read(DataInput in) throws IOException
-	{
-		this.value = in.readUnsignedShort();
-	}
-
-
-	@Override
-	public int hashCode()
-	{
-		return this.value;
-	}
-
-
-	@Override
-	public boolean equals(Object obj)
-	{
-		if (obj instanceof UnsignedShortType) {
-			UnsignedShortType other = (UnsignedShortType) obj;
-			return this.value == other.value;
-		} else {
-			return false;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/library/FileLineReadWriteTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/library/FileLineReadWriteTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/library/FileLineReadWriteTest.java
deleted file mode 100644
index f88d7bf..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/library/FileLineReadWriteTest.java
+++ /dev/null
@@ -1,134 +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.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 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.nephele.io.RecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
-import eu.stratosphere.nephele.template.InputSplitProvider;
-
-/**
- * This class checks the functionality of the {@link FileLineReader} and the {@link 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/2db78a8d/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
index 6d29521..1e2be47 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/DoubleSourceTask.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/DoubleSourceTask.java
@@ -19,7 +19,7 @@ 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.io.RecordWriter;
+import eu.stratosphere.runtime.io.api.RecordWriter;
 import eu.stratosphere.nephele.template.AbstractFileInputTask;
 import eu.stratosphere.runtime.fs.LineReader;
 
@@ -31,6 +31,8 @@ public class DoubleSourceTask extends AbstractFileInputTask {
 
 	@Override
 	public void invoke() throws Exception {
+		this.output1.initializeSerializers();
+		this.output2.initializeSerializers();
 
 		final Iterator<FileInputSplit> splitIterator = getFileInputSplits();
 
@@ -65,12 +67,15 @@ public class DoubleSourceTask extends AbstractFileInputTask {
 			// Close the stream;
 			lineReader.close();
 		}
+
+		this.output1.flush();
+		this.output2.flush();
 	}
 
 	@Override
 	public void registerInputOutput() {
-		this.output1 = new RecordWriter<StringRecord>(this, StringRecord.class);
-		this.output2 = new RecordWriter<StringRecord>(this, StringRecord.class);
+		this.output1 = new RecordWriter<StringRecord>(this);
+		this.output2 = new RecordWriter<StringRecord>(this);
 	}
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 cea9dd2..f0ca435 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
@@ -14,8 +14,8 @@
 package eu.stratosphere.nephele.jobmanager;
 
 import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.RecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
+import eu.stratosphere.runtime.io.api.RecordReader;
+import eu.stratosphere.runtime.io.api.RecordWriter;
 import eu.stratosphere.nephele.template.AbstractTask;
 
 public class DoubleTargetTask extends AbstractTask {
@@ -29,6 +29,8 @@ public class DoubleTargetTask extends AbstractTask {
 	@Override
 	public void invoke() throws Exception {
 
+		this.output.initializeSerializers();
+
 		while (this.input1.hasNext()) {
 
 			StringRecord s = input1.next();
@@ -41,13 +43,15 @@ public class DoubleTargetTask extends AbstractTask {
 			this.output.emit(s);
 		}
 
+		this.output.flush();
+
 	}
 
 	@Override
 	public void registerInputOutput() {
 		this.input1 = new RecordReader<StringRecord>(this, StringRecord.class);
 		this.input2 = new RecordReader<StringRecord>(this, StringRecord.class);
-		this.output = new RecordWriter<StringRecord>(this, StringRecord.class);
+		this.output = new RecordWriter<StringRecord>(this);
 	}
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 baae3b9..77b4f96 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,8 +14,8 @@
 package eu.stratosphere.nephele.jobmanager;
 
 import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.RecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
+import eu.stratosphere.runtime.io.api.RecordReader;
+import eu.stratosphere.runtime.io.api.RecordWriter;
 import eu.stratosphere.nephele.template.AbstractTask;
 
 /**
@@ -57,7 +57,7 @@ public class ExceptionTask extends AbstractTask {
 	public void registerInputOutput() {
 
 		new RecordReader<StringRecord>(this, StringRecord.class);
-		new RecordWriter<StringRecord>(this, StringRecord.class);
+		new RecordWriter<StringRecord>(this);
 	}
 
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 be7be66..96be668 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
@@ -14,8 +14,8 @@
 package eu.stratosphere.nephele.jobmanager;
 
 import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.RecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
+import eu.stratosphere.runtime.io.api.RecordReader;
+import eu.stratosphere.runtime.io.api.RecordWriter;
 import eu.stratosphere.nephele.template.AbstractTask;
 
 public class ForwardTask extends AbstractTask {
@@ -26,16 +26,20 @@ public class ForwardTask extends AbstractTask {
 	@Override
 	public void invoke() throws Exception {
 
+		this.output.initializeSerializers();
+
 		while (this.input.hasNext()) {
 
 			StringRecord s = input.next();
 			this.output.emit(s);
 		}
+
+		this.output.flush();
 	}
 
 	@Override
 	public void registerInputOutput() {
 		this.input = new RecordReader<StringRecord>(this, StringRecord.class);
-		this.output = new RecordWriter<StringRecord>(this, StringRecord.class);
+		this.output = new RecordWriter<StringRecord>(this);
 	}
 }


[17/30] Offer buffer-oriented API for I/O (#25)

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java
deleted file mode 100644
index c1553d8..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java
+++ /dev/null
@@ -1,303 +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.taskmanager.runtime;
-
-import java.io.IOException;
-import java.util.ArrayDeque;
-import java.util.Iterator;
-import java.util.Queue;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.io.channels.Buffer;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedInputChannel;
-import eu.stratosphere.nephele.io.channels.bytebuffered.BufferOrEvent;
-import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedInputChannelBroker;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferAvailabilityListener;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.InputChannelContext;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.ReceiverNotFoundEvent;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher;
-
-
-final class RuntimeInputChannelContext implements InputChannelContext, ByteBufferedInputChannelBroker {
-
-	private static final Log LOG = LogFactory.getLog(RuntimeInputChannelContext.class);
-
-	private final RuntimeInputGateContext inputGateContext;
-
-	private final AbstractByteBufferedInputChannel<?> byteBufferedInputChannel;
-
-	private final TransferEnvelopeDispatcher transferEnvelopeDispatcher;
-
-	private final Queue<TransferEnvelope> queuedEnvelopes = new ArrayDeque<TransferEnvelope>();
-	
-	private Iterator<AbstractEvent> pendingEvents;
-
-	private int lastReceivedEnvelope = -1;
-
-	private boolean destroyCalled = false;
-
-	RuntimeInputChannelContext(final RuntimeInputGateContext inputGateContext,
-			final TransferEnvelopeDispatcher transferEnvelopeDispatcher,
-			final AbstractByteBufferedInputChannel<?> byteBufferedInputChannel) {
-
-		this.inputGateContext = inputGateContext;
-		this.transferEnvelopeDispatcher = transferEnvelopeDispatcher;
-		this.byteBufferedInputChannel = byteBufferedInputChannel;
-		this.byteBufferedInputChannel.setInputChannelBroker(this);
-	}
-
-
-	@Override
-	public BufferOrEvent getNextBufferOrEvent() throws IOException {
-		// return pending events first
-		if (this.pendingEvents != null) {
-			// if the field is not null, it must always have a next value!
-			BufferOrEvent next = new BufferOrEvent(this.pendingEvents.next());
-			if (!this.pendingEvents.hasNext()) {
-				this.pendingEvents = null;
-			}
-			return next;
-		}
-
-		// if no events are pending, get the next buffer
-		TransferEnvelope nextEnvelope;
-		synchronized (this.queuedEnvelopes) {
-			if (this.queuedEnvelopes.isEmpty()) {
-				return null;
-			}
-			nextEnvelope = this.queuedEnvelopes.poll();
-		}
-
-		// schedule events as pending, because events come always after the buffer!
-		if (nextEnvelope.getEventList() != null) {
-			Iterator<AbstractEvent> events = nextEnvelope.getEventList().iterator();
-			if (events.hasNext()) {
-				this.pendingEvents = events;
-			}
-		}
-		
-		// get the buffer, if there is one
-		if (nextEnvelope.getBuffer() != null) {
-			return new BufferOrEvent(nextEnvelope.getBuffer());
-		}
-		else if (this.pendingEvents != null) {
-			// if the field is not null, it must always have a next value!
-			BufferOrEvent next = new BufferOrEvent(this.pendingEvents.next());
-			if (!this.pendingEvents.hasNext()) {
-				this.pendingEvents = null;
-			}
-			
-			return next;
-		}
-		else {
-			// no buffer and no events, this should be an error
-			throw new IOException("Received an envelope with neither data nor events.");
-		}
-	}
-
-	@Override
-	public void transferEventToOutputChannel(AbstractEvent event) throws IOException, InterruptedException {
-		TransferEnvelope ephemeralTransferEnvelope = new TransferEnvelope(0, getJobID(), getChannelID());
-		ephemeralTransferEnvelope.addEvent(event);
-		
-		this.transferEnvelopeDispatcher.processEnvelopeFromInputChannel(ephemeralTransferEnvelope);
-	}
-
-	@Override
-	public void queueTransferEnvelope(TransferEnvelope transferEnvelope) {
-
-		if (ReceiverNotFoundEvent.isReceiverNotFoundEvent(transferEnvelope)) {
-			return;
-		}
-		
-		// The sequence number of the envelope to be queued
-		final int sequenceNumber = transferEnvelope.getSequenceNumber();
-
-		synchronized (this.queuedEnvelopes) {
-
-			if (this.destroyCalled) {
-				final Buffer buffer = transferEnvelope.getBuffer();
-				if (buffer != null) {
-					buffer.recycleBuffer();
-				}
-				return;
-			}
-
-			final int expectedSequenceNumber = this.lastReceivedEnvelope + 1;
-			if (sequenceNumber != expectedSequenceNumber) {
-				// This is a problem, now we are actually missing some data
-				this.byteBufferedInputChannel.reportIOException(new IOException("Expected data packet "
-						+ expectedSequenceNumber + " but received " + sequenceNumber));
-				
-				// notify that something (an exception) is available
-				this.byteBufferedInputChannel.notifyGateThatInputIsAvailable();
-
-				if (LOG.isDebugEnabled()) {
-					LOG.debug("Input channel " + getChannelName() + " expected envelope " + expectedSequenceNumber
-						+ " but received " + sequenceNumber);
-				}
-
-				// rescue the buffer
-				final Buffer buffer = transferEnvelope.getBuffer();
-				if (buffer != null) {
-					buffer.recycleBuffer();
-				}
-			} else {
-
-				this.queuedEnvelopes.add(transferEnvelope);
-				this.lastReceivedEnvelope = sequenceNumber;
-
-				// Notify the channel about the new data. notify as much as there is (buffer plus once per event)
-				if (transferEnvelope.getBuffer() != null) {
-					this.byteBufferedInputChannel.notifyGateThatInputIsAvailable();
-				}
-				if (transferEnvelope.getEventList() != null) {
-					for (int i = 0; i < transferEnvelope.getEventList().size(); i++) {
-						this.byteBufferedInputChannel.notifyGateThatInputIsAvailable();
-					}
-				}
-			}
-		}
-	}
-
-	@Override
-	public ChannelID getChannelID() {
-		return this.byteBufferedInputChannel.getID();
-	}
-
-	@Override
-	public ChannelID getConnectedChannelID() {
-		return this.byteBufferedInputChannel.getConnectedChannelID();
-	}
-
-	@Override
-	public JobID getJobID() {
-		return this.byteBufferedInputChannel.getJobID();
-	}
-
-	@Override
-	public boolean isInputChannel() {
-		return this.byteBufferedInputChannel.isInputChannel();
-	}
-
-	@Override
-	public void destroy() {
-		final Queue<Buffer> buffersToRecycle = new ArrayDeque<Buffer>();
-
-		synchronized (this.queuedEnvelopes) {
-			this.destroyCalled = true;
-
-			while (!this.queuedEnvelopes.isEmpty()) {
-				final TransferEnvelope envelope = this.queuedEnvelopes.poll();
-				if (envelope.getBuffer() != null) {
-					buffersToRecycle.add(envelope.getBuffer());
-				}
-			}
-		}
-
-		while (!buffersToRecycle.isEmpty()) {
-			buffersToRecycle.poll().recycleBuffer();
-		}
-	}
-
-	@Override
-	public void logQueuedEnvelopes() {
-		int numberOfQueuedEnvelopes = 0;
-		int numberOfQueuedEnvelopesWithMemoryBuffers = 0;
-		int numberOfQueuedEnvelopesWithFileBuffers = 0;
-
-		synchronized (this.queuedEnvelopes) {
-
-			final Iterator<TransferEnvelope> it = this.queuedEnvelopes.iterator();
-			while (it.hasNext()) {
-
-				final TransferEnvelope envelope = it.next();
-				++numberOfQueuedEnvelopes;
-				final Buffer buffer = envelope.getBuffer();
-				if (buffer == null) {
-					continue;
-				}
-
-				if (buffer.isBackedByMemory()) {
-					++numberOfQueuedEnvelopesWithMemoryBuffers;
-				} else {
-					++numberOfQueuedEnvelopesWithFileBuffers;
-				}
-			}
-		}
-
-		System.out.println("\t\t" + getChannelName() + ": " + numberOfQueuedEnvelopes + " ("
-			+ numberOfQueuedEnvelopesWithMemoryBuffers + ", " + numberOfQueuedEnvelopesWithFileBuffers + ")");
-
-	}
-
-	@Override
-	public Buffer requestEmptyBuffer(int minimumSizeOfBuffer) throws IOException {
-		return this.inputGateContext.requestEmptyBuffer(minimumSizeOfBuffer);
-	}
-
-	@Override
-	public Buffer requestEmptyBufferBlocking(int minimumSizeOfBuffer) throws IOException, InterruptedException {
-		return this.inputGateContext.requestEmptyBufferBlocking(minimumSizeOfBuffer);
-	}
-
-	@Override
-	public int getMaximumBufferSize() {
-		return this.inputGateContext.getMaximumBufferSize();
-	}
-
-	@Override
-	public boolean isShared() {
-		return this.inputGateContext.isShared();
-	}
-
-	@Override
-	public void reportAsynchronousEvent() {
-		this.inputGateContext.reportAsynchronousEvent();
-	}
-
-	@Override
-	public ChannelType getType() {
-		return this.byteBufferedInputChannel.getType();
-	}
-
-	/**
-	 * Constructs and returns a human-readable name of this channel used for debugging.
-	 * 
-	 * @return a human-readable name of this channel used for debugging
-	 */
-	private String getChannelName() {
-		StringBuilder sb = new StringBuilder(this.inputGateContext.getTaskName());
-		sb.append(' ');
-		sb.append('(');
-		sb.append(this.byteBufferedInputChannel.getChannelIndex());
-		sb.append(',');
-		sb.append(' ');
-		sb.append(this.byteBufferedInputChannel.getID());
-		sb.append(')');
-		return sb.toString();
-	}
-
-	@Override
-	public boolean registerBufferAvailabilityListener(final BufferAvailabilityListener bufferAvailabilityListener) {
-		return this.inputGateContext.registerBufferAvailabilityListener(bufferAvailabilityListener);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java
deleted file mode 100644
index 7e2d492..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java
+++ /dev/null
@@ -1,183 +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.taskmanager.runtime;
-
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.io.GateID;
-import eu.stratosphere.nephele.io.InputGate;
-import eu.stratosphere.nephele.io.channels.AbstractInputChannel;
-import eu.stratosphere.nephele.io.channels.Buffer;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedInputChannel;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferAvailabilityListener;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPool;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPoolOwner;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.InputChannelContext;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.InputGateContext;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher;
-
-final class RuntimeInputGateContext implements BufferProvider, InputGateContext, LocalBufferPoolOwner {
-
-	private final String taskName;
-
-	private final LocalBufferPool localBufferPool;
-
-	private final TransferEnvelopeDispatcher transferEnvelopeDispatcher;
-
-	private final InputGate<? extends IOReadableWritable> inputGate;
-
-	RuntimeInputGateContext(final String taskName, final TransferEnvelopeDispatcher transferEnvelopeDispatcher,
-			final InputGate<? extends IOReadableWritable> inputGate) {
-
-		this.taskName = taskName;
-		this.localBufferPool = new LocalBufferPool(1, false);
-
-		this.transferEnvelopeDispatcher = transferEnvelopeDispatcher;
-		this.inputGate = inputGate;
-	}
-
-	@Override
-	public Buffer requestEmptyBuffer(final int minimumSizeOfBuffer) throws IOException {
-
-		return this.localBufferPool.requestEmptyBuffer(minimumSizeOfBuffer);
-	}
-
-
-	@Override
-	public Buffer requestEmptyBufferBlocking(final int minimumSizeOfBuffer) throws IOException, InterruptedException {
-
-		final Buffer buffer = this.localBufferPool.requestEmptyBuffer(minimumSizeOfBuffer);
-		if (buffer != null) {
-			return buffer;
-		}
-
-		return this.localBufferPool.requestEmptyBufferBlocking(minimumSizeOfBuffer);
-	}
-
-
-	@Override
-	public int getMaximumBufferSize() {
-
-		return this.localBufferPool.getMaximumBufferSize();
-	}
-
-
-	@Override
-	public boolean isShared() {
-
-		return this.localBufferPool.isShared();
-	}
-
-
-	@Override
-	public void reportAsynchronousEvent() {
-
-		this.localBufferPool.reportAsynchronousEvent();
-	}
-
-	@Override
-	public int getNumberOfChannels() {
-
-		return this.inputGate.getNumberOfInputChannels();
-	}
-
-
-	@Override
-	public void setDesignatedNumberOfBuffers(int numberOfBuffers) {
-
-		this.localBufferPool.setDesignatedNumberOfBuffers(numberOfBuffers);
-	}
-
-
-	@Override
-	public void clearLocalBufferPool() {
-
-		this.localBufferPool.destroy();
-	}
-
-
-	@Override
-	public void logBufferUtilization() {
-
-		final int ava = this.localBufferPool.getNumberOfAvailableBuffers();
-		final int req = this.localBufferPool.getRequestedNumberOfBuffers();
-		final int des = this.localBufferPool.getDesignatedNumberOfBuffers();
-
-		System.out
-			.println("\t\tInput gate " + this.inputGate.getIndex() + " of " + this.taskName + ": " + ava
-				+ " available, " + req + " requested, " + des + " designated");
-	}
-
-
-	@Override
-	public GateID getGateID() {
-
-		return this.inputGate.getGateID();
-	}
-
-
-	@Override
-	public InputChannelContext createInputChannelContext(final ChannelID channelID,
-			final InputChannelContext previousContext) {
-
-		AbstractInputChannel<? extends IOReadableWritable> channel = null;
-		for (int i = 0; i < this.inputGate.getNumberOfInputChannels(); ++i) {
-			AbstractInputChannel<? extends IOReadableWritable> candidateChannel = this.inputGate.getInputChannel(i);
-			if (candidateChannel.getID().equals(channelID)) {
-				channel = candidateChannel;
-				break;
-			}
-		}
-
-		if (channel == null) {
-			throw new IllegalArgumentException("Cannot find input channel with ID " + channelID);
-		}
-
-		if (!(channel instanceof AbstractByteBufferedInputChannel)) {
-			throw new IllegalStateException("Channel with ID" + channelID
-				+ " is not of type AbstractByteBufferedInputChannel");
-		}
-
-		return new RuntimeInputChannelContext(this, this.transferEnvelopeDispatcher,
-			(AbstractByteBufferedInputChannel<? extends IOReadableWritable>) channel);
-	}
-
-
-	@Override
-	public LocalBufferPoolOwner getLocalBufferPoolOwner() {
-
-		return this;
-	}
-
-	/**
-	 * Returns the name of the task this gate belongs to.
-	 * 
-	 * @return the name of the task this gate belongs to
-	 */
-	String getTaskName() {
-
-		return this.taskName;
-	}
-
-
-	@Override
-	public boolean registerBufferAvailabilityListener(final BufferAvailabilityListener bufferAvailabilityListener) {
-
-		return this.localBufferPool.registerBufferAvailabilityListener(bufferAvailabilityListener);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java
deleted file mode 100644
index 41fc7ff..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java
+++ /dev/null
@@ -1,206 +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.taskmanager.runtime;
-
-import java.io.IOException;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
-import eu.stratosphere.nephele.io.channels.Buffer;
-import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedOutputChannel;
-import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedChannelCloseEvent;
-import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedOutputChannelBroker;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.AbstractOutputChannelForwarder;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwardingChain;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.ReceiverNotFoundEvent;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope;
-
-final class RuntimeOutputChannelBroker extends AbstractOutputChannelForwarder implements
-		ByteBufferedOutputChannelBroker {
-
-	/**
-	 * The byte buffered output channel this context belongs to.
-	 */
-	private final AbstractByteBufferedOutputChannel<?> byteBufferedOutputChannel;
-
-	/**
-	 * The buffer provider this channel broker to obtain buffers from.
-	 */
-	private final RuntimeOutputGateContext outputGateContext;
-
-	/**
-	 * The forwarding chain along which the created transfer envelopes will be pushed.
-	 */
-	private OutputChannelForwardingChain forwardingChain;
-
-	/**
-	 * Points to the {@link TransferEnvelope} object that will be passed to the framework upon
-	 * the next <code>releaseWriteBuffers</code> call.
-	 */
-	private TransferEnvelope outgoingTransferEnvelope = null;
-
-	/**
-	 * Stores whether the receiver has acknowledged the close request from this channel.
-	 */
-	private boolean closeAcknowledgmentReceived = false;
-
-	/**
-	 * Stores the last sequence number of the transfer envelope for which the receiver could not be found.
-	 */
-	private int lastSequenceNumberWithReceiverNotFound = -1;
-
-	/**
-	 * The sequence number for the next {@link TransferEnvelope} to be created.
-	 */
-	private int sequenceNumber = 0;
-
-	RuntimeOutputChannelBroker(final RuntimeOutputGateContext outputGateContext,
-			final AbstractByteBufferedOutputChannel<?> byteBufferedOutputChannel,
-			final AbstractOutputChannelForwarder next) {
-
-		super(next);
-
-		if (next == null) {
-			throw new IllegalArgumentException("Argument next must not be null");
-		}
-
-		this.outputGateContext = outputGateContext;
-		this.byteBufferedOutputChannel = byteBufferedOutputChannel;
-		this.byteBufferedOutputChannel.setByteBufferedOutputChannelBroker(this);
-	}
-
-	public void setForwardingChain(final OutputChannelForwardingChain forwardingChain) {
-		this.forwardingChain = forwardingChain;
-	}
-
-
-	@Override
-	public boolean hasDataLeft() throws IOException, InterruptedException {
-
-		if (this.closeAcknowledgmentReceived) {
-			return getNext().hasDataLeft();
-		}
-
-		if ((this.lastSequenceNumberWithReceiverNotFound + 1) == this.sequenceNumber) {
-			return getNext().hasDataLeft();
-		}
-
-		return true;
-	}
-
-
-	@Override
-	public void processEvent(final AbstractEvent event) {
-
-		if (event instanceof ByteBufferedChannelCloseEvent) {
-			this.closeAcknowledgmentReceived = true;
-		} else if (event instanceof ReceiverNotFoundEvent) {
-			this.lastSequenceNumberWithReceiverNotFound = ((ReceiverNotFoundEvent) event).getSequenceNumber();
-		} else if (event instanceof AbstractTaskEvent) {
-			throw new IllegalStateException("Received synchronous task event " + event);
-		}
-
-		getNext().processEvent(event);
-	}
-
-
-	@Override
-	public Buffer requestEmptyWriteBuffer() throws InterruptedException, IOException {
-
-		if (this.outgoingTransferEnvelope == null) {
-			this.outgoingTransferEnvelope = createNewOutgoingTransferEnvelope();
-		}
-
-		final int uncompressedBufferSize = calculateBufferSize();
-
-		return this.outputGateContext.requestEmptyBufferBlocking(uncompressedBufferSize);
-	}
-
-	/**
-	 * Creates a new {@link TransferEnvelope} object. The method assigns
-	 * and increases the sequence number. Moreover, it will look up the list of receivers for this transfer envelope.
-	 * This method will block until the lookup is completed.
-	 * 
-	 * @return a new {@link TransferEnvelope} object containing the correct sequence number and receiver list
-	 */
-	private TransferEnvelope createNewOutgoingTransferEnvelope() {
-
-		final TransferEnvelope transferEnvelope = new TransferEnvelope(this.sequenceNumber++,
-			this.byteBufferedOutputChannel.getJobID(),
-			this.byteBufferedOutputChannel.getID());
-
-		return transferEnvelope;
-	}
-
-	/**
-	 * Calculates the recommended size of the next buffer to be
-	 * handed to the attached channel object in bytes.
-	 * 
-	 * @return the recommended size of the next buffer in bytes
-	 */
-	private int calculateBufferSize() {
-
-		// TODO: Include latency considerations
-		return this.outputGateContext.getMaximumBufferSize();
-	}
-
-
-	@Override
-	public void releaseWriteBuffer(final Buffer buffer) throws IOException, InterruptedException {
-
-		// Check for events
-		this.forwardingChain.processQueuedEvents();
-
-		if (this.outgoingTransferEnvelope == null) {
-			throw new IllegalStateException("Cannot find transfer envelope for channel with ID "
-				+ this.byteBufferedOutputChannel.getID());
-		}
-
-		// Consistency check
-		if (this.outgoingTransferEnvelope.getBuffer() != null) {
-			throw new IllegalStateException("Channel " + this.byteBufferedOutputChannel.getID()
-				+ " has already a buffer attached");
-		}
-		buffer.flip();
-		this.outgoingTransferEnvelope.setBuffer(buffer);
-
-		this.forwardingChain.pushEnvelope(this.outgoingTransferEnvelope);
-		this.outgoingTransferEnvelope = null;
-	}
-
-
-	@Override
-	public boolean hasDataLeftToTransmit() throws IOException, InterruptedException {
-
-		// Check for events
-		this.forwardingChain.processQueuedEvents();
-
-		return this.forwardingChain.anyForwarderHasDataLeft();
-	}
-
-
-	@Override
-	public void transferEventToInputChannel(final AbstractEvent event) throws IOException, InterruptedException {
-
-		if (this.outgoingTransferEnvelope != null) {
-			this.outgoingTransferEnvelope.addEvent(event);
-		} else {
-
-			final TransferEnvelope ephemeralTransferEnvelope = createNewOutgoingTransferEnvelope();
-			ephemeralTransferEnvelope.addEvent(event);
-
-			this.forwardingChain.pushEnvelope(ephemeralTransferEnvelope);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java
deleted file mode 100644
index 01d8ba2..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java
+++ /dev/null
@@ -1,76 +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.taskmanager.runtime;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedOutputChannel;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.AbstractOutputChannelContext;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwardingChain;
-
-public final class RuntimeOutputChannelContext extends AbstractOutputChannelContext {
-
-	private final AbstractByteBufferedOutputChannel<?> byteBufferedOutputChannel;
-
-	RuntimeOutputChannelContext(final AbstractByteBufferedOutputChannel<?> byteBufferedOutputChannel,
-			final OutputChannelForwardingChain forwardingChain) {
-		super(forwardingChain);
-
-		this.byteBufferedOutputChannel = byteBufferedOutputChannel;
-	}
-
-
-	@Override
-	public boolean isInputChannel() {
-
-		return false;
-	}
-
-
-	@Override
-	public ChannelID getChannelID() {
-
-		return this.byteBufferedOutputChannel.getID();
-	}
-
-
-	@Override
-	public ChannelID getConnectedChannelID() {
-
-		return this.byteBufferedOutputChannel.getConnectedChannelID();
-	}
-
-
-	@Override
-	public JobID getJobID() {
-
-		return this.byteBufferedOutputChannel.getJobID();
-	}
-
-
-	@Override
-	public ChannelType getType() {
-
-		return this.byteBufferedOutputChannel.getType();
-	}
-
-
-	@Override
-	protected void processEventAsynchronously(final AbstractEvent event) {
-
-		this.byteBufferedOutputChannel.processEvent(event);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java
deleted file mode 100644
index 72e5047..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java
+++ /dev/null
@@ -1,159 +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.taskmanager.runtime;
-
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.io.AbstractID;
-import eu.stratosphere.nephele.io.GateID;
-import eu.stratosphere.nephele.io.OutputGate;
-import eu.stratosphere.nephele.io.channels.AbstractOutputChannel;
-import eu.stratosphere.nephele.io.channels.Buffer;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedOutputChannel;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferAvailabilityListener;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.AbstractOutputChannelForwarder;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelContext;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwardingChain;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputGateContext;
-
-final class RuntimeOutputGateContext implements BufferProvider, OutputGateContext {
-
-	private final RuntimeTaskContext taskContext;
-
-	private final OutputGate<? extends IOReadableWritable> outputGate;
-
-	RuntimeOutputGateContext(final RuntimeTaskContext taskContext, final OutputGate<? extends IOReadableWritable> outputGate) {
-
-		this.taskContext = taskContext;
-		this.outputGate = outputGate;
-	}
-
-	AbstractID getFileOwnerID() {
-
-		return this.taskContext.getFileOwnerID();
-	}
-
-
-	@Override
-	public int getMaximumBufferSize() {
-
-		return this.taskContext.getMaximumBufferSize();
-	}
-
-
-	@Override
-	public Buffer requestEmptyBuffer(int minimumSizeOfBuffer) throws IOException {
-
-		return this.taskContext.requestEmptyBuffer(minimumSizeOfBuffer);
-	}
-
-
-	@Override
-	public Buffer requestEmptyBufferBlocking(int minimumSizeOfBuffer) throws IOException, InterruptedException {
-
-		Buffer buffer = this.taskContext.requestEmptyBuffer(minimumSizeOfBuffer);
-
-		// No memory-based buffer available
-		if (buffer == null) {
-			// Wait until a memory-based buffer is available
-			buffer = this.taskContext.requestEmptyBufferBlocking(minimumSizeOfBuffer);
-		}
-
-		return buffer;
-	}
-
-
-	@Override
-	public boolean isShared() {
-
-		return this.taskContext.isShared();
-	}
-
-
-	@Override
-	public void reportAsynchronousEvent() {
-
-		this.taskContext.reportAsynchronousEvent();
-	}
-
-
-	@Override
-	public GateID getGateID() {
-
-		return this.outputGate.getGateID();
-	}
-
-
-	@Override
-	public OutputChannelContext createOutputChannelContext(ChannelID channelID, OutputChannelContext previousContext,
-			boolean isReceiverRunning, boolean mergeSpillBuffers) {
-
-		if (previousContext != null) {
-			throw new IllegalStateException("Found previous output context for channel " + channelID);
-		}
-
-		AbstractOutputChannel<? extends IOReadableWritable> channel = null;
-		for (int i = 0; i < this.outputGate.getNumberOfOutputChannels(); ++i) {
-			AbstractOutputChannel<? extends IOReadableWritable> candidateChannel = this.outputGate.getOutputChannel(i);
-			if (candidateChannel.getID().equals(channelID)) {
-				channel = candidateChannel;
-				break;
-			}
-		}
-
-		if (channel == null) {
-			throw new IllegalArgumentException("Cannot find output channel with ID " + channelID);
-		}
-
-		if (!(channel instanceof AbstractByteBufferedOutputChannel)) {
-			throw new IllegalStateException("Channel with ID" + channelID
-				+ " is not of type AbstractByteBufferedOutputChannel");
-		}
-
-		// The output channel for this context
-		final AbstractByteBufferedOutputChannel<? extends IOReadableWritable> outputChannel = (AbstractByteBufferedOutputChannel<? extends IOReadableWritable>) channel;
-
-		// Construct the forwarding chain
-		RuntimeOutputChannelBroker outputChannelBroker;
-		AbstractOutputChannelForwarder last;
-		// Construction for in-memory and network channels
-		final RuntimeDispatcher runtimeDispatcher = new RuntimeDispatcher(
-			this.taskContext.getTransferEnvelopeDispatcher());
-		/*
-		 * final SpillingBarrier spillingBarrier = new SpillingBarrier(isReceiverRunning, mergeSpillBuffers,
-		 * runtimeDispatcher);
-		 * final ForwardingBarrier forwardingBarrier = new ForwardingBarrier(channelID, spillingBarrier);
-		 */
-		final ForwardingBarrier forwardingBarrier = new ForwardingBarrier(channelID, runtimeDispatcher);
-		outputChannelBroker = new RuntimeOutputChannelBroker(this, outputChannel, forwardingBarrier);
-		last = runtimeDispatcher;
-
-		final OutputChannelForwardingChain forwardingChain = new OutputChannelForwardingChain(outputChannelBroker, last);
-
-		// Set forwarding chain for broker
-		outputChannelBroker.setForwardingChain(forwardingChain);
-
-		return new RuntimeOutputChannelContext(outputChannel, forwardingChain);
-	}
-
-
-	@Override
-	public boolean registerBufferAvailabilityListener(final BufferAvailabilityListener bufferAvailabilityListener) {
-
-		return this.taskContext.registerBufferAvailabilityListener(bufferAvailabilityListener);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java
deleted file mode 100644
index 5a44530..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java
+++ /dev/null
@@ -1,346 +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.taskmanager.runtime;
-
-import java.util.Iterator;
-import java.util.Queue;
-import java.util.concurrent.ConcurrentLinkedQueue;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.nephele.execution.Environment;
-import eu.stratosphere.nephele.execution.ExecutionListener;
-import eu.stratosphere.nephele.execution.ExecutionObserver;
-import eu.stratosphere.nephele.execution.ExecutionState;
-import eu.stratosphere.nephele.execution.ExecutionStateTransition;
-import eu.stratosphere.nephele.execution.RuntimeEnvironment;
-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.taskmanager.Task;
-import eu.stratosphere.nephele.taskmanager.TaskManager;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPoolOwner;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.TaskContext;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher;
-import eu.stratosphere.nephele.template.AbstractInvokable;
-import eu.stratosphere.util.StringUtils;
-
-public final class RuntimeTask implements Task, ExecutionObserver {
-
-	/**
-	 * The log object used for debugging.
-	 */
-	private static final Log LOG = LogFactory.getLog(RuntimeTask.class);
-
-	private final ExecutionVertexID vertexID;
-
-	private final RuntimeEnvironment environment;
-
-	private final TaskManager taskManager;
-
-	/**
-	 * Stores whether the task has been canceled.
-	 */
-	private volatile boolean isCanceled = false;
-
-	/**
-	 * The current execution state of the task
-	 */
-	private volatile ExecutionState executionState = ExecutionState.STARTING;
-
-	private Queue<ExecutionListener> registeredListeners = new ConcurrentLinkedQueue<ExecutionListener>();
-
-	public RuntimeTask(final ExecutionVertexID vertexID, final RuntimeEnvironment environment,
-			final TaskManager taskManager) {
-
-		this.vertexID = vertexID;
-		this.environment = environment;
-		this.taskManager = taskManager;
-
-		this.environment.setExecutionObserver(this);
-	}
-
-
-	@Override
-	public void executionStateChanged(final ExecutionState newExecutionState, final String optionalMessage) {
-
-		// Check the state transition
-		ExecutionStateTransition.checkTransition(false, getTaskName(), this.executionState, newExecutionState);
-
-		// Make sure the reason for a transition to FAILED appears in the log files
-		if (newExecutionState == ExecutionState.FAILED) {
-			LOG.error(optionalMessage);
-		}
-
-		// Notify all listener objects
-		final Iterator<ExecutionListener> it = this.registeredListeners.iterator();
-		while (it.hasNext()) {
-			it.next().executionStateChanged(this.environment.getJobID(), this.vertexID, newExecutionState,
-				optionalMessage);
-		}
-
-		// Store the new execution state
-		this.executionState = newExecutionState;
-
-		// Finally propagate the state change to the job manager
-		this.taskManager.executionStateChanged(this.environment.getJobID(), this.vertexID, newExecutionState,
-			optionalMessage);
-	}
-
-	/**
-	 * Returns the name of the task associated with this observer object.
-	 * 
-	 * @return the name of the task associated with this observer object
-	 */
-	private String getTaskName() {
-
-		return this.environment.getTaskName() + " (" + (this.environment.getIndexInSubtaskGroup() + 1) + "/"
-			+ this.environment.getCurrentNumberOfSubtasks() + ")";
-	}
-
-
-	@Override
-	public void userThreadStarted(final Thread userThread) {
-
-		// Notify the listeners
-		final Iterator<ExecutionListener> it = this.registeredListeners.iterator();
-		while (it.hasNext()) {
-			it.next().userThreadStarted(this.environment.getJobID(), this.vertexID, userThread);
-		}
-	}
-
-
-	@Override
-	public void userThreadFinished(final Thread userThread) {
-
-		// Notify the listeners
-		final Iterator<ExecutionListener> it = this.registeredListeners.iterator();
-		while (it.hasNext()) {
-			it.next().userThreadFinished(this.environment.getJobID(), this.vertexID, userThread);
-		}
-	}
-
-	/**
-	 * Registers the {@link ExecutionListener} object for this task. This object
-	 * will be notified about important events during the task execution.
-	 * 
-	 * @param executionListener
-	 *        the object to be notified for important events during the task execution
-	 */
-
-	public void registerExecutionListener(final ExecutionListener executionListener) {
-
-		this.registeredListeners.add(executionListener);
-	}
-
-	/**
-	 * Unregisters the {@link ExecutionListener} object for this environment. This object
-	 * will no longer be notified about important events during the task execution.
-	 * 
-	 * @param executionListener
-	 *        the lister object to be unregistered
-	 */
-
-	public void unregisterExecutionListener(final ExecutionListener executionListener) {
-
-		this.registeredListeners.remove(executionListener);
-	}
-
-
-	@Override
-	public void markAsFailed() {
-
-		executionStateChanged(ExecutionState.FAILED, "Execution thread died unexpectedly");
-	}
-
-
-	@Override
-	public void cancelExecution() {
-
-		cancelOrKillExecution(true);
-	}
-
-
-	@Override
-	public void killExecution() {
-
-		cancelOrKillExecution(false);
-	}
-
-	/**
-	 * Cancels or kills the task.
-	 * 
-	 * @param cancel
-	 *        <code>true/code> if the task shall be canceled, <code>false</code> if it shall be killed
-	 */
-	private void cancelOrKillExecution(final boolean cancel) {
-
-		final Thread executingThread = this.environment.getExecutingThread();
-
-		if (executingThread == null) {
-			return;
-		}
-
-		if (this.executionState != ExecutionState.RUNNING && this.executionState != ExecutionState.FINISHING) {
-			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) {}
-			
-			if (!executingThread.isAlive()) {
-				break;
-			}
-
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("Sending repeated " + (cancel == true ? "canceling" : "killing") + " signal to " +
-					this.environment.getTaskName() + " with state " + this.executionState);
-			}
-		}
-	}
-
-
-	@Override
-	public void startExecution() {
-
-		final Thread thread = this.environment.getExecutingThread();
-		thread.start();
-	}
-
-
-	@Override
-	public boolean isCanceled() {
-
-		return this.isCanceled;
-	}
-
-
-	@Override
-	public boolean isTerminated() {
-
-		final Thread executingThread = this.environment.getExecutingThread();
-		if (executingThread.getState() == Thread.State.TERMINATED) {
-			return true;
-		}
-
-		return false;
-	}
-
-
-	@Override
-	public Environment getEnvironment() {
-
-		return this.environment;
-	}
-
-	/**
-	 * Returns the runtime environment associated with this task.
-	 * 
-	 * @return the runtime environment associated with this task
-	 */
-	public RuntimeEnvironment getRuntimeEnvironment() {
-
-		return this.environment;
-	}
-
-
-	@Override
-	public JobID getJobID() {
-
-		return this.environment.getJobID();
-	}
-
-
-	@Override
-	public ExecutionVertexID getVertexID() {
-
-		return this.vertexID;
-	}
-
-
-	@Override
-	public void registerProfiler(final TaskManagerProfiler taskManagerProfiler, final Configuration jobConfiguration) {
-
-		taskManagerProfiler.registerExecutionListener(this, jobConfiguration);
-	}
-
-
-	@Override
-	public void unregisterMemoryManager(final MemoryManager memoryManager) {
-
-		if (memoryManager != null) {
-			memoryManager.releaseAll(this.environment.getInvokable());
-		}
-	}
-
-
-	@Override
-	public void unregisterProfiler(final TaskManagerProfiler taskManagerProfiler) {
-
-		if (taskManagerProfiler != null) {
-			taskManagerProfiler.unregisterExecutionListener(this.vertexID);
-		}
-	}
-
-
-	@Override
-	public TaskContext createTaskContext(final TransferEnvelopeDispatcher transferEnvelopeDispatcher,
-			final LocalBufferPoolOwner previousBufferPoolOwner) {
-
-		if (previousBufferPoolOwner != null) {
-			throw new IllegalStateException("Vertex " + this.vertexID + " has a previous buffer pool owner");
-		}
-
-		return new RuntimeTaskContext(this, transferEnvelopeDispatcher);
-	}
-
-
-	@Override
-	public ExecutionState getExecutionState() {
-
-		return this.executionState;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java
deleted file mode 100644
index 7f3a9a0..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java
+++ /dev/null
@@ -1,211 +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.taskmanager.runtime;
-
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.execution.RuntimeEnvironment;
-import eu.stratosphere.nephele.io.AbstractID;
-import eu.stratosphere.nephele.io.GateID;
-import eu.stratosphere.nephele.io.InputGate;
-import eu.stratosphere.nephele.io.OutputGate;
-import eu.stratosphere.nephele.io.channels.Buffer;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferAvailabilityListener;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPool;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.InputGateContext;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputGateContext;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.TaskContext;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher;
-
-public final class RuntimeTaskContext implements BufferProvider, TaskContext {
-
-	private final LocalBufferPool localBufferPool;
-
-	private final RuntimeTask task;
-
-	private final int numberOfOutputChannels;
-
-	private final TransferEnvelopeDispatcher transferEnvelopeDispatcher;
-
-	RuntimeTaskContext(final RuntimeTask task, final TransferEnvelopeDispatcher transferEnvelopeDispatcher) {
-
-		this.localBufferPool = new LocalBufferPool(1, false);
-		this.task = task;
-
-		final RuntimeEnvironment environment = task.getRuntimeEnvironment();
-
-		// Compute number of output input channels
-		int nooc = 0;
-		for (int i = 0; i < environment.getNumberOfOutputGates(); ++i) {
-			final OutputGate<? extends IOReadableWritable> outputGate = environment.getOutputGate(i);
-			if (outputGate.isBroadcast()) {
-				++nooc;
-			} else {
-				nooc += outputGate.getNumberOfOutputChannels();
-			}
-		}
-		this.numberOfOutputChannels = nooc;
-
-		this.transferEnvelopeDispatcher = transferEnvelopeDispatcher;
-	}
-
-	TransferEnvelopeDispatcher getTransferEnvelopeDispatcher() {
-
-		return this.transferEnvelopeDispatcher;
-	}
-
-
-
-	@Override
-	public Buffer requestEmptyBuffer(final int minimumSizeOfBuffer) throws IOException {
-
-		return this.localBufferPool.requestEmptyBuffer(minimumSizeOfBuffer);
-	}
-
-
-	@Override
-	public Buffer requestEmptyBufferBlocking(int minimumSizeOfBuffer) throws IOException,
-			InterruptedException {
-
-		return this.localBufferPool.requestEmptyBufferBlocking(minimumSizeOfBuffer);
-	}
-
-
-	@Override
-	public int getMaximumBufferSize() {
-
-		return this.localBufferPool.getMaximumBufferSize();
-	}
-
-
-	@Override
-	public void clearLocalBufferPool() {
-
-		// Clear the buffer cache
-		this.localBufferPool.destroy();
-
-	}
-
-
-	@Override
-	public boolean isShared() {
-
-		return false;
-	}
-
-
-	@Override
-	public void logBufferUtilization() {
-
-		final int ava = this.localBufferPool.getNumberOfAvailableBuffers();
-		final int req = this.localBufferPool.getRequestedNumberOfBuffers();
-		final int des = this.localBufferPool.getDesignatedNumberOfBuffers();
-
-		final RuntimeEnvironment environment = this.task.getRuntimeEnvironment();
-
-		System.out.println("\t\t" + environment.getTaskNameWithIndex() + ": " + ava + " available, " + req
-			+ " requested, " + des + " designated");
-	}
-
-
-
-	@Override
-	public void reportAsynchronousEvent() {
-
-		this.localBufferPool.reportAsynchronousEvent();
-	}
-
-
-	@Override
-	public int getNumberOfChannels() {
-
-		return this.numberOfOutputChannels;
-	}
-
-
-	@Override
-	public void setDesignatedNumberOfBuffers(int numberOfBuffers) {
-
-		this.localBufferPool.setDesignatedNumberOfBuffers(numberOfBuffers);
-	}
-
-	AbstractID getFileOwnerID() {
-
-		return this.task.getVertexID();
-	}
-
-
-	@Override
-	public OutputGateContext createOutputGateContext(final GateID gateID) {
-
-		if (gateID == null) {
-			throw new IllegalArgumentException("Argument gateID must not be null");
-		}
-
-		OutputGate<? extends IOReadableWritable> outputGate = null;
-		final RuntimeEnvironment re = this.task.getRuntimeEnvironment();
-		for (int i = 0; i < re.getNumberOfOutputGates(); ++i) {
-			final OutputGate<? extends IOReadableWritable> candidateGate = re.getOutputGate(i);
-			if (candidateGate.getGateID().equals(gateID)) {
-				outputGate = candidateGate;
-				break;
-			}
-		}
-
-		if (outputGate == null) {
-			throw new IllegalStateException("Cannot find output gate with ID " + gateID);
-		}
-
-		return new RuntimeOutputGateContext(this, outputGate);
-	}
-
-
-	@Override
-	public InputGateContext createInputGateContext(final GateID gateID) {
-
-		if (gateID == null) {
-			throw new IllegalArgumentException("Argument gateID must not be null");
-		}
-
-		InputGate<? extends IOReadableWritable> inputGate = null;
-		final RuntimeEnvironment re = this.task.getRuntimeEnvironment();
-		for (int i = 0; i < re.getNumberOfInputGates(); ++i) {
-			final InputGate<? extends IOReadableWritable> candidateGate = re.getInputGate(i);
-			if (candidateGate.getGateID().equals(gateID)) {
-				inputGate = candidateGate;
-				break;
-			}
-		}
-
-		if (inputGate == null) {
-			throw new IllegalStateException("Cannot find input gate with ID " + gateID);
-		}
-
-		return new RuntimeInputGateContext(re.getTaskNameWithIndex(), this.transferEnvelopeDispatcher, inputGate);
-	}
-
-	public LocalBufferPool getLocalBufferPool() {
-
-		return this.localBufferPool;
-	}
-
-
-	@Override
-	public boolean registerBufferAvailabilityListener(final BufferAvailabilityListener bufferAvailabilityListener) {
-
-		return this.localBufferPool.registerBufferAvailabilityListener(bufferAvailabilityListener);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/AbstractDeserializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/AbstractDeserializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/AbstractDeserializer.java
deleted file mode 100644
index 3a810f6..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/AbstractDeserializer.java
+++ /dev/null
@@ -1,355 +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.taskmanager.transferenvelope;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.ReadableByteChannel;
-
-import eu.stratosphere.nephele.event.task.EventList;
-import eu.stratosphere.nephele.io.channels.Buffer;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.DefaultDeserializer;
-import eu.stratosphere.nephele.jobgraph.JobID;
-
-public abstract class AbstractDeserializer {
-
-	private enum DeserializationState {
-		NOTDESERIALIZED,
-		SEQUENCENUMBERDESERIALIZED,
-		JOBIDDESERIALIZED,
-		SOURCEDESERIALIZED,
-		NOTIFICATIONSDESERIALIZED,
-		FULLYDESERIALIZED
-	};
-
-	private static final int SIZEOFINT = 4;
-
-	private TransferEnvelope transferEnvelope = null;
-
-	private DeserializationState deserializationState = DeserializationState.NOTDESERIALIZED;
-
-	private final DefaultDeserializer<ChannelID> channelIDDeserializationBuffer = new DefaultDeserializer<ChannelID>(
-			ChannelID.class, true);
-
-	private final DefaultDeserializer<JobID> jobIDDeserializationBuffer = new DefaultDeserializer<JobID>(
-			JobID.class, true);
-
-	private final DefaultDeserializer<EventList> notificationListDeserializationBuffer = new DefaultDeserializer<EventList>(
-			EventList.class, true);
-
-	private final ByteBuffer tempBuffer = ByteBuffer.allocate(8); // TODO: Make this configurable
-
-	private boolean bufferExistanceDeserialized = false;
-
-	private boolean eventListExistanceDeserialized = false;
-
-	private boolean sequenceNumberDeserializationStarted = false;
-
-	private int sizeOfBuffer = -1;
-
-	private int deserializedSequenceNumber = -1;
-
-	private Buffer buffer = null;
-
-	private JobID deserializedJobID = null;
-
-	private ChannelID deserializedSourceID = null;
-
-	private EventList deserializedEventList = null;
-
-	public void read(ReadableByteChannel readableByteChannel) throws IOException, NoBufferAvailableException {
-
-		while (true) {
-
-			// System.out.println("INCOMING State: " + this.deserializationState);
-
-			boolean waitingForMoreData = false;
-
-			switch (deserializationState) {
-			case NOTDESERIALIZED:
-				waitingForMoreData = readSequenceNumber(readableByteChannel);
-				break;
-			case SEQUENCENUMBERDESERIALIZED:
-				waitingForMoreData = readID(readableByteChannel);
-				break;
-			case JOBIDDESERIALIZED:
-				waitingForMoreData = readID(readableByteChannel);
-				break;
-			case SOURCEDESERIALIZED:
-				waitingForMoreData = readNotificationList(readableByteChannel);
-				break;
-			case NOTIFICATIONSDESERIALIZED:
-				waitingForMoreData = readBuffer(readableByteChannel);
-				break;
-			case FULLYDESERIALIZED:
-				return;
-			}
-
-			if (waitingForMoreData) {
-				return;
-			}
-
-		}
-	}
-
-	protected final ByteBuffer getTempBuffer() {
-		return this.tempBuffer;
-	}
-
-	protected void setBuffer(final Buffer buffer) {
-		this.buffer = buffer;
-	}
-
-	protected int getSizeOfBuffer() {
-		return this.sizeOfBuffer;
-	}
-
-	protected JobID getDeserializedJobID() {
-		return this.deserializedJobID;
-	}
-
-	protected ChannelID getDeserializedSourceID() {
-		return this.deserializedSourceID;
-	}
-
-	private boolean readSequenceNumber(ReadableByteChannel readableByteChannel) throws IOException {
-
-		if (!this.sequenceNumberDeserializationStarted) {
-			this.tempBuffer.position(0);
-			this.tempBuffer.limit(SIZEOFINT);
-			this.sequenceNumberDeserializationStarted = true;
-		}
-
-		if (readableByteChannel.read(this.tempBuffer) == -1) {
-
-			if (this.tempBuffer.position() == 0) {
-				// Regular end of stream
-				throw new EOFException();
-			} else {
-				throw new IOException("Unexpected end of stream while deserializing the sequence number");
-			}
-		}
-
-		if (!this.tempBuffer.hasRemaining()) {
-
-			this.deserializedSequenceNumber = byteBufferToInteger(this.tempBuffer, 0);
-			if (this.deserializedSequenceNumber < 0) {
-				throw new IOException("Received invalid sequence number: " + this.deserializedSequenceNumber);
-			}
-
-			this.deserializationState = DeserializationState.SEQUENCENUMBERDESERIALIZED;
-			this.sequenceNumberDeserializationStarted = false;
-			this.transferEnvelope = null;
-			this.sizeOfBuffer = -1;
-			this.bufferExistanceDeserialized = false;
-			this.eventListExistanceDeserialized = false;
-			this.tempBuffer.clear();
-			this.buffer = null;
-			this.jobIDDeserializationBuffer.clear();
-			this.channelIDDeserializationBuffer.clear();
-			this.deserializedEventList = null;
-			return false;
-		}
-
-		return true;
-	}
-
-	private boolean readID(ReadableByteChannel readableByteChannel) throws IOException {
-
-		if (this.deserializationState == DeserializationState.SEQUENCENUMBERDESERIALIZED) {
-
-			this.deserializedJobID = this.jobIDDeserializationBuffer.readData(null, readableByteChannel);
-			if (this.deserializedJobID == null) {
-				return true;
-			}
-
-			this.deserializationState = DeserializationState.JOBIDDESERIALIZED;
-
-		} else {
-
-			this.deserializedSourceID = this.channelIDDeserializationBuffer.readData(null, readableByteChannel);
-			if (this.deserializedSourceID == null) {
-				return true;
-			}
-
-			this.deserializationState = DeserializationState.SOURCEDESERIALIZED;
-		}
-
-		return false;
-	}
-
-	private boolean readNotificationList(ReadableByteChannel readableByteChannel) throws IOException {
-
-		if (!this.eventListExistanceDeserialized) {
-			this.tempBuffer.position(0);
-			this.tempBuffer.limit(1);
-			readableByteChannel.read(this.tempBuffer);
-
-			if (this.tempBuffer.hasRemaining()) {
-				return true;
-			}
-
-			this.eventListExistanceDeserialized = true;
-			final boolean eventListFollows = (this.tempBuffer.get(0) == (byte) 1);
-			this.tempBuffer.clear();
-
-			if (!eventListFollows) {
-				// No event list here
-				this.transferEnvelope = new TransferEnvelope(this.deserializedSequenceNumber, this.deserializedJobID,
-					this.deserializedSourceID, this.deserializedEventList);
-				this.deserializationState = DeserializationState.NOTIFICATIONSDESERIALIZED;
-				return false;
-			}
-		}
-
-		this.deserializedEventList = this.notificationListDeserializationBuffer.readData(null, readableByteChannel);
-		if (this.deserializedEventList == null) {
-			return true;
-		} else {
-			this.transferEnvelope = new TransferEnvelope(this.deserializedSequenceNumber, this.deserializedJobID,
-				this.deserializedSourceID, this.deserializedEventList);
-			this.deserializationState = DeserializationState.NOTIFICATIONSDESERIALIZED;
-			return false;
-		}
-	}
-
-	/**
-	 * Read the buffer's actual data from the stream.
-	 * 
-	 * @param readableByteChannel
-	 *        the stream to read the buffer data from
-	 * @return <code>true</code> if more buffer data need to be read from the stream, <code>false</code> otherwise
-	 * @throws IOException
-	 *         thrown if an I/O error occurred while reading data from the stream
-	 * @throws NoBufferAvailableException
-	 *         thrown if the deserialization process could not be continued due to a lack of buffers
-	 */
-	protected abstract boolean readBufferData(ReadableByteChannel readableByteChannel) throws IOException,
-			NoBufferAvailableException;
-
-	private boolean readBuffer(final ReadableByteChannel readableByteChannel) throws IOException,
-			NoBufferAvailableException {
-
-		if (!this.bufferExistanceDeserialized) {
-
-			this.tempBuffer.position(0);
-			this.tempBuffer.limit(1);
-
-			final int bytesRead = readableByteChannel.read(this.tempBuffer);
-			if (bytesRead == -1) {
-				if (this.tempBuffer.get(0) == 0 && this.tempBuffer.position() == 1) { // Regular end, no
-					// buffer will follow
-					throw new EOFException();
-				} else {
-					throw new IOException("Deserialization error: Expected at least "
-						+ this.tempBuffer.remaining() + " more bytes to follow");
-				}
-			} else if (bytesRead == 0) {
-				try {
-					Thread.sleep(50);
-				} catch (InterruptedException e) {
-				}
-			}
-
-			if (!this.tempBuffer.hasRemaining()) {
-				this.bufferExistanceDeserialized = true;
-				this.tempBuffer.position(0);
-				this.tempBuffer.limit(SIZEOFINT);
-				if (this.tempBuffer.get(0) == 0) {
-					// No buffer will follow, we are done
-					this.transferEnvelope.setBuffer(null);
-					this.deserializationState = DeserializationState.FULLYDESERIALIZED;
-					return false;
-				}
-			} else {
-				return true;
-			}
-		}
-
-		if (this.sizeOfBuffer < 0) {
-
-			// We need to deserialize the size of the buffer
-			final int bytesRead = readableByteChannel.read(this.tempBuffer);
-			if (bytesRead == -1) {
-				throw new IOException("Deserialization error: Expected at least " + this.tempBuffer.remaining()
-					+ " more bytes to follow");
-			}
-
-			if (!this.tempBuffer.hasRemaining()) {
-				this.sizeOfBuffer = byteBufferToInteger(this.tempBuffer, 0);
-				// System.out.println("INCOMING: Buffer size is " + this.sizeOfBuffer);
-
-				if (this.sizeOfBuffer <= 0) {
-					throw new IOException("Invalid buffer size: " + this.sizeOfBuffer);
-				}
-			} else {
-				return true;
-			}
-		}
-
-		if (readBufferData(readableByteChannel)) {
-			return true;
-		}
-
-		this.transferEnvelope.setBuffer(this.buffer);
-		this.deserializationState = DeserializationState.FULLYDESERIALIZED;
-		return false;
-	}
-
-	public TransferEnvelope getFullyDeserializedTransferEnvelope() {
-
-		if (this.deserializationState == DeserializationState.FULLYDESERIALIZED) {
-			this.deserializationState = DeserializationState.NOTDESERIALIZED;
-			return this.transferEnvelope;
-		}
-
-		return null;
-	}
-
-	public Buffer getBuffer() {
-		return this.buffer;
-	}
-
-	public void reset() {
-		this.deserializationState = DeserializationState.NOTDESERIALIZED;
-		this.sequenceNumberDeserializationStarted = false;
-	}
-
-	public boolean hasUnfinishedData() {
-
-		if (this.deserializationState != DeserializationState.NOTDESERIALIZED) {
-			return true;
-		}
-
-		return this.channelIDDeserializationBuffer.hasUnfinishedData();
-	}
-
-	private int byteBufferToInteger(ByteBuffer byteBuffer, int offset) throws IOException {
-
-		int integer = 0;
-
-		if ((offset + SIZEOFINT) > byteBuffer.limit()) {
-			throw new IOException("Cannot convert byte buffer to integer, not enough data in byte buffer ("
-				+ byteBuffer.limit() + ")");
-		}
-
-		for (int i = 0; i < SIZEOFINT; ++i) {
-			integer |= (byteBuffer.get((offset + SIZEOFINT - 1) - i) & 0xff) << (i << 3);
-		}
-
-		return integer;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/AbstractSerializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/AbstractSerializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/AbstractSerializer.java
deleted file mode 100644
index bdbc592..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/AbstractSerializer.java
+++ /dev/null
@@ -1,274 +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.taskmanager.transferenvelope;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.WritableByteChannel;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.event.task.EventList;
-import eu.stratosphere.nephele.io.AbstractID;
-import eu.stratosphere.nephele.io.channels.Buffer;
-import eu.stratosphere.nephele.io.channels.SerializationBuffer;
-
-public abstract class AbstractSerializer {
-
-	private enum SerializationState {
-		NOTSERIALIZED,
-		SEQUENCENUMBERSERIALIZED,
-		JOBIDSERIALIZED,
-		SOURCESERIALIZED,
-		NOTIFICATIONSSERIALIZED,
-		FULLYSERIALIZED
-	};
-
-	private final static int SIZEOFINT = 4;
-
-	private TransferEnvelope transferEnvelope = null;
-
-	private SerializationState serializationState;
-
-	private final SerializationBuffer<IOReadableWritable> serializationBuffer = new SerializationBuffer<IOReadableWritable>();
-
-	private final ByteBuffer tempBuffer = ByteBuffer.allocate(64); // TODO: Make this configurable
-
-	private boolean serializationStarted = false;
-
-	private boolean bufferExistanceSerialized = false;
-
-	private boolean eventListExistanceSerialized = false;
-
-	public final void setTransferEnvelope(TransferEnvelope transferEnvelope) {
-
-		this.transferEnvelope = transferEnvelope;
-		reset();
-	}
-
-	protected final SerializationBuffer<IOReadableWritable> getSerializationBuffer() {
-
-		return this.serializationBuffer;
-	}
-
-	protected final ByteBuffer getTempBuffer() {
-		return this.tempBuffer;
-	}
-
-	public final boolean write(WritableByteChannel writableByteChannel) throws IOException {
-
-		while (true) {
-
-			boolean moreDataFollows = false;
-
-			// System.out.println("OUTGOING State: " + this.serializationState);
-
-			switch (serializationState) {
-			case NOTSERIALIZED:
-				moreDataFollows = writeSequenceNumber(writableByteChannel, this.transferEnvelope.getSequenceNumber());
-				break;
-			case SEQUENCENUMBERSERIALIZED:
-				moreDataFollows = writeID(writableByteChannel, this.transferEnvelope.getJobID());
-				break;
-			case JOBIDSERIALIZED:
-				moreDataFollows = writeID(writableByteChannel, this.transferEnvelope.getSource());
-				break;
-			case SOURCESERIALIZED:
-				moreDataFollows = writeNotification(writableByteChannel, this.transferEnvelope.getEventList());
-				break;
-			case NOTIFICATIONSSERIALIZED:
-				moreDataFollows = writeBuffer(writableByteChannel, this.transferEnvelope.getBuffer());
-				break;
-			case FULLYSERIALIZED:
-				return false;
-			}
-
-			if (moreDataFollows) {
-				return true;
-			}
-		}
-	}
-
-	private boolean writeSequenceNumber(WritableByteChannel writableByteChannel, int sequenceNumber) throws IOException {
-
-		if (sequenceNumber < 0) {
-			throw new IOException("Invalid sequence number: " + sequenceNumber);
-		}
-
-		if (!this.serializationStarted) {
-			this.tempBuffer.clear();
-			integerToByteBuffer(sequenceNumber, 0, this.tempBuffer);
-			this.serializationStarted = true;
-		}
-
-		if (writableByteChannel.write(this.tempBuffer) == -1) {
-			throw new IOException("Unexpected end of stream while serializing the sequence number");
-		}
-
-		if (!this.tempBuffer.hasRemaining()) {
-			this.serializationState = SerializationState.SEQUENCENUMBERSERIALIZED;
-			this.serializationStarted = false;
-			return false;
-		}
-
-		return true;
-	}
-
-	private boolean writeID(WritableByteChannel writableByteChannel, AbstractID id) throws IOException {
-
-		if (!writeIOReadableWritable(writableByteChannel, id)) {
-			// We're done, all the data has been written to the channel
-			if (this.serializationState == SerializationState.SEQUENCENUMBERSERIALIZED) {
-				// System.out.println("OUTGOING Serialized source: " + channelID);
-				this.serializationState = SerializationState.JOBIDSERIALIZED;
-			} else {
-				// System.out.println("OUTGOING Serialized target: " + channelID);
-				this.serializationState = SerializationState.SOURCESERIALIZED;
-			}
-			return false;
-		}
-
-		return true;
-	}
-
-	private boolean writeIOReadableWritable(WritableByteChannel writableByteChannel,
-			IOReadableWritable ioReadableWritable) throws IOException {
-
-		if (!this.serializationStarted) {
-			this.serializationBuffer.clear();
-			this.serializationBuffer.serialize(ioReadableWritable);
-			this.serializationStarted = true;
-		}
-
-		if (this.serializationBuffer.dataLeftFromPreviousSerialization()) {
-			this.serializationBuffer.read(writableByteChannel);
-		} else {
-			this.serializationStarted = false;
-			return false;
-		}
-
-		return true;
-	}
-
-	private boolean writeNotification(WritableByteChannel writableByteChannel, EventList notificationList)
-			throws IOException {
-
-		if (!this.eventListExistanceSerialized) {
-			this.tempBuffer.position(0);
-			if (notificationList == null) {
-				this.tempBuffer.put(0, (byte) 0);
-			} else {
-				this.tempBuffer.put(0, (byte) 1);
-			}
-			this.tempBuffer.limit(1);
-
-			writableByteChannel.write(this.tempBuffer);
-			if (this.tempBuffer.hasRemaining()) {
-				return true;
-			}
-
-			this.eventListExistanceSerialized = true;
-		}
-
-		if (notificationList != null) {
-			if (writeIOReadableWritable(writableByteChannel, notificationList)) {
-				return true;
-			}
-		}
-
-		this.serializationState = SerializationState.NOTIFICATIONSSERIALIZED;
-		return false;
-	}
-
-	public void reset() {
-		this.serializationState = SerializationState.NOTSERIALIZED;
-		this.serializationStarted = false;
-		this.bufferExistanceSerialized = false;
-		this.eventListExistanceSerialized = false;
-	}
-
-	private boolean writeBuffer(WritableByteChannel writableByteChannel, Buffer buffer) throws IOException {
-
-		while (true) {
-
-			if (!this.bufferExistanceSerialized) {
-
-				if (!this.serializationStarted) {
-					this.tempBuffer.position(0);
-
-					if (buffer == null) {
-						this.tempBuffer.put(0, (byte) 0);
-						this.tempBuffer.limit(1);
-					} else {
-						this.tempBuffer.put(0, (byte) 1);
-						// System.out.println("OUTGOING: Buffer size is " + buffer.size());
-						integerToByteBuffer(buffer.size(), 1, this.tempBuffer);
-					}
-					this.serializationStarted = true;
-				}
-
-				if (this.tempBuffer.hasRemaining()) {
-					if (writableByteChannel.write(tempBuffer) == 0) {
-						return true;
-					}
-				} else {
-					this.bufferExistanceSerialized = true;
-					this.serializationStarted = false;
-					if (buffer == null) {
-						// That's it, we're done. No buffer will follow
-						this.serializationState = SerializationState.FULLYSERIALIZED;
-						return false;
-					}
-				}
-
-			} else {
-
-				if (!writeBufferData(writableByteChannel, buffer)) {
-					this.serializationState = SerializationState.FULLYSERIALIZED;
-					return false;
-				}
-
-				return true;
-			}
-		}
-	}
-
-	/**
-	 * Writes the buffer's actual data.
-	 * 
-	 * @param writableByteChannel
-	 *        the channel to write the buffer data to
-	 * @param buffer
-	 *        the buffer whose data shall be written
-	 * @return <code>true</code> if the buffer has more data to be written, <code>false</code> otherwise
-	 * @throws IOException
-	 *         thrown if an I/O error occurs while writing the buffer's data
-	 */
-	protected abstract boolean writeBufferData(WritableByteChannel writableByteChannel, Buffer buffer)
-			throws IOException;
-
-	private void integerToByteBuffer(int integerToSerialize, int offset, ByteBuffer byteBuffer) throws IOException {
-
-		if ((offset + SIZEOFINT) > byteBuffer.capacity()) {
-			throw new IOException("Cannot convert integer to byte buffer, buffer is too small (" + byteBuffer.limit()
-				+ ", required " + (offset + SIZEOFINT) + ")");
-		}
-
-		byteBuffer.limit(offset + SIZEOFINT);
-
-		for (int i = 0; i < SIZEOFINT; ++i) {
-			final int shift = i << 3; // i * 8
-			byteBuffer.put((offset + SIZEOFINT - 1) - i, (byte) ((integerToSerialize & (0xff << shift)) >>> shift));
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/CapacityConstrainedArrayQueue.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/CapacityConstrainedArrayQueue.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/CapacityConstrainedArrayQueue.java
deleted file mode 100644
index 9e9ac66..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/CapacityConstrainedArrayQueue.java
+++ /dev/null
@@ -1,322 +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.taskmanager.transferenvelope;
-
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-import java.util.Queue;
-
-/**
- * This class implements a simple, capacity-constrained queue. The implementation is highlighted by its small
- * overhead in terms of memory consumption and management.
- * <p>
- * This class is not thread-safe.
- * 
- * @param <E>
- *        the type of the elements stored in this queue
- */
-public final class CapacityConstrainedArrayQueue<E> implements Queue<E> {
-
-	/**
-	 * The maximum capacity supported by this type of queue.
-	 */
-	private static final int MAX_CAPACITY = 128;
-
-	/**
-	 * The array storing the actual elements of the queue.
-	 */
-	private final E[] elements;
-
-	/**
-	 * Index to the current head of the queue.
-	 */
-	private byte head = 0;
-
-	/**
-	 * Index to the current tail of the queue.
-	 */
-	private byte tail = 0;
-
-	/**
-	 * The current size of the queue.
-	 */
-	private byte size = 0;
-
-	/**
-	 * Constructs a new capacity-constrained array queue.
-	 * 
-	 * @param capacity
-	 *        the capacity limit of the queue
-	 */
-	@SuppressWarnings("unchecked")
-	public CapacityConstrainedArrayQueue(final int capacity) {
-
-		if (capacity > MAX_CAPACITY) {
-			throw new IllegalArgumentException("This queue does only support capacities up to " + MAX_CAPACITY);
-		}
-
-		this.elements = (E[]) new Object[capacity];
-	}
-
-
-	@Override
-	public boolean addAll(final Collection<? extends E> c) {
-
-		throw new UnsupportedOperationException("addAll is not supported on this type of queue");
-	}
-
-	/**
-	 * Checks if there is capacity left in the queue.
-	 * 
-	 * @return <code>true</code> if there is capacity left in the queue, <code>false</code> otherwise
-	 */
-	private boolean capacityLeft() {
-
-		return ((this.elements.length - this.size) > 0);
-	}
-
-	/**
-	 * Increments the head of the queue.
-	 */
-	private void incrementHead() {
-
-		if (++this.head == this.elements.length) {
-			this.head = 0;
-		}
-	}
-
-	/**
-	 * Increments the tail of the queue.
-	 */
-	private void incrementTail() {
-
-		if (++this.tail == this.elements.length) {
-			this.tail = 0;
-		}
-	}
-
-
-	@Override
-	public void clear() {
-
-		this.head = 0;
-		this.tail = 0;
-		this.size = 0;
-	}
-
-
-	@Override
-	public boolean containsAll(final Collection<?> c) {
-
-		throw new UnsupportedOperationException("containsAll is not supported on this type of queue");
-	}
-
-
-	@Override
-	public boolean isEmpty() {
-
-		return (this.size == 0);
-	}
-
-
-	@Override
-	public boolean removeAll(final Collection<?> c) {
-
-		throw new UnsupportedOperationException("removeAll is not supported on this type of queue");
-	}
-
-
-	@Override
-	public boolean retainAll(final Collection<?> c) {
-
-		throw new UnsupportedOperationException("retainAll is not supported on this type of queue");
-	}
-
-
-	@Override
-	public Object[] toArray() {
-
-		throw new UnsupportedOperationException("toArray is not supported on this type of queue");
-	}
-
-
-	@Override
-	public <T> T[] toArray(T[] a) {
-
-		throw new UnsupportedOperationException("toArray is not supported on this type of queue");
-	}
-
-
-	@Override
-	public boolean add(final E arg0) {
-
-		throw new UnsupportedOperationException("add is not supported on this type of queue");
-	}
-
-
-	@Override
-	public boolean contains(final Object arg0) {
-
-		throw new UnsupportedOperationException("contains is not supported on this type of queue");
-	}
-
-
-	@Override
-	public E element() {
-
-		throw new UnsupportedOperationException("element is not supported on this type of queue");
-	}
-
-
-	@Override
-	public Iterator<E> iterator() {
-
-		return new CapacityConstrainedArrayQueueIterator(this.head);
-	}
-
-
-	@Override
-	public boolean offer(final E arg0) {
-
-		if (!capacityLeft()) {
-			return false;
-		}
-
-		this.elements[this.tail] = arg0;
-		incrementTail();
-		++this.size;
-
-		return true;
-	}
-
-
-	@Override
-	public E peek() {
-
-		if (isEmpty()) {
-			return null;
-		}
-
-		return this.elements[this.head];
-	}
-
-
-	@Override
-	public E poll() {
-
-		if (isEmpty()) {
-			return null;
-		}
-
-		final E retVal = this.elements[this.head];
-		incrementHead();
-		--this.size;
-
-		return retVal;
-	}
-
-
-	@Override
-	public E remove() {
-
-		final E retVal = poll();
-		if (retVal == null) {
-			throw new NoSuchElementException();
-		}
-
-		return retVal;
-	}
-
-
-	@Override
-	public boolean remove(final Object arg0) {
-
-		throw new UnsupportedOperationException("remove is not supported on this type of queue");
-	}
-
-
-	@Override
-	public int size() {
-
-		return this.size;
-	}
-
-	/**
-	 * This class implements an iterator for the capacity-constrained array queue.
-	 * 
-	 */
-	private final class CapacityConstrainedArrayQueueIterator implements Iterator<E> {
-
-		/**
-		 * The current position of the iterator.
-		 */
-		private byte pos;
-
-		/**
-		 * Counter how many this the position index has been modified.
-		 */
-		private byte count = 0;
-
-		/**
-		 * Constructs a new capacity-constrained array queue iterator.
-		 * 
-		 * @param startPos
-		 *        the start position of the iterator
-		 */
-		private CapacityConstrainedArrayQueueIterator(final byte startPos) {
-			this.pos = startPos;
-		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public boolean hasNext() {
-
-			if (this.count < size) {
-				return true;
-			}
-
-			return false;
-		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public E next() {
-
-			final E retVal = elements[this.pos];
-
-			if (++this.pos == elements.length) {
-				this.pos = 0;
-			}
-
-			++this.count;
-
-			return retVal;
-		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public void remove() {
-
-			throw new UnsupportedOperationException("remove is not supported by this iterator");
-		}
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultDeserializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultDeserializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultDeserializer.java
deleted file mode 100644
index 4ff6965..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultDeserializer.java
+++ /dev/null
@@ -1,94 +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.taskmanager.transferenvelope;
-
-import java.io.IOException;
-import java.nio.channels.ReadableByteChannel;
-
-import eu.stratosphere.nephele.io.channels.Buffer;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProviderBroker;
-
-public final class DefaultDeserializer extends AbstractDeserializer {
-
-	private final BufferProviderBroker bufferProviderBroker;
-
-	private BufferProvider bufferProvider = null;
-
-	private JobID lastDeserializedJobID = null;
-
-	private ChannelID lastDeserializedSourceID = null;
-
-	public DefaultDeserializer(final BufferProviderBroker bufferProviderBroker) {
-		this.bufferProviderBroker = bufferProviderBroker;
-	}
-
-
-	@Override
-	protected boolean readBufferData(final ReadableByteChannel readableByteChannel) throws IOException,
-			NoBufferAvailableException {
-
-		if (getBuffer() == null) {
-
-			// Find buffer provider for this channel
-			if (!getDeserializedJobID().equals(this.lastDeserializedJobID)
-				|| !getDeserializedSourceID().equals(this.lastDeserializedSourceID)) {
-
-				try {
-					this.bufferProvider = this.bufferProviderBroker.getBufferProvider(getDeserializedJobID(),
-						getDeserializedSourceID());
-				} catch (InterruptedException e) {
-					return true;
-				}
-
-				this.lastDeserializedJobID = getDeserializedJobID();
-				this.lastDeserializedSourceID = getDeserializedSourceID();
-			}
-
-			final Buffer buf = this.bufferProvider.requestEmptyBuffer(getSizeOfBuffer());
-
-			if (buf == null) {
-				throw new NoBufferAvailableException(this.bufferProvider);
-			}
-
-			setBuffer(buf);
-
-		} else {
-
-			final Buffer buffer = getBuffer();
-
-			final int bytesWritten = buffer.write(readableByteChannel);
-
-			if (!buffer.hasRemaining()) {
-				// We are done, the buffer has been fully read
-				buffer.flip();
-				return false;
-			} else {
-				if (bytesWritten == -1) {
-					throw new IOException("Deserialization error: Expected at least " + buffer.remaining()
-						+ " more bytes to follow");
-				}
-			}
-		}
-
-		return true;
-	}
-
-	public BufferProvider getBufferProvider() {
-
-		return this.bufferProvider;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultSerializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultSerializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultSerializer.java
deleted file mode 100644
index 34be592..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultSerializer.java
+++ /dev/null
@@ -1,38 +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.taskmanager.transferenvelope;
-
-import java.io.IOException;
-import java.nio.channels.WritableByteChannel;
-
-import eu.stratosphere.nephele.io.channels.Buffer;
-
-/**
- * This class is the default implementation to serialize a {@link TransferEnvelope} into a byte stream. In case the
- * transfer envelope contains a buffer, this implementation copies the buffer's data into the byte stream.
- * 
- */
-public class DefaultSerializer extends AbstractSerializer {
-
-
-	@Override
-	protected boolean writeBufferData(final WritableByteChannel writableByteChannel, final Buffer buffer)
-			throws IOException {
-
-		buffer.writeTo(writableByteChannel);
-
-		return buffer.hasRemaining();
-	}
-
-}


[02/30] git commit: Fix maven enforcer plugin warnings in Eclipse

Posted by rm...@apache.org.
Fix maven enforcer plugin warnings in Eclipse


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

Branch: refs/heads/master
Commit: 2a9ed093f9b462956af87ef09c1d5f67b89f9e1c
Parents: 2b0baea
Author: StephanEwen <st...@tu-berlin.de>
Authored: Fri Jun 6 15:18:19 2014 +0200
Committer: StephanEwen <st...@tu-berlin.de>
Committed: Fri Jun 6 15:59:22 2014 +0200

----------------------------------------------------------------------
 pom.xml | 15 ++++++++++++++-
 1 file changed, 14 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2a9ed093/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index f473b56..96acd1d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -541,7 +541,20 @@
 										</goals>
 									</pluginExecutionFilter>
 									<action>
-										<ignore></ignore>
+										<ignore />
+									</action>
+								</pluginExecution>
+								<pluginExecution>
+									<pluginExecutionFilter>
+										<groupId>org.apache.maven.plugins</groupId>
+										<artifactId>maven-enforcer-plugin</artifactId>
+										<versionRange>[1.0.0,)</versionRange>
+										<goals>
+											<goal>enforce</goal>
+										</goals>
+									</pluginExecutionFilter>
+									<action>
+										<ignore />
 									</action>
 								</pluginExecution>
 							</pluginExecutions>


[28/30] git commit: Merge cleanups

Posted by rm...@apache.org.
Merge cleanups


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

Branch: refs/heads/master
Commit: 24d00598ab85186dd75f263563273cb3402304c7
Parents: b0f8ba0
Author: StephanEwen <st...@tu-berlin.de>
Authored: Thu Jun 5 15:45:49 2014 +0200
Committer: StephanEwen <st...@tu-berlin.de>
Committed: Sat Jun 7 09:41:22 2014 +0200

----------------------------------------------------------------------
 stratosphere-core/src/main/java/eu/stratosphere/types/Record.java  | 2 +-
 .../java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java | 1 -
 2 files changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/24d00598/stratosphere-core/src/main/java/eu/stratosphere/types/Record.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/types/Record.java b/stratosphere-core/src/main/java/eu/stratosphere/types/Record.java
index 5b06547..73671fa 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/types/Record.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/types/Record.java
@@ -70,7 +70,7 @@ public final class Record implements Value, CopyableValue<Record> {
 	
 	private Value[] writeFields;		// the cache for objects into which the binary representations are read
 	
-	public int binaryLen;				// the length of the contents in the binary buffer that is valid
+	private int binaryLen;				// the length of the contents in the binary buffer that is valid
 	
 	private int numFields;				// the number of fields in the record
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/24d00598/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 29d6853..cc542c0 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
@@ -201,7 +201,6 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 	 * @param inputSplitProvider the input split provider for this environment
 	 * @throws Exception thrown if an error occurs while instantiating the invokable class
 	 */
-	@SuppressWarnings({"unchecked", "rawtypes"})
 	public RuntimeEnvironment(final TaskDeploymentDescriptor tdd,
 							final MemoryManager memoryManager, final IOManager ioManager,
 							final InputSplitProvider inputSplitProvider,


[04/30] Replace custom Java NIO TCP/IP code with Netty 4 library

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/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
new file mode 100644
index 0000000..40aefd3
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoderTest.java
@@ -0,0 +1,857 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.network.netty;
+
+import eu.stratosphere.core.memory.MemorySegment;
+import eu.stratosphere.nephele.event.task.AbstractEvent;
+import eu.stratosphere.nephele.jobgraph.JobID;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.BufferRecycler;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferAvailabilityListener;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider.BufferAvailabilityRegistration;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferProviderBroker;
+import eu.stratosphere.runtime.io.network.Envelope;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.CompositeByteBuf;
+import io.netty.channel.embedded.EmbeddedChannel;
+import junit.framework.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Matchers;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Random;
+
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class InboundEnvelopeDecoderTest {
+
+	@Mock
+	private BufferProvider bufferProvider;
+
+	@Mock
+	private BufferProviderBroker bufferProviderBroker;
+
+	@Before
+	public void initMocks() throws IOException {
+		MockitoAnnotations.initMocks(this);
+	}
+
+	@Test
+	public void testBufferStaging() throws Exception {
+		final InboundEnvelopeDecoder decoder = new InboundEnvelopeDecoder(this.bufferProviderBroker);
+		final EmbeddedChannel ch = new EmbeddedChannel(
+				new OutboundEnvelopeEncoder(),
+				decoder);
+
+		when(this.bufferProviderBroker.getBufferProvider(anyJobId(), anyChannelId()))
+				.thenReturn(this.bufferProvider);
+
+		// --------------------------------------------------------------------
+
+		Envelope[] envelopes = nextEnvelopes(3, true);
+
+		ByteBuf buf = encode(ch, envelopes);
+
+		when(this.bufferProvider.registerBufferAvailabilityListener(Matchers.<BufferAvailabilityListener>anyObject()))
+				.thenReturn(BufferAvailabilityRegistration.REGISTERED);
+
+		Buffer buffer = allocBuffer(envelopes[2].getBuffer().size());
+
+		when(this.bufferProvider.requestBuffer(anyInt()))
+				.thenReturn(null, null, buffer, null);
+
+		// --------------------------------------------------------------------
+
+		// slices: [0] => full envelope, [1] => half envelope, [2] => remaining half + full envelope
+		ByteBuf[] slices = slice(buf,
+				OutboundEnvelopeEncoder.HEADER_SIZE + envelopes[0].getBuffer().size(),
+				OutboundEnvelopeEncoder.HEADER_SIZE + envelopes[1].getBuffer().size() / 2);
+
+		// 1. no buffer available, incoming slice contains all data
+		int refCount = slices[0].refCnt();
+
+		decodeAndVerify(ch, slices[0]);
+
+		Assert.assertEquals(refCount + 1, slices[0].refCnt());
+		Assert.assertFalse(ch.config().isAutoRead());
+
+		// notify of available buffer (=> bufferAvailable() callback does return a buffer
+		// of the current network buffer size; the decoder needs to adjust its size to the
+		// requested size
+		decoder.bufferAvailable(allocBuffer(envelopes[0].getBuffer().size() * 2));
+		ch.runPendingTasks();
+
+		Assert.assertEquals(refCount - 1, slices[0].refCnt());
+		Assert.assertTrue(ch.config().isAutoRead());
+
+		decodeAndVerify(ch, envelopes[0]);
+
+		// 2. no buffer available, incoming slice does NOT contain all data
+		refCount = slices[1].refCnt();
+
+		decodeAndVerify(ch, slices[1]);
+
+		Assert.assertEquals(refCount + 1, slices[1].refCnt());
+		Assert.assertFalse(ch.config().isAutoRead());
+
+		decoder.bufferAvailable(allocBuffer());
+		ch.runPendingTasks();
+
+		Assert.assertEquals(refCount - 1, slices[1].refCnt());
+		Assert.assertTrue(ch.config().isAutoRead());
+
+		decodeAndVerify(ch);
+
+		// 3. buffer available
+		refCount = slices[2].refCnt();
+
+		decodeAndVerify(ch, slices[2], envelopes[1], envelopes[2]);
+
+		Assert.assertEquals(refCount - 1, slices[2].refCnt());
+		Assert.assertTrue(ch.config().isAutoRead());
+
+		Assert.assertEquals(1, buf.refCnt());
+		buf.release();
+	}
+
+	@Test
+	public void testBufferStagingStagedBufferException() throws Exception {
+		final EmbeddedChannel ch = new EmbeddedChannel(
+				new OutboundEnvelopeEncoder(),
+				new InboundEnvelopeDecoder(this.bufferProviderBroker));
+
+		when(this.bufferProviderBroker.getBufferProvider(anyJobId(), anyChannelId()))
+				.thenReturn(this.bufferProvider);
+
+		// --------------------------------------------------------------------
+
+		ByteBuf buf = encode(ch, nextEnvelope(true));
+
+		when(this.bufferProvider.requestBuffer(anyInt()))
+				.thenReturn(null);
+
+		when(this.bufferProvider.registerBufferAvailabilityListener(Matchers.<BufferAvailabilityListener>anyObject()))
+				.thenReturn(BufferAvailabilityRegistration.REGISTERED);
+
+		// --------------------------------------------------------------------
+
+		int refCount = buf.refCnt();
+
+		decodeAndVerify(ch, buf);
+
+		Assert.assertFalse(ch.config().isAutoRead());
+		Assert.assertEquals(refCount + 1, buf.refCnt());
+
+		try {
+			decodeAndVerify(ch, buf);
+			Assert.fail("Expected IllegalStateException not thrown");
+		} catch (IllegalStateException e) {
+			// expected exception
+		}
+
+		buf.release();
+	}
+
+	@Test
+	public void testBufferAvailabilityRegistrationBufferAvailable() throws Exception {
+		final EmbeddedChannel ch = new EmbeddedChannel(
+				new OutboundEnvelopeEncoder(),
+				new InboundEnvelopeDecoder(this.bufferProviderBroker));
+
+		when(this.bufferProviderBroker.getBufferProvider(anyJobId(), anyChannelId()))
+				.thenReturn(this.bufferProvider);
+
+		// --------------------------------------------------------------------
+
+		Envelope[] envelopes = new Envelope[]{nextEnvelope(true), nextEnvelope()};
+
+		when(this.bufferProvider.registerBufferAvailabilityListener(Matchers.<BufferAvailabilityListener>anyObject()))
+				.thenReturn(BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_AVAILABLE);
+
+		when(this.bufferProvider.requestBuffer(anyInt()))
+				.thenReturn(null)
+				.thenReturn(allocBuffer(envelopes[0].getBuffer().size()));
+
+		// --------------------------------------------------------------------
+
+		ByteBuf buf = encode(ch, envelopes);
+
+		decodeAndVerify(ch, buf, envelopes);
+		Assert.assertEquals(0, buf.refCnt());
+	}
+
+	@Test
+	public void testBufferAvailabilityRegistrationBufferPoolDestroyedSkipBytes() throws Exception {
+		final EmbeddedChannel ch = new EmbeddedChannel(
+				new OutboundEnvelopeEncoder(),
+				new InboundEnvelopeDecoder(this.bufferProviderBroker));
+
+		when(this.bufferProviderBroker.getBufferProvider(anyJobId(), anyChannelId()))
+				.thenReturn(this.bufferProvider);
+
+		when(this.bufferProvider.requestBuffer(anyInt()))
+				.thenReturn(null);
+
+		when(this.bufferProvider.registerBufferAvailabilityListener(Matchers.<BufferAvailabilityListener>anyObject()))
+				.thenReturn(BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_POOL_DESTROYED);
+
+		// --------------------------------------------------------------------
+
+		Envelope[] envelopes = new Envelope[]{nextEnvelope(true), nextEnvelope(), nextEnvelope()};
+		Envelope[] expectedEnvelopes = new Envelope[]{envelopes[1], envelopes[2]};
+
+		ByteBuf buf = encode(ch, envelopes);
+
+		int bufferSize = envelopes[0].getBuffer().size();
+
+		// --------------------------------------------------------------------
+		// 1) skip in current buffer only
+		// --------------------------------------------------------------------
+		{
+			// skip last bytes in current buffer
+			ByteBuf[] slices = slice(buf, OutboundEnvelopeEncoder.HEADER_SIZE + bufferSize);
+
+			int refCount = slices[0].refCnt();
+			decodeAndVerify(ch, slices[0]);
+			Assert.assertEquals(refCount - 1, slices[0].refCnt());
+
+			refCount = slices[1].refCnt();
+			decodeAndVerify(ch, slices[1], expectedEnvelopes);
+			Assert.assertEquals(refCount - 1, slices[1].refCnt());
+		}
+
+		{
+			// skip bytes in current buffer, leave last 16 bytes from next envelope
+			ByteBuf[] slices = slice(buf, OutboundEnvelopeEncoder.HEADER_SIZE + bufferSize + 16);
+
+			int refCount = slices[0].refCnt();
+			decodeAndVerify(ch, slices[0]);
+			Assert.assertEquals(refCount - 1, slices[0].refCnt());
+
+			refCount = slices[1].refCnt();
+			decodeAndVerify(ch, slices[1], expectedEnvelopes);
+			Assert.assertEquals(refCount - 1, slices[1].refCnt());
+		}
+
+		{
+			// skip bytes in current buffer, then continue with full envelope from same buffer
+			ByteBuf[] slices = slice(buf, OutboundEnvelopeEncoder.HEADER_SIZE + bufferSize + OutboundEnvelopeEncoder.HEADER_SIZE);
+
+			int refCount = slices[0].refCnt();
+			decodeAndVerify(ch, slices[0], expectedEnvelopes[0]);
+			Assert.assertEquals(refCount - 1, slices[0].refCnt());
+
+			refCount = slices[1].refCnt();
+			decodeAndVerify(ch, slices[1], expectedEnvelopes[1]);
+			Assert.assertEquals(refCount - 1, slices[1].refCnt());
+		}
+
+		// --------------------------------------------------------------------
+		// 2) skip in current and next buffer
+		// --------------------------------------------------------------------
+
+		{
+			// skip bytes in current buffer, then continue to skip last 32 bytes in next buffer
+			ByteBuf[] slices = slice(buf, OutboundEnvelopeEncoder.HEADER_SIZE + bufferSize - 32);
+
+			int refCount = slices[0].refCnt();
+			decodeAndVerify(ch, slices[0]);
+			Assert.assertEquals(refCount - 1, slices[0].refCnt());
+
+			refCount = slices[1].refCnt();
+			decodeAndVerify(ch, slices[1], expectedEnvelopes);
+			Assert.assertEquals(refCount - 1, slices[1].refCnt());
+		}
+
+		{
+			// skip bytes in current buffer, then continue to skip in next two buffers
+			ByteBuf[] slices = slice(buf, OutboundEnvelopeEncoder.HEADER_SIZE + bufferSize - 32, 16);
+
+			int refCount = slices[0].refCnt();
+			decodeAndVerify(ch, slices[0]);
+			Assert.assertEquals(refCount - 1, slices[0].refCnt());
+
+			refCount = slices[1].refCnt();
+			decodeAndVerify(ch, slices[1]);
+			Assert.assertEquals(refCount - 1, slices[1].refCnt());
+
+			refCount = slices[2].refCnt();
+			decodeAndVerify(ch, slices[2], expectedEnvelopes);
+			Assert.assertEquals(refCount - 1, slices[2].refCnt());
+		}
+
+		// ref count should be 1, because slices shared the ref count
+		Assert.assertEquals(1, buf.refCnt());
+	}
+
+	@Test
+	public void testEncodeDecode() throws Exception {
+		final EmbeddedChannel ch = new EmbeddedChannel(
+				new OutboundEnvelopeEncoder(), new InboundEnvelopeDecoder(this.bufferProviderBroker));
+
+		when(this.bufferProviderBroker.getBufferProvider(anyJobId(), anyChannelId()))
+				.thenReturn(this.bufferProvider);
+
+		when(this.bufferProvider.requestBuffer(anyInt())).thenAnswer(new Answer<Object>() {
+			@Override
+			public Object answer(InvocationOnMock invocation) throws Throwable {
+				// fulfill the buffer request
+				return allocBuffer((Integer) invocation.getArguments()[0]);
+			}
+		});
+
+		// --------------------------------------------------------------------
+
+		Envelope[] envelopes = new Envelope[]{
+				nextEnvelope(0),
+				nextEnvelope(2),
+				nextEnvelope(32768),
+				nextEnvelope(3782, new TestEvent1(34872527)),
+				nextEnvelope(88, new TestEvent1(8749653), new TestEvent1(365345)),
+				nextEnvelope(0, new TestEvent2(34563456), new TestEvent1(598432), new TestEvent2(976293845)),
+				nextEnvelope(23)
+		};
+
+		ByteBuf buf = encode(ch, envelopes);
+
+		// 1. complete ByteBuf as input
+		int refCount = buf.retain().refCnt();
+
+		decodeAndVerify(ch, buf, envelopes);
+		Assert.assertEquals(refCount - 1, buf.refCnt());
+
+		// 2. random slices
+		buf.readerIndex(0);
+		ByteBuf[] slices = randomSlices(buf);
+
+		ch.writeInbound(slices);
+
+		for (ByteBuf slice : slices) {
+			Assert.assertEquals(1, slice.refCnt());
+		}
+
+		decodeAndVerify(ch, envelopes);
+
+		buf.release();
+	}
+
+	@Test
+	public void testEncodeDecodeRandomEnvelopes() throws Exception {
+		final InboundEnvelopeDecoder decoder = new InboundEnvelopeDecoder(this.bufferProviderBroker);
+		final EmbeddedChannel ch = new EmbeddedChannel(
+				new OutboundEnvelopeEncoder(), decoder);
+
+		when(this.bufferProviderBroker.getBufferProvider(anyJobId(), anyChannelId()))
+				.thenReturn(this.bufferProvider);
+
+		when(this.bufferProvider.requestBuffer(anyInt())).thenAnswer(new Answer<Object>() {
+			@Override
+			public Object answer(InvocationOnMock invocation) throws Throwable {
+				// fulfill the buffer request with the requested size
+				return allocBuffer((Integer) invocation.getArguments()[0]);
+			}
+		});
+
+		Random randomAnswerSource = new Random(RANDOM_SEED);
+
+		RandomBufferRequestAnswer randomBufferRequestAnswer = new RandomBufferRequestAnswer(randomAnswerSource);
+
+		RandomBufferAvailabilityRegistrationAnswer randomBufferAvailabilityRegistrationAnswer =
+				new RandomBufferAvailabilityRegistrationAnswer(randomAnswerSource, randomBufferRequestAnswer);
+
+		when(this.bufferProvider.requestBuffer(anyInt())).thenAnswer(randomBufferRequestAnswer);
+
+		when(this.bufferProvider.registerBufferAvailabilityListener(Matchers.<BufferAvailabilityListener>anyObject()))
+				.thenAnswer(randomBufferAvailabilityRegistrationAnswer);
+
+		// --------------------------------------------------------------------
+
+		Envelope[] envelopes = nextRandomEnvelopes(1024);
+
+		ByteBuf buf = encode(ch, envelopes);
+
+		ByteBuf[] slices = randomSlices(buf);
+
+		for (ByteBuf slice : slices) {
+			int refCount = slice.refCnt();
+			ch.writeInbound(slice);
+
+			// registered BufferAvailabilityListener => call bufferAvailable(buffer)
+			while (randomBufferAvailabilityRegistrationAnswer.isRegistered()) {
+				randomBufferAvailabilityRegistrationAnswer.unregister();
+
+				Assert.assertFalse(ch.config().isAutoRead());
+				Assert.assertEquals(refCount + 1, slice.refCnt());
+
+				// return a buffer of max size => decoder needs to limit buffer size
+				decoder.bufferAvailable(allocBuffer(MAX_BUFFER_SIZE));
+				ch.runPendingTasks();
+			}
+
+			Assert.assertEquals(refCount - 1, slice.refCnt());
+			Assert.assertTrue(ch.config().isAutoRead());
+		}
+
+		Envelope[] expected = randomBufferAvailabilityRegistrationAnswer.removeSkippedEnvelopes(envelopes);
+
+		decodeAndVerify(ch, expected);
+
+		Assert.assertEquals(1, buf.refCnt());
+
+		buf.release();
+	}
+
+	// ========================================================================
+	// helpers
+	// ========================================================================
+
+	private final static long RANDOM_SEED = 520346508276087l;
+
+	private final static Random random = new Random(RANDOM_SEED);
+
+	private final static int[] BUFFER_SIZES = new int[]{8192, 16384, 32768};
+
+	private final static int MAX_BUFFER_SIZE = BUFFER_SIZES[2];
+
+	private final static int MAX_NUM_EVENTS = 5;
+
+	private final static int MAX_SLICE_SIZE = MAX_BUFFER_SIZE / 3;
+
+	private final static int MIN_SLICE_SIZE = 1;
+
+	private final static BufferRecycler RECYCLER = mock(BufferRecycler.class);
+
+	// ------------------------------------------------------------------------
+	// envelopes
+	// ------------------------------------------------------------------------
+
+	private static Buffer allocBuffer() {
+		return allocBuffer(MAX_BUFFER_SIZE);
+	}
+
+	private static Buffer allocBuffer(int bufferSize) {
+		return spy(new Buffer(new MemorySegment(new byte[bufferSize]), bufferSize, RECYCLER));
+	}
+
+	private Envelope nextEnvelope() {
+		return nextEnvelope(false, false);
+	}
+
+	private Envelope nextEnvelope(boolean withBuffer) {
+		return nextEnvelope(withBuffer, false);
+	}
+
+	private Envelope nextEnvelope(int bufferSize, AbstractEvent... events) {
+		Envelope env = new Envelope(random.nextInt(), new JobID(), new ChannelID());
+		if (bufferSize > 0) {
+			byte[] data = new byte[bufferSize];
+			random.nextBytes(data);
+
+			env.setBuffer(spy(new Buffer(new MemorySegment(data), bufferSize, RECYCLER)));
+		}
+
+		if (events != null && events.length > 0) {
+			env.serializeEventList(Arrays.asList(events));
+		}
+
+		return env;
+	}
+
+	private Envelope nextEnvelope(boolean withBuffer, boolean withEvents) {
+		int bufferSize = 0;
+		AbstractEvent[] events = null;
+
+		if (withBuffer) {
+			bufferSize = BUFFER_SIZES[random.nextInt(BUFFER_SIZES.length)];
+		}
+
+		if (withEvents) {
+			events = new AbstractEvent[random.nextInt(MAX_NUM_EVENTS) + 1];
+
+			for (int i = 0; i < events.length; i++) {
+				events[i] = (random.nextBoolean()
+						? new TestEvent1(random.nextLong())
+						: new TestEvent2(random.nextLong()));
+			}
+		}
+
+		return nextEnvelope(bufferSize, events);
+	}
+
+	private Envelope[] nextEnvelopes(int numEnvelopes, boolean withBuffer) {
+		Envelope[] envelopes = new Envelope[numEnvelopes];
+		for (int i = 0; i < numEnvelopes; i++) {
+			envelopes[i] = nextEnvelope(withBuffer, false);
+		}
+		return envelopes;
+	}
+
+	private Envelope[] nextRandomEnvelopes(int numEnvelopes) {
+		Envelope[] envelopes = new Envelope[numEnvelopes];
+		for (int i = 0; i < numEnvelopes; i++) {
+			envelopes[i] = nextEnvelope(random.nextBoolean(), random.nextBoolean());
+		}
+		return envelopes;
+	}
+
+	// ------------------------------------------------------------------------
+	// channel encode/decode
+	// ------------------------------------------------------------------------
+
+	private static ByteBuf encode(EmbeddedChannel ch, Envelope... envelopes) {
+		for (Envelope env : envelopes) {
+			ch.writeOutbound(env);
+
+			if (env.getBuffer() != null) {
+				verify(env.getBuffer(), times(1)).recycleBuffer();
+			}
+		}
+
+		CompositeByteBuf encodedEnvelopes = new CompositeByteBuf(ByteBufAllocator.DEFAULT, false, envelopes.length);
+
+		ByteBuf buf;
+		while ((buf = (ByteBuf) ch.readOutbound()) != null) {
+			encodedEnvelopes.addComponent(buf);
+		}
+
+		return encodedEnvelopes.writerIndex(encodedEnvelopes.capacity());
+	}
+
+	private static void decodeAndVerify(EmbeddedChannel ch, ByteBuf buf, Envelope... expectedEnvelopes) {
+		ch.writeInbound(buf);
+
+		decodeAndVerify(ch, expectedEnvelopes);
+	}
+
+	private static void decodeAndVerify(EmbeddedChannel ch, Envelope... expectedEnvelopes) {
+		if (expectedEnvelopes == null) {
+			Assert.assertNull(ch.readInbound());
+		}
+		else {
+			for (Envelope expected : expectedEnvelopes) {
+				Envelope actual = (Envelope) ch.readInbound();
+
+				if (actual == null) {
+					Assert.fail("No inbound envelope available, but expected one");
+				}
+
+				assertEqualEnvelopes(expected, actual);
+			}
+		}
+	}
+
+	private static void assertEqualEnvelopes(Envelope expected, Envelope actual) {
+		Assert.assertTrue(expected.getSequenceNumber() == actual.getSequenceNumber() &&
+				expected.getJobID().equals(actual.getJobID()) &&
+				expected.getSource().equals(actual.getSource()));
+
+		if (expected.getBuffer() == null) {
+			Assert.assertNull(actual.getBuffer());
+		}
+		else {
+			Assert.assertNotNull(actual.getBuffer());
+
+			ByteBuffer expectedByteBuffer = expected.getBuffer().getMemorySegment().wrap(0, expected.getBuffer().size());
+			ByteBuffer actualByteBuffer = actual.getBuffer().getMemorySegment().wrap(0, actual.getBuffer().size());
+
+			Assert.assertEquals(0, expectedByteBuffer.compareTo(actualByteBuffer));
+		}
+
+		if (expected.getEventsSerialized() == null) {
+			Assert.assertNull(actual.getEventsSerialized());
+		}
+		else {
+			Assert.assertNotNull(actual.getEventsSerialized());
+
+			// this is needed, because the encoding of the byte buffer
+			// alters the state of the buffer
+			expected.getEventsSerialized().clear();
+
+			List<? extends AbstractEvent> expectedEvents = expected.deserializeEvents();
+			List<? extends AbstractEvent> actualEvents = actual.deserializeEvents();
+
+			Assert.assertEquals(expectedEvents.size(), actualEvents.size());
+
+			for (int i = 0; i < expectedEvents.size(); i++) {
+				AbstractEvent expectedEvent = expectedEvents.get(i);
+				AbstractEvent actualEvent = actualEvents.get(i);
+
+				Assert.assertEquals(expectedEvent.getClass(), actualEvent.getClass());
+				Assert.assertEquals(expectedEvent, actualEvent);
+			}
+		}
+	}
+
+	private static ByteBuf[] randomSlices(ByteBuf buf) {
+		List<Integer> sliceSizes = new LinkedList<Integer>();
+
+		if (buf.readableBytes() < MIN_SLICE_SIZE) {
+			throw new IllegalStateException("Buffer to slice is smaller than required minimum slice size");
+		}
+
+		int available = buf.readableBytes() - MIN_SLICE_SIZE;
+
+		while (available > 0) {
+			int size = Math.min(available, Math.max(MIN_SLICE_SIZE, random.nextInt(MAX_SLICE_SIZE) + 1));
+			available -= size;
+			sliceSizes.add(size);
+		}
+
+		int[] slices = new int[sliceSizes.size()];
+		for (int i = 0; i < sliceSizes.size(); i++) {
+			slices[i] = sliceSizes.get(i);
+		}
+
+		return slice(buf, slices);
+	}
+
+	/**
+	 * Returns slices with the specified sizes of the given buffer.
+	 * <p/>
+	 * When given n indexes, n+1 slices will be returned:
+	 * <ul>
+	 * <li>0 - sliceSizes[0]</li>
+	 * <li>sliceSizes[0] - sliceSizes[1]</li>
+	 * <li>...</li>
+	 * <li>sliceSizes[n-1] - buf.capacity()</li>
+	 * </ul>
+	 *
+	 * @return slices with the specified sizes of the given buffer
+	 */
+	private static ByteBuf[] slice(ByteBuf buf, int... sliceSizes) {
+		if (sliceSizes.length == 0) {
+			throw new IllegalStateException("Need to provide at least one slice size");
+		}
+
+		int numSlices = sliceSizes.length;
+		// transform slice sizes to buffer indexes
+		for (int i = 1; i < numSlices; i++) {
+			sliceSizes[i] += sliceSizes[i - 1];
+		}
+
+		for (int i = 0; i < sliceSizes.length - 1; i++) {
+			if (sliceSizes[i] >= sliceSizes[i + 1] || sliceSizes[i] <= 0 || sliceSizes[i] >= buf.capacity()) {
+				throw new IllegalStateException(
+						String.format("Slice size %s are off for %s", Arrays.toString(sliceSizes), buf));
+			}
+		}
+
+		ByteBuf[] slices = new ByteBuf[numSlices + 1];
+
+		// slice at slice indexes
+		slices[0] = buf.slice(0, sliceSizes[0]).retain();
+		for (int i = 1; i < numSlices; i++) {
+			slices[i] = buf.slice(sliceSizes[i - 1], sliceSizes[i] - sliceSizes[i - 1]).retain();
+		}
+		slices[numSlices] = buf.slice(sliceSizes[numSlices - 1], buf.capacity() - sliceSizes[numSlices - 1]).retain();
+
+		return slices;
+	}
+
+	// ------------------------------------------------------------------------
+	// mocking
+	// ------------------------------------------------------------------------
+
+	private static JobID anyJobId() {
+		return Matchers.anyObject();
+	}
+
+	private static ChannelID anyChannelId() {
+		return Matchers.anyObject();
+	}
+
+	// these following two Answer classes are quite ugly, but they allow to implement a randomized
+	// test of encoding and decoding envelopes
+	private static class RandomBufferRequestAnswer implements Answer<Buffer> {
+
+		private final Random random;
+
+		private boolean forced;
+
+		private RandomBufferRequestAnswer(Random random) {
+			this.random = random;
+		}
+
+		@Override
+		public Buffer answer(InvocationOnMock invocation) throws Throwable {
+			if (this.forced) {
+				Buffer toReturn = allocBuffer((Integer) invocation.getArguments()[0]);
+				this.forced = false;
+
+				return toReturn;
+			}
+
+			return this.random.nextBoolean() ? allocBuffer((Integer) invocation.getArguments()[0]) : null;
+		}
+
+		public void forceBufferAvailable() {
+			this.forced = true;
+		}
+	}
+
+	private static class RandomBufferAvailabilityRegistrationAnswer implements Answer<BufferAvailabilityRegistration> {
+
+		private final Random random;
+
+		private final RandomBufferRequestAnswer bufferRequestAnswer;
+
+		private boolean isRegistered = false;
+
+		private int numSkipped;
+
+		private RandomBufferAvailabilityRegistrationAnswer(Random random, RandomBufferRequestAnswer bufferRequestAnswer) {
+			this.random = random;
+			this.bufferRequestAnswer = bufferRequestAnswer;
+		}
+
+		@Override
+		public BufferAvailabilityRegistration answer(InvocationOnMock invocation) throws Throwable {
+			if (this.random.nextBoolean()) {
+				this.isRegistered = true;
+				return BufferAvailabilityRegistration.REGISTERED;
+			}
+			else if (this.random.nextBoolean()) {
+				this.bufferRequestAnswer.forceBufferAvailable();
+				return BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_AVAILABLE;
+			}
+			else {
+				this.numSkipped++;
+				return BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_POOL_DESTROYED;
+			}
+		}
+
+		public Envelope[] removeSkippedEnvelopes(Envelope[] envelopes) {
+			this.random.setSeed(RANDOM_SEED);
+			Envelope[] envelopesWithoutSkipped = new Envelope[envelopes.length - this.numSkipped];
+			int numEnvelopes = 0;
+
+			for (Envelope env : envelopes) {
+				if (env.getBuffer() != null) {
+					// skip envelope if returned NOT_REGISTERED_BUFFER_POOL_DESTROYED
+					if (!this.random.nextBoolean() && !this.random.nextBoolean() && !this.random.nextBoolean()) {
+						continue;
+					}
+				}
+
+				envelopesWithoutSkipped[numEnvelopes++] = env;
+			}
+
+			return envelopesWithoutSkipped;
+		}
+
+		public boolean isRegistered() {
+			return this.isRegistered;
+		}
+
+		public void unregister() {
+			this.isRegistered = false;
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	public static final class TestEvent1 extends AbstractEvent {
+
+		private long id;
+
+		public TestEvent1() {
+		}
+
+		public TestEvent1(long id) {
+			this.id = id;
+		}
+
+		@Override
+		public void write(DataOutput out) throws IOException {
+			out.writeLong(id);
+		}
+
+		@Override
+		public void read(DataInput in) throws IOException {
+			id = in.readLong();
+		}
+
+
+		@Override
+		public boolean equals(Object obj) {
+			return obj.getClass() == TestEvent1.class && ((TestEvent1) obj).id == this.id;
+		}
+
+		@Override
+		public int hashCode() {
+			return ((int) id) ^ ((int) (id >>> 32));
+		}
+
+		@Override
+		public String toString() {
+			return "TestEvent1 (" + id + ")";
+		}
+	}
+
+	public static final class TestEvent2 extends AbstractEvent {
+
+		private long id;
+
+		public TestEvent2() {
+		}
+
+		public TestEvent2(long id) {
+			this.id = id;
+		}
+
+		@Override
+		public void write(DataOutput out) throws IOException {
+			out.writeLong(id);
+		}
+
+		@Override
+		public void read(DataInput in) throws IOException {
+			id = in.readLong();
+		}
+
+		@Override
+		public boolean equals(Object obj) {
+			return obj.getClass() == TestEvent2.class && ((TestEvent2) obj).id == this.id;
+		}
+
+		@Override
+		public int hashCode() {
+			return ((int) id) ^ ((int) (id >>> 32));
+		}
+
+		@Override
+		public String toString() {
+			return "TestEvent2 (" + id + ")";
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManagerTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManagerTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManagerTest.java
new file mode 100644
index 0000000..c424a1f
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManagerTest.java
@@ -0,0 +1,196 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.network.netty;
+
+import eu.stratosphere.nephele.jobgraph.JobID;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.network.ChannelManager;
+import eu.stratosphere.runtime.io.network.Envelope;
+import eu.stratosphere.runtime.io.network.RemoteReceiver;
+import junit.framework.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.mockito.Matchers;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Random;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CountDownLatch;
+
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+
+@RunWith(Parameterized.class)
+public class NettyConnectionManagerTest {
+
+	private final static long RANDOM_SEED = 520346508276087l;
+
+	private final static Random random = new Random(RANDOM_SEED);
+
+	private final static int BIND_PORT = 20000;
+
+	private final static int HIGH_WATERMARK = 32 * 1024;
+
+	private int numSubtasks;
+
+	private int numToSendPerSubtask;
+
+	private int numInThreads;
+
+	private int numOutThreads;
+
+	private int numChannels;
+
+	public NettyConnectionManagerTest(int numSubtasks, int numToSendPerSubtask, int numChannels, int numInThreads, int numOutThreads) {
+		this.numSubtasks = numSubtasks;
+		this.numToSendPerSubtask = numToSendPerSubtask;
+		this.numChannels = numChannels;
+		this.numInThreads = numInThreads;
+		this.numOutThreads = numOutThreads;
+	}
+
+	@Parameterized.Parameters
+	public static Collection configure() {
+		return Arrays.asList(
+				new Object[][]{
+						{64, 4096, 1, 1, 1},
+						{128, 2048, 1, 1, 1},
+						{256, 1024, 1, 1, 1},
+						{512, 512, 1, 1, 1},
+						{64, 4096, 4, 1, 1},
+						{128, 2048, 4, 1, 1},
+						{256, 1024, 4, 1, 1},
+						{512, 512, 4, 1, 1},
+						{64, 4096, 4, 2, 2},
+						{128, 2048, 4, 2, 2},
+						{256, 1024, 4, 2, 2},
+						{512, 512, 4, 2, 2}
+				}
+		);
+	}
+
+	@Test
+	public void testEnqueueRaceAndDeadlockFreeMultipleChannels() throws Exception {
+		final InetAddress localhost = InetAddress.getLocalHost();
+		final CountDownLatch latch = new CountDownLatch(this.numSubtasks);
+
+		// --------------------------------------------------------------------
+		// setup
+		// --------------------------------------------------------------------
+		ChannelManager channelManager = mock(ChannelManager.class);
+		doAnswer(new VerifyEnvelopes(latch)).when(channelManager).dispatchFromNetwork(Matchers.<Envelope>anyObject());
+
+		NettyConnectionManager connManagerToTest = new NettyConnectionManager(channelManager, localhost,
+				BIND_PORT, HIGH_WATERMARK, this.numInThreads, this.numOutThreads, -1, -1);
+
+		NettyConnectionManager connManagerReceiver = new NettyConnectionManager(channelManager, localhost,
+				BIND_PORT + 1, HIGH_WATERMARK, this.numInThreads, this.numOutThreads, -1, -1);
+
+		// --------------------------------------------------------------------
+		// start sender threads
+		// --------------------------------------------------------------------
+		RemoteReceiver[] receivers = new RemoteReceiver[this.numChannels];
+
+		for (int i = 0; i < this.numChannels; i++) {
+			receivers[i] = new RemoteReceiver(new InetSocketAddress(localhost, BIND_PORT + 1), i);
+		}
+
+		for (int i = 0; i < this.numSubtasks; i++) {
+			RemoteReceiver receiver = receivers[random.nextInt(this.numChannels)];
+			new Thread(new SubtaskSenderThread(connManagerToTest, receiver)).start();
+		}
+
+		latch.await();
+
+		connManagerToTest.shutdown();
+		connManagerReceiver.shutdown();
+	}
+
+
+	private class VerifyEnvelopes implements Answer {
+
+		private final ConcurrentMap<ChannelID, Integer> received = new ConcurrentHashMap<ChannelID, Integer>();
+
+		private final CountDownLatch latch;
+
+		private VerifyEnvelopes(CountDownLatch latch) {
+			this.latch = latch;
+		}
+
+		@Override
+		public Object answer(InvocationOnMock invocation) throws Throwable {
+			Envelope env = (Envelope) invocation.getArguments()[0];
+
+			ChannelID channelId = env.getSource();
+			int seqNum = env.getSequenceNumber();
+
+			if (seqNum == 0) {
+				Assert.assertNull(
+						String.format("Received envelope from %s before, but current seq num is 0", channelId),
+						this.received.putIfAbsent(channelId, seqNum));
+			}
+			else {
+				Assert.assertTrue(
+						String.format("Received seq num %d from %s, but previous was not %d", seqNum, channelId, seqNum - 1),
+						this.received.replace(channelId, seqNum - 1, seqNum));
+			}
+
+			// count down the latch if all envelopes received for this source
+			if (seqNum == numToSendPerSubtask - 1) {
+				this.latch.countDown();
+			}
+
+			return null;
+		}
+	}
+
+	private class SubtaskSenderThread implements Runnable {
+
+		private final NettyConnectionManager connectionManager;
+
+		private final RemoteReceiver receiver;
+
+		private final JobID jobId = new JobID();
+
+		private final ChannelID channelId = new ChannelID();
+
+		private int seqNum = 0;
+
+		private SubtaskSenderThread(NettyConnectionManager connectionManager, RemoteReceiver receiver) {
+			this.connectionManager = connectionManager;
+			this.receiver = receiver;
+		}
+
+		@Override
+		public void run() {
+			// enqueue envelopes with ascending seq nums
+			while (this.seqNum < numToSendPerSubtask) {
+				try {
+					Envelope env = new Envelope(this.seqNum++, this.jobId, this.channelId);
+					this.connectionManager.enqueue(env, receiver);
+				} catch (IOException e) {
+					throw new RuntimeException("Unexpected exception while enqueing envelope");
+				}
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/OutboundEnvelopeEncoderTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/OutboundEnvelopeEncoderTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/OutboundEnvelopeEncoderTest.java
new file mode 100644
index 0000000..ba4edc7
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/OutboundEnvelopeEncoderTest.java
@@ -0,0 +1,97 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.network.netty;
+
+import eu.stratosphere.core.memory.MemorySegment;
+import eu.stratosphere.nephele.jobgraph.JobID;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.network.Envelope;
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.embedded.EmbeddedChannel;
+import junit.framework.Assert;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.util.Random;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class OutboundEnvelopeEncoderTest {
+
+	private final long RANDOM_SEED = 520346508276087l;
+
+	private final Random random = new Random(RANDOM_SEED);
+
+	private static final int NUM_RANDOM_ENVELOPES = 512;
+
+	private static final int MAX_EVENTS_SIZE = 1024;
+
+	private static final int MAX_BUFFER_SIZE = 32768;
+
+	@Test
+	public void testEncodedSizeAndBufferRecycling() {
+		final ByteBuffer events = ByteBuffer.allocate(MAX_EVENTS_SIZE);
+		final MemorySegment segment = new MemorySegment(new byte[MAX_BUFFER_SIZE]);
+
+		final Buffer buffer = mock(Buffer.class);
+		when(buffer.getMemorySegment()).thenReturn(segment);
+
+		final EmbeddedChannel channel = new EmbeddedChannel(new OutboundEnvelopeEncoder());
+
+		int numBuffers = 0;
+		for (int i = 0; i < NUM_RANDOM_ENVELOPES; i++) {
+			Envelope env = new Envelope(i, new JobID(), new ChannelID());
+			int expectedEncodedMsgSize = OutboundEnvelopeEncoder.HEADER_SIZE;
+
+			if (random.nextBoolean()) {
+				int eventsSize = random.nextInt(MAX_EVENTS_SIZE + 1);
+				expectedEncodedMsgSize += eventsSize;
+
+				events.clear();
+				events.limit(eventsSize);
+
+				env.setEventsSerialized(events);
+			}
+
+			if (random.nextBoolean()) {
+				numBuffers++;
+
+				int bufferSize = random.nextInt(MAX_BUFFER_SIZE + 1);
+				when(buffer.size()).thenReturn(bufferSize);
+				env.setBuffer(buffer);
+
+				expectedEncodedMsgSize += bufferSize;
+			}
+
+			Assert.assertTrue(channel.writeOutbound(env));
+
+			// --------------------------------------------------------------------
+			// verify encoded ByteBuf size
+			// --------------------------------------------------------------------
+			ByteBuf encodedMsg = (ByteBuf) channel.readOutbound();
+			Assert.assertEquals(expectedEncodedMsgSize, encodedMsg.readableBytes());
+
+			encodedMsg.release();
+		}
+
+		// --------------------------------------------------------------------
+		// verify buffers are recycled
+		// --------------------------------------------------------------------
+		verify(buffer, times(numBuffers)).recycleBuffer();
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/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 c9323a6..8cda32f 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
@@ -14,6 +14,7 @@
  **********************************************************************************************************************/
 package eu.stratosphere.test.broadcastvars;
 
+import eu.stratosphere.nephele.jobgraph.DistributionPattern;
 import org.apache.log4j.Level;
 
 import eu.stratosphere.api.common.operators.util.UserCodeObjectWrapper;
@@ -22,8 +23,6 @@ import eu.stratosphere.api.common.typeutils.TypeSerializerFactory;
 import eu.stratosphere.api.java.record.io.CsvInputFormat;
 import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.core.fs.Path;
-import eu.stratosphere.nephele.io.DistributionPattern;
-import eu.stratosphere.nephele.io.channels.ChannelType;
 import eu.stratosphere.nephele.jobgraph.JobGraph;
 import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException;
 import eu.stratosphere.nephele.jobgraph.JobInputVertex;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/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 802fd0b..984ecc2 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,9 +23,8 @@ import eu.stratosphere.client.LocalExecutor;
 import eu.stratosphere.test.recordJobs.wordcount.WordCount;
 import eu.stratosphere.test.testdata.WordCountData;
 
-
 public class LocalExecutorITCase {
-	
+
 	@Test
 	public void testLocalExecutorWithWordCount() {
 		try {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/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 2eaa8a9..4a60836 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
@@ -20,7 +20,7 @@ import eu.stratosphere.test.util.JavaProgramTestBase;
 
 
 public class WordCountITCase extends JavaProgramTestBase {
-	
+
 	protected String textPath;
 	protected String resultPath;
 
@@ -28,7 +28,6 @@ public class WordCountITCase extends JavaProgramTestBase {
 		setNumTaskManager(2);
 	}
 
-	
 	@Override
 	protected void preSubmit() throws Exception {
 		textPath = createTempFile("text.txt", WordCountData.TEXT);
@@ -39,7 +38,7 @@ public class WordCountITCase extends JavaProgramTestBase {
 	protected void postSubmit() throws Exception {
 		compareResultsByLinesInMemory(WordCountData.COUNTS, resultPath);
 	}
-	
+
 	@Override
 	protected void testProgram() throws Exception {
 		WordCount.main(new String[] { textPath, resultPath });

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/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 0a63af1..40068b7 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
@@ -16,6 +16,8 @@ package eu.stratosphere.test.iterative.nephele;
 import java.io.BufferedReader;
 import java.util.Collection;
 
+import eu.stratosphere.nephele.jobgraph.DistributionPattern;
+import eu.stratosphere.runtime.io.channels.ChannelType;
 import eu.stratosphere.test.util.RecordAPITestBase;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -31,8 +33,6 @@ import eu.stratosphere.api.java.record.io.CsvInputFormat;
 import eu.stratosphere.api.java.record.io.CsvOutputFormat;
 import eu.stratosphere.api.java.record.io.FileOutputFormat;
 import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.nephele.io.DistributionPattern;
-import eu.stratosphere.nephele.io.channels.ChannelType;
 import eu.stratosphere.nephele.jobgraph.JobGraph;
 import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException;
 import eu.stratosphere.nephele.jobgraph.JobInputVertex;
@@ -425,11 +425,11 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		JobGraphUtils.connect(head, intermediate, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
 		intermediateConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, numSubTasks);
 
-		JobGraphUtils.connect(intermediate, tail, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+		JobGraphUtils.connect(intermediate, tail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
 		tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
 
-		JobGraphUtils.connect(head, output, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
-		JobGraphUtils.connect(tail, fakeTail, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+		JobGraphUtils.connect(head, output, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
+		JobGraphUtils.connect(tail, fakeTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
 
 		JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE);
 
@@ -567,16 +567,16 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		JobGraphUtils.connect(intermediate, ssJoinIntermediate, ChannelType.NETWORK, DistributionPattern.POINTWISE);
 		ssJoinIntermediateConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
 
-		JobGraphUtils.connect(ssJoinIntermediate, ssTail, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+		JobGraphUtils.connect(ssJoinIntermediate, ssTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
 		ssTailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
 
-		JobGraphUtils.connect(ssJoinIntermediate, wsTail, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+		JobGraphUtils.connect(ssJoinIntermediate, wsTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
 		wsTailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
 
-		JobGraphUtils.connect(head, output, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+		JobGraphUtils.connect(head, output, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
 
-		JobGraphUtils.connect(ssTail, ssFakeTail, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
-		JobGraphUtils.connect(wsTail, wsFakeTail, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+		JobGraphUtils.connect(ssTail, ssFakeTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
+		JobGraphUtils.connect(wsTail, wsFakeTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
 
 		JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE);
 
@@ -695,12 +695,12 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 			DistributionPattern.POINTWISE);
 		wsUpdateConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
 
-		JobGraphUtils.connect(wsUpdateIntermediate, ssTail, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+		JobGraphUtils.connect(wsUpdateIntermediate, ssTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
 		ssTailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
 
-		JobGraphUtils.connect(head, output, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+		JobGraphUtils.connect(head, output, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
 
-		JobGraphUtils.connect(ssTail, fakeTail, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+		JobGraphUtils.connect(ssTail, fakeTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
 
 		JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE);
 
@@ -815,12 +815,12 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		JobGraphUtils.connect(intermediate, ssJoinIntermediate, ChannelType.NETWORK, DistributionPattern.POINTWISE);
 		ssJoinIntermediateConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
 
-		JobGraphUtils.connect(ssJoinIntermediate, wsTail, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+		JobGraphUtils.connect(ssJoinIntermediate, wsTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
 		wsTailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
 
-		JobGraphUtils.connect(head, output, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+		JobGraphUtils.connect(head, output, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
 
-		JobGraphUtils.connect(wsTail, fakeTail, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+		JobGraphUtils.connect(wsTail, fakeTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
 
 		JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE);
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/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 ef83aef..ef7c9d2 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
@@ -15,6 +15,8 @@ package eu.stratosphere.test.iterative.nephele;
 import java.util.Collection;
 import java.util.Iterator;
 
+import eu.stratosphere.nephele.jobgraph.DistributionPattern;
+import eu.stratosphere.runtime.io.channels.ChannelType;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
@@ -25,8 +27,6 @@ import eu.stratosphere.api.java.record.functions.MapFunction;
 import eu.stratosphere.api.java.record.functions.ReduceFunction;
 import eu.stratosphere.api.java.record.io.FileOutputFormat;
 import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.nephele.io.DistributionPattern;
-import eu.stratosphere.nephele.io.channels.ChannelType;
 import eu.stratosphere.nephele.jobgraph.JobGraph;
 import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException;
 import eu.stratosphere.nephele.jobgraph.JobInputVertex;
@@ -232,16 +232,16 @@ public class IterationWithChainingNepheleITCase extends RecordAPITestBase {
 		// --------------------------------------------------------------------------------------------------------------
 		// 2. EDGES
 		// --------------------------------------------------------------------------------------------------------------
-		JobGraphUtils.connect(input, head, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+		JobGraphUtils.connect(input, head, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
 
-		JobGraphUtils.connect(head, tail, ChannelType.INMEMORY, DistributionPattern.BIPARTITE);
+		JobGraphUtils.connect(head, tail, ChannelType.IN_MEMORY, DistributionPattern.BIPARTITE);
 		tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, numSubTasks);
 
-		JobGraphUtils.connect(head, output, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+		JobGraphUtils.connect(head, output, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
 
 		JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE);
 
-		JobGraphUtils.connect(tail, fakeTail, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+		JobGraphUtils.connect(tail, fakeTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
 
 		// --------------------------------------------------------------------------------------------------------------
 		// 3. INSTANCE SHARING

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/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 624fd9b..30ce102 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
@@ -22,8 +22,6 @@ import eu.stratosphere.api.java.record.io.CsvOutputFormat;
 import eu.stratosphere.api.java.record.io.TextInputFormat;
 import eu.stratosphere.api.java.record.operators.MapOperator;
 import eu.stratosphere.api.java.record.operators.ReduceOperator;
-import eu.stratosphere.nephele.io.MutableUnionRecordReader;
-import eu.stratosphere.nephele.io.UnionRecordReader;
 import eu.stratosphere.test.recordJobs.wordcount.WordCount.CountWords;
 import eu.stratosphere.test.recordJobs.wordcount.WordCount.TokenizeLine;
 import eu.stratosphere.test.testdata.WordCountData;
@@ -39,8 +37,6 @@ import eu.stratosphere.types.StringValue;
  * 
  * @see {@link https://github.com/stratosphere/stratosphere/issues/192}
  * @see {@link https://github.com/stratosphere/stratosphere/issues/124}
- * @see {@link UnionRecordReader}
- * @see {@link MutableUnionRecordReader}
  */
 public class WordCountUnionReduceITCase extends RecordAPITestBase {
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackNepheleITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackNepheleITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackNepheleITCase.java
new file mode 100644
index 0000000..f5beda4
--- /dev/null
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackNepheleITCase.java
@@ -0,0 +1,286 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.runtime;
+
+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.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 NetworkStackNepheleITCase extends RecordAPITestBase {
+
+	private static final Log LOG = LogFactory.getLog(NetworkStackNepheleITCase.class);
+
+	private static final String DATA_VOLUME_GB_CONFIG_KEY = "data.volume.gb";
+
+	private static final String USE_FORWARDER_CONFIG_KEY = "use.forwarder";
+
+	private static final String NUM_SUBTASKS_CONFIG_KEY = "num.subtasks";
+
+	private static final String NUM_SUBTASKS_PER_INSTANCE_CONFIG_KEY = "num.subtasks.instance";
+
+	private static final String IS_SLOW_SENDER_CONFIG_KEY = "is.slow.sender";
+
+	private static final String IS_SLOW_RECEIVER_CONFIG_KEY = "is.slow.receiver";
+
+	private static final int IS_SLOW_SLEEP_MS = 10;
+
+	private static final int IS_SLOW_EVERY_NUM_RECORDS = (2 * 32 * 1024) / SpeedTestRecord.RECORD_SIZE;
+
+	// ------------------------------------------------------------------------
+
+	public NetworkStackNepheleITCase(Configuration config) {
+		super(config);
+
+		setNumTaskManager(2);
+		LogUtils.initializeDefaultConsoleLogger();
+	}
+
+	@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},
+		};
+
+		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(NUM_SUBTASKS_CONFIG_KEY, (Integer) p[4]);
+			config.setInteger(NUM_SUBTASKS_PER_INSTANCE_CONFIG_KEY, (Integer) p[5]);
+
+			configs.add(config);
+		}
+
+		return toParameterList(configs);
+	}
+
+	// ------------------------------------------------------------------------
+
+	@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);
+	}
+
+	@After
+	public void calculateThroughput() {
+		if (getJobExecutionResult() != null) {
+			int dataVolumeGb = this.config.getInteger(DATA_VOLUME_GB_CONFIG_KEY, 1);
+
+			double dataVolumeMbit = dataVolumeGb * 8192.0;
+			double runtimeSecs = getJobExecutionResult().getNetRuntime() / 1000.0;
+
+			int mbitPerSecond = (int) Math.round(dataVolumeMbit / runtimeSecs);
+
+			LOG.info(String.format("Test finished with throughput of %d MBit/s (" +
+					"runtime [secs]: %.2f, data volume [mbits]: %.2f)", mbitPerSecond, runtimeSecs, dataVolumeMbit));
+		}
+	}
+
+	private JobGraph createJobGraph(int dataVolumeGb, boolean useForwarder, boolean isSlowSender, boolean isSlowReceiver,
+									int numSubtasks, int numSubtasksPerInstance) 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);
+
+		JobTaskVertex forwarder = null;
+		if (useForwarder) {
+			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) {
+			producer.connectTo(forwarder, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
+			forwarder.connectTo(consumer, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
+
+			producer.setVertexToShareInstancesWith(forwarder);
+			forwarder.setVertexToShareInstancesWith(consumer);
+		}
+		else {
+			producer.connectTo(consumer, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
+			producer.setVertexToShareInstancesWith(consumer);
+		}
+
+		return jobGraph;
+	}
+
+	// ------------------------------------------------------------------------
+
+	public static class SpeedTestProducer extends AbstractGenericInputTask {
+
+		private RecordWriter<SpeedTestRecord> writer;
+
+		@Override
+		public void registerInputOutput() {
+			this.writer = new RecordWriter<SpeedTestRecord>(this);
+		}
+
+		@Override
+		public void invoke() throws Exception {
+			this.writer.initializeSerializers();
+
+			// Determine the amount of data to send per subtask
+			int dataVolumeGb = getTaskConfiguration().getInteger(NetworkStackNepheleITCase.DATA_VOLUME_GB_CONFIG_KEY, 1);
+
+			long dataMbPerSubtask = (dataVolumeGb * 1024) / getCurrentNumberOfSubtasks();
+			long numRecordsToEmit = (dataMbPerSubtask * 1024 * 1024) / SpeedTestRecord.RECORD_SIZE;
+
+			LOG.info(String.format("%d/%d: Producing %d records (each record: %d bytes, total: %.2f GB)",
+					getIndexInSubtaskGroup() + 1, getCurrentNumberOfSubtasks(), numRecordsToEmit,
+					SpeedTestRecord.RECORD_SIZE, dataMbPerSubtask/1024.0));
+
+			boolean isSlow = getTaskConfiguration().getBoolean(IS_SLOW_SENDER_CONFIG_KEY, false);
+
+			int numRecords = 0;
+			SpeedTestRecord record = new SpeedTestRecord();
+			for (long i = 0; i < numRecordsToEmit; i++) {
+				if (isSlow && (numRecords++ % IS_SLOW_EVERY_NUM_RECORDS) == 0) {
+					Thread.sleep(IS_SLOW_SLEEP_MS);
+				}
+
+				this.writer.emit(record);
+			}
+
+			this.writer.flush();
+		}
+	}
+
+	public static class SpeedTestForwarder extends AbstractTask {
+
+		private RecordReader<SpeedTestRecord> reader;
+
+		private RecordWriter<SpeedTestRecord> writer;
+
+		@Override
+		public void registerInputOutput() {
+			this.reader = new RecordReader<SpeedTestRecord>(this, SpeedTestRecord.class);
+			this.writer = new RecordWriter<SpeedTestRecord>(this);
+		}
+
+		@Override
+		public void invoke() throws Exception {
+			this.writer.initializeSerializers();
+
+			SpeedTestRecord record;
+			while ((record = this.reader.next()) != null) {
+				this.writer.emit(record);
+			}
+
+			this.writer.flush();
+		}
+	}
+
+	public static class SpeedTestConsumer extends AbstractOutputTask {
+
+		private RecordReader<SpeedTestRecord> reader;
+
+		@Override
+		public void registerInputOutput() {
+			this.reader = new RecordReader<SpeedTestRecord>(this, SpeedTestRecord.class);
+		}
+
+		@Override
+		public void invoke() throws Exception {
+			boolean isSlow = getTaskConfiguration().getBoolean(IS_SLOW_RECEIVER_CONFIG_KEY, false);
+
+			int numRecords = 0;
+			while (this.reader.next() != null) {
+				if (isSlow && (numRecords++ % IS_SLOW_EVERY_NUM_RECORDS) == 0) {
+					Thread.sleep(IS_SLOW_SLEEP_MS);
+				}
+			}
+		}
+	}
+
+	public static class SpeedTestRecord implements IOReadableWritable {
+
+		private static final int RECORD_SIZE = 128;
+
+		private final byte[] buf = new byte[RECORD_SIZE];
+
+		public SpeedTestRecord() {
+			for (int i = 0; i < RECORD_SIZE; ++i) {
+				this.buf[i] = (byte) (i % 128);
+			}
+		}
+
+		@Override
+		public void write(DataOutput out) throws IOException {
+			out.write(this.buf);
+		}
+
+		@Override
+		public void read(DataInput in) throws IOException {
+			in.readFully(this.buf);
+		}
+	}
+}


[27/30] git commit: Rename BufferAvailabilityRegistration constants and add initial LocalBufferPoolTest

Posted by rm...@apache.org.
Rename BufferAvailabilityRegistration constants and add initial LocalBufferPoolTest


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

Branch: refs/heads/master
Commit: 6a40d41ac213772b11267203797ed8c83a937392
Parents: 3beca62
Author: uce <u....@fu-berlin.de>
Authored: Tue Jun 3 14:59:16 2014 +0200
Committer: StephanEwen <st...@tu-berlin.de>
Committed: Sat Jun 7 09:41:22 2014 +0200

----------------------------------------------------------------------
 .../network/bufferprovider/BufferProvider.java  |  16 +-
 .../bufferprovider/DiscardBufferPool.java       |   2 +-
 .../network/bufferprovider/LocalBufferPool.java |  12 +-
 .../network/netty/InboundEnvelopeDecoder.java   |   6 +-
 .../pact/runtime/test/util/MockEnvironment.java |   2 +-
 .../bufferprovider/LocalBufferPoolTest.java     | 360 +++++++++++++++++++
 .../netty/InboundEnvelopeDecoderTest.java       |  16 +-
 7 files changed, 381 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6a40d41a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProvider.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProvider.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProvider.java
index d82b427..25c8a7e 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProvider.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProvider.java
@@ -68,18 +68,8 @@ public interface BufferProvider {
 	BufferAvailabilityRegistration registerBufferAvailabilityListener(BufferAvailabilityListener listener);
 
 	public enum BufferAvailabilityRegistration {
-		NOT_REGISTERED_BUFFER_AVAILABLE(false),
-		NOT_REGISTERED_BUFFER_POOL_DESTROYED(false),
-		REGISTERED(true);
-
-		private final boolean isSuccessful;
-
-		private BufferAvailabilityRegistration(boolean isSuccessful) {
-			this.isSuccessful = isSuccessful;
-		}
-
-		public boolean isSuccessful() {
-			return isSuccessful;
-		}
+		SUCCEEDED_REGISTERED(),
+		FAILED_BUFFER_AVAILABLE(),
+		FAILED_BUFFER_POOL_DESTROYED()
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6a40d41a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/DiscardBufferPool.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/DiscardBufferPool.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/DiscardBufferPool.java
index 5daa509..9b3d434 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/DiscardBufferPool.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/DiscardBufferPool.java
@@ -41,7 +41,7 @@ public final class DiscardBufferPool implements BufferProvider, BufferRecycler {
 
 	@Override
 	public BufferAvailabilityRegistration registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
-		return BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_POOL_DESTROYED;
+		return BufferAvailabilityRegistration.FAILED_BUFFER_POOL_DESTROYED;
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6a40d41a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java
index 6285726..933df76 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java
@@ -187,17 +187,17 @@ public final class LocalBufferPool implements BufferProvider {
 	public BufferAvailabilityRegistration registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
 		synchronized (this.buffers) {
 			if (!this.buffers.isEmpty()) {
-				return BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_AVAILABLE;
+				return BufferAvailabilityRegistration.FAILED_BUFFER_AVAILABLE;
 			}
 
 			if (this.isDestroyed) {
-				return BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_POOL_DESTROYED;
+				return BufferAvailabilityRegistration.FAILED_BUFFER_POOL_DESTROYED;
 			}
 
 			this.listeners.add(listener);
 		}
 
-		return BufferAvailabilityRegistration.REGISTERED;
+		return BufferAvailabilityRegistration.SUCCEEDED_REGISTERED;
 	}
 
 	/**
@@ -300,10 +300,7 @@ public final class LocalBufferPool implements BufferProvider {
 					this.globalBufferPool.returnBuffer(buffer);
 					this.numRequestedBuffers--;
 
-					return;
-				}
-
-				if (!this.listeners.isEmpty()) {
+				} else if (!this.listeners.isEmpty()) {
 					Buffer availableBuffer = new Buffer(buffer, buffer.size(), this.recycler);
 					try {
 						this.listeners.poll().bufferAvailable(availableBuffer);
@@ -311,6 +308,7 @@ public final class LocalBufferPool implements BufferProvider {
 						this.buffers.add(buffer);
 						this.buffers.notify();
 					}
+
 				} else {
 					this.buffers.add(buffer);
 					this.buffers.notify();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6a40d41a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoder.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoder.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoder.java
index 54f4617..1135577 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoder.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoder.java
@@ -124,7 +124,7 @@ public class InboundEnvelopeDecoder extends ChannelInboundHandlerAdapter impleme
 			}
 			else if (decoderState == DecoderState.NO_BUFFER_AVAILABLE) {
 				switch (this.currentBufferProvider.registerBufferAvailabilityListener(this)) {
-					case REGISTERED:
+					case SUCCEEDED_REGISTERED:
 						if (ctx.channel().config().isAutoRead()) {
 							ctx.channel().config().setAutoRead(false);
 
@@ -137,10 +137,10 @@ public class InboundEnvelopeDecoder extends ChannelInboundHandlerAdapter impleme
 						this.stagedBuffer.retain();
 						return false;
 
-					case NOT_REGISTERED_BUFFER_AVAILABLE:
+					case FAILED_BUFFER_AVAILABLE:
 						continue;
 
-					case NOT_REGISTERED_BUFFER_POOL_DESTROYED:
+					case FAILED_BUFFER_POOL_DESTROYED:
 						this.bytesToSkip = skipBytes(in, this.currentBufferRequestSize);
 
 						this.currentBufferRequestSize = 0;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6a40d41a/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 b715a4e..2585a74 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
@@ -127,7 +127,7 @@ public class MockEnvironment implements Environment, BufferProvider, LocalBuffer
 
 	@Override
 	public BufferAvailabilityRegistration registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
-		return BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_POOL_DESTROYED;
+		return BufferAvailabilityRegistration.FAILED_BUFFER_POOL_DESTROYED;
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6a40d41a/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
new file mode 100644
index 0000000..c7d8d41
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPoolTest.java
@@ -0,0 +1,360 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.network.bufferprovider;
+
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider.BufferAvailabilityRegistration;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.Matchers;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.IOException;
+import java.util.Timer;
+import java.util.TimerTask;
+
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+public class LocalBufferPoolTest {
+
+	private final static int NUM_BUFFERS = 2048;
+
+	private final static int BUFFER_SIZE = 1024;
+
+	private final static GlobalBufferPool GLOBAL_BUFFER_POOL = new GlobalBufferPool(NUM_BUFFERS, BUFFER_SIZE);
+
+	private final static RecyclingBufferAvailableAnswer RECYCLING_BUFFER_AVAILABLE_ANSWER = new RecyclingBufferAvailableAnswer();
+
+	@BeforeClass
+	public static void setupGlobalBufferPoolOnce() {
+		Assert.assertEquals("GlobalBufferPool does not have required number of buffers.",
+				NUM_BUFFERS, GLOBAL_BUFFER_POOL.numBuffers());
+		Assert.assertEquals("GlobalBufferPool does not have required number of available buffers.",
+				NUM_BUFFERS, GLOBAL_BUFFER_POOL.numAvailableBuffers());
+	}
+
+	@After
+	public void verifyAllBuffersReturnedToGlobalBufferPool() {
+		Assert.assertEquals("Did not return all buffers to GlobalBufferPool after test.",
+				NUM_BUFFERS, GLOBAL_BUFFER_POOL.numAvailableBuffers());
+	}
+
+	@Test
+	public void testSingleConsumerNonBlockingRequestAndRecycle() throws IOException {
+		final LocalBufferPool bufferPool = new LocalBufferPool(GLOBAL_BUFFER_POOL, NUM_BUFFERS);
+
+		Assert.assertEquals(0, bufferPool.numRequestedBuffers());
+
+		// this request-recycle cycle should only take a single buffer out of
+		// the GlobalBufferPool as it is recycled over and over again
+		for (int numRequested = 0; numRequested < NUM_BUFFERS; numRequested++) {
+			Buffer buffer = bufferPool.requestBuffer(BUFFER_SIZE);
+
+			Assert.assertEquals(BUFFER_SIZE, buffer.size());
+
+			Assert.assertEquals("Expected single buffer request in buffer pool.",
+					1, bufferPool.numRequestedBuffers());
+			Assert.assertEquals("Expected no available buffer in buffer pool.",
+					0, bufferPool.numAvailableBuffers());
+
+			buffer.recycleBuffer();
+
+			Assert.assertEquals("Expected single available buffer after recycle.",
+					1, bufferPool.numAvailableBuffers());
+		}
+
+		bufferPool.destroy();
+	}
+
+	@Test
+	public void testSingleConsumerNonBlockingRequestMoreThanAvailable() throws IOException {
+		final LocalBufferPool bufferPool = new LocalBufferPool(GLOBAL_BUFFER_POOL, NUM_BUFFERS);
+
+		Assert.assertEquals(0, bufferPool.numRequestedBuffers());
+
+		// request all buffers from the buffer pool
+		Buffer[] requestedBuffers = new Buffer[NUM_BUFFERS];
+		for (int i = 0; i < NUM_BUFFERS; i++) {
+			requestedBuffers[i] = bufferPool.requestBuffer(BUFFER_SIZE);
+		}
+
+		Assert.assertEquals("Expected no available buffer in buffer pool.",
+				0, bufferPool.numAvailableBuffers());
+
+		Assert.assertNull("Expected null return value for buffer request with no available buffer.",
+				bufferPool.requestBuffer(BUFFER_SIZE));
+
+		// recycle all buffers and destroy buffer pool
+		for (Buffer buffer : requestedBuffers) {
+			buffer.recycleBuffer();
+		}
+
+		bufferPool.destroy();
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testSingleConsumerNonBlockingRequestTooLarge() throws IOException {
+		final LocalBufferPool bufferPool = new LocalBufferPool(GLOBAL_BUFFER_POOL, NUM_BUFFERS);
+
+		// request too large buffer for the pool
+		bufferPool.requestBuffer(BUFFER_SIZE * 2);
+	}
+
+	@Test
+	public void testSingleConsumerNonBlockingRequestSmall() throws IOException {
+		final LocalBufferPool bufferPool = new LocalBufferPool(GLOBAL_BUFFER_POOL, NUM_BUFFERS);
+
+		// request smaller buffer and verify size
+		Buffer buffer = bufferPool.requestBuffer(BUFFER_SIZE / 2);
+
+		Assert.assertEquals(BUFFER_SIZE / 2, buffer.size());
+
+		buffer.recycleBuffer();
+
+		bufferPool.destroy();
+	}
+
+	@Test
+	public void testSingleConsumerBlockingRequest() throws Exception {
+		final LocalBufferPool bufferPool = new LocalBufferPool(GLOBAL_BUFFER_POOL, NUM_BUFFERS);
+
+		final Buffer[] requestedBuffers = new Buffer[NUM_BUFFERS];
+		for (int i = 0; i < NUM_BUFFERS; i++) {
+			requestedBuffers[i] = bufferPool.requestBuffer(BUFFER_SIZE);
+		}
+
+		final Buffer[] bufferFromBlockingRequest = new Buffer[1];
+
+		// --------------------------------------------------------------------
+		// 1. blocking call: interrupt thread
+		// --------------------------------------------------------------------
+		Assert.assertEquals(NUM_BUFFERS, bufferPool.numRequestedBuffers());
+		Assert.assertEquals(0, bufferPool.numAvailableBuffers());
+
+		Thread blockingBufferRequestThread = new Thread(new Runnable() {
+			@Override
+			public void run() {
+				try {
+					bufferFromBlockingRequest[0] = bufferPool.requestBufferBlocking(BUFFER_SIZE);
+					Assert.fail("Unexpected return from blocking buffer request.");
+				} catch (IOException e) {
+					Assert.fail("Unexpected IOException during test.");
+				} catch (InterruptedException e) {
+					// expected interruption
+				}
+			}
+		});
+
+		// start blocking request thread, sleep, interrupt blocking request thread
+		blockingBufferRequestThread.start();
+
+		Thread.sleep(500);
+
+		blockingBufferRequestThread.interrupt();
+
+		Assert.assertNull(bufferFromBlockingRequest[0]);
+		Assert.assertEquals(NUM_BUFFERS, bufferPool.numRequestedBuffers());
+		Assert.assertEquals(0, bufferPool.numAvailableBuffers());
+
+		// --------------------------------------------------------------------
+		// 2. blocking call: recycle buffer in different thread
+		// --------------------------------------------------------------------
+		// recycle the buffer soon
+		new Timer().schedule(new TimerTask() {
+			@Override
+			public void run() {
+				requestedBuffers[0].recycleBuffer();
+			}
+		}, 500);
+
+		//
+		try {
+			Buffer buffer = bufferPool.requestBufferBlocking(BUFFER_SIZE);
+			Assert.assertNotNull(buffer);
+
+			buffer.recycleBuffer();
+		} catch (InterruptedException e) {
+			Assert.fail("Unexpected InterruptedException during test.");
+		}
+
+		// recycle remaining buffers
+		for (int i = 1; i < requestedBuffers.length; i++) {
+			requestedBuffers[i].recycleBuffer();
+		}
+
+		bufferPool.destroy();
+	}
+
+	@Test
+	public void testSingleConsumerRecycleAfterDestroy() throws IOException {
+		final LocalBufferPool bufferPool = new LocalBufferPool(GLOBAL_BUFFER_POOL, NUM_BUFFERS);
+
+		Buffer[] requestedBuffers = new Buffer[NUM_BUFFERS];
+		for (int i = 0; i < NUM_BUFFERS; i++) {
+			requestedBuffers[i] = bufferPool.requestBuffer(BUFFER_SIZE);
+		}
+
+		bufferPool.destroy();
+
+		// recycle should return buffers to GlobalBufferPool
+		// => verified in verifyAllBuffersReturned()
+		for (Buffer buffer : requestedBuffers) {
+			buffer.recycleBuffer();
+		}
+	}
+
+	@Test
+	public void testSingleConsumerBufferAvailabilityListenerRegistration() throws Exception {
+		final LocalBufferPool bufferPool = new LocalBufferPool(GLOBAL_BUFFER_POOL, NUM_BUFFERS);
+
+		BufferAvailabilityListener listener = mock(BufferAvailabilityListener.class);
+
+		// recycle buffer when listener mock is called back
+		doAnswer(RECYCLING_BUFFER_AVAILABLE_ANSWER).when(listener).bufferAvailable(Matchers.<Buffer>anyObject());
+
+		// request all buffers of the pool
+		Buffer[] requestedBuffers = new Buffer[NUM_BUFFERS];
+		for (int i = 0; i < NUM_BUFFERS; i++) {
+			requestedBuffers[i] = bufferPool.requestBuffer(BUFFER_SIZE);
+		}
+
+		BufferAvailabilityRegistration registration;
+		// --------------------------------------------------------------------
+		// 1. success
+		// --------------------------------------------------------------------
+		registration = bufferPool.registerBufferAvailabilityListener(listener);
+		Assert.assertEquals(BufferAvailabilityRegistration.SUCCEEDED_REGISTERED, registration);
+
+		// verify call to buffer listener after recycle
+		requestedBuffers[0].recycleBuffer();
+		verify(listener, times(1)).bufferAvailable(Matchers.<Buffer>anyObject());
+
+		Assert.assertEquals("Expected single available buffer after recycle call in mock listener.",
+				1, bufferPool.numAvailableBuffers());
+
+		// --------------------------------------------------------------------
+		// 2. failure: buffer is available
+		// --------------------------------------------------------------------
+		registration = bufferPool.registerBufferAvailabilityListener(listener);
+		Assert.assertEquals(BufferAvailabilityRegistration.FAILED_BUFFER_AVAILABLE, registration);
+
+		Buffer buffer = bufferPool.requestBuffer(BUFFER_SIZE);
+		Assert.assertNotNull(buffer);
+
+		buffer.recycleBuffer();
+
+		// --------------------------------------------------------------------
+		// 3. failure: buffer pool destroyed
+		// --------------------------------------------------------------------
+		bufferPool.destroy();
+
+		registration = bufferPool.registerBufferAvailabilityListener(listener);
+		Assert.assertEquals(BufferAvailabilityRegistration.FAILED_BUFFER_POOL_DESTROYED, registration);
+
+		// recycle remaining buffers
+		for (int i = 1; i < requestedBuffers.length; i++) {
+			requestedBuffers[i].recycleBuffer();
+		}
+	}
+
+	@Test
+	public void testSingleConsumerReturnExcessBuffers() throws Exception {
+		final LocalBufferPool bufferPool = new LocalBufferPool(GLOBAL_BUFFER_POOL, NUM_BUFFERS);
+
+		// request all buffers of the pool
+		Buffer[] requestedBuffers = new Buffer[NUM_BUFFERS];
+		for (int i = 0; i < NUM_BUFFERS; i++) {
+			requestedBuffers[i] = bufferPool.requestBuffer(BUFFER_SIZE);
+		}
+
+		Assert.assertEquals(NUM_BUFFERS, bufferPool.numRequestedBuffers());
+		Assert.assertEquals(0, bufferPool.numAvailableBuffers());
+
+		// recycle first half of the buffers
+		// => leave requested number of buffers unchanged
+		// => increase available number of buffers
+		for (int i = 0; i < NUM_BUFFERS / 2; i++) {
+			requestedBuffers[i].recycleBuffer();
+		}
+
+		Assert.assertEquals(NUM_BUFFERS, bufferPool.numRequestedBuffers());
+		Assert.assertEquals(NUM_BUFFERS / 2, bufferPool.numAvailableBuffers());
+
+		// reduce designated number of buffers
+		// => available buffers (1/2th) should be returned immediately
+		// => non-available buffers (1/4th) should be returned later
+		bufferPool.setNumDesignatedBuffers((NUM_BUFFERS / 2) - (NUM_BUFFERS / 4));
+
+		Assert.assertEquals(NUM_BUFFERS / 2, bufferPool.numRequestedBuffers());
+		Assert.assertEquals(0, bufferPool.numAvailableBuffers());
+
+		// recycle second half of the buffers
+		// => previously non-available buffers (1/4th) should be returned immediately
+		// => remaining buffers are the available ones (1/4th)
+		for (int i = NUM_BUFFERS / 2; i < NUM_BUFFERS; i++) {
+			requestedBuffers[i].recycleBuffer();
+		}
+
+		Assert.assertEquals("Expected current number of requested buffers to be equal to the number of designated buffers.",
+				bufferPool.numDesignatedBuffers(), bufferPool.numRequestedBuffers());
+
+		Assert.assertEquals("Expected current number of requested and available buffers to be equal, " +
+				"because all requested buffers have been recycled and become available again.",
+				bufferPool.numRequestedBuffers(), bufferPool.numAvailableBuffers());
+
+		// re-request remaining buffers and register buffer availability listener
+		int remaining = bufferPool.numRequestedBuffers();
+		for (int i = 0; i < remaining; i++) {
+			requestedBuffers[i] = bufferPool.requestBuffer(BUFFER_SIZE);
+		}
+
+		BufferAvailabilityListener listener = mock(BufferAvailabilityListener.class);
+		doAnswer(RECYCLING_BUFFER_AVAILABLE_ANSWER).when(listener).bufferAvailable(Matchers.<Buffer>anyObject());
+
+		BufferAvailabilityRegistration registration = bufferPool.registerBufferAvailabilityListener(listener);
+		Assert.assertEquals(BufferAvailabilityRegistration.SUCCEEDED_REGISTERED, registration);
+
+		// reduce number of designated buffers and recycle all buffers
+		bufferPool.setNumDesignatedBuffers(bufferPool.numDesignatedBuffers() - 1);
+
+		for (int i = 0; i < remaining; i++) {
+			requestedBuffers[i].recycleBuffer();
+		}
+
+		Assert.assertEquals(remaining - 1, bufferPool.numRequestedBuffers());
+		Assert.assertEquals(remaining - 1, bufferPool.numAvailableBuffers());
+
+		bufferPool.destroy();
+	}
+
+	// --------------------------------------------------------------------
+
+	private static class RecyclingBufferAvailableAnswer implements Answer<Void> {
+
+		@Override
+		public Void answer(InvocationOnMock invocation) throws Throwable {
+			Buffer buffer = (Buffer) invocation.getArguments()[0];
+			buffer.recycleBuffer();
+
+			return null;
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6a40d41a/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 40aefd3..f695979 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
@@ -83,7 +83,7 @@ public class InboundEnvelopeDecoderTest {
 		ByteBuf buf = encode(ch, envelopes);
 
 		when(this.bufferProvider.registerBufferAvailabilityListener(Matchers.<BufferAvailabilityListener>anyObject()))
-				.thenReturn(BufferAvailabilityRegistration.REGISTERED);
+				.thenReturn(BufferAvailabilityRegistration.SUCCEEDED_REGISTERED);
 
 		Buffer buffer = allocBuffer(envelopes[2].getBuffer().size());
 
@@ -161,7 +161,7 @@ public class InboundEnvelopeDecoderTest {
 				.thenReturn(null);
 
 		when(this.bufferProvider.registerBufferAvailabilityListener(Matchers.<BufferAvailabilityListener>anyObject()))
-				.thenReturn(BufferAvailabilityRegistration.REGISTERED);
+				.thenReturn(BufferAvailabilityRegistration.SUCCEEDED_REGISTERED);
 
 		// --------------------------------------------------------------------
 
@@ -196,7 +196,7 @@ public class InboundEnvelopeDecoderTest {
 		Envelope[] envelopes = new Envelope[]{nextEnvelope(true), nextEnvelope()};
 
 		when(this.bufferProvider.registerBufferAvailabilityListener(Matchers.<BufferAvailabilityListener>anyObject()))
-				.thenReturn(BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_AVAILABLE);
+				.thenReturn(BufferAvailabilityRegistration.FAILED_BUFFER_AVAILABLE);
 
 		when(this.bufferProvider.requestBuffer(anyInt()))
 				.thenReturn(null)
@@ -223,7 +223,7 @@ public class InboundEnvelopeDecoderTest {
 				.thenReturn(null);
 
 		when(this.bufferProvider.registerBufferAvailabilityListener(Matchers.<BufferAvailabilityListener>anyObject()))
-				.thenReturn(BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_POOL_DESTROYED);
+				.thenReturn(BufferAvailabilityRegistration.FAILED_BUFFER_POOL_DESTROYED);
 
 		// --------------------------------------------------------------------
 
@@ -738,15 +738,15 @@ public class InboundEnvelopeDecoderTest {
 		public BufferAvailabilityRegistration answer(InvocationOnMock invocation) throws Throwable {
 			if (this.random.nextBoolean()) {
 				this.isRegistered = true;
-				return BufferAvailabilityRegistration.REGISTERED;
+				return BufferAvailabilityRegistration.SUCCEEDED_REGISTERED;
 			}
 			else if (this.random.nextBoolean()) {
 				this.bufferRequestAnswer.forceBufferAvailable();
-				return BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_AVAILABLE;
+				return BufferAvailabilityRegistration.FAILED_BUFFER_AVAILABLE;
 			}
 			else {
 				this.numSkipped++;
-				return BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_POOL_DESTROYED;
+				return BufferAvailabilityRegistration.FAILED_BUFFER_POOL_DESTROYED;
 			}
 		}
 
@@ -757,7 +757,7 @@ public class InboundEnvelopeDecoderTest {
 
 			for (Envelope env : envelopes) {
 				if (env.getBuffer() != null) {
-					// skip envelope if returned NOT_REGISTERED_BUFFER_POOL_DESTROYED
+					// skip envelope if returned FAILED_BUFFER_POOL_DESTROYED
 					if (!this.random.nextBoolean() && !this.random.nextBoolean() && !this.random.nextBoolean()) {
 						continue;
 					}


[09/30] Offer buffer-oriented API for I/O (#25)

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/DefaultChannelSelectorTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/DefaultChannelSelectorTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/DefaultChannelSelectorTest.java
new file mode 100644
index 0000000..890b34f
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/DefaultChannelSelectorTest.java
@@ -0,0 +1,47 @@
+/***********************************************************************************************************************
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+
+import eu.stratosphere.runtime.io.api.RoundRobinChannelSelector;
+import org.junit.Test;
+
+import eu.stratosphere.core.io.StringRecord;
+
+/**
+ * This class checks the functionality of the {@link RoundRobinChannelSelector} class.
+ * 
+ */
+public class DefaultChannelSelectorTest {
+
+	/**
+	 * This test checks the channel selection
+	 */
+	@Test
+	public void channelSelect() {
+
+		final StringRecord dummyRecord = new StringRecord("abc");
+		final RoundRobinChannelSelector<StringRecord> selector = new RoundRobinChannelSelector<StringRecord>();
+		// Test with two channels
+		final int numberOfOutputChannels = 2;
+		int[] selectedChannels = selector.selectChannels(dummyRecord, numberOfOutputChannels);
+		assertEquals(1, selectedChannels.length);
+		assertEquals(1, selectedChannels[0]);
+		selectedChannels = selector.selectChannels(dummyRecord, numberOfOutputChannels);
+		assertEquals(1, selectedChannels.length);
+		assertEquals(0, selectedChannels[0]);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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
new file mode 100644
index 0000000..17c2f58
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/library/FileLineReadWriteTest.java
@@ -0,0 +1,136 @@
+/***********************************************************************************************************************
+ * 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/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReaderWriterTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReaderWriterTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReaderWriterTest.java
new file mode 100644
index 0000000..b8914a8
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReaderWriterTest.java
@@ -0,0 +1,394 @@
+/***********************************************************************************************************************
+ * 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.network.envelope;
+
+import eu.stratosphere.core.memory.MemorySegment;
+import eu.stratosphere.nephele.event.task.AbstractEvent;
+import eu.stratosphere.nephele.jobgraph.JobID;
+import eu.stratosphere.nephele.util.DiscardingRecycler;
+import eu.stratosphere.nephele.util.TestBufferProvider;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferProviderBroker;
+import eu.stratosphere.runtime.io.BufferRecycler;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+
+public class EnvelopeReaderWriterTest {
+	
+	private final long RANDOM_SEED = 520346508276087l;
+
+	private static final int BUFFER_SIZE = 32768;
+	
+	private static final byte BUFFER_CONTENT = 13;
+	
+	private final int[] BUFFER_SIZES = { 0, 2, BUFFER_SIZE, 3782, 88, 0, 23};
+	
+	private final AbstractEvent[][] EVENT_LISTS = {
+		{},
+		{},
+		{},
+		{ new TestEvent1(34872527) },
+		{ new TestEvent1(8749653), new TestEvent1(365345) },
+		{ new TestEvent2(34563456), new TestEvent1(598432), new TestEvent2(976293845) },
+		{}
+	};
+
+	@Test
+	public void testWriteAndRead() {
+		
+		Assert.assertTrue("Test broken.", BUFFER_SIZES.length == EVENT_LISTS.length);
+
+		File testFile = null;
+		RandomAccessFile raf = null;
+		try {
+			testFile = File.createTempFile("envelopes", ".tmp");
+			raf = new RandomAccessFile(testFile, "rw");
+			
+			// write
+			FileChannel c = raf.getChannel();
+			writeEnvelopes(c);
+			
+			// read
+			c.position(0);
+			readEnvelopes(c, -1.0f);
+			c.close();
+		}
+		catch (Exception e) {
+			System.err.println(e.getMessage());
+			e.printStackTrace();
+			Assert.fail(e.getMessage());
+		}
+		finally {
+			if (raf != null)
+				try { raf.close(); } catch (Throwable t) {}
+			
+			if (testFile != null)
+				testFile.delete();
+		}
+	}
+	
+	@Test
+	public void testWriteAndReadChunked() {
+		
+		Assert.assertTrue("Test broken.", BUFFER_SIZES.length == EVENT_LISTS.length);
+
+		File testFile = null;
+		RandomAccessFile raf = null;
+		try {
+			testFile = File.createTempFile("envelopes", ".tmp");
+			raf = new RandomAccessFile(testFile, "rw");
+			
+			// write
+			FileChannel c = raf.getChannel();
+			writeEnvelopes(new ChunkedWriteableChannel(c));
+			
+			// read
+			c.position(0);
+			readEnvelopes(new ChunkedReadableChannel(c), 0.75f);
+			c.close();
+		}
+		catch (Exception e) {
+			System.err.println(e.getMessage());
+			e.printStackTrace();
+			Assert.fail(e.getMessage());
+		}
+		finally {
+			if (raf != null)
+				try { raf.close(); } catch (Throwable t) {}
+			
+			if (testFile != null)
+				testFile.delete();
+		}
+	}
+
+	private void writeEnvelopes(WritableByteChannel channel) throws IOException {
+
+		final BufferRecycler recycler = new DiscardingRecycler();
+		final Random rand = new Random(RANDOM_SEED);
+		
+		final EnvelopeWriter serializer = new EnvelopeWriter();
+		
+		final int NUM_ENVS = BUFFER_SIZES.length;
+		
+		for (int i = 0; i < NUM_ENVS; i++) {
+			int seqNum = Math.abs(rand.nextInt());
+			JobID jid = new JobID(rand.nextLong(), rand.nextLong());
+			ChannelID sid = new ChannelID(rand.nextLong(), rand.nextLong());
+			
+			Envelope env = new Envelope(seqNum, jid, sid);
+			if (EVENT_LISTS[i].length > 0) {
+				env.serializeEventList(Arrays.asList(EVENT_LISTS[i]));
+			}
+			
+			int bufferSize = BUFFER_SIZES[i];
+			if (bufferSize > 0) {
+				MemorySegment ms = new MemorySegment(new byte[BUFFER_SIZE]);
+				for (int x = 0; x < bufferSize; x++) {
+					ms.put(x, BUFFER_CONTENT);
+				}
+				
+				Buffer mb = new Buffer(ms, bufferSize, recycler);
+				env.setBuffer(mb);
+			}
+			
+			serializer.setEnvelopeForWriting(env);
+			
+			while (serializer.writeNextChunk(channel));
+		}
+	}
+	
+	private void readEnvelopes(ReadableByteChannel channel, float probabilityForNoBufferCurrently) throws IOException {
+		
+		final Random rand = new Random(RANDOM_SEED);
+		
+		final EnvelopeReader reader = new EnvelopeReader(new OneForAllBroker(BUFFER_SIZE, probabilityForNoBufferCurrently));
+		
+		final int NUM_ENVS = BUFFER_SIZES.length;
+		
+		for (int i = 0; i < NUM_ENVS; i++) {
+			int expectedSeqNum = Math.abs(rand.nextInt());
+			JobID expectedJid = new JobID(rand.nextLong(), rand.nextLong());
+			ChannelID expectedSid = new ChannelID(rand.nextLong(), rand.nextLong());
+			
+			// read the next envelope
+			while (reader.readNextChunk(channel) != EnvelopeReader.DeserializationState.COMPLETE);
+			Envelope env = reader.getFullyDeserializedTransferEnvelope();
+			
+			// check the basic fields from the header
+			Assert.assertEquals(expectedSeqNum, env.getSequenceNumber());
+			Assert.assertEquals(expectedJid, env.getJobID());
+			Assert.assertEquals(expectedSid, env.getSource());
+			
+			// check the events
+			List<? extends AbstractEvent> events = env.deserializeEvents();
+			Assert.assertEquals(EVENT_LISTS[i].length, events.size());
+			
+			for (int n = 0; n < EVENT_LISTS[i].length; n++) {
+				AbstractEvent expectedEvent = EVENT_LISTS[i][n];
+				AbstractEvent actualEvent = events.get(n);
+				
+				Assert.assertEquals(expectedEvent.getClass(), actualEvent.getClass());
+				Assert.assertEquals(expectedEvent, actualEvent);
+			}
+			
+			// check the buffer
+			Buffer buf = env.getBuffer();
+			if (buf == null) {
+				Assert.assertTrue(BUFFER_SIZES[i] == 0);
+			} else {
+				Assert.assertEquals(BUFFER_SIZES[i], buf.size());
+				for (int k = 0; k < BUFFER_SIZES[i]; k++) {
+					Assert.assertEquals(BUFFER_CONTENT, buf.getMemorySegment().get(k));
+				}
+			}
+			
+			reader.reset();
+		}
+		
+	}
+	
+	
+	public  static final class TestEvent1 extends AbstractEvent {
+
+		private long id;
+		
+		public TestEvent1() {}
+		
+		public TestEvent1(long id) {
+			this.id = id;
+		}
+		
+		@Override
+		public void write(DataOutput out) throws IOException {
+			out.writeLong(id);
+		}
+
+		@Override
+		public void read(DataInput in) throws IOException {
+			id = in.readLong();
+		}
+		
+
+		@Override
+		public boolean equals(Object obj) {
+			if (obj.getClass() == TestEvent1.class) {
+				return ((TestEvent1) obj).id == this.id;
+			} else {
+				return false;
+			}
+		}
+		
+		@Override
+		public int hashCode() {
+			return ((int) id) ^ ((int) (id >>> 32));
+		}
+		
+		@Override
+		public String toString() {
+			return "TestEvent1 (" + id + ")";
+		}
+	}
+	
+	public static final class TestEvent2 extends AbstractEvent {
+
+		private long id;
+		
+		public TestEvent2() {}
+		
+		public TestEvent2(long id) {
+			this.id = id;
+		}
+		
+		@Override
+		public void write(DataOutput out) throws IOException {
+			out.writeLong(id);
+		}
+
+		@Override
+		public void read(DataInput in) throws IOException {
+			id = in.readLong();
+		}
+		
+
+		@Override
+		public boolean equals(Object obj) {
+			if (obj.getClass() == TestEvent2.class) {
+				return ((TestEvent2) obj).id == this.id;
+			} else {
+				return false;
+			}
+		}
+		
+		@Override
+		public int hashCode() {
+			return ((int) id) ^ ((int) (id >>> 32));
+		}
+		
+		@Override
+		public String toString() {
+			return "TestEvent2 (" + id + ")";
+		}
+	}
+	
+	private static final class ChunkedWriteableChannel implements WritableByteChannel {
+		
+		private final WritableByteChannel delegate;
+		
+		private final Random rnd;
+		
+		private ChunkedWriteableChannel(WritableByteChannel delegate) {
+			this.delegate = delegate;
+			this.rnd = new Random();
+		}
+
+		@Override
+		public boolean isOpen() {
+			return this.delegate.isOpen();
+		}
+
+		@Override
+		public void close() throws IOException {
+			this.delegate.close();
+		}
+
+		@Override
+		public int write(ByteBuffer src) throws IOException {
+			final int available = src.remaining();
+			final int oldLimit = src.limit();
+			
+			int toWrite = rnd.nextInt(available) + 1;
+			toWrite = Math.min(Math.max(toWrite, 8), available);
+			
+			src.limit(src.position() + toWrite);
+			
+			int written = this.delegate.write(src);
+			
+			src.limit(oldLimit);
+			
+			return written;
+		}
+	}
+	
+	private static final class ChunkedReadableChannel implements ReadableByteChannel {
+		
+		private final ReadableByteChannel delegate;
+		
+		private final Random rnd;
+		
+		private ChunkedReadableChannel(ReadableByteChannel delegate) {
+			this.delegate = delegate;
+			this.rnd = new Random();
+		}
+
+		@Override
+		public boolean isOpen() {
+			return this.delegate.isOpen();
+		}
+
+		@Override
+		public void close() throws IOException {
+			this.delegate.close();
+		}
+
+		@Override
+		public int read(ByteBuffer dst) throws IOException {
+			final int available = dst.remaining();
+			final int oldLimit = dst.limit();
+			
+			int toRead = rnd.nextInt(available) + 1;
+			toRead = Math.min(Math.max(toRead, 8), available);
+			
+			dst.limit(dst.position() + toRead);
+			
+			int read = this.delegate.read(dst);
+			
+			dst.limit(oldLimit);
+			
+			return read;
+		}
+	}
+	
+	private static final class OneForAllBroker implements BufferProviderBroker {
+
+		private final TestBufferProvider provider;
+
+		private OneForAllBroker(int sizeOfMemorySegments) {
+			this.provider = new TestBufferProvider(sizeOfMemorySegments);
+		}
+		
+		private OneForAllBroker(int sizeOfMemorySegments, float probabilityForNoBufferCurrently) {
+			this.provider = new TestBufferProvider(sizeOfMemorySegments, probabilityForNoBufferCurrently);
+		}
+		
+		@Override
+		public BufferProvider getBufferProvider(JobID jobID, ChannelID sourceChannelID) {
+			return this.provider;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/DataInputOutputSerializerTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/DataInputOutputSerializerTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/DataInputOutputSerializerTest.java
new file mode 100644
index 0000000..55c0243
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/DataInputOutputSerializerTest.java
@@ -0,0 +1,115 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.serialization;
+
+import eu.stratosphere.core.memory.MemorySegment;
+import eu.stratosphere.runtime.io.serialization.types.SerializationTestType;
+import eu.stratosphere.runtime.io.serialization.types.SerializationTestTypeFactory;
+import eu.stratosphere.runtime.io.serialization.types.Util;
+import junit.framework.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+
+public class DataInputOutputSerializerTest {
+
+	@Test
+	public void testWrapAsByteBuffer() {
+		SerializationTestType randomInt = Util.randomRecord(SerializationTestTypeFactory.INT);
+
+		DataOutputSerializer serializer = new DataOutputSerializer(randomInt.length());
+		MemorySegment segment = new MemorySegment(new byte[randomInt.length()]);
+
+		try {
+			// empty buffer, read buffer should be empty
+			ByteBuffer wrapper = serializer.wrapAsByteBuffer();
+
+			Assert.assertEquals(0, wrapper.position());
+			Assert.assertEquals(0, wrapper.limit());
+
+			// write to data output, read buffer should still be empty
+			randomInt.write(serializer);
+
+			Assert.assertEquals(0, wrapper.position());
+			Assert.assertEquals(0, wrapper.limit());
+
+			// get updated read buffer, read buffer should contain written data
+			wrapper = serializer.wrapAsByteBuffer();
+
+			Assert.assertEquals(0, wrapper.position());
+			Assert.assertEquals(randomInt.length(), wrapper.limit());
+
+			// clear data output, read buffer should still contain written data
+			serializer.clear();
+
+			Assert.assertEquals(0, wrapper.position());
+			Assert.assertEquals(randomInt.length(), wrapper.limit());
+
+			// get updated read buffer, should be empty
+			wrapper = serializer.wrapAsByteBuffer();
+
+			Assert.assertEquals(0, wrapper.position());
+			Assert.assertEquals(0, wrapper.limit());
+
+			// write to data output and read back to memory
+			randomInt.write(serializer);
+			wrapper = serializer.wrapAsByteBuffer();
+
+			segment.put(0, wrapper, randomInt.length());
+
+			Assert.assertEquals(randomInt.length(), wrapper.position());
+			Assert.assertEquals(randomInt.length(), wrapper.limit());
+		} catch (IOException e) {
+			e.printStackTrace();
+			Assert.fail("Test encountered an unexpected exception.");
+		}
+	}
+
+	@Test
+	public void testRandomValuesWriteRead() {
+		final int numElements = 100000;
+		final ArrayDeque<SerializationTestType> reference = new ArrayDeque<SerializationTestType>();
+
+		DataOutputSerializer serializer = new DataOutputSerializer(1);
+
+		for (SerializationTestType value : Util.randomRecords(numElements)) {
+			reference.add(value);
+
+			try {
+				value.write(serializer);
+			} catch (IOException e) {
+				e.printStackTrace();
+				Assert.fail("Test encountered an unexpected exception.");
+			}
+		}
+
+		DataInputDeserializer deserializer = new DataInputDeserializer(serializer.wrapAsByteBuffer());
+
+		for (SerializationTestType expected : reference) {
+			try {
+				SerializationTestType actual = expected.getClass().newInstance();
+				actual.read(deserializer);
+
+				Assert.assertEquals(expected, actual);
+			} catch (Exception e) {
+				e.printStackTrace();
+				Assert.fail("Test encountered an unexpected exception.");
+			}
+		}
+
+		reference.clear();
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/PagedViewsTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/PagedViewsTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/PagedViewsTest.java
new file mode 100644
index 0000000..817c0e6
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/PagedViewsTest.java
@@ -0,0 +1,160 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.serialization;
+
+import eu.stratosphere.core.memory.MemorySegment;
+import eu.stratosphere.nephele.services.memorymanager.AbstractPagedInputView;
+import eu.stratosphere.nephele.services.memorymanager.AbstractPagedOutputView;
+import eu.stratosphere.runtime.io.serialization.types.SerializationTestType;
+import eu.stratosphere.runtime.io.serialization.types.SerializationTestTypeFactory;
+import eu.stratosphere.runtime.io.serialization.types.Util;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.*;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+public class PagedViewsTest {
+
+	@Test
+	public void testSequenceOfIntegersWithAlignedBuffers() {
+		try {
+			final int NUM_INTS = 1000000;
+
+			testSequenceOfTypes(Util.randomRecords(NUM_INTS, SerializationTestTypeFactory.INT), 2048);
+
+		} catch (Exception e) {
+			e.printStackTrace();
+			fail("Test encountered an unexpected exception.");
+		}
+	}
+
+	@Test
+	public void testSequenceOfIntegersWithUnalignedBuffers() {
+		try {
+			final int NUM_INTS = 1000000;
+
+			testSequenceOfTypes(Util.randomRecords(NUM_INTS, SerializationTestTypeFactory.INT), 2047);
+
+		} catch (Exception e) {
+			e.printStackTrace();
+			fail("Test encountered an unexpected exception.");
+		}
+	}
+
+	@Test
+	public void testRandomTypes() {
+		try {
+			final int NUM_TYPES = 100000;
+
+			// test with an odd buffer size to force many unaligned cases
+			testSequenceOfTypes(Util.randomRecords(NUM_TYPES), 57);
+
+		} catch (Exception e) {
+			e.printStackTrace();
+			fail("Test encountered an unexpected exception.");
+		}
+	}
+
+	private static void testSequenceOfTypes(Iterable<SerializationTestType> sequence, int segmentSize) throws Exception {
+
+		List<SerializationTestType> elements = new ArrayList<SerializationTestType>(512);
+		TestOutputView outView = new TestOutputView(segmentSize);
+
+		// write
+		for (SerializationTestType type : sequence) {
+			// serialize the record
+			type.write(outView);
+			elements.add(type);
+		}
+		outView.close();
+
+		// check the records
+		TestInputView inView = new TestInputView(outView.segments);
+
+		for (SerializationTestType reference : elements) {
+			SerializationTestType result = reference.getClass().newInstance();
+			result.read(inView);
+			assertEquals(reference, result);
+		}
+	}
+
+	// ============================================================================================
+
+	private static final class SegmentWithPosition {
+
+		private final MemorySegment segment;
+		private final int position;
+
+		public SegmentWithPosition(MemorySegment segment, int position) {
+			this.segment = segment;
+			this.position = position;
+		}
+	}
+
+	private static final class TestOutputView extends AbstractPagedOutputView {
+
+		private final List<SegmentWithPosition> segments = new ArrayList<SegmentWithPosition>();
+
+		private final int segmentSize;
+
+		private TestOutputView(int segmentSize) {
+			super(new MemorySegment(new byte[segmentSize]), segmentSize, 0);
+
+			this.segmentSize = segmentSize;
+		}
+
+		@Override
+		protected MemorySegment nextSegment(MemorySegment current, int positionInCurrent) throws IOException {
+			segments.add(new SegmentWithPosition(current, positionInCurrent));
+			return new MemorySegment(new byte[segmentSize]);
+		}
+
+		public void close() {
+			segments.add(new SegmentWithPosition(getCurrentSegment(), getCurrentPositionInSegment()));
+		}
+	}
+
+	private static final class TestInputView extends AbstractPagedInputView {
+
+		private final List<SegmentWithPosition> segments;
+
+		private int num;
+
+
+		private TestInputView(List<SegmentWithPosition> segments) {
+			super(segments.get(0).segment, segments.get(0).position, 0);
+
+			this.segments = segments;
+			this.num = 0;
+		}
+
+		@Override
+		protected MemorySegment nextSegment(MemorySegment current) throws IOException {
+			num++;
+			if (num < segments.size()) {
+				return segments.get(num).segment;
+			} else {
+				return null;
+			}
+		}
+
+		@Override
+		protected int getLimitForSegment(MemorySegment segment) {
+			return segments.get(num).position;
+		}
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializationTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializationTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializationTest.java
new file mode 100644
index 0000000..094e597
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializationTest.java
@@ -0,0 +1,164 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.serialization;
+
+import eu.stratosphere.core.memory.MemorySegment;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.serialization.RecordDeserializer.DeserializationResult;
+import eu.stratosphere.runtime.io.serialization.types.SerializationTestType;
+import eu.stratosphere.runtime.io.serialization.types.SerializationTestTypeFactory;
+import eu.stratosphere.runtime.io.serialization.types.Util;
+import junit.framework.Assert;
+import org.junit.Test;
+
+import java.util.ArrayDeque;
+
+public class SpanningRecordSerializationTest {
+
+	@Test
+	public void testIntRecordsSpanningMultipleSegments() {
+		final int SEGMENT_SIZE = 1;
+		final int NUM_VALUES = 10;
+
+		try {
+			test(Util.randomRecords(NUM_VALUES, SerializationTestTypeFactory.INT), SEGMENT_SIZE);
+		} catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail("Test encountered an unexpected exception.");
+		}
+	}
+
+	@Test
+	public void testIntRecordsWithAlignedBuffers () {
+		final int SEGMENT_SIZE = 64;
+		final int NUM_VALUES = 64;
+
+		try {
+			test(Util.randomRecords(NUM_VALUES, SerializationTestTypeFactory.INT), SEGMENT_SIZE);
+		} catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail("Test encountered an unexpected exception.");
+		}
+	}
+
+	@Test
+	public void testIntRecordsWithUnalignedBuffers () {
+		final int SEGMENT_SIZE = 31;
+		final int NUM_VALUES = 248;
+
+		try {
+			test(Util.randomRecords(NUM_VALUES, SerializationTestTypeFactory.INT), SEGMENT_SIZE);
+		} catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail("Test encountered an unexpected exception.");
+		}
+	}
+
+	@Test
+	 public void testRandomRecords () {
+		final int SEGMENT_SIZE = 127;
+		final int NUM_VALUES = 10000;
+
+		try {
+			test(Util.randomRecords(NUM_VALUES), SEGMENT_SIZE);
+		} catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail("Test encountered an unexpected exception.");
+		}
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+
+	/**
+	 * Iterates over the provided records and tests whether {@link SpanningRecordSerializer} and {@link AdaptiveSpanningRecordDeserializer}
+	 * interact as expected.
+	 * <p>
+	 * Only a single {@link MemorySegment} will be allocated.
+	 *
+	 * @param records records to test
+	 * @param segmentSize size for the {@link MemorySegment}
+	 */
+	private void test (Util.MockRecords records, int segmentSize) throws Exception {
+		final int SERIALIZATION_OVERHEAD = 4; // length encoding
+
+		final RecordSerializer<SerializationTestType> serializer = new SpanningRecordSerializer<SerializationTestType>();
+		final RecordDeserializer<SerializationTestType> deserializer = new AdaptiveSpanningRecordDeserializer<SerializationTestType>();
+
+		final Buffer buffer = new Buffer(new MemorySegment(new byte[segmentSize]), segmentSize, null);
+
+		final ArrayDeque<SerializationTestType> serializedRecords = new ArrayDeque<SerializationTestType>();
+
+		// -------------------------------------------------------------------------------------------------------------
+
+		serializer.setNextBuffer(buffer);
+
+		int numBytes = 0;
+		int numRecords = 0;
+		for (SerializationTestType record : records) {
+
+			serializedRecords.add(record);
+
+			numRecords++;
+			numBytes += record.length() + SERIALIZATION_OVERHEAD;
+
+			// serialize record
+			if (serializer.addRecord(record).isFullBuffer()) {
+				// buffer is full => start deserializing
+				deserializer.setNextMemorySegment(serializer.getCurrentBuffer().getMemorySegment(), segmentSize);
+
+				while (!serializedRecords.isEmpty()) {
+					SerializationTestType expected = serializedRecords.poll();
+					SerializationTestType actual = expected.getClass().newInstance();
+
+					if (deserializer.getNextRecord(actual).isFullRecord()) {
+						Assert.assertEquals(expected, actual);
+						numRecords--;
+					} else {
+						serializedRecords.addFirst(expected);
+						break;
+					}
+				}
+
+				while (serializer.setNextBuffer(buffer).isFullBuffer()) {
+					deserializer.setNextMemorySegment(serializer.getCurrentBuffer().getMemorySegment(), segmentSize);
+				}
+
+
+
+			}
+		}
+
+		// deserialize left over records
+		deserializer.setNextMemorySegment(serializer.getCurrentBuffer().getMemorySegment(), (numBytes % segmentSize));
+
+		serializer.clear();
+
+		while (!serializedRecords.isEmpty()) {
+			SerializationTestType expected = serializedRecords.poll();
+
+			SerializationTestType actual = expected.getClass().newInstance();
+			DeserializationResult result = deserializer.getNextRecord(actual);
+
+			Assert.assertTrue(result.isFullRecord());
+			Assert.assertEquals(expected, actual);
+			numRecords--;
+		}
+
+
+		// assert that all records have been serialized and deserialized
+		Assert.assertEquals(0, numRecords);
+		Assert.assertFalse(serializer.hasData());
+		Assert.assertFalse(deserializer.hasUnfinishedData());
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializerTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializerTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializerTest.java
new file mode 100644
index 0000000..637b7d5
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializerTest.java
@@ -0,0 +1,219 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2014 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.serialization;
+
+import eu.stratosphere.core.memory.MemorySegment;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.serialization.RecordSerializer.SerializationResult;
+import eu.stratosphere.runtime.io.serialization.types.SerializationTestType;
+import eu.stratosphere.runtime.io.serialization.types.SerializationTestTypeFactory;
+import eu.stratosphere.runtime.io.serialization.types.Util;
+import junit.framework.Assert;
+import org.junit.Test;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Random;
+
+public class SpanningRecordSerializerTest {
+
+	@Test
+	public void testHasData() {
+		final int SEGMENT_SIZE = 16;
+
+		final SpanningRecordSerializer<SerializationTestType> serializer = new SpanningRecordSerializer<SerializationTestType>();
+		final Buffer buffer = new Buffer(new MemorySegment(new byte[SEGMENT_SIZE]), SEGMENT_SIZE, null);
+		final SerializationTestType randomIntRecord = Util.randomRecord(SerializationTestTypeFactory.INT);
+
+		Assert.assertFalse(serializer.hasData());
+
+		try {
+			serializer.addRecord(randomIntRecord);
+			Assert.assertTrue(serializer.hasData());
+
+			serializer.setNextBuffer(buffer);
+			Assert.assertTrue(serializer.hasData());
+
+			serializer.clear();
+			Assert.assertFalse(serializer.hasData());
+
+			serializer.setNextBuffer(buffer);
+
+			serializer.addRecord(randomIntRecord);
+			Assert.assertTrue(serializer.hasData());
+
+			serializer.addRecord(randomIntRecord);
+			Assert.assertTrue(serializer.hasData());
+		} catch (IOException e) {
+			e.printStackTrace();
+		}
+
+	}
+
+	@Test
+	public void testEmptyRecords() {
+		final int SEGMENT_SIZE = 11;
+
+		final SpanningRecordSerializer<SerializationTestType> serializer = new SpanningRecordSerializer<SerializationTestType>();
+		final Buffer buffer = new Buffer(new MemorySegment(new byte[SEGMENT_SIZE]), SEGMENT_SIZE, null);
+
+		try {
+			Assert.assertEquals(SerializationResult.FULL_RECORD, serializer.setNextBuffer(buffer));
+		} catch (IOException e) {
+			e.printStackTrace();
+		}
+
+		try {
+			SerializationTestType emptyRecord = new SerializationTestType() {
+				@Override
+				public SerializationTestType getRandom(Random rnd) {
+					throw new UnsupportedOperationException();
+				}
+
+				@Override
+				public int length() {
+					throw new UnsupportedOperationException();
+				}
+
+				@Override
+				public void write(DataOutput out) throws IOException {
+				}
+
+				@Override
+				public void read(DataInput in) throws IOException {
+				}
+
+				@Override
+				public int hashCode() {
+					throw new UnsupportedOperationException();
+				}
+
+				@Override
+				public boolean equals(Object obj) {
+					throw new UnsupportedOperationException();
+				}
+			};
+
+			SerializationResult result = serializer.addRecord(emptyRecord);
+			Assert.assertEquals(SerializationResult.FULL_RECORD, result);
+
+			result = serializer.addRecord(emptyRecord);
+			Assert.assertEquals(SerializationResult.FULL_RECORD, result);
+
+			result = serializer.addRecord(emptyRecord);
+			Assert.assertEquals(SerializationResult.PARTIAL_RECORD_MEMORY_SEGMENT_FULL, result);
+
+			result = serializer.setNextBuffer(buffer);
+			Assert.assertEquals(SerializationResult.FULL_RECORD, result);
+		} catch (IOException e) {
+			e.printStackTrace();
+		}
+	}
+
+	@Test
+	public void testIntRecordsSpanningMultipleSegments() {
+		final int SEGMENT_SIZE = 1;
+		final int NUM_VALUES = 10;
+
+		try {
+			test(Util.randomRecords(NUM_VALUES, SerializationTestTypeFactory.INT), SEGMENT_SIZE);
+		} catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail("Test encountered an unexpected exception.");
+		}
+	}
+
+	@Test
+	public void testIntRecordsWithAlignedSegments() {
+		final int SEGMENT_SIZE = 64;
+		final int NUM_VALUES = 64;
+
+		try {
+			test(Util.randomRecords(NUM_VALUES, SerializationTestTypeFactory.INT), SEGMENT_SIZE);
+		} catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail("Test encountered an unexpected exception.");
+		}
+	}
+
+	@Test
+	public void testIntRecordsWithUnalignedSegments() {
+		final int SEGMENT_SIZE = 31;
+		final int NUM_VALUES = 248; // least common multiple => last record should align
+
+		try {
+			test(Util.randomRecords(NUM_VALUES, SerializationTestTypeFactory.INT), SEGMENT_SIZE);
+		} catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail("Test encountered an unexpected exception.");
+		}
+	}
+
+	@Test
+	public void testRandomRecords() {
+		final int SEGMENT_SIZE = 127;
+		final int NUM_VALUES = 100000;
+
+		try {
+			test(Util.randomRecords(NUM_VALUES), SEGMENT_SIZE);
+		} catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail("Test encountered an unexpected exception.");
+		}
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+
+	/**
+	 * Iterates over the provided records and tests whether the {@link SpanningRecordSerializer} returns the expected
+	 * {@link SerializationResult} values.
+	 * <p>
+	 * Only a single {@link MemorySegment} will be allocated.
+	 *
+	 * @param records records to test
+	 * @param segmentSize size for the {@link MemorySegment}
+	 */
+	private void test(Util.MockRecords records, int segmentSize) throws Exception {
+		final int SERIALIZATION_OVERHEAD = 4; // length encoding
+
+		final SpanningRecordSerializer<SerializationTestType> serializer = new SpanningRecordSerializer<SerializationTestType>();
+		final Buffer buffer = new Buffer(new MemorySegment(new byte[segmentSize]), segmentSize, null);
+
+		// -------------------------------------------------------------------------------------------------------------
+
+		serializer.setNextBuffer(buffer);
+
+		int numBytes = 0;
+		for (SerializationTestType record : records) {
+			SerializationResult result = serializer.addRecord(record);
+			numBytes += record.length() + SERIALIZATION_OVERHEAD;
+
+			if (numBytes < segmentSize) {
+				Assert.assertEquals(SerializationResult.FULL_RECORD, result);
+			} else if (numBytes == segmentSize) {
+				Assert.assertEquals(SerializationResult.FULL_RECORD_MEMORY_SEGMENT_FULL, result);
+				serializer.setNextBuffer(buffer);
+				numBytes = 0;
+			} else {
+				Assert.assertEquals(SerializationResult.PARTIAL_RECORD_MEMORY_SEGMENT_FULL, result);
+
+				while (result.isFullBuffer()) {
+					numBytes -= segmentSize;
+					result = serializer.setNextBuffer(buffer);
+				}
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/AsciiStringType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/AsciiStringType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/AsciiStringType.java
new file mode 100644
index 0000000..7aadc7c
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/AsciiStringType.java
@@ -0,0 +1,77 @@
+/***********************************************************************************************************************
+ * 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.serialization.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Random;
+
+public class AsciiStringType implements SerializationTestType {
+
+	private static final int MAX_LEN = 1500;
+
+	public String value;
+
+	public AsciiStringType() {
+		this.value = "";
+	}
+
+	private AsciiStringType(String value) {
+		this.value = value;
+	}
+
+	@Override
+	public AsciiStringType getRandom(Random rnd) {
+		final StringBuilder bld = new StringBuilder();
+		final int len = rnd.nextInt(MAX_LEN + 1);
+
+		for (int i = 0; i < len; i++) {
+			// 1--127
+			bld.append((char) (rnd.nextInt(126) + 1));
+		}
+
+		return new AsciiStringType(bld.toString());
+	}
+
+	@Override
+	public int length() {
+		return value.getBytes().length + 2;
+	}
+
+	@Override
+	public void write(DataOutput out) throws IOException {
+		out.writeUTF(this.value);
+	}
+
+	@Override
+	public void read(DataInput in) throws IOException {
+		this.value = in.readUTF();
+	}
+
+	@Override
+	public int hashCode() {
+		return this.value.hashCode();
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj instanceof AsciiStringType) {
+			AsciiStringType other = (AsciiStringType) obj;
+			return this.value.equals(other.value);
+		} else {
+			return false;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/BooleanType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/BooleanType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/BooleanType.java
new file mode 100644
index 0000000..32b2ba3
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/BooleanType.java
@@ -0,0 +1,67 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ **********************************************************************************************************************/
+
+package eu.stratosphere.runtime.io.serialization.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Random;
+
+public class BooleanType implements SerializationTestType {
+
+	private boolean value;
+
+	public BooleanType() {
+		this.value = false;
+	}
+
+	private BooleanType(boolean value) {
+		this.value = value;
+	}
+
+	@Override
+	public BooleanType getRandom(Random rnd) {
+		return new BooleanType(rnd.nextBoolean());
+	}
+
+	@Override
+	public int length() {
+		return 1;
+	}
+
+	@Override
+	public void write(DataOutput out) throws IOException {
+		out.writeBoolean(this.value);
+	}
+
+	@Override
+	public void read(DataInput in) throws IOException {
+		this.value = in.readBoolean();
+	}
+
+	@Override
+	public int hashCode() {
+		return this.value ? 1 : 0;
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj instanceof BooleanType) {
+			BooleanType other = (BooleanType) obj;
+			return this.value == other.value;
+		} else {
+			return false;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteArrayType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteArrayType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteArrayType.java
new file mode 100644
index 0000000..cb29a5c
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteArrayType.java
@@ -0,0 +1,76 @@
+/***********************************************************************************************************************
+ * 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.serialization.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Random;
+
+public class ByteArrayType implements SerializationTestType {
+
+	private static final int MAX_LEN = 512 * 15;
+
+	private byte[] data;
+
+	public ByteArrayType() {
+		this.data = new byte[0];
+	}
+
+	public ByteArrayType(byte[] data) {
+		this.data = data;
+	}
+
+	@Override
+	public ByteArrayType getRandom(Random rnd) {
+		final int len = rnd.nextInt(MAX_LEN) + 1;
+		final byte[] data = new byte[len];
+		rnd.nextBytes(data);
+		return new ByteArrayType(data);
+	}
+
+	@Override
+	public int length() {
+		return data.length + 4;
+	}
+
+	@Override
+	public void write(DataOutput out) throws IOException {
+		out.writeInt(this.data.length);
+		out.write(this.data);
+	}
+
+	@Override
+	public void read(DataInput in) throws IOException {
+		final int len = in.readInt();
+		this.data = new byte[len];
+		in.readFully(this.data);
+	}
+
+	@Override
+	public int hashCode() {
+		return Arrays.hashCode(this.data);
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj instanceof ByteArrayType) {
+			ByteArrayType other = (ByteArrayType) obj;
+			return Arrays.equals(this.data, other.data);
+		} else {
+			return false;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteSubArrayType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteSubArrayType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteSubArrayType.java
new file mode 100644
index 0000000..2b683d2
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteSubArrayType.java
@@ -0,0 +1,91 @@
+/***********************************************************************************************************************
+ * 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.serialization.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Random;
+
+public class ByteSubArrayType implements SerializationTestType {
+
+	private static final int MAX_LEN = 512;
+
+	private final byte[] data;
+
+	private int len;
+
+	public ByteSubArrayType() {
+		this.data = new byte[MAX_LEN];
+		this.len = 0;
+	}
+
+	@Override
+	public ByteSubArrayType getRandom(Random rnd) {
+		final int len = rnd.nextInt(MAX_LEN) + 1;
+		final ByteSubArrayType t = new ByteSubArrayType();
+		t.len = len;
+
+		final byte[] data = t.data;
+		for (int i = 0; i < len; i++) {
+			data[i] = (byte) rnd.nextInt(256);
+		}
+
+		return t;
+	}
+
+	@Override
+	public int length() {
+		return len + 4;
+	}
+
+	@Override
+	public void write(DataOutput out) throws IOException {
+		out.writeInt(this.len);
+		out.write(this.data, 0, this.len);
+	}
+
+	@Override
+	public void read(DataInput in) throws IOException {
+		this.len = in.readInt();
+		in.readFully(this.data, 0, this.len);
+	}
+
+	@Override
+	public int hashCode() {
+		final byte[] copy = new byte[this.len];
+		System.arraycopy(this.data, 0, copy, 0, this.len);
+		return Arrays.hashCode(copy);
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj instanceof ByteSubArrayType) {
+			ByteSubArrayType other = (ByteSubArrayType) obj;
+			if (this.len == other.len) {
+				for (int i = 0; i < this.len; i++) {
+					if (this.data[i] != other.data[i]) {
+						return false;
+					}
+				}
+				return true;
+			} else {
+				return false;
+			}
+		} else {
+			return false;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteType.java
new file mode 100644
index 0000000..52abdcb
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteType.java
@@ -0,0 +1,67 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ **********************************************************************************************************************/
+
+package eu.stratosphere.runtime.io.serialization.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Random;
+
+public class ByteType implements SerializationTestType {
+
+	private byte value;
+
+	public ByteType() {
+		this.value = (byte) 0;
+	}
+
+	private ByteType(byte value) {
+		this.value = value;
+	}
+
+	@Override
+	public ByteType getRandom(Random rnd) {
+		return new ByteType((byte) rnd.nextInt(256));
+	}
+
+	@Override
+	public int length() {
+		return 1;
+	}
+
+	@Override
+	public void write(DataOutput out) throws IOException {
+		out.writeByte(this.value);
+	}
+
+	@Override
+	public void read(DataInput in) throws IOException {
+		this.value = in.readByte();
+	}
+
+	@Override
+	public int hashCode() {
+		return this.value;
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj instanceof ByteType) {
+			ByteType other = (ByteType) obj;
+			return this.value == other.value;
+		} else {
+			return false;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/CharType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/CharType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/CharType.java
new file mode 100644
index 0000000..25df737
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/CharType.java
@@ -0,0 +1,68 @@
+/***********************************************************************************************************************
+ * 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.serialization.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Random;
+
+public class CharType implements SerializationTestType {
+
+	private char value;
+
+	public CharType() {
+		this.value = 0;
+	}
+
+	private CharType(char value) {
+		this.value = value;
+	}
+
+	@Override
+	public CharType getRandom(Random rnd) {
+		return new CharType((char) rnd.nextInt(10000));
+	}
+
+	@Override
+	public int length() {
+		return 2;
+	}
+
+	@Override
+	public void write(DataOutput out) throws IOException {
+		out.writeChar(this.value);
+	}
+
+	@Override
+	public void read(DataInput in) throws IOException {
+		this.value = in.readChar();
+	}
+
+	@Override
+	public int hashCode() {
+		return this.value;
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj instanceof CharType) {
+			CharType other = (CharType) obj;
+			return this.value == other.value;
+		} else {
+			return false;
+		}
+	}
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/DoubleType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/DoubleType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/DoubleType.java
new file mode 100644
index 0000000..1046e75
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/DoubleType.java
@@ -0,0 +1,68 @@
+/***********************************************************************************************************************
+ * 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.serialization.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Random;
+
+public class DoubleType implements SerializationTestType {
+
+	private double value;
+
+	public DoubleType() {
+		this.value = 0;
+	}
+
+	private DoubleType(double value) {
+		this.value = value;
+	}
+
+	@Override
+	public DoubleType getRandom(Random rnd) {
+		return new DoubleType(rnd.nextDouble());
+	}
+
+	@Override
+	public int length() {
+		return 8;
+	}
+
+	@Override
+	public void write(DataOutput out) throws IOException {
+		out.writeDouble(this.value);
+	}
+
+	@Override
+	public void read(DataInput in) throws IOException {
+		this.value = in.readDouble();
+	}
+
+	@Override
+	public int hashCode() {
+		final long l = Double.doubleToLongBits(this.value);
+		return (int) (l ^ l >>> 32);
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj instanceof DoubleType) {
+			DoubleType other = (DoubleType) obj;
+			return Double.doubleToLongBits(this.value) == Double.doubleToLongBits(other.value);
+		} else {
+			return false;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/FloatType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/FloatType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/FloatType.java
new file mode 100644
index 0000000..4a50873
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/FloatType.java
@@ -0,0 +1,67 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ **********************************************************************************************************************/
+
+package eu.stratosphere.runtime.io.serialization.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Random;
+
+public class FloatType implements SerializationTestType {
+
+	private float value;
+
+	public FloatType() {
+		this.value = 0;
+	}
+
+	private FloatType(float value) {
+		this.value = value;
+	}
+
+	@Override
+	public FloatType getRandom(Random rnd) {
+		return new FloatType(rnd.nextFloat());
+	}
+
+	@Override
+	public int length() {
+		return 4;
+	}
+
+	@Override
+	public void write(DataOutput out) throws IOException {
+		out.writeFloat(this.value);
+	}
+
+	@Override
+	public void read(DataInput in) throws IOException {
+		this.value = in.readFloat();
+	}
+
+	@Override
+	public int hashCode() {
+		return Float.floatToIntBits(this.value);
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj instanceof FloatType) {
+			FloatType other = (FloatType) obj;
+			return Float.floatToIntBits(this.value) == Float.floatToIntBits(other.value);
+		} else {
+			return false;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/IntType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/IntType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/IntType.java
new file mode 100644
index 0000000..50a3546
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/IntType.java
@@ -0,0 +1,67 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ **********************************************************************************************************************/
+
+package eu.stratosphere.runtime.io.serialization.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Random;
+
+public class IntType implements SerializationTestType {
+
+	private int value;
+
+	public IntType() {
+		this.value = 0;
+	}
+
+	public IntType(int value) {
+		this.value = value;
+	}
+
+	@Override
+	public IntType getRandom(Random rnd) {
+		return new IntType(rnd.nextInt());
+	}
+
+	@Override
+	public int length() {
+		return 4;
+	}
+
+	@Override
+	public void write(DataOutput out) throws IOException {
+		out.writeInt(this.value);
+	}
+
+	@Override
+	public void read(DataInput in) throws IOException {
+		this.value = in.readInt();
+	}
+
+	@Override
+	public int hashCode() {
+		return this.value;
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj instanceof IntType) {
+			IntType other = (IntType) obj;
+			return this.value == other.value;
+		} else {
+			return false;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/LongType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/LongType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/LongType.java
new file mode 100644
index 0000000..1402fb5
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/LongType.java
@@ -0,0 +1,67 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ **********************************************************************************************************************/
+
+package eu.stratosphere.runtime.io.serialization.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Random;
+
+public class LongType implements SerializationTestType {
+
+	private long value;
+
+	public LongType() {
+		this.value = 0;
+	}
+
+	private LongType(long value) {
+		this.value = value;
+	}
+
+	@Override
+	public LongType getRandom(Random rnd) {
+		return new LongType(rnd.nextLong());
+	}
+
+	@Override
+	public int length() {
+		return 8;
+	}
+
+	@Override
+	public void write(DataOutput out) throws IOException {
+		out.writeLong(this.value);
+	}
+
+	@Override
+	public void read(DataInput in) throws IOException {
+		this.value = in.readLong();
+	}
+
+	@Override
+	public int hashCode() {
+		return (int) (this.value ^ this.value >>> 32);
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj instanceof LongType) {
+			LongType other = (LongType) obj;
+			return this.value == other.value;
+		} else {
+			return false;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/SerializationTestType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/SerializationTestType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/SerializationTestType.java
new file mode 100644
index 0000000..a827b07
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/SerializationTestType.java
@@ -0,0 +1,26 @@
+/***********************************************************************************************************************
+ * 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.serialization.types;
+
+import java.util.Random;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+
+public interface SerializationTestType extends IOReadableWritable {
+
+	public SerializationTestType getRandom(Random rnd);
+
+	public int length();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/SerializationTestTypeFactory.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/SerializationTestTypeFactory.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/SerializationTestTypeFactory.java
new file mode 100644
index 0000000..127a0ec
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/SerializationTestTypeFactory.java
@@ -0,0 +1,40 @@
+/***********************************************************************************************************************
+ * 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.serialization.types;
+
+public enum SerializationTestTypeFactory {
+	BOOLEAN(new BooleanType()),
+	BYTE_ARRAY(new ByteArrayType()),
+	BYTE_SUB_ARRAY(new ByteSubArrayType()),
+	BYTE(new ByteType()),
+	CHAR(new CharType()),
+	DOUBLE(new DoubleType()),
+	FLOAT(new FloatType()),
+	INT(new IntType()),
+	LONG(new LongType()),
+	SHORT(new ShortType()),
+	UNSIGNED_BYTE(new UnsignedByteType()),
+	UNSIGNED_SHORT(new UnsignedShortType()),
+	STRING(new AsciiStringType());
+
+	private final SerializationTestType factory;
+
+	SerializationTestTypeFactory(SerializationTestType type) {
+		this.factory = type;
+	}
+
+	public SerializationTestType factory() {
+		return this.factory;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ShortType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ShortType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ShortType.java
new file mode 100644
index 0000000..7711e88
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ShortType.java
@@ -0,0 +1,67 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ **********************************************************************************************************************/
+
+package eu.stratosphere.runtime.io.serialization.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Random;
+
+public class ShortType implements SerializationTestType {
+
+	private short value;
+
+	public ShortType() {
+		this.value = (short) 0;
+	}
+
+	private ShortType(short value) {
+		this.value = value;
+	}
+
+	@Override
+	public ShortType getRandom(Random rnd) {
+		return new ShortType((short) rnd.nextInt(65536));
+	}
+
+	@Override
+	public int length() {
+		return 2;
+	}
+
+	@Override
+	public void write(DataOutput out) throws IOException {
+		out.writeShort(this.value);
+	}
+
+	@Override
+	public void read(DataInput in) throws IOException {
+		this.value = in.readShort();
+	}
+
+	@Override
+	public int hashCode() {
+		return this.value;
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj instanceof ShortType) {
+			ShortType other = (ShortType) obj;
+			return this.value == other.value;
+		} else {
+			return false;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/UnsignedByteType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/UnsignedByteType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/UnsignedByteType.java
new file mode 100644
index 0000000..9a1f1fb
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/UnsignedByteType.java
@@ -0,0 +1,67 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ **********************************************************************************************************************/
+
+package eu.stratosphere.runtime.io.serialization.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Random;
+
+public class UnsignedByteType implements SerializationTestType {
+
+	private int value;
+
+	public UnsignedByteType() {
+		this.value = 0;
+	}
+
+	private UnsignedByteType(int value) {
+		this.value = value;
+	}
+
+	@Override
+	public UnsignedByteType getRandom(Random rnd) {
+		return new UnsignedByteType(rnd.nextInt(128) + 128);
+	}
+
+	@Override
+	public int length() {
+		return 1;
+	}
+
+	@Override
+	public void write(DataOutput out) throws IOException {
+		out.writeByte(this.value);
+	}
+
+	@Override
+	public void read(DataInput in) throws IOException {
+		this.value = in.readUnsignedByte();
+	}
+
+	@Override
+	public int hashCode() {
+		return this.value;
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj instanceof UnsignedByteType) {
+			UnsignedByteType other = (UnsignedByteType) obj;
+			return this.value == other.value;
+		} else {
+			return false;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/UnsignedShortType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/UnsignedShortType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/UnsignedShortType.java
new file mode 100644
index 0000000..ac80ef7
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/UnsignedShortType.java
@@ -0,0 +1,67 @@
+/***********************************************************************************************************************
+ * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ **********************************************************************************************************************/
+
+package eu.stratosphere.runtime.io.serialization.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Random;
+
+public class UnsignedShortType implements SerializationTestType {
+
+	private int value;
+
+	public UnsignedShortType() {
+		this.value = 0;
+	}
+
+	private UnsignedShortType(int value) {
+		this.value = value;
+	}
+
+	@Override
+	public UnsignedShortType getRandom(Random rnd) {
+		return new UnsignedShortType(rnd.nextInt(32768) + 32768);
+	}
+
+	@Override
+	public int length() {
+		return 2;
+	}
+
+	@Override
+	public void write(DataOutput out) throws IOException {
+		out.writeShort(this.value);
+	}
+
+	@Override
+	public void read(DataInput in) throws IOException {
+		this.value = in.readUnsignedShort();
+	}
+
+	@Override
+	public int hashCode() {
+		return this.value;
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj instanceof UnsignedShortType) {
+			UnsignedShortType other = (UnsignedShortType) obj;
+			return this.value == other.value;
+		} else {
+			return false;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/Util.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/Util.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/Util.java
new file mode 100644
index 0000000..ef14f7f
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/Util.java
@@ -0,0 +1,90 @@
+/***********************************************************************************************************************
+ * 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.serialization.types;
+
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.Random;
+
+public class Util {
+
+	private static final long SEED = 64871654635745873L;
+
+	private static Random random = new Random(SEED);
+
+	public static SerializationTestType randomRecord(SerializationTestTypeFactory type) {
+		return type.factory().getRandom(Util.random);
+	}
+
+	public static MockRecords randomRecords(final int numElements, final SerializationTestTypeFactory type) {
+
+		return new MockRecords(numElements) {
+			@Override
+			protected SerializationTestType getRecord() {
+				return type.factory().getRandom(Util.random);
+			}
+		};
+	}
+
+	public static MockRecords randomRecords(final int numElements) {
+
+		return new MockRecords(numElements) {
+			@Override
+			protected SerializationTestType getRecord() {
+				// select random test type factory
+				SerializationTestTypeFactory[] types = SerializationTestTypeFactory.values();
+				int i = Util.random.nextInt(types.length);
+
+				return types[i].factory().getRandom(Util.random);
+			}
+		};
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+	public abstract static class MockRecords implements Iterable<SerializationTestType> {
+
+		private int numRecords;
+
+		public MockRecords(int numRecords) {
+			this.numRecords = numRecords;
+		}
+
+		@Override
+		public Iterator<SerializationTestType> iterator() {
+			return new Iterator<SerializationTestType>() {
+				@Override
+				public boolean hasNext() {
+					return numRecords > 0;
+				}
+
+				@Override
+				public SerializationTestType next() {
+					if (numRecords > 0) {
+						numRecords--;
+
+						return getRecord();
+					}
+
+					throw new NoSuchElementException();
+				}
+
+				@Override
+				public void remove() {
+					throw new UnsupportedOperationException();
+				}
+			};
+		}
+
+		abstract protected SerializationTestType getRecord();
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/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 db843dd..b80810b 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
@@ -21,6 +21,8 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import eu.stratosphere.test.util.RecordAPITestBase;
+import eu.stratosphere.nephele.jobgraph.DistributionPattern;
+import eu.stratosphere.runtime.io.channels.ChannelType;
 import org.junit.Assert;
 
 import eu.stratosphere.api.common.operators.util.UserCodeClassWrapper;
@@ -31,8 +33,6 @@ import eu.stratosphere.api.java.record.io.CsvInputFormat;
 import eu.stratosphere.api.java.record.io.CsvOutputFormat;
 import eu.stratosphere.configuration.Configuration;
 import eu.stratosphere.core.fs.Path;
-import eu.stratosphere.nephele.io.DistributionPattern;
-import eu.stratosphere.nephele.io.channels.ChannelType;
 import eu.stratosphere.nephele.jobgraph.JobGraph;
 import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException;
 import eu.stratosphere.nephele.jobgraph.JobInputVertex;