You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by uc...@apache.org on 2015/03/18 17:48:50 UTC

[01/13] flink git commit: [runtime] [tests] Run ProcessFailureBatchRecoveryITCase in BATCH and PIPELINED execution mode

Repository: flink
Updated Branches:
  refs/heads/master 1930678fb -> 4a49a73a7


[runtime] [tests] Run ProcessFailureBatchRecoveryITCase in BATCH and PIPELINED execution mode


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

Branch: refs/heads/master
Commit: cf6c63c897927d7c827fd7a22e657e2270ff82d0
Parents: 9c77f07
Author: Ufuk Celebi <uc...@apache.org>
Authored: Tue Mar 17 15:40:33 2015 +0100
Committer: Ufuk Celebi <uc...@apache.org>
Committed: Wed Mar 18 17:44:40 2015 +0100

----------------------------------------------------------------------
 .../partition/SpilledSubpartitionViewTest.java  |  3 +-
 .../ProcessFailureBatchRecoveryITCase.java      | 30 +++++++++++++++-----
 2 files changed, 24 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/cf6c63c8/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewTest.java
index 11037dc..fff7bc6 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewTest.java
@@ -76,8 +76,7 @@ public class SpilledSubpartitionViewTest {
 	public static Collection<Object[]> ioMode() {
 		return Arrays.asList(new Object[][]{
 				{IOMode.SYNC},
-				{IOMode.ASYNC}
-		});
+				{IOMode.ASYNC}});
 	}
 
 	@Test

http://git-wip-us.apache.org/repos/asf/flink/blob/cf6c63c8/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureBatchRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureBatchRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureBatchRecoveryITCase.java
index 7bb094b..6f5e698 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureBatchRecoveryITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureBatchRecoveryITCase.java
@@ -20,11 +20,10 @@ package org.apache.flink.test.recovery;
 
 import akka.actor.ActorRef;
 import akka.actor.ActorSystem;
-
 import akka.pattern.Patterns;
 import akka.util.Timeout;
 import org.apache.commons.io.FileUtils;
-
+import org.apache.flink.api.common.ExecutionMode;
 import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.java.DataSet;
@@ -37,12 +36,11 @@ import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.net.NetUtils;
 import org.apache.flink.runtime.taskmanager.TaskManager;
 import org.apache.flink.runtime.testutils.CommonTestUtils;
-
 import org.junit.Test;
-
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import scala.Some;
 import scala.Tuple2;
 import scala.concurrent.Await;
@@ -54,14 +52,17 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.StringWriter;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
 import static org.apache.flink.runtime.testutils.CommonTestUtils.getCurrentClasspath;
 import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath;
-
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.fail;
 
 /**
  * This test verifies the behavior of the recovery in the case when a TaskManager
@@ -77,6 +78,7 @@ import static org.junit.Assert.*;
  * the original task managers. The recovery should restart the tasks on the new TaskManager.
  */
 @SuppressWarnings("serial")
+@RunWith(Parameterized.class)
 public class ProcessFailureBatchRecoveryITCase {
 
 	private static final String READY_MARKER_FILE_PREFIX = "ready_";
@@ -84,6 +86,19 @@ public class ProcessFailureBatchRecoveryITCase {
 
 	private static final int PARALLELISM = 4;
 
+	private ExecutionMode executionMode;
+
+	public ProcessFailureBatchRecoveryITCase(ExecutionMode executionMode) {
+		this.executionMode = executionMode;
+	}
+
+	@Parameterized.Parameters
+	public static Collection<Object[]> executionMode() {
+		return Arrays.asList(new Object[][]{
+				{ExecutionMode.PIPELINED},
+				{ExecutionMode.BATCH}});
+	}
+
 	@Test
 	public void testTaskManagerProcessFailure() {
 
@@ -161,6 +176,7 @@ public class ProcessFailureBatchRecoveryITCase {
 			ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment("localhost", jobManagerPort);
 			env.setDegreeOfParallelism(PARALLELISM);
 			env.setNumberOfExecutionRetries(1);
+			env.getConfig().setExecutionMode(executionMode);
 
 			final long NUM_ELEMENTS = 100000L;
 			final DataSet<Long> result = env.generateSequence(1, NUM_ELEMENTS)


[08/13] flink git commit: [FLINK-1350] [runtime] Add blocking result partition variant

Posted by uc...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/IntermediateResultPartition.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/IntermediateResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/IntermediateResultPartition.java
deleted file mode 100644
index 174211a..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/IntermediateResultPartition.java
+++ /dev/null
@@ -1,319 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.io.network.partition;
-
-import akka.actor.ActorRef;
-import akka.dispatch.OnFailure;
-import akka.pattern.Patterns;
-import com.google.common.base.Optional;
-import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.deployment.PartitionDeploymentDescriptor;
-import org.apache.flink.runtime.execution.RuntimeEnvironment;
-import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.io.network.NetworkEnvironment;
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.buffer.BufferPool;
-import org.apache.flink.runtime.io.network.buffer.BufferPoolOwner;
-import org.apache.flink.runtime.io.network.buffer.BufferProvider;
-import org.apache.flink.runtime.io.network.partition.queue.IllegalQueueIteratorRequestException;
-import org.apache.flink.runtime.io.network.partition.queue.IntermediateResultPartitionQueue;
-import org.apache.flink.runtime.io.network.partition.queue.IntermediateResultPartitionQueueIterator;
-import org.apache.flink.runtime.io.network.partition.queue.PipelinedPartitionQueue;
-import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
-import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionType;
-import org.apache.flink.runtime.jobgraph.JobID;
-import org.apache.flink.runtime.messages.JobManagerMessages.ScheduleOrUpdateConsumers;
-import org.apache.flink.runtime.messages.TaskManagerMessages.FailTask;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.concurrent.Future;
-
-import java.io.IOException;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
-import static com.google.common.base.Preconditions.checkState;
-
-public class IntermediateResultPartition implements BufferPoolOwner {
-
-	private static final Logger LOG = LoggerFactory.getLogger(IntermediateResultPartition.class);
-
-	private final RuntimeEnvironment environment;
-
-	/**
-	 * Note: This index needs to correspond to the index of the partition in
-	 * {@link ExecutionVertex#resultPartitions}, which might be a little
-	 * fragile as the data availability notifications use it.
-	 */
-	private final int partitionIndex;
-
-	private final JobID jobId;
-
-	private final ExecutionAttemptID producerExecutionId;
-
-	private final IntermediateResultPartitionID partitionId;
-
-	private final IntermediateResultPartitionType partitionType;
-
-	private final NetworkEnvironment networkEnvironment;
-
-	private final IntermediateResultPartitionQueue[] queues;
-
-	private volatile boolean hasNotifiedConsumers;
-
-	private volatile boolean isReleased;
-
-	private boolean isFinished;
-
-	private BufferPool bufferPool;
-
-	public IntermediateResultPartition(RuntimeEnvironment environment, int partitionIndex, JobID jobId,
-			ExecutionAttemptID executionId, IntermediateResultPartitionID partitionId, IntermediateResultPartitionType partitionType,
-			IntermediateResultPartitionQueue[] partitionQueues, NetworkEnvironment networkEnvironment) {
-		this.environment = environment;
-		this.partitionIndex = partitionIndex;
-		this.jobId = jobId;
-		this.producerExecutionId = executionId;
-		this.partitionId = partitionId;
-		this.partitionType = partitionType;
-		this.networkEnvironment = networkEnvironment;
-		this.queues = partitionQueues;
-	}
-
-	// ------------------------------------------------------------------------
-	// Properties
-	// ------------------------------------------------------------------------
-
-	public void setBufferPool(BufferPool bufferPool) {
-		checkArgument(bufferPool.getNumberOfRequiredMemorySegments() == getNumberOfQueues(),
-				"Buffer pool has not enough buffers for this intermediate result.");
-		checkState(this.bufferPool == null, "Buffer pool has already been set for intermediate result partition.");
-
-		this.bufferPool = checkNotNull(bufferPool);
-	}
-
-	public ExecutionAttemptID getProducerExecutionId() {
-		return producerExecutionId;
-	}
-
-	public IntermediateResultPartitionID getPartitionId() {
-		return partitionId;
-	}
-
-	public JobID getJobId() {
-		return jobId;
-	}
-
-	public int getNumberOfQueues() {
-		return queues.length;
-	}
-
-	public BufferProvider getBufferProvider() {
-		return bufferPool;
-	}
-
-	public boolean isFinished() {
-		return isFinished;
-	}
-
-	// ------------------------------------------------------------------------
-	// Produce
-	// ------------------------------------------------------------------------
-
-	public void add(Buffer buffer, int targetQueue) throws IOException {
-		synchronized (queues) {
-			if (isReleased) {
-				buffer.recycle();
-			}
-			else {
-				checkInProducePhase();
-				queues[targetQueue].add(buffer);
-			}
-		}
-
-		maybeNotifyConsumers(partitionType.isPipelined());
-	}
-
-	public void finish() throws IOException {
-		boolean success = false;
-
-		synchronized (queues) {
-			checkInProducePhase();
-
-			try {
-				if (!isReleased) {
-					for (IntermediateResultPartitionQueue queue : queues) {
-						queue.finish();
-					}
-
-					success = true;
-				}
-			}
-			finally {
-				isFinished = true;
-			}
-		}
-
-		if (success) {
-			// Notify at this point in any case either because of the end
-			// of a blocking result or an empty pipelined result.
-			maybeNotifyConsumers(true);
-
-			if (!partitionType.isPersistent() && bufferPool != null) {
-				// If this partition is not persistent, immediately destroy
-				// the buffer pool. For persistent intermediate results, the
-				// partition manager needs to release the buffer pool.
-				bufferPool.destroy();
-			}
-		}
-	}
-
-	public void releaseAllResources() throws IOException {
-		synchronized (queues) {
-			LOG.debug("Release all resources of {}.", this);
-
-			if (!isReleased) {
-				try {
-					for (IntermediateResultPartitionQueue queue : queues) {
-						try {
-							queue.discard();
-						}
-						catch (Throwable t) {
-							LOG.error("Error while discarding queue: " + t.getMessage(), t);
-						}
-					}
-
-					if (bufferPool != null) {
-						bufferPool.destroy();
-					}
-				}
-				finally {
-					isReleased = true;
-				}
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	// Consume
-	// ------------------------------------------------------------------------
-
-	public IntermediateResultPartitionQueueIterator getQueueIterator(int queueIndex, Optional<BufferProvider> bufferProvider)
-			throws IOException {
-		synchronized (queues) {
-			if (isReleased) {
-				throw new IllegalQueueIteratorRequestException("Intermediate result partition has already been released.");
-			}
-
-			if (queueIndex < 0 || queueIndex >= queues.length) {
-				throw new IllegalQueueIteratorRequestException("Illegal queue index: " + queueIndex + ", allowed: 0-" + (queues.length - 1));
-			}
-
-			return queues[queueIndex].getQueueIterator(bufferProvider);
-		}
-	}
-
-	// ------------------------------------------------------------------------
-
-	private void checkInProducePhase() {
-		checkState(!isReleased, "Partition has already been discarded.");
-		checkState(!isFinished, "Partition has already been finished.");
-	}
-
-	/**
-	 * Maybe notifies consumers of this result partition.
-	 */
-	private void maybeNotifyConsumers(boolean doNotify) throws IOException {
-		if (doNotify && !hasNotifiedConsumers) {
-			scheduleOrUpdateConsumers();
-			hasNotifiedConsumers = true;
-		}
-	}
-
-	private void scheduleOrUpdateConsumers() throws IOException {
-		if(!isReleased){
-			ScheduleOrUpdateConsumers msg = new ScheduleOrUpdateConsumers(jobId,
-					producerExecutionId, partitionIndex);
-
-			Future<Object> futureResponse = Patterns.ask(networkEnvironment.getJobManager(), msg,
-					networkEnvironment.getJobManagerTimeout());
-
-			futureResponse.onFailure(new OnFailure(){
-				@Override
-				public void onFailure(Throwable failure) throws Throwable {
-					LOG.error("Could not schedule or update consumers at the JobManager.", failure);
-
-					// Fail task at the TaskManager
-					FailTask failMsg = new FailTask(producerExecutionId,
-							new RuntimeException("Could not schedule or update consumers at " +
-									"the JobManager.", failure));
-
-					networkEnvironment.getTaskManager().tell(failMsg, ActorRef.noSender());
-				}
-			}, AkkaUtils.globalExecutionContext());
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	// Buffer pool owner methods
-	// ------------------------------------------------------------------------
-
-	/**
-	 * If this partition is registered as the {@link BufferPoolOwner} of a
-	 * {@link BufferPool}, it will forward the requests to the queues.
-	 */
-	@Override
-	public void recycleBuffers(int numBuffersToRecycle) throws IOException {
-		int numRecycledBuffers = 0;
-
-		for (IntermediateResultPartitionQueue queue : queues) {
-			numRecycledBuffers += queue.recycleBuffers();
-
-			if (numRecycledBuffers >= numBuffersToRecycle) {
-				break;
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-
-	public static IntermediateResultPartition create(RuntimeEnvironment environment, int partitionIndex, JobID jobId,
-			ExecutionAttemptID executionId, NetworkEnvironment networkEnvironment, PartitionDeploymentDescriptor desc) {
-		final IntermediateResultPartitionID partitionId = checkNotNull(desc.getPartitionId());
-		final IntermediateResultPartitionType partitionType = checkNotNull(desc.getPartitionType());
-
-		final IntermediateResultPartitionQueue[] partitionQueues = new IntermediateResultPartitionQueue[desc.getNumberOfQueues()];
-
-		// TODO The queues need to be created depending on the result type
-		for (int i = 0; i < partitionQueues.length; i++) {
-			partitionQueues[i] = new PipelinedPartitionQueue();
-		}
-
-		return new IntermediateResultPartition(environment, partitionIndex, jobId, executionId, partitionId, partitionType,
-				partitionQueues, networkEnvironment);
-	}
-
-	@Override
-	public String toString() {
-		return String.format("IntermediateResultPartition(JobID: %s, ExecutionID: %s, " +
-				"PartitionID: %s, PartitionType: %s, [num queues: %d, (isFinished: %b)",
-				jobId, producerExecutionId, partitionId, partitionType, queues.length, isFinished);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/IntermediateResultPartitionManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/IntermediateResultPartitionManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/IntermediateResultPartitionManager.java
deleted file mode 100644
index d5b8fe5..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/IntermediateResultPartitionManager.java
+++ /dev/null
@@ -1,139 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.io.network.partition;
-
-import com.google.common.base.Optional;
-import com.google.common.collect.HashBasedTable;
-import com.google.common.collect.Table;
-import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.io.network.buffer.BufferProvider;
-import org.apache.flink.runtime.io.network.partition.queue.IllegalQueueIteratorRequestException;
-import org.apache.flink.runtime.io.network.partition.queue.IntermediateResultPartitionQueueIterator;
-import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-/**
- * The intermediate result partition manager keeps track of all available
- * partitions of a task manager and
- */
-public class IntermediateResultPartitionManager implements IntermediateResultPartitionProvider {
-
-	private static final Logger LOG = LoggerFactory.getLogger(IntermediateResultPartitionManager.class);
-
-	private final Table<ExecutionAttemptID, IntermediateResultPartitionID,
-			IntermediateResultPartition> partitions = HashBasedTable.create();
-
-	private boolean isShutdown;
-
-	public void registerIntermediateResultPartition(IntermediateResultPartition partition) throws IOException {
-		synchronized (partitions) {
-			LOG.debug("Register intermediate result partition {}.", partition);
-
-			if (isShutdown) {
-				throw new IOException("Intermediate result partition manager has already been shut down.");
-			}
-
-			if (partitions.put(partition.getProducerExecutionId(), partition.getPartitionId(), partition) != null) {
-				throw new IOException("Tried to re-register intermediate result partition.");
-			}
-		}
-	}
-
-	public void failIntermediateResultPartitions(ExecutionAttemptID producerExecutionId) {
-		synchronized (partitions) {
-			List<IntermediateResultPartition> partitionsToFail = new ArrayList<IntermediateResultPartition>();
-
-			for (IntermediateResultPartitionID partitionId : partitions.row(producerExecutionId).keySet()) {
-				partitionsToFail.add(partitions.get(producerExecutionId, partitionId));
-			}
-
-			for(IntermediateResultPartition partition : partitionsToFail) {
-				failIntermediateResultPartition(partition);
-			}
-		}
-	}
-
-	private void failIntermediateResultPartition(IntermediateResultPartition partition) {
-		if (partition != null) {
-			try {
-				partition.releaseAllResources();
-			}
-			catch (Throwable t) {
-				LOG.error("Error during release of produced intermediate result partition: " + t.getMessage(), t);
-			}
-		}
-	}
-
-	public void shutdown() {
-		synchronized (partitions) {
-			for (IntermediateResultPartition partition : partitions.values()) {
-				try {
-					partition.releaseAllResources();
-				}
-				catch (IOException e) {
-					LOG.error("Error while releasing intermediate result partition: " + e.getMessage(), e);
-				}
-			}
-
-			isShutdown = true;
-		}
-	}
-
-	public int getNumberOfRegisteredPartitions() {
-		synchronized (partitions) {
-			return partitions.size();
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	// Intermediate result partition provider
-	// ------------------------------------------------------------------------
-
-	@Override
-	public IntermediateResultPartitionQueueIterator getIntermediateResultPartitionIterator(
-			ExecutionAttemptID producerExecutionId,
-			IntermediateResultPartitionID partitionId,
-			int queueIndex,
-			Optional<BufferProvider> bufferProvider) throws IOException {
-
-		synchronized (partitions) {
-			IntermediateResultPartition partition = partitions.get(producerExecutionId, partitionId);
-
-			if (partition == null) {
-				if (!partitions.containsRow(producerExecutionId)) {
-					LOG.debug("Could not find producer execution ID {}. Registered producer" +
-							" execution IDs {}.", producerExecutionId,
-							Arrays.toString(partitions.rowKeySet().toArray()));
-
-					throw new IllegalQueueIteratorRequestException("Unknown producer execution ID " + producerExecutionId + ".");
-				}
-
-				throw new IllegalQueueIteratorRequestException("Unknown partition " + partitionId + ".");
-			}
-
-			return partition.getQueueIterator(queueIndex, bufferProvider);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/IntermediateResultPartitionProvider.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/IntermediateResultPartitionProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/IntermediateResultPartitionProvider.java
deleted file mode 100644
index b18b3fc..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/IntermediateResultPartitionProvider.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.io.network.partition;
-
-import com.google.common.base.Optional;
-import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.io.network.buffer.BufferProvider;
-import org.apache.flink.runtime.io.network.partition.queue.IntermediateResultPartitionQueueIterator;
-import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
-
-import java.io.IOException;
-
-public interface IntermediateResultPartitionProvider {
-
-	IntermediateResultPartitionQueueIterator getIntermediateResultPartitionIterator(
-			ExecutionAttemptID producerExecutionId,
-			IntermediateResultPartitionID partitionId,
-			int requestedQueueIndex,
-			Optional<BufferProvider> bufferProvider) throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
new file mode 100644
index 0000000..a8a0a7b
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition;
+
+import com.google.common.base.Optional;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferProvider;
+import org.apache.flink.runtime.util.event.NotificationListener;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayDeque;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * A pipelined in-memory only subpartition, which can be consumed once.
+ */
+class PipelinedSubpartition extends ResultSubpartition {
+
+	private static final Logger LOG = LoggerFactory.getLogger(PipelinedSubpartition.class);
+
+	/** Flag indicating whether the subpartition has been finished. */
+	private boolean isFinished;
+
+	/** Flag indicating whether the subpartition has been released. */
+	private boolean isReleased;
+
+	/**
+	 * A data availability listener. Registered, when the consuming task is faster than the
+	 * producing task.
+	 */
+	private NotificationListener registeredListener;
+
+	/** The read view to consume this subpartition. */
+	private PipelinedSubpartitionView readView;
+
+	/** All buffers of this subpartition. Access to the buffers is synchronized on this object. */
+	final ArrayDeque<Buffer> buffers = new ArrayDeque<Buffer>();
+
+	PipelinedSubpartition(int index, ResultPartition parent) {
+		super(index, parent);
+	}
+
+	@Override
+	public boolean add(Buffer buffer) {
+		checkNotNull(buffer);
+
+		final NotificationListener listener;
+
+		synchronized (buffers) {
+			if (isReleased || isFinished) {
+				return false;
+			}
+
+			// Add the buffer and update the stats
+			buffers.add(buffer);
+			updateStatistics(buffer);
+
+			// Get the listener...
+			listener = registeredListener;
+			registeredListener = null;
+		}
+
+		// Notify the listener outside of the synchronized block
+		if (listener != null) {
+			listener.onNotification();
+		}
+
+		return true;
+	}
+
+	@Override
+	public void finish() {
+		final NotificationListener listener;
+
+		synchronized (buffers) {
+			if (isReleased || isFinished) {
+				return;
+			}
+
+			final Buffer buffer = EventSerializer.toBuffer(EndOfPartitionEvent.INSTANCE);
+
+			buffers.add(buffer);
+			updateStatistics(buffer);
+
+			isFinished = true;
+
+			LOG.debug("Finished {}.", this);
+
+			// Get the listener...
+			listener = registeredListener;
+			registeredListener = null;
+		}
+
+		// Notify the listener outside of the synchronized block
+		if (listener != null) {
+			listener.onNotification();
+		}
+	}
+
+	@Override
+	public void release() {
+		final NotificationListener listener;
+		final PipelinedSubpartitionView view;
+
+		synchronized (buffers) {
+			if (isReleased) {
+				return;
+			}
+
+			// Release all available buffers
+			Buffer buffer;
+			while ((buffer = buffers.poll()) != null) {
+				if (!buffer.isRecycled()) {
+					buffer.recycle();
+				}
+			}
+
+			// Get the view...
+			view = readView;
+			readView = null;
+
+			// Get the listener...
+			listener = registeredListener;
+			registeredListener = null;
+
+			// Make sure that no further buffers are added to the subpartition
+			isReleased = true;
+
+			LOG.debug("Released {}.", this);
+		}
+
+		// Release all resources of the view
+		if (view != null) {
+			view.releaseAllResources();
+		}
+
+		// Notify the listener outside of the synchronized block
+		if (listener != null) {
+			listener.onNotification();
+		}
+	}
+
+	@Override
+	public int releaseMemory() {
+		// The pipelined subpartition does not react to memory release requests. The buffers will be
+		// recycled by the consuming task.
+		return 0;
+	}
+
+	@Override
+	public PipelinedSubpartitionView getReadView(Optional<BufferProvider> bufferProvider) {
+		synchronized (buffers) {
+			if (readView != null) {
+				throw new IllegalStateException("Subpartition is being or already has been " +
+						"consumed, but pipelined subpartitions can only be consumed once.");
+			}
+
+			readView = new PipelinedSubpartitionView(this);
+
+			LOG.debug("Created {}.", readView);
+
+			return readView;
+		}
+	}
+
+	@Override
+	public String toString() {
+		synchronized (buffers) {
+			return String.format("PipelinedSubpartition [number of buffers: %d (%d bytes), " +
+							"finished? %s, read view? %s]",
+					getTotalNumberOfBuffers(), getTotalNumberOfBytes(), isFinished, readView != null);
+		}
+	}
+
+	/**
+	 * Registers a listener with this subpartition and returns whether the registration was
+	 * successful.
+	 *
+	 * <p> A registered listener is notified when the state of the subpartition changes. After a
+	 * notification, the listener is unregistered. Only a single listener is allowed to be
+	 * registered.
+	 */
+	boolean registerListener(NotificationListener listener) {
+		synchronized (buffers) {
+			if (!buffers.isEmpty() || isReleased) {
+				return false;
+			}
+
+			if (registeredListener == null) {
+				registeredListener = listener;
+
+				return true;
+			}
+
+			throw new IllegalStateException("Already registered listener.");
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionView.java
new file mode 100644
index 0000000..822e33a
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionView.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition;
+
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.util.event.NotificationListener;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * View over a pipelined in-memory only subpartition.
+ */
+class PipelinedSubpartitionView implements ResultSubpartitionView {
+
+	/** The subpartition this view belongs to. */
+	private final PipelinedSubpartition parent;
+
+	/** Flag indicating whether this view has been released. */
+	private AtomicBoolean isReleased = new AtomicBoolean();
+
+	PipelinedSubpartitionView(PipelinedSubpartition parent) {
+		this.parent = checkNotNull(parent);
+	}
+
+	@Override
+	public Buffer getNextBuffer() {
+		synchronized (parent.buffers) {
+			return parent.buffers.poll();
+		}
+	}
+
+	@Override
+	public boolean registerListener(NotificationListener listener) {
+		return !isReleased.get() && parent.registerListener(listener);
+
+	}
+
+	@Override
+	public void notifySubpartitionConsumed() {
+		releaseAllResources();
+	}
+
+	@Override
+	public void releaseAllResources() {
+		if (isReleased.compareAndSet(false, true)) {
+			// The view doesn't hold any resources and the parent cannot be restarted. Therefore,
+			// it's OK to notify about consumption as well.
+			parent.onConsumedSubpartition();
+		}
+	}
+
+	@Override
+	public boolean isReleased() {
+		return isReleased.get();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java
new file mode 100644
index 0000000..95aa636
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java
@@ -0,0 +1,421 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition;
+
+import akka.actor.ActorRef;
+import akka.dispatch.OnFailure;
+import akka.pattern.Patterns;
+import com.google.common.base.Optional;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.executiongraph.IntermediateResultPartition;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.io.network.NetworkEnvironment;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.BufferPoolOwner;
+import org.apache.flink.runtime.io.network.buffer.BufferProvider;
+import org.apache.flink.runtime.io.network.partition.consumer.LocalInputChannel;
+import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.runtime.taskmanager.TaskManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.concurrent.Future;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
+import static org.apache.flink.runtime.messages.JobManagerMessages.ScheduleOrUpdateConsumers;
+import static org.apache.flink.runtime.messages.TaskManagerMessages.FailTask;
+
+/**
+ * A result partition for data produced by a single task.
+ *
+ * <p> This class is the runtime part of a logical {@link IntermediateResultPartition}. Essentially,
+ * a result partition is a collection of {@link Buffer} instances. The buffers are organized in one
+ * or more {@link ResultSubpartition} instances, which further partition the data depending on the
+ * number of consuming tasks and the data {@link DistributionPattern}.
+ *
+ * <p> Tasks, which consume a result partition have to request one of its subpartitions. The request
+ * happens either remotely (see {@link RemoteInputChannel}) or locally (see {@link LocalInputChannel})
+ *
+ * <h2>Life-cycle</h2>
+ *
+ * The life-cycle of each result partition has three (possibly overlapping) phases:
+ * <ol>
+ * <li><strong>Produce</strong>: </li>
+ * <li><strong>Consume</strong>: </li>
+ * <li><strong>Release</strong>: </li>
+ * </ol>
+ *
+ * <h2>Lazy deployment and updates of consuming tasks</h2>
+ *
+ * Before a consuming task can request the result, it has to be deployed. The time of deployment
+ * depends on the PIPELINED vs. BLOCKING characteristic of the result partition. With pipelined
+ * results, receivers are deployed as soon as the first buffer is added to the result partition.
+ * With blocking results on the other hand, receivers are deployed after the partition is finished.
+ *
+ * <h2>Buffer management</h2>
+ *
+ * <h2>State management</h2>
+ */
+public class ResultPartition implements BufferPoolOwner {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ResultPartition.class);
+
+	private final JobID jobId;
+
+	private final ResultPartitionID partitionId;
+
+	/** Type of this partition. Defines the concrete subpartition implementation to use. */
+	private final ResultPartitionType partitionType;
+
+	/** The subpartitions of this partition. At least one. */
+	private final ResultSubpartition[] subpartitions;
+
+	private final NetworkEnvironment networkEnvironment;
+
+	// - Runtime state --------------------------------------------------------
+
+	private final AtomicBoolean isReleased = new AtomicBoolean();
+
+	/**
+	 * The total number of references to subpartitions of this result. The result partition can be
+	 * safely released, iff the reference count is zero. A reference count of -1 denotes that the
+	 * result partition has been released.
+	 */
+	private final AtomicInteger pendingReferences = new AtomicInteger();
+
+	private BufferPool bufferPool;
+
+	private boolean hasNotifiedPipelinedConsumers;
+
+	private boolean isFinished;
+
+	// - Statistics ----------------------------------------------------------
+
+	/** The total number of buffers (both data and event buffers) */
+	private int totalNumberOfBuffers;
+
+	/** The total number of bytes (both data and event buffers) */
+	private long totalNumberOfBytes;
+
+	public ResultPartition(
+			JobID jobId,
+			ResultPartitionID partitionId,
+			ResultPartitionType partitionType,
+			int numberOfSubpartitions,
+			NetworkEnvironment networkEnvironment,
+			IOManager ioManager) {
+
+		this.jobId = checkNotNull(jobId);
+		this.partitionId = checkNotNull(partitionId);
+		this.partitionType = checkNotNull(partitionType);
+		this.subpartitions = new ResultSubpartition[numberOfSubpartitions];
+		this.networkEnvironment = checkNotNull(networkEnvironment);
+
+		// Create the subpartitions.
+		switch (partitionType) {
+			case BLOCKING:
+				for (int i = 0; i < subpartitions.length; i++) {
+					subpartitions[i] = new SpillableSubpartition(
+							i, this, ioManager, networkEnvironment.getDefaultIOMode());
+				}
+
+				break;
+
+			case PIPELINED:
+				for (int i = 0; i < subpartitions.length; i++) {
+					subpartitions[i] = new PipelinedSubpartition(i, this);
+				}
+
+				break;
+
+			default:
+				throw new IllegalArgumentException("Unsupported result partition type.");
+		}
+
+		// Initially, partitions should be consumed once before release.
+		pin();
+
+		LOG.debug("Initialized {}", this);
+	}
+
+	/**
+	 * Registers a buffer pool with this result partition.
+	 * <p>
+	 * There is one pool for each result partition, which is shared by all its sub partitions.
+	 * <p>
+	 * The pool is registered with the partition *after* it as been constructed in order to conform
+	 * to the life-cycle of task registrations in the {@link TaskManager}.
+	 */
+	public void registerBufferPool(BufferPool bufferPool) {
+		checkArgument(bufferPool.getNumberOfRequiredMemorySegments() == getNumberOfSubpartitions(),
+				"Bug in result partition setup logic: Buffer pool has not enough guaranteed buffers for this result partition.");
+
+		checkState(this.bufferPool == null, "Bug in result partition setup logic: Already registered buffer pool.");
+
+		this.bufferPool = checkNotNull(bufferPool);
+
+		// If the partition type is back pressure-free, we register with the buffer pool for
+		// callbacks to release memory.
+		if (!partitionType.hasBackPressure()) {
+			bufferPool.setBufferPoolOwner(this);
+		}
+	}
+
+	public JobID getJobId() {
+		return jobId;
+	}
+
+	public ResultPartitionID getPartitionId() {
+		return partitionId;
+	}
+
+	public int getNumberOfSubpartitions() {
+		return subpartitions.length;
+	}
+
+	public BufferProvider getBufferProvider() {
+		return bufferPool;
+	}
+
+	public int getTotalNumberOfBuffers() {
+		return totalNumberOfBuffers;
+	}
+
+	public long getTotalNumberOfBytes() {
+		return totalNumberOfBytes;
+	}
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Adds a buffer to the subpartition with the given index.
+	 *
+	 * <p> For PIPELINED results, this will trigger the deployment of consuming tasks after the
+	 * first buffer has been added.
+	 */
+	public void add(Buffer buffer, int subpartitionIndex) throws IOException {
+		boolean success = false;
+
+		try {
+			checkInProduceState();
+
+			final ResultSubpartition subpartition = subpartitions[subpartitionIndex];
+
+			synchronized (subpartition) {
+				success = subpartition.add(buffer);
+
+				// Update statistics
+				totalNumberOfBuffers++;
+				totalNumberOfBytes += buffer.getSize();
+			}
+		}
+		finally {
+			if (success) {
+				notifyPipelinedConsumers();
+			}
+			else {
+				buffer.recycle();
+			}
+		}
+	}
+
+	/**
+	 * Finishes the result partition.
+	 *
+	 * <p> After this operation, it is not possible to add further data to the result partition.
+	 *
+	 * <p> For BLOCKING results, this will trigger the deployment of consuming tasks.
+	 */
+	public void finish() throws IOException {
+		boolean success = false;
+
+		try {
+			checkInProduceState();
+
+			for (ResultSubpartition subpartition : subpartitions) {
+				synchronized (subpartition) {
+					subpartition.finish();
+				}
+			}
+
+			success = true;
+		}
+		finally {
+			if (success) {
+				isFinished = true;
+
+				notifyPipelinedConsumers();
+			}
+		}
+	}
+
+	/**
+	 * Releases the result partition.
+	 */
+	public void release() {
+		if (isReleased.compareAndSet(false, true)) {
+			LOG.debug("Releasing {}", this);
+
+			try {
+				for (ResultSubpartition subpartition : subpartitions) {
+					try {
+						synchronized (subpartition) {
+							subpartition.release();
+						}
+					}
+					// Catch this in order to ensure that release is called on all subpartitions
+					catch (Throwable t) {
+						LOG.error("Error during release of result subpartition: " + t.getMessage(), t);
+					}
+				}
+			}
+			finally {
+				if (bufferPool != null) {
+					bufferPool.lazyDestroy();
+				}
+			}
+		}
+	}
+
+	/**
+	 * Returns the requested subpartition.
+	 */
+	public ResultSubpartitionView getSubpartition(int index, Optional<BufferProvider> bufferProvider) throws IOException {
+		int refCnt = pendingReferences.get();
+
+		checkState(refCnt != -1, "Partition released.");
+		checkState(refCnt > 0, "Partition not pinned.");
+
+		return subpartitions[index].getReadView(bufferProvider);
+	}
+
+	/**
+	 * Releases buffers held by this result partition.
+	 *
+	 * <p> This is a callback from the buffer pool, which is registered for result partitions, which
+	 * are back pressure-free.
+	 */
+	@Override
+	public void releaseMemory(int toRelease) throws IOException {
+		checkArgument(toRelease > 0);
+
+		for (ResultSubpartition subpartition : subpartitions) {
+			toRelease -= subpartition.releaseMemory();
+
+			// Only release as much memory as needed
+			if (toRelease <= 0) {
+				break;
+			}
+		}
+	}
+
+	@Override
+	public String toString() {
+		return "ResultPartition " + partitionId.toString() + " [" + partitionType + ", "
+				+ subpartitions.length + " subpartitions, "
+				+ pendingReferences + " pending references]";
+	}
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Pins the result partition.
+	 *
+	 * <p> The partition can only be released after each subpartition has been consumed once per pin
+	 * operation.
+	 */
+	void pin() {
+		while (true) {
+			int refCnt = pendingReferences.get();
+
+			if (refCnt >= 0) {
+				if (pendingReferences.compareAndSet(refCnt, refCnt + subpartitions.length)) {
+					break;
+				}
+			}
+			else {
+				throw new IllegalStateException("Released.");
+			}
+		}
+	}
+
+	/**
+	 * Notification when a subpartition is released.
+	 */
+	void onConsumedSubpartition(int subpartitionIndex) {
+
+		if (isReleased.get()) {
+			return;
+		}
+
+		int refCnt = pendingReferences.decrementAndGet();
+
+		if (refCnt == 0) {
+			networkEnvironment.getPartitionManager().onConsumedPartition(this);
+		}
+		else if (refCnt < 0) {
+			throw new IllegalStateException("All references released.");
+		}
+
+		LOG.debug("{}: Received release notification for subpartition {} (reference count now at: {}).",
+				this, subpartitionIndex, pendingReferences);
+	}
+
+	// ------------------------------------------------------------------------
+
+	private void checkInProduceState() {
+		checkState(!isFinished, "Partition already finished.");
+	}
+
+	/**
+	 * Notifies pipelined consumers of this result partition once.
+	 */
+	private void notifyPipelinedConsumers() throws IOException {
+		if (partitionType.isPipelined() && !hasNotifiedPipelinedConsumers) {
+			ScheduleOrUpdateConsumers msg = new ScheduleOrUpdateConsumers(jobId, partitionId);
+
+			Future<Object> futureResponse = Patterns.ask(networkEnvironment.getJobManager(), msg,
+					networkEnvironment.getJobManagerTimeout());
+
+			futureResponse.onFailure(new OnFailure() {
+				@Override
+				public void onFailure(Throwable failure) throws Throwable {
+					LOG.error("Could not schedule or update consumers at the JobManager.", failure);
+
+					// Fail task at the TaskManager
+					FailTask failMsg = new FailTask(partitionId.getProducerId(),
+							new RuntimeException("Could not schedule or update consumers at " +
+									"the JobManager.", failure));
+
+					networkEnvironment.getTaskManager().tell(failMsg, ActorRef.noSender());
+				}
+			}, AkkaUtils.globalExecutionContext());
+
+			hasNotifiedPipelinedConsumers = true;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionID.java
new file mode 100644
index 0000000..af2970d
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionID.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition;
+
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.IntermediateResultPartition;
+import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
+
+import java.io.Serializable;
+
+/**
+ * Runtime identifier of a produced {@link IntermediateResultPartition}.
+ *
+ * <p> In failure cases the {@link IntermediateResultPartitionID} is not enough to uniquely
+ * identify a result partition. It needs to be associated with the producing task as well to ensure
+ * correct tracking of failed/restarted tasks.
+ */
+public final class ResultPartitionID implements Serializable {
+
+	private final IntermediateResultPartitionID partitionId;
+
+	private final ExecutionAttemptID producerId;
+
+	public ResultPartitionID() {
+		this(new IntermediateResultPartitionID(), new ExecutionAttemptID());
+	}
+
+	public ResultPartitionID(IntermediateResultPartitionID partitionId, ExecutionAttemptID producerId) {
+		this.partitionId = partitionId;
+		this.producerId = producerId;
+	}
+
+	public IntermediateResultPartitionID getPartitionId() {
+		return partitionId;
+	}
+
+	public ExecutionAttemptID getProducerId() {
+		return producerId;
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj != null && obj.getClass() == ResultPartitionID.class) {
+			ResultPartitionID o = (ResultPartitionID) obj;
+
+			return o.getPartitionId().equals(partitionId) && o.getProducerId().equals(producerId);
+		}
+
+		return false;
+	}
+
+	@Override
+	public int hashCode() {
+		return partitionId.hashCode() ^ producerId.hashCode();
+	}
+
+	@Override
+	public String toString() {
+		return partitionId.toShortString() + "@" + producerId.toShortString();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManager.java
new file mode 100644
index 0000000..c120de8
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManager.java
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Table;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.io.network.buffer.BufferProvider;
+import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Map;
+
+import static com.google.common.base.Preconditions.checkState;
+
+/**
+ * The result partition manager keeps track of all currently produced/consumed partitions of a
+ * task manager.
+ */
+public class ResultPartitionManager implements ResultPartitionProvider {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ResultPartitionManager.class);
+
+	public final Table<ExecutionAttemptID, IntermediateResultPartitionID, ResultPartition>
+			registeredPartitions = HashBasedTable.create();
+
+	private boolean isShutdown;
+
+	public void registerIntermediateResultPartition(ResultPartition partition) throws IOException {
+		synchronized (registeredPartitions) {
+			checkState(!isShutdown, "Result partition manager already shut down.");
+
+			ResultPartitionID partitionId = partition.getPartitionId();
+
+			ResultPartition previous = registeredPartitions.put(
+					partitionId.getProducerId(), partitionId.getPartitionId(), partition);
+
+			if (previous != null) {
+				throw new IllegalStateException("Result partition already registered.");
+			}
+
+			LOG.debug("Registered {}.", partition);
+		}
+	}
+
+	@Override
+	public ResultSubpartitionView getSubpartition(
+			ResultPartitionID partitionId,
+			int subpartitionIndex,
+			Optional<BufferProvider> bufferProvider) throws IOException {
+
+		synchronized (registeredPartitions) {
+			final ResultPartition partition = registeredPartitions.get(partitionId.getProducerId(),
+					partitionId.getPartitionId());
+
+			if (partition == null) {
+				throw new IOException("Unknown partition " + partitionId + ".");
+			}
+
+			LOG.debug("Requested partition {}.", partition);
+
+			return partition.getSubpartition(subpartitionIndex, bufferProvider);
+		}
+	}
+
+	public void releasePartitionsProducedBy(ExecutionAttemptID executionId) {
+		synchronized (registeredPartitions) {
+			final Map<IntermediateResultPartitionID, ResultPartition> partitions =
+					registeredPartitions.row(executionId);
+
+			for (ResultPartition partition : partitions.values()) {
+				partition.release();
+			}
+
+			for (IntermediateResultPartitionID partitionId : ImmutableList
+					.copyOf(partitions.keySet())) {
+
+				registeredPartitions.remove(executionId, partitionId);
+			}
+
+			LOG.debug("Released all partitions produced by {}.", executionId);
+		}
+	}
+
+	public void shutdown() {
+		synchronized (registeredPartitions) {
+
+			LOG.debug("Releasing {} partitions because of shutdown.",
+					registeredPartitions.values().size());
+
+			for (ResultPartition partition : registeredPartitions.values()) {
+				partition.release();
+			}
+
+			registeredPartitions.clear();
+
+			isShutdown = true;
+
+			LOG.debug("Successful shutdown.");
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	// Notifications
+	// ------------------------------------------------------------------------
+
+	void onConsumedPartition(ResultPartition partition) {
+		final ResultPartition previous;
+
+		LOG.debug("Received consume notification from {}.", partition);
+
+		synchronized (registeredPartitions) {
+			ResultPartitionID partitionId = partition.getPartitionId();
+
+			previous = registeredPartitions.remove(partitionId.getProducerId(),
+					partitionId.getPartitionId());
+		}
+
+		// Release the partition if it was successfully removed
+		if (partition == previous) {
+			partition.release();
+
+			LOG.debug("Released {}.", partition);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionProvider.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionProvider.java
new file mode 100644
index 0000000..1f35f59
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionProvider.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition;
+
+import com.google.common.base.Optional;
+import org.apache.flink.runtime.io.network.buffer.BufferProvider;
+
+import java.io.IOException;
+
+public interface ResultPartitionProvider {
+
+	/**
+	 * Returns the requested intermediate result partition input view.
+	 */
+	ResultSubpartitionView getSubpartition(ResultPartitionID partitionId, int index, Optional<BufferProvider> bufferProvider) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionType.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionType.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionType.java
new file mode 100644
index 0000000..65d49ed
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionType.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition;
+
+public enum ResultPartitionType {
+
+	BLOCKING(true, false, false),
+
+	PIPELINED(false, true, true),
+
+	PIPELINED_PERSISTENT(true, true, true);
+
+	/** Does the partition live longer than the consuming task? */
+	private final boolean isPersistent;
+
+	/** Can the partition be consumed while being produced? */
+	private final boolean isPipelined;
+
+	/** Does the partition produce back pressure when not consumed? */
+	private final boolean hasBackPressure;
+
+	/**
+	 * Specifies the behaviour of an intermediate result partition at runtime.
+	 */
+	ResultPartitionType(boolean isPersistent, boolean isPipelined, boolean hasBackPressure) {
+		this.isPersistent = isPersistent;
+		this.isPipelined = isPipelined;
+		this.hasBackPressure = hasBackPressure;
+	}
+
+	public boolean hasBackPressure() {
+		return hasBackPressure;
+	}
+
+	public boolean isBlocking() {
+		return !isPipelined;
+	}
+
+	public boolean isPipelined() {
+		return isPipelined;
+	}
+
+	public boolean isPersistent() {
+		return isPersistent;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java
new file mode 100644
index 0000000..1538a1a
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition;
+
+import com.google.common.base.Optional;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferProvider;
+
+import java.io.IOException;
+
+/**
+ * A single subpartition of a {@link ResultPartition} instance.
+ */
+public abstract class ResultSubpartition {
+
+	/** The index of the subpartition at the parent partition. */
+	protected final int index;
+
+	/** The parent partition this subpartition belongs to. */
+	protected final ResultPartition parent;
+
+	// - Statistics ----------------------------------------------------------
+
+	/** The total number of buffers (both data and event buffers) */
+	private int totalNumberOfBuffers;
+
+	/** The total number of bytes (both data and event buffers) */
+	private long totalNumberOfBytes;
+
+	public ResultSubpartition(int index, ResultPartition parent) {
+		this.index = index;
+		this.parent = parent;
+	}
+
+	protected void updateStatistics(Buffer buffer) {
+		totalNumberOfBuffers++;
+		totalNumberOfBytes += buffer.getSize();
+	}
+
+	protected int getTotalNumberOfBuffers() {
+		return totalNumberOfBuffers;
+	}
+
+	protected long getTotalNumberOfBytes() {
+		return totalNumberOfBytes;
+	}
+
+	/**
+	 * Notifies the parent partition about a consumed {@link ResultSubpartitionView}.
+	 */
+	protected void onConsumedSubpartition() {
+		parent.onConsumedSubpartition(index);
+	}
+
+	abstract public boolean add(Buffer buffer) throws IOException;
+
+	abstract public void finish() throws IOException;
+
+	abstract public void release() throws IOException;
+
+	abstract public ResultSubpartitionView getReadView(Optional<BufferProvider> bufferProvider) throws IOException;
+
+	abstract int releaseMemory() throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartitionView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartitionView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartitionView.java
new file mode 100644
index 0000000..82cee6c
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartitionView.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition;
+
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.util.event.NotificationListener;
+
+import java.io.IOException;
+
+/**
+ * A view to consume a {@link ResultSubpartition} instance.
+ */
+public interface ResultSubpartitionView {
+
+	/**
+	 * Returns the next {@link Buffer} instance of this queue iterator.
+	 * <p>
+	 * If there is currently no instance available, it will return <code>null</code>.
+	 * This might happen for example when a pipelined queue producer is slower
+	 * than the consumer or a spilled queue needs to read in more data.
+	 * <p>
+	 * <strong>Important</strong>: The consumer has to make sure that each
+	 * buffer instance will eventually be recycled with {@link Buffer#recycle()}
+	 * after it has been consumed.
+	 */
+	Buffer getNextBuffer() throws IOException, InterruptedException;
+
+	/**
+	 * Subscribes to data availability notifications.
+	 * <p>
+	 * Returns whether the subscription was successful. A subscription fails,
+	 * if there is data available.
+	 */
+	boolean registerListener(NotificationListener listener) throws IOException;
+
+
+	void releaseAllResources() throws IOException;
+
+	void notifySubpartitionConsumed() throws IOException;
+
+	boolean isReleased();
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java
new file mode 100644
index 0000000..da6a847
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition;
+
+import com.google.common.base.Optional;
+import org.apache.flink.runtime.io.disk.iomanager.BufferFileWriter;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * A blocking in-memory subpartition, which is able to spill to disk.
+ *
+ * <p> Buffers are kept in-memory as long as possible. If not possible anymore, all buffers are
+ * spilled to disk.
+ */
+class SpillableSubpartition extends ResultSubpartition {
+
+	private static final Logger LOG = LoggerFactory.getLogger(SpillableSubpartition.class);
+
+	/** All buffers of this subpartition. */
+	final ArrayList<Buffer> buffers = new ArrayList<Buffer>();
+
+	/** The I/O manager to create the spill writer from. */
+	final IOManager ioManager;
+
+	/** The default I/O mode to use. */
+	final IOMode ioMode;
+
+	/** The writer used for spilling. As long as this is null, we are in-memory. */
+	BufferFileWriter spillWriter;
+
+	/** Flag indicating whether the subpartition has been finished. */
+	private boolean isFinished;
+
+	/** Flag indicating whether the subpartition has been released. */
+	private boolean isReleased;
+
+	/** The read view to consume this subpartition. */
+	private ResultSubpartitionView readView;
+
+	SpillableSubpartition(int index, ResultPartition parent, IOManager ioManager, IOMode ioMode) {
+		super(index, parent);
+
+		this.ioManager = checkNotNull(ioManager);
+		this.ioMode = checkNotNull(ioMode);
+	}
+
+	@Override
+	public boolean add(Buffer buffer) throws IOException {
+		checkNotNull(buffer);
+
+		synchronized (buffers) {
+			if (isFinished || isReleased) {
+				return false;
+			}
+
+			// In-memory
+			if (spillWriter == null) {
+				buffers.add(buffer);
+
+				return true;
+			}
+		}
+
+		// Else: Spilling
+		spillWriter.writeBlock(buffer);
+
+		return true;
+	}
+
+	@Override
+	public void finish() throws IOException {
+		synchronized (buffers) {
+			if (add(EventSerializer.toBuffer(EndOfPartitionEvent.INSTANCE))) {
+				// If we are spilling/have spilled, wait for the writer to finish.
+				if (spillWriter != null) {
+					spillWriter.close();
+				}
+
+				isFinished = true;
+			}
+		}
+	}
+
+	@Override
+	public void release() throws IOException {
+		final ResultSubpartitionView view;
+
+		synchronized (buffers) {
+			if (isReleased) {
+				return;
+			}
+
+			// Recycle all in-memory buffers
+			for (Buffer buffer : buffers) {
+				buffer.recycle();
+			}
+
+			buffers.clear();
+			buffers.trimToSize();
+
+			// If we are spilling/have spilled, wait for the writer to finish and delete the file.
+			if (spillWriter != null) {
+				spillWriter.closeAndDelete();
+			}
+
+			// Get the view...
+			view = readView;
+			readView = null;
+
+			isReleased = true;
+		}
+
+		// Release the view outside of the synchronized block
+		if (view != null) {
+			view.notifySubpartitionConsumed();
+		}
+	}
+
+	@Override
+	public int releaseMemory() throws IOException {
+		synchronized (buffers) {
+			if (spillWriter == null) {
+				// Create the spill writer
+				spillWriter = ioManager.createBufferFileWriter(ioManager.createChannel());
+
+				final int numberOfBuffers = buffers.size();
+
+				// Spill all buffers
+				for (int i = 0; i < numberOfBuffers; i++) {
+					spillWriter.writeBlock(buffers.remove(0));
+				}
+
+				LOG.debug("Spilling {} buffers of {}.", numberOfBuffers, this);
+
+				return numberOfBuffers;
+			}
+		}
+
+		// Else: We have already spilled and don't hold any buffers
+		return 0;
+	}
+
+	@Override
+	public ResultSubpartitionView getReadView(Optional<BufferProvider> bufferProvider) throws IOException {
+		synchronized (buffers) {
+			if (!isFinished) {
+				throw new IllegalStateException("Subpartition has not been finished yet, " +
+						"but blocking subpartitions can only be consumed after they have " +
+						"been finished.");
+			}
+
+			if (readView != null) {
+				throw new IllegalStateException("Subpartition is being or already has been " +
+						"consumed, but we currently allow subpartitions to only be consumed once.");
+			}
+
+			// Spilled if closed and no outstanding write requests
+			boolean isSpilled = spillWriter != null && (spillWriter.isClosed()
+					|| spillWriter.getNumberOfOutstandingRequests() == 0);
+
+			if (isSpilled) {
+				if (ioMode.isSynchronous()) {
+					readView = new SpilledSubpartitionViewSyncIO(
+							this,
+							bufferProvider.get().getMemorySegmentSize(),
+							spillWriter.getChannelID(),
+							0);
+				}
+				else {
+					readView = new SpilledSubpartitionViewAsyncIO(
+							this,
+							bufferProvider.get(),
+							ioManager,
+							spillWriter.getChannelID(),
+							0);
+				}
+			}
+			else {
+				readView = new SpillableSubpartitionView(
+						this, bufferProvider.get(), buffers.size(), ioMode);
+			}
+
+			return readView;
+		}
+	}
+
+	@Override
+	public String toString() {
+		return String.format("SpillableSubpartition [%d number of buffers (%d bytes)," +
+						"finished? %s, read view? %s, spilled? %s]",
+				getTotalNumberOfBuffers(), getTotalNumberOfBytes(), isFinished, readView != null,
+				spillWriter != null);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java
new file mode 100644
index 0000000..59b1464
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java
@@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition;
+
+import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferProvider;
+import org.apache.flink.runtime.util.event.NotificationListener;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+class SpillableSubpartitionView implements ResultSubpartitionView {
+
+	/** The subpartition this view belongs to. */
+	private final SpillableSubpartition parent;
+
+	/** The buffer provider to read buffers into (spilling case). */
+	private final BufferProvider bufferProvider;
+
+	/** The number of buffers in-memory at the subpartition. */
+	private final int numberOfBuffers;
+
+	/** The default I/O mode to use. */
+	private final IOMode ioMode;
+
+	private ResultSubpartitionView spilledView;
+
+	private int currentQueuePosition;
+
+	private long currentBytesRead;
+
+	private final AtomicBoolean isReleased = new AtomicBoolean(false);
+
+	public SpillableSubpartitionView(
+			SpillableSubpartition parent,
+			BufferProvider bufferProvider,
+			int numberOfBuffers,
+			IOMode ioMode) {
+
+		this.parent = checkNotNull(parent);
+		this.bufferProvider = checkNotNull(bufferProvider);
+		checkArgument(numberOfBuffers >= 0);
+		this.numberOfBuffers = numberOfBuffers;
+		this.ioMode = checkNotNull(ioMode);
+	}
+
+	@Override
+	public Buffer getNextBuffer() throws IOException, InterruptedException {
+
+		if (isReleased.get()) {
+			return null;
+		}
+
+		// 1) In-memory
+		synchronized (parent.buffers) {
+			if (parent.spillWriter == null) {
+				if (currentQueuePosition < numberOfBuffers) {
+					Buffer buffer = parent.buffers.get(currentQueuePosition);
+
+					buffer.retain();
+
+					// TODO Fix hard coding of 8 bytes for the header
+					currentBytesRead += buffer.getSize() + 8;
+					currentQueuePosition++;
+
+					return buffer;
+				}
+
+				return null;
+			}
+		}
+
+		// 2) Spilled
+		if (spilledView != null) {
+			return spilledView.getNextBuffer();
+		}
+
+		// 3) Spilling
+		// Make sure that all buffers are written before consuming them. We can't block here,
+		// because this might be called from an network I/O thread.
+		if (parent.spillWriter.getNumberOfOutstandingRequests() > 0) {
+			return null;
+		}
+
+		if (ioMode.isSynchronous()) {
+			spilledView = new SpilledSubpartitionViewSyncIO(
+					parent,
+					bufferProvider.getMemorySegmentSize(),
+					parent.spillWriter.getChannelID(),
+					0);
+		}
+		else {
+			spilledView = new SpilledSubpartitionViewAsyncIO(
+					parent,
+					bufferProvider,
+					parent.ioManager,
+					parent.spillWriter.getChannelID(),
+					0);
+		}
+
+		return spilledView.getNextBuffer();
+	}
+
+	@Override
+	public boolean registerListener(NotificationListener listener) throws IOException {
+		if (spilledView == null) {
+			synchronized (parent.buffers) {
+				// Didn't spill yet, buffers should be in-memory
+				if (parent.spillWriter == null) {
+					return false;
+				}
+			}
+
+			// Spilling
+			if (parent.spillWriter.getNumberOfOutstandingRequests() > 0) {
+				return parent.spillWriter.registerAllRequestsProcessedListener(listener);
+			}
+
+			return false;
+		}
+
+		return spilledView.registerListener(listener);
+	}
+
+	@Override
+	public void notifySubpartitionConsumed() throws IOException {
+		parent.onConsumedSubpartition();
+	}
+
+	@Override
+	public void releaseAllResources() throws IOException {
+		if (isReleased.compareAndSet(false, true)) {
+			if (spilledView != null) {
+				spilledView.releaseAllResources();
+			}
+		}
+	}
+
+	@Override
+	public boolean isReleased() {
+		return isReleased.get();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewAsyncIO.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewAsyncIO.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewAsyncIO.java
new file mode 100644
index 0000000..1d4a9ab
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewAsyncIO.java
@@ -0,0 +1,365 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition;
+
+import org.apache.flink.runtime.io.disk.iomanager.BufferFileReader;
+import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.io.disk.iomanager.RequestDoneCallback;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferProvider;
+import org.apache.flink.runtime.util.event.EventListener;
+import org.apache.flink.runtime.util.event.NotificationListener;
+
+import java.io.IOException;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * View over a spilled subpartition.
+ *
+ * <p> Reads are triggered asynchronously in batches of configurable size.
+ */
+class SpilledSubpartitionViewAsyncIO implements ResultSubpartitionView {
+
+	private final static int DEFAULT_READ_BATCH_SIZE = 2;
+
+	private final Object lock = new Object();
+
+	/** The subpartition this view belongs to. */
+	private final ResultSubpartition parent;
+
+	/** The buffer provider to get the buffer read everything into. */
+	private final BufferProvider bufferProvider;
+
+	/** The buffer availability listener to be notified on available buffers. */
+	private final BufferProviderCallback bufferAvailabilityListener;
+
+	/** The size of read batches. */
+	private final int readBatchSize;
+
+	/**
+	 * The size of the current batch (>= 0 and <= the configured batch size). Reads are only
+	 * triggered when the size of the current batch is 0.
+	 */
+	private final AtomicInteger currentBatchSize = new AtomicInteger();
+
+	/** The asynchronous file reader to do the actual I/O. */
+	private final BufferFileReader asyncFileReader;
+
+	/** The buffers, which have been returned from the file reader. */
+	private final ConcurrentLinkedQueue<Buffer> returnedBuffers = new ConcurrentLinkedQueue<Buffer>();
+
+	/** A data availability listener. */
+	private NotificationListener registeredListener;
+
+	/** Error, which has occurred in the I/O thread. */
+	private volatile IOException errorInIOThread;
+
+	/** Flag indicating whether all resources have been released. */
+	private volatile boolean isReleased;
+
+	/** Flag indicating whether we reached EOF at the file reader. */
+	private volatile boolean hasReachedEndOfFile;
+
+	SpilledSubpartitionViewAsyncIO(
+			ResultSubpartition parent,
+			BufferProvider bufferProvider,
+			IOManager ioManager,
+			FileIOChannel.ID channelId,
+			long initialSeekPosition) throws IOException {
+
+		this(parent, bufferProvider, ioManager, channelId, initialSeekPosition, DEFAULT_READ_BATCH_SIZE);
+	}
+
+	SpilledSubpartitionViewAsyncIO(
+			ResultSubpartition parent,
+			BufferProvider bufferProvider,
+			IOManager ioManager,
+			FileIOChannel.ID channelId,
+			long initialSeekPosition,
+			int readBatchSize) throws IOException {
+
+		checkArgument(initialSeekPosition >= 0, "Initial seek position is < 0.");
+		checkArgument(readBatchSize >= 1, "Batch read size < 1.");
+
+		this.parent = checkNotNull(parent);
+		this.bufferProvider = checkNotNull(bufferProvider);
+		this.bufferAvailabilityListener = new BufferProviderCallback(this);
+
+		this.asyncFileReader = ioManager.createBufferFileReader(channelId, new IOThreadCallback(this));
+
+		if (initialSeekPosition > 0) {
+			asyncFileReader.seekToPosition(initialSeekPosition);
+		}
+
+		this.readBatchSize = readBatchSize;
+
+		// Trigger the initial read requests
+		readNextBatchAsync();
+	}
+
+	@Override
+	public Buffer getNextBuffer() throws IOException {
+		checkError();
+
+		final Buffer buffer = returnedBuffers.poll();
+
+		// No buffer returned from the I/O thread currently. Either the current batch is in progress
+		// or we trigger the next one.
+		if (buffer == null) {
+			if (currentBatchSize.get() == 0) {
+				readNextBatchAsync();
+			}
+		}
+		else {
+			currentBatchSize.decrementAndGet();
+		}
+
+		return buffer;
+	}
+
+	@Override
+	public boolean registerListener(NotificationListener listener) throws IOException {
+		checkNotNull(listener);
+
+		checkError();
+
+		synchronized (lock) {
+			if (isReleased || !returnedBuffers.isEmpty()) {
+				return false;
+			}
+
+			if (registeredListener == null) {
+				registeredListener = listener;
+
+				return true;
+			}
+		}
+
+		throw new IllegalStateException("Already registered listener.");
+	}
+
+	@Override
+	public void notifySubpartitionConsumed() throws IOException {
+		parent.onConsumedSubpartition();
+	}
+
+	@Override
+	public void releaseAllResources() throws IOException {
+		try {
+			synchronized (lock) {
+				if (!isReleased) {
+					// Recycle all buffers. Buffers, which are in flight are recycled as soon as
+					// they return from the I/O thread.
+					Buffer buffer;
+					while ((buffer = returnedBuffers.poll()) != null) {
+						buffer.recycle();
+					}
+
+					isReleased = true;
+				}
+			}
+		}
+		finally {
+			asyncFileReader.close();
+		}
+	}
+
+	@Override
+	public boolean isReleased() {
+		return isReleased;
+	}
+
+	/**
+	 * Requests buffers from the buffer provider and triggers asynchronous read requests to fill
+	 * them.
+	 *
+	 * <p> The number of requested buffers/triggered I/O read requests per call depends on the
+	 * configured size of batch reads.
+	 */
+	private void readNextBatchAsync() throws IOException {
+		// This does not need to be fully synchronized with actually reaching EOF as long as
+		// we eventually notice it. In the worst case, we trigger some discarded reads and
+		// notice it when the buffers are returned.
+		//
+		// We only trigger reads if the current batch size is 0.
+		if (hasReachedEndOfFile || currentBatchSize.get() != 0) {
+			return;
+		}
+
+		// Number of successful buffer requests or callback registrations. The call back will
+		// trigger the read as soon as a buffer becomes available again.
+		int i = 0;
+
+		while (i < readBatchSize) {
+			final Buffer buffer = bufferProvider.requestBuffer();
+
+			if (buffer == null) {
+				// Listen for buffer availability.
+				currentBatchSize.incrementAndGet();
+
+				if (bufferProvider.addListener(bufferAvailabilityListener)) {
+					i++;
+				}
+				else if (bufferProvider.isDestroyed()) {
+					currentBatchSize.decrementAndGet();
+					return;
+				}
+				else {
+					// Buffer available again
+					currentBatchSize.decrementAndGet();
+				}
+			}
+			else {
+				currentBatchSize.incrementAndGet();
+
+				asyncFileReader.readInto(buffer);
+			}
+		}
+	}
+
+	/**
+	 * Returns a buffer from the buffer provider.
+	 *
+	 * <p> Note: This method is called from the thread recycling the available buffer.
+	 */
+	private void onAvailableBuffer(Buffer buffer) {
+		try {
+			asyncFileReader.readInto(buffer);
+		}
+		catch (IOException e) {
+			notifyError(e);
+		}
+	}
+
+	/**
+	 * Returns a successful buffer read request.
+	 *
+	 * <p> Note: This method is always called from the same I/O thread.
+	 */
+	private void returnBufferFromIOThread(Buffer buffer) {
+		final NotificationListener listener;
+
+		synchronized (lock) {
+			if (hasReachedEndOfFile || isReleased) {
+				buffer.recycle();
+
+				return;
+			}
+
+			returnedBuffers.add(buffer);
+
+			listener = registeredListener;
+			registeredListener = null;
+
+			// If this was the last buffer before we reached EOF, set the corresponding flag to
+			// ensure that further buffers are correctly recycled and eventually no further reads
+			// are triggered.
+			if (asyncFileReader.hasReachedEndOfFile()) {
+				hasReachedEndOfFile = true;
+			}
+		}
+
+		if (listener != null) {
+			listener.onNotification();
+		}
+	}
+
+	/**
+	 * Notifies the view about an error.
+	 */
+	private void notifyError(IOException error) {
+		if (errorInIOThread == null) {
+			errorInIOThread = error;
+		}
+
+		final NotificationListener listener;
+
+		synchronized (lock) {
+			listener = registeredListener;
+			registeredListener = null;
+		}
+
+		if (listener != null) {
+			listener.onNotification();
+		}
+	}
+
+	/**
+	 * Checks whether an error has been reported and rethrow the respective Exception, if available.
+	 */
+	private void checkError() throws IOException {
+		if (errorInIOThread != null) {
+			throw errorInIOThread;
+		}
+	}
+
+	/**
+	 * Callback from the I/O thread.
+	 *
+	 * <p> Successful buffer read requests add the buffer to the subpartition view, and failed ones
+	 * notify about the error.
+	 */
+	private static class IOThreadCallback implements RequestDoneCallback<Buffer> {
+
+		private final SpilledSubpartitionViewAsyncIO subpartitionView;
+
+		public IOThreadCallback(SpilledSubpartitionViewAsyncIO subpartitionView) {
+			this.subpartitionView = subpartitionView;
+		}
+
+		@Override
+		public void requestSuccessful(Buffer buffer) {
+			subpartitionView.returnBufferFromIOThread(buffer);
+		}
+
+		@Override
+		public void requestFailed(Buffer buffer, IOException error) {
+			// Recycle the buffer and forward the error
+			buffer.recycle();
+
+			subpartitionView.notifyError(error);
+		}
+	}
+
+	/**
+	 * Callback from the buffer provider.
+	 */
+	private static class BufferProviderCallback implements EventListener<Buffer> {
+
+		private final SpilledSubpartitionViewAsyncIO subpartitionView;
+
+		private BufferProviderCallback(SpilledSubpartitionViewAsyncIO subpartitionView) {
+			this.subpartitionView = subpartitionView;
+		}
+
+		@Override
+		public void onEvent(Buffer buffer) {
+			if (buffer == null) {
+				return;
+			}
+
+			subpartitionView.onAvailableBuffer(buffer);
+		}
+	}
+}


[10/13] flink git commit: [FLINK-1350] [runtime] Add blocking result partition variant

Posted by uc...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannel.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannel.java
index 8889f70..cca04b7 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannel.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannel.java
@@ -19,10 +19,13 @@
 package org.apache.flink.runtime.io.disk.iomanager;
 
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.util.event.NotificationListener;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import static com.google.common.base.Preconditions.checkNotNull;
@@ -37,36 +40,42 @@ import static com.google.common.base.Preconditions.checkNotNull;
  * @param <R> The type of request (e.g. <tt>ReadRequest</tt> or <tt>WriteRequest</tt> issued by this access to the I/O threads.
  */
 public abstract class AsynchronousFileIOChannel<T, R extends IORequest> extends AbstractFileIOChannel {
-	
-	/** The lock that is used during closing to synchronize the thread that waits for all
-	 * requests to be handled with the asynchronous I/O thread. */
+
+	private final Object listenerLock = new Object();
+
+	/**
+	 * The lock that is used during closing to synchronize the thread that waits for all
+	 * requests to be handled with the asynchronous I/O thread.
+	 */
 	protected final Object closeLock = new Object();
-	
+
 	/** A request queue for submitting asynchronous requests to the corresponding IO worker thread. */
 	protected final RequestQueue<R> requestQueue;
-	
+
 	/** An atomic integer that counts the number of requests that we still wait for to return. */
 	protected final AtomicInteger requestsNotReturned = new AtomicInteger(0);
 	
 	/** Handler for completed requests */
 	protected final RequestDoneCallback<T> resultHandler;
-	
-	/** An exception that was encountered by the asynchronous request handling thread.*/
+
+	/** An exception that was encountered by the asynchronous request handling thread. */
 	protected volatile IOException exception;
-	
+
 	/** Flag marking this channel as closed */
 	protected volatile boolean closed;
 
+	private NotificationListener allRequestsProcessedListener;
+
 	// --------------------------------------------------------------------------------------------
-	
+
 	/**
 	 * Creates a new channel access to the path indicated by the given ID. The channel accepts buffers to be
-	 * read/written and hands them to the asynchronous I/O thread. After being processed, the buffers 
+	 * read/written and hands them to the asynchronous I/O thread. After being processed, the buffers
 	 * are returned by adding the to the given queue.
-	 * 
-	 * @param channelID The id describing the path of the file that the channel accessed.
+	 *
+	 * @param channelID    The id describing the path of the file that the channel accessed.
 	 * @param requestQueue The queue that this channel hands its IO requests to.
-	 * @param callback The callback to be invoked when a request is done.
+	 * @param callback     The callback to be invoked when a request is done.
 	 * @param writeEnabled Flag describing whether the channel should be opened in read/write mode, rather
 	 *                     than in read-only mode.
 	 * @throws IOException Thrown, if the channel could no be opened.
@@ -79,21 +88,25 @@ public abstract class AsynchronousFileIOChannel<T, R extends IORequest> extends
 		this.requestQueue = checkNotNull(requestQueue);
 		this.resultHandler = checkNotNull(callback);
 	}
-	
+
 	// --------------------------------------------------------------------------------------------
-	
+
 	@Override
 	public boolean isClosed() {
 		return this.closed;
 	}
-	
+
 	/**
-	 * Closes the reader and waits until all pending asynchronous requests are
-	 * handled. Even if an exception interrupts the closing, the underlying <tt>FileChannel</tt> is closed.
-	 * 
+	 * Closes the channel and waits until all pending asynchronous requests are processed. The
+	 * underlying <code>FileChannel</code> is closed even if an exception interrupts the closing.
+	 *
+	 * <p> <strong>Important:</strong> the {@link #isClosed()} method returns <code>true</code>
+	 * immediately after this method has been called even when there are outstanding requests.
+	 *
 	 * @throws IOException Thrown, if an I/O exception occurred while waiting for the buffers, or if
 	 *                     the closing was interrupted.
 	 */
+	@Override
 	public void close() throws IOException {
 		// atomically set the close flag
 		synchronized (this.closeLock) {
@@ -101,7 +114,7 @@ public abstract class AsynchronousFileIOChannel<T, R extends IORequest> extends
 				return;
 			}
 			this.closed = true;
-			
+
 			try {
 				// wait until as many buffers have been returned as were written
 				// only then is everything guaranteed to be consistent.
@@ -136,9 +149,10 @@ public abstract class AsynchronousFileIOChannel<T, R extends IORequest> extends
 	 * <p>
 	 * Even if an exception interrupts the closing, such that not all request are handled,
 	 * the underlying <tt>FileChannel</tt> is closed and deleted.
-	 * 
+	 *
 	 * @throws IOException Thrown, if an I/O exception occurred while waiting for the buffers, or if the closing was interrupted.
 	 */
+	@Override
 	public void closeAndDelete() throws IOException {
 		try {
 			close();
@@ -147,11 +161,11 @@ public abstract class AsynchronousFileIOChannel<T, R extends IORequest> extends
 			deleteChannel();
 		}
 	}
-	
+
 	/**
 	 * Checks the exception state of this channel. The channel is erroneous, if one of its requests could not
 	 * be processed correctly.
-	 * 
+	 *
 	 * @throws IOException Thrown, if the channel is erroneous. The thrown exception contains the original exception
 	 *                     that defined the erroneous state as its cause.
 	 */
@@ -160,15 +174,15 @@ public abstract class AsynchronousFileIOChannel<T, R extends IORequest> extends
 			throw this.exception;
 		}
 	}
-	
+
 	/**
 	 * Handles a processed <tt>Buffer</tt>. This method is invoked by the
 	 * asynchronous IO worker threads upon completion of the IO request with the
 	 * provided buffer and/or an exception that occurred while processing the request
 	 * for that buffer.
-	 * 
+	 *
 	 * @param buffer The buffer to be processed.
-	 * @param ex The exception that occurred in the I/O threads when processing the buffer's request.
+	 * @param ex     The exception that occurred in the I/O threads when processing the buffer's request.
 	 */
 	final protected void handleProcessedBuffer(T buffer, IOException ex) {
 		if (buffer == null) {
@@ -186,13 +200,26 @@ public abstract class AsynchronousFileIOChannel<T, R extends IORequest> extends
 			}
 		}
 		finally {
-			// decrement the number of missing buffers. If we are currently closing, notify the waiters
+			NotificationListener listener = null;
+
+			// Decrement the number of outstanding requests. If we are currently closing, notify the
+			// waiters. If there is a listener, notify her as well.
 			synchronized (this.closeLock) {
-				final int num = this.requestsNotReturned.decrementAndGet();
-				if (this.closed && num == 0) {
-					this.closeLock.notifyAll();
+				if (this.requestsNotReturned.decrementAndGet() == 0) {
+					if (this.closed) {
+						this.closeLock.notifyAll();
+					}
+
+					synchronized (listenerLock) {
+						listener = allRequestsProcessedListener;
+						allRequestsProcessedListener = null;
+					}
 				}
 			}
+
+			if (listener != null) {
+				listener.onNotification();
+			}
 		}
 	}
 
@@ -202,14 +229,57 @@ public abstract class AsynchronousFileIOChannel<T, R extends IORequest> extends
 
 		// write the current buffer and get the next one
 		this.requestsNotReturned.incrementAndGet();
+
 		if (this.closed || this.requestQueue.isClosed()) {
 			// if we found ourselves closed after the counter increment,
 			// decrement the counter again and do not forward the request
 			this.requestsNotReturned.decrementAndGet();
+
+			final NotificationListener listener;
+
+			synchronized (listenerLock) {
+				listener = allRequestsProcessedListener;
+				allRequestsProcessedListener = null;
+			}
+
+			if (listener != null) {
+				listener.onNotification();
+			}
+
 			throw new IOException("I/O channel already closed. Could not fulfill: " + request);
 		}
+
 		this.requestQueue.add(request);
 	}
+
+	/**
+	 * Registers a listener to be notified when all outstanding requests have been processed.
+	 *
+	 * <p> New requests can arrive right after the listener got notified. Therefore, it is not safe
+	 * to assume that the number of outstanding requests is still zero after a notification unless
+	 * there was a close right before the listener got called.
+	 *
+	 * <p> Returns <code>true</code>, if the registration was successful. A registration can fail,
+	 * if there are no outstanding requests when trying to register a listener.
+	 */
+	protected boolean registerAllRequestsProcessedListener(NotificationListener listener) throws IOException {
+		checkNotNull(listener);
+
+		synchronized (listenerLock) {
+			if (allRequestsProcessedListener == null) {
+				// There was a race with the processing of the last outstanding request
+				if (requestsNotReturned.get() == 0) {
+					return false;
+				}
+
+				allRequestsProcessedListener = listener;
+
+				return true;
+			}
+		}
+
+		throw new IllegalStateException("Already subscribed.");
+	}
 }
 
 //--------------------------------------------------------------------------------------------
@@ -218,11 +288,11 @@ public abstract class AsynchronousFileIOChannel<T, R extends IORequest> extends
  * Read request that reads an entire memory segment from a block reader.
  */
 final class SegmentReadRequest implements ReadRequest {
-	
+
 	private final AsynchronousFileIOChannel<MemorySegment, ReadRequest> channel;
-	
+
 	private final MemorySegment segment;
-	
+
 	protected SegmentReadRequest(AsynchronousFileIOChannel<MemorySegment, ReadRequest> targetChannel, MemorySegment segment) {
 		this.channel = targetChannel;
 		this.segment = segment;
@@ -254,11 +324,11 @@ final class SegmentReadRequest implements ReadRequest {
  * Write request that writes an entire memory segment to the block writer.
  */
 final class SegmentWriteRequest implements WriteRequest {
-	
+
 	private final AsynchronousFileIOChannel<MemorySegment, WriteRequest> channel;
-	
+
 	private final MemorySegment segment;
-	
+
 	protected SegmentWriteRequest(AsynchronousFileIOChannel<MemorySegment, WriteRequest> targetChannel, MemorySegment segment) {
 		this.channel = targetChannel;
 		this.segment = segment;
@@ -280,6 +350,135 @@ final class SegmentWriteRequest implements WriteRequest {
 	}
 }
 
+final class BufferWriteRequest implements WriteRequest {
+
+	private final AsynchronousFileIOChannel<Buffer, WriteRequest> channel;
+
+	private final Buffer buffer;
+
+	protected BufferWriteRequest(AsynchronousFileIOChannel<Buffer, WriteRequest> targetChannel, Buffer buffer) {
+		this.channel = checkNotNull(targetChannel);
+		this.buffer = checkNotNull(buffer);
+	}
+
+	@Override
+	public void write() throws IOException {
+		final ByteBuffer header = ByteBuffer.allocateDirect(8);
+
+		header.putInt(buffer.isBuffer() ? 1 : 0);
+		header.putInt(buffer.getSize());
+		header.flip();
+
+		channel.fileChannel.write(header);
+		channel.fileChannel.write(buffer.getNioBuffer());
+	}
+
+	@Override
+	public void requestDone(IOException error) {
+		channel.handleProcessedBuffer(buffer, error);
+	}
+}
+
+final class BufferReadRequest implements ReadRequest {
+
+	private final AsynchronousFileIOChannel<Buffer, ReadRequest> channel;
+
+	private final Buffer buffer;
+
+	private final AtomicBoolean hasReachedEndOfFile;
+
+	protected BufferReadRequest(AsynchronousFileIOChannel<Buffer, ReadRequest> targetChannel, Buffer buffer, AtomicBoolean hasReachedEndOfFile) {
+		this.channel = targetChannel;
+		this.buffer = buffer;
+		this.hasReachedEndOfFile = hasReachedEndOfFile;
+	}
+
+	@Override
+	public void read() throws IOException {
+
+		final FileChannel fileChannel = channel.fileChannel;
+
+		if (fileChannel.size() - fileChannel.position() > 0) {
+			final ByteBuffer header = ByteBuffer.allocateDirect(8);
+
+			fileChannel.read(header);
+			header.flip();
+
+			final boolean isBuffer = header.getInt() == 1;
+			final int size = header.getInt();
+
+			if (size > buffer.getMemorySegment().size()) {
+				throw new IllegalStateException("Buffer is too small for data: " + buffer.getMemorySegment().size() + " bytes available, but " + size + " needed. This is most likely due to an serialized event, which is larger than the buffer size.");
+			}
+
+			buffer.setSize(size);
+
+			fileChannel.read(buffer.getNioBuffer());
+
+			if (!isBuffer) {
+				buffer.tagAsEvent();
+			}
+
+			hasReachedEndOfFile.set(fileChannel.size() - fileChannel.position() == 0);
+		}
+		else {
+			hasReachedEndOfFile.set(true);
+		}
+	}
+
+	@Override
+	public void requestDone(IOException error) {
+		channel.handleProcessedBuffer(buffer, error);
+	}
+}
+
+final class FileSegmentReadRequest implements ReadRequest {
+
+	private final AsynchronousFileIOChannel<FileSegment, ReadRequest> channel;
+
+	private final AtomicBoolean hasReachedEndOfFile;
+
+	private FileSegment fileSegment;
+
+	protected FileSegmentReadRequest(AsynchronousFileIOChannel<FileSegment, ReadRequest> targetChannel, AtomicBoolean hasReachedEndOfFile) {
+		this.channel = targetChannel;
+		this.hasReachedEndOfFile = hasReachedEndOfFile;
+	}
+
+	@Override
+	public void read() throws IOException {
+
+		final FileChannel fileChannel = channel.fileChannel;
+
+		if (fileChannel.size() - fileChannel.position() > 0) {
+			final ByteBuffer header = ByteBuffer.allocateDirect(8);
+
+			fileChannel.read(header);
+			header.flip();
+
+			final long position = fileChannel.position();
+
+			final boolean isBuffer = header.getInt() == 1;
+			final int length = header.getInt();
+
+			fileSegment = new FileSegment(fileChannel, position, length, isBuffer);
+
+			// Skip the binary dataa
+			fileChannel.position(position + length);
+
+			hasReachedEndOfFile.set(fileChannel.size() - fileChannel.position() == 0);
+		}
+		else {
+			hasReachedEndOfFile.set(true);
+		}
+	}
+
+	@Override
+	public void requestDone(IOException error) {
+		channel.handleProcessedBuffer(fileSegment, error);
+	}
+}
+
 /**
  * Request that seeks the underlying file channel to the given position.
  */

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelReader.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelReader.java
index 8f7f218..957052e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelReader.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelReader.java
@@ -21,15 +21,13 @@ package org.apache.flink.runtime.io.disk.iomanager;
 import java.io.IOException;
 import java.util.concurrent.LinkedBlockingQueue;
 
-import org.apache.flink.core.memory.MemorySegment;
-
 /**
  * A reader that reads data in blocks from a file channel. The reader reads the blocks into a 
  * {@link org.apache.flink.core.memory.MemorySegment}. To support asynchronous implementations,
  * the read method does not immediately return the full memory segment, but rather adds it to
  * a blocking queue of finished read operations.
  */
-public interface BlockChannelReader extends FileIOChannel {
+public interface BlockChannelReader<T> extends FileIOChannel {
 
 	/**
 	 * Issues a read request, which will fill the given segment with the next block in the
@@ -39,33 +37,27 @@ public interface BlockChannelReader extends FileIOChannel {
 	 * @param segment The segment to read the block into.
 	 * @throws IOException Thrown, when the reader encounters an I/O error.
 	 */
-	void readBlock(MemorySegment segment) throws IOException;
+	void readBlock(T segment) throws IOException;
+
+	void seekToPosition(long position) throws IOException;
 	
 	/**
 	 * Gets the next memory segment that has been filled with data by the reader. This method blocks until
 	 * such a segment is available, or until an error occurs in the reader, or the reader is closed.
 	 * <p>
 	 * WARNING: If this method is invoked without any segment ever returning (for example, because the
-	 * {@link #readBlock(MemorySegment)} method has not been invoked appropriately), the method may block
+	 * {@link #readBlock(T)} method has not been invoked appropriately), the method may block
 	 * forever.
 	 * 
 	 * @return The next memory segment from the reader's return queue.
 	 * @throws IOException Thrown, if an I/O error occurs in the reader while waiting for the request to return.
 	 */
-	public MemorySegment getNextReturnedSegment() throws IOException;
+	public T getNextReturnedBlock() throws IOException;
 	
 	/**
 	 * Gets the queue in which the full memory segments are queued after the read is complete.
 	 * 
 	 * @return The queue with the full memory segments.
 	 */
-	LinkedBlockingQueue<MemorySegment> getReturnQueue();
-	
-	/**
-	 * Seeks the underlying file channel to the given position.
-	 * 
-	 * @param position The position to seek to.
-	 */
-	void seekToPosition(long position) throws IOException;
+	LinkedBlockingQueue<T> getReturnQueue();
 }
-	
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelWriter.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelWriter.java
index 25c74e4..ccf065a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelWriter.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelWriter.java
@@ -21,15 +21,13 @@ package org.apache.flink.runtime.io.disk.iomanager;
 import java.io.IOException;
 import java.util.concurrent.LinkedBlockingQueue;
 
-import org.apache.flink.core.memory.MemorySegment;
-
 /**
  * A writer that writes data in blocks to a file channel. The writer receives the data blocks in the form of 
  * {@link org.apache.flink.core.memory.MemorySegment}, which it writes entirely to the channel,
  * regardless of how space in the segment is used. The writing may be realized synchronously, or asynchronously,
  * depending on the implementation.
  */
-public interface BlockChannelWriter extends BlockChannelWriterWithCallback {
+public interface BlockChannelWriter<T> extends BlockChannelWriterWithCallback<T> {
 	
 	/**
 	 * Gets the next memory segment that has been written and is available again.
@@ -37,13 +35,13 @@ public interface BlockChannelWriter extends BlockChannelWriterWithCallback {
 	 * writer is closed.
 	 * <p>
 	 * NOTE: If this method is invoked without any segment ever returning (for example, because the
-	 * {@link #writeBlock(MemorySegment)} method has not been invoked accordingly), the method may block
+	 * {@link #writeBlock(T)} method has not been invoked accordingly), the method may block
 	 * forever.
 	 * 
 	 * @return The next memory segment from the writers's return queue.
 	 * @throws IOException Thrown, if an I/O error occurs in the writer while waiting for the request to return.
 	 */
-	MemorySegment getNextReturnedSegment() throws IOException;
+	T getNextReturnedBlock() throws IOException;
 	
 	/**
 	 * Gets the queue in which the memory segments are queued after the asynchronous write
@@ -51,5 +49,5 @@ public interface BlockChannelWriter extends BlockChannelWriterWithCallback {
 	 * 
 	 * @return The queue with the written memory segments.
 	 */
-	LinkedBlockingQueue<MemorySegment> getReturnQueue();
+	LinkedBlockingQueue<T> getReturnQueue();
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelWriterWithCallback.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelWriterWithCallback.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelWriterWithCallback.java
index 57bc7e0..f7618e4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelWriterWithCallback.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelWriterWithCallback.java
@@ -20,16 +20,14 @@ package org.apache.flink.runtime.io.disk.iomanager;
 
 import java.io.IOException;
 
-import org.apache.flink.core.memory.MemorySegment;
+public interface BlockChannelWriterWithCallback<T> extends FileIOChannel {
 
-public interface BlockChannelWriterWithCallback extends FileIOChannel {
-	
 	/**
-	 * Writes the given memory segment. The request may be executed synchronously, or asynchronously, depending
+	 * Writes the given block. The request may be executed synchronously, or asynchronously, depending
 	 * on the implementation.
-	 * 
-	 * @param segment The segment to be written.
+	 *
+	 * @param block The segment to be written.
 	 * @throws IOException Thrown, when the writer encounters an I/O error.
 	 */
-	void writeBlock(MemorySegment segment) throws IOException;
+	void writeBlock(T block) throws IOException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileReader.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileReader.java
new file mode 100644
index 0000000..74999e2
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileReader.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.disk.iomanager;
+
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+
+import java.io.IOException;
+
+public interface BufferFileReader extends FileIOChannel {
+
+	void readInto(Buffer buffer) throws IOException;
+
+	void seekToPosition(long position) throws IOException;
+
+	boolean hasReachedEndOfFile();
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileSegmentReader.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileSegmentReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileSegmentReader.java
new file mode 100644
index 0000000..fa25d4f
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileSegmentReader.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.disk.iomanager;
+
+import java.io.IOException;
+
+public interface BufferFileSegmentReader extends FileIOChannel {
+
+	void read() throws IOException;
+
+	void seekTo(long position) throws IOException;
+
+	boolean hasReachedEndOfFile();
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriter.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriter.java
new file mode 100644
index 0000000..704aad2
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriter.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.disk.iomanager;
+
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.util.event.NotificationListener;
+
+import java.io.IOException;
+
+public interface BufferFileWriter extends BlockChannelWriterWithCallback<Buffer> {
+
+	/**
+	 * Returns the number of outstanding requests.
+	 */
+	int getNumberOfOutstandingRequests();
+
+	/**
+	 * Registers a listener, which is notified after all outstanding requests have been processed.
+	 */
+	boolean registerAllRequestsProcessedListener(NotificationListener listener) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelReaderInputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelReaderInputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelReaderInputView.java
index d85ec82..b919034 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelReaderInputView.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelReaderInputView.java
@@ -35,7 +35,7 @@ import org.apache.flink.runtime.memorymanager.AbstractPagedInputView;
  */
 public class ChannelReaderInputView extends AbstractPagedInputView {
 	
-	protected final BlockChannelReader reader;		// the block reader that reads memory segments
+	protected final BlockChannelReader<MemorySegment> reader;		// the block reader that reads memory segments
 	
 	protected int numRequestsRemaining;				// the number of block requests remaining
 	
@@ -63,7 +63,7 @@ public class ChannelReaderInputView extends AbstractPagedInputView {
 	 * @throws IOException Thrown, if the read requests for the first blocks fail to be
 	 *                     served by the reader.
 	 */
-	public ChannelReaderInputView(BlockChannelReader reader, List<MemorySegment> memory, boolean waitForFirstBlock)
+	public ChannelReaderInputView(BlockChannelReader<MemorySegment> reader, List<MemorySegment> memory, boolean waitForFirstBlock)
 	throws IOException
 	{
 		this(reader, memory, -1, waitForFirstBlock);
@@ -89,7 +89,7 @@ public class ChannelReaderInputView extends AbstractPagedInputView {
 	 * @throws IOException Thrown, if the read requests for the first blocks fail to be
 	 *                     served by the reader.
 	 */
-	public ChannelReaderInputView(BlockChannelReader reader, List<MemorySegment> memory, 
+	public ChannelReaderInputView(BlockChannelReader<MemorySegment> reader, List<MemorySegment> memory,
 														int numBlocks, boolean waitForFirstBlock)
 	throws IOException
 	{
@@ -117,7 +117,7 @@ public class ChannelReaderInputView extends AbstractPagedInputView {
 	 * 
 	 * @throws IOException
 	 */
-	ChannelReaderInputView(BlockChannelReader reader, List<MemorySegment> memory, 
+	ChannelReaderInputView(BlockChannelReader<MemorySegment> reader, List<MemorySegment> memory,
 				int numBlocks, int headerLen, boolean waitForFirstBlock)
 	throws IOException
 	{
@@ -225,7 +225,7 @@ public class ChannelReaderInputView extends AbstractPagedInputView {
 		}
 		
 		// get the next segment
-		final MemorySegment seg = this.reader.getNextReturnedSegment();
+		final MemorySegment seg = this.reader.getNextReturnedBlock();
 		
 		// check the header
 		if (seg.getShort(0) != ChannelWriterOutputView.HEADER_MAGIC_NUMBER) {

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelWriterOutputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelWriterOutputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelWriterOutputView.java
index 9824d34..089e10a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelWriterOutputView.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelWriterOutputView.java
@@ -61,7 +61,7 @@ public final class ChannelWriterOutputView extends AbstractPagedOutputView {
 	
 	// --------------------------------------------------------------------------------------------
 	
-	private final BlockChannelWriter writer;		// the writer to the channel
+	private final BlockChannelWriter<MemorySegment> writer;		// the writer to the channel
 	
 	private long bytesBeforeSegment;				// the number of bytes written before the current memory segment
 	
@@ -81,7 +81,7 @@ public final class ChannelWriterOutputView extends AbstractPagedOutputView {
 	 * @param memory The memory used to buffer data, or null, to utilize solely the return queue.
 	 * @param segmentSize The size of the memory segments.
 	 */
-	public ChannelWriterOutputView(BlockChannelWriter writer, List<MemorySegment> memory, int segmentSize) {
+	public ChannelWriterOutputView(BlockChannelWriter<MemorySegment> writer, List<MemorySegment> memory, int segmentSize) {
 		super(segmentSize, HEADER_LENGTH);
 		
 		if (writer == null) {
@@ -123,7 +123,7 @@ public final class ChannelWriterOutputView extends AbstractPagedOutputView {
 	 * @param writer The writer to write to.
 	 * @param segmentSize The size of the memory segments.
 	 */
-	public ChannelWriterOutputView(BlockChannelWriter writer, int segmentSize)
+	public ChannelWriterOutputView(BlockChannelWriter<MemorySegment> writer, int segmentSize)
 	{
 		this(writer, null, segmentSize);
 	}
@@ -203,7 +203,7 @@ public final class ChannelWriterOutputView extends AbstractPagedOutputView {
 			writeSegment(current, posInSegment, false);
 		}
 		
-		final MemorySegment next = this.writer.getNextReturnedSegment();
+		final MemorySegment next = this.writer.getNextReturnedBlock();
 		this.blockCount++;
 		return next;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/FileIOChannel.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/FileIOChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/FileIOChannel.java
index c5a3daa..f9ee90c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/FileIOChannel.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/FileIOChannel.java
@@ -20,6 +20,7 @@ package org.apache.flink.runtime.io.disk.iomanager;
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.channels.FileChannel;
 import java.util.Random;
 
 import org.apache.flink.util.StringUtils;
@@ -73,6 +74,8 @@ public interface FileIOChannel {
 	* @throws IOException Thrown, if an error occurred while waiting for pending requests.
 	*/
 	public void closeAndDelete() throws IOException;
+
+	FileChannel getNioFileChannel();
 	
 	// --------------------------------------------------------------------------------------------
 	// --------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/FileSegment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/FileSegment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/FileSegment.java
new file mode 100644
index 0000000..7c3a83e
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/FileSegment.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.disk.iomanager;
+
+import java.nio.channels.FileChannel;
+
+public class FileSegment {
+
+	private final FileChannel fileChannel;
+	private final long position;
+	private final int length;
+	private final boolean isBuffer;
+
+	public FileSegment(FileChannel fileChannel, long position, int length, boolean isBuffer) {
+		this.fileChannel = fileChannel;
+		this.position = position;
+		this.length = length;
+		this.isBuffer = isBuffer;
+	}
+
+	public FileChannel getFileChannel() {
+		return fileChannel;
+	}
+
+	public long getPosition() {
+		return position;
+	}
+
+	public int getLength() {
+		return length;
+	}
+
+	public boolean isBuffer() {
+		return isBuffer;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/HeaderlessChannelReaderInputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/HeaderlessChannelReaderInputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/HeaderlessChannelReaderInputView.java
index cdad3fb..63e86c9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/HeaderlessChannelReaderInputView.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/HeaderlessChannelReaderInputView.java
@@ -60,7 +60,7 @@ public class HeaderlessChannelReaderInputView extends ChannelReaderInputView
 	 * @throws IOException Thrown, if the read requests for the first blocks fail to be
 	 *                     served by the reader.
 	 */
-	public HeaderlessChannelReaderInputView(BlockChannelReader reader, List<MemorySegment> memory, int numBlocks,
+	public HeaderlessChannelReaderInputView(BlockChannelReader<MemorySegment> reader, List<MemorySegment> memory, int numBlocks,
 			int numBytesInLastBlock, boolean waitForFirstBlock)
 	throws IOException
 	{
@@ -87,7 +87,7 @@ public class HeaderlessChannelReaderInputView extends ChannelReaderInputView
 		
 		// get the next segment
 		this.numBlocksRemaining--;
-		return this.reader.getNextReturnedSegment();
+		return this.reader.getNextReturnedBlock();
 	}
 	
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManager.java
index c04ba97..c1a4b84 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManager.java
@@ -20,6 +20,7 @@ package org.apache.flink.runtime.io.disk.iomanager;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -35,6 +36,21 @@ import java.util.concurrent.LinkedBlockingQueue;
  */
 public abstract class IOManager {
 
+	public enum IOMode {
+
+		SYNC(true), ASYNC(false);
+
+		private final boolean isSynchronous;
+
+		IOMode(boolean isSynchronous) {
+			this.isSynchronous = isSynchronous;
+		}
+
+		public boolean isSynchronous() {
+			return isSynchronous;
+		}
+	}
+
 	/** Logging */
 	protected static final Logger LOG = LoggerFactory.getLogger(IOManager.class);
 
@@ -190,7 +206,7 @@ public abstract class IOManager {
 	 * @return A block channel writer that writes to the given channel.
 	 * @throws IOException Thrown, if the channel for the writer could not be opened.
 	 */
-	public BlockChannelWriter createBlockChannelWriter(FileIOChannel.ID channelID) throws IOException {
+	public BlockChannelWriter<MemorySegment> createBlockChannelWriter(FileIOChannel.ID channelID) throws IOException {
 		return createBlockChannelWriter(channelID, new LinkedBlockingQueue<MemorySegment>());
 	}
 
@@ -203,7 +219,7 @@ public abstract class IOManager {
 	 * @return A block channel writer that writes to the given channel.
 	 * @throws IOException Thrown, if the channel for the writer could not be opened.
 	 */
-	public abstract BlockChannelWriter createBlockChannelWriter(FileIOChannel.ID channelID,
+	public abstract BlockChannelWriter<MemorySegment> createBlockChannelWriter(FileIOChannel.ID channelID,
 				LinkedBlockingQueue<MemorySegment> returnQueue) throws IOException;
 
 	/**
@@ -216,7 +232,7 @@ public abstract class IOManager {
 	 * @return A block channel writer that writes to the given channel.
 	 * @throws IOException Thrown, if the channel for the writer could not be opened.
 	 */
-	public abstract BlockChannelWriterWithCallback createBlockChannelWriter(FileIOChannel.ID channelID, RequestDoneCallback<MemorySegment> callback) throws IOException;
+	public abstract BlockChannelWriterWithCallback<MemorySegment> createBlockChannelWriter(FileIOChannel.ID channelID, RequestDoneCallback<MemorySegment> callback) throws IOException;
 
 	/**
 	 * Creates a block channel reader that reads blocks from the given channel. The reader pushed
@@ -227,7 +243,7 @@ public abstract class IOManager {
 	 * @return A block channel reader that reads from the given channel.
 	 * @throws IOException Thrown, if the channel for the reader could not be opened.
 	 */
-	public BlockChannelReader createBlockChannelReader(FileIOChannel.ID channelID) throws IOException {
+	public BlockChannelReader<MemorySegment> createBlockChannelReader(FileIOChannel.ID channelID) throws IOException {
 		return createBlockChannelReader(channelID, new LinkedBlockingQueue<MemorySegment>());
 	}
 
@@ -240,9 +256,15 @@ public abstract class IOManager {
 	 * @return A block channel reader that reads from the given channel.
 	 * @throws IOException Thrown, if the channel for the reader could not be opened.
 	 */
-	public abstract BlockChannelReader createBlockChannelReader(FileIOChannel.ID channelID,
+	public abstract BlockChannelReader<MemorySegment> createBlockChannelReader(FileIOChannel.ID channelID,
 										LinkedBlockingQueue<MemorySegment> returnQueue) throws IOException;
 
+	public abstract BufferFileWriter createBufferFileWriter(FileIOChannel.ID channelID) throws IOException;
+
+	public abstract BufferFileReader createBufferFileReader(FileIOChannel.ID channelID, RequestDoneCallback<Buffer> callback) throws IOException;
+
+	public abstract BufferFileSegmentReader createBufferFileSegmentReader(FileIOChannel.ID channelID, RequestDoneCallback<FileSegment> callback) throws IOException;
+
 	/**
 	 * Creates a block channel reader that reads all blocks from the given channel directly in one bulk.
 	 * The reader draws segments to read the blocks into from a supplied list, which must contain as many

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsync.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsync.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsync.java
index 2396665..e615913 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsync.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsync.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.io.disk.iomanager;
 
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
 import org.apache.flink.runtime.util.EnvironmentInformation;
 
 import java.io.IOException;
@@ -143,7 +144,7 @@ public class IOManagerAsync extends IOManager implements UncaughtExceptionHandle
 			}
 		}
 		finally {
-			// make sure we all the super implementation in any case and at the last point,
+			// make sure we call the super implementation in any case and at the last point,
 			// because this will clean up the I/O directories
 			super.shutdown();
 		}
@@ -182,7 +183,7 @@ public class IOManagerAsync extends IOManager implements UncaughtExceptionHandle
 	// ------------------------------------------------------------------------
 	
 	@Override
-	public BlockChannelWriter createBlockChannelWriter(FileIOChannel.ID channelID,
+	public BlockChannelWriter<MemorySegment> createBlockChannelWriter(FileIOChannel.ID channelID,
 								LinkedBlockingQueue<MemorySegment> returnQueue) throws IOException
 	{
 		checkState(!isShutdown.get(), "I/O-Manger is shut down.");
@@ -190,7 +191,7 @@ public class IOManagerAsync extends IOManager implements UncaughtExceptionHandle
 	}
 	
 	@Override
-	public BlockChannelWriterWithCallback createBlockChannelWriter(FileIOChannel.ID channelID, RequestDoneCallback<MemorySegment> callback) throws IOException {
+	public BlockChannelWriterWithCallback<MemorySegment> createBlockChannelWriter(FileIOChannel.ID channelID, RequestDoneCallback<MemorySegment> callback) throws IOException {
 		checkState(!isShutdown.get(), "I/O-Manger is shut down.");
 		return new AsynchronousBlockWriterWithCallback(channelID, this.writers[channelID.getThreadNum()].requestQueue, callback);
 	}
@@ -206,13 +207,34 @@ public class IOManagerAsync extends IOManager implements UncaughtExceptionHandle
 	 * @throws IOException Thrown, if the channel for the reader could not be opened.
 	 */
 	@Override
-	public BlockChannelReader createBlockChannelReader(FileIOChannel.ID channelID,
+	public BlockChannelReader<MemorySegment> createBlockChannelReader(FileIOChannel.ID channelID,
 										LinkedBlockingQueue<MemorySegment> returnQueue) throws IOException
 	{
 		checkState(!isShutdown.get(), "I/O-Manger is shut down.");
 		return new AsynchronousBlockReader(channelID, this.readers[channelID.getThreadNum()].requestQueue, returnQueue);
 	}
-	
+
+	@Override
+	public BufferFileWriter createBufferFileWriter(FileIOChannel.ID channelID) throws IOException {
+		checkState(!isShutdown.get(), "I/O-Manger is shut down.");
+
+		return new AsynchronousBufferFileWriter(channelID, writers[channelID.getThreadNum()].requestQueue);
+	}
+
+	@Override
+	public BufferFileReader createBufferFileReader(FileIOChannel.ID channelID, RequestDoneCallback<Buffer> callback) throws IOException {
+		checkState(!isShutdown.get(), "I/O-Manger is shut down.");
+
+		return new AsynchronousBufferFileReader(channelID, readers[channelID.getThreadNum()].requestQueue, callback);
+	}
+
+	@Override
+	public BufferFileSegmentReader createBufferFileSegmentReader(FileIOChannel.ID channelID, RequestDoneCallback<FileSegment> callback) throws IOException {
+		checkState(!isShutdown.get(), "I/O-Manger is shut down.");		
+
+		return new AsynchronousBufferFileSegmentReader(channelID, readers[channelID.getThreadNum()].requestQueue, callback);
+	}
+
 	/**
 	 * Creates a block channel reader that reads all blocks from the given channel directly in one bulk.
 	 * The reader draws segments to read the blocks into from a supplied list, which must contain as many

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/QueuingCallback.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/QueuingCallback.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/QueuingCallback.java
index 95f3dc7..a2e3e82 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/QueuingCallback.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/QueuingCallback.java
@@ -21,26 +21,24 @@ package org.apache.flink.runtime.io.disk.iomanager;
 import java.io.IOException;
 import java.util.concurrent.LinkedBlockingQueue;
 
-import org.apache.flink.core.memory.MemorySegment;
-
 /**
  * A {@link RequestDoneCallback} that adds the memory segments to a blocking queue.
  */
-public class QueuingCallback implements RequestDoneCallback<MemorySegment> {
+public class QueuingCallback<T> implements RequestDoneCallback<T> {
+
+	private final LinkedBlockingQueue<T> queue;
 
-	private final LinkedBlockingQueue<MemorySegment> queue;
-	
-	public QueuingCallback(LinkedBlockingQueue<MemorySegment> queue) {
+	public QueuingCallback(LinkedBlockingQueue<T> queue) {
 		this.queue = queue;
 	}
 
 	@Override
-	public void requestSuccessful(MemorySegment buffer) {
+	public void requestSuccessful(T buffer) {
 		queue.add(buffer);
 	}
 
 	@Override
-	public void requestFailed(MemorySegment buffer, IOException e) {
+	public void requestFailed(T buffer, IOException e) {
 		// the I/O error is recorded in the writer already
 		queue.add(buffer);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/SynchronousBufferFileReader.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/SynchronousBufferFileReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/SynchronousBufferFileReader.java
new file mode 100644
index 0000000..27189cb
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/SynchronousBufferFileReader.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.disk.iomanager;
+
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * A synchronous {@link BufferFileReader} implementation.
+ *
+ * <p> This currently bypasses the I/O manager as it is the only synchronous implementation, which
+ * is currently in use.
+ *
+ * TODO Refactor I/O manager setup and refactor this into it
+ */
+public class SynchronousBufferFileReader extends SynchronousFileIOChannel implements BufferFileReader {
+
+	private final ByteBuffer header = ByteBuffer.allocateDirect(8);
+
+	private boolean hasReachedEndOfFile;
+
+	public SynchronousBufferFileReader(ID channelID, boolean writeEnabled) throws IOException {
+		super(channelID, writeEnabled);
+	}
+
+	@Override
+	public void readInto(Buffer buffer) throws IOException {
+		if (fileChannel.size() - fileChannel.position() > 0) {
+			// This is the synchronous counter part to the asynchronous buffer read request
+
+			// Read header
+			header.clear();
+			fileChannel.read(header);
+			header.flip();
+
+			final boolean isBuffer = header.getInt() == 1;
+			final int size = header.getInt();
+
+			if (size > buffer.getMemorySegment().size()) {
+				throw new IllegalStateException("Buffer is too small for data: " + buffer.getMemorySegment().size() + " bytes available, but " + size + " needed. This is most likely due to an serialized event, which is larger than the buffer size.");
+			}
+
+			buffer.setSize(size);
+
+			fileChannel.read(buffer.getNioBuffer());
+
+			if (!isBuffer) {
+				buffer.tagAsEvent();
+			}
+
+			hasReachedEndOfFile = fileChannel.size() - fileChannel.position() == 0;
+		}
+		else {
+			buffer.recycle();
+		}
+	}
+
+	@Override
+	public void seekToPosition(long position) throws IOException {
+		fileChannel.position(position);
+	}
+
+	@Override
+	public boolean hasReachedEndOfFile() {
+		return hasReachedEndOfFile;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/SynchronousFileIOChannel.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/SynchronousFileIOChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/SynchronousFileIOChannel.java
index fd6c230..19a0fc9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/SynchronousFileIOChannel.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/SynchronousFileIOChannel.java
@@ -42,4 +42,4 @@ public abstract class SynchronousFileIOChannel extends AbstractFileIOChannel {
 			this.fileChannel.close();
 		}
 	}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ConnectionID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ConnectionID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ConnectionID.java
new file mode 100644
index 0000000..5a31c3f
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ConnectionID.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network;
+
+import org.apache.flink.runtime.executiongraph.IntermediateResult;
+import org.apache.flink.runtime.instance.InstanceConnectionInfo;
+
+import java.io.Serializable;
+import java.net.InetSocketAddress;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * A {@link ConnectionID} identifies a connection to a remote task manager by the socket address and
+ * a connection index. This allows multiple connections to the same task manager to be distinguished
+ * by their connection index.
+ *
+ * <p> The connection index is assigned by the {@link IntermediateResult} and ensures that it is
+ * safe to multiplex multiple data transfers over the same physical TCP connection.
+ */
+public class ConnectionID implements Serializable {
+
+	private final InetSocketAddress address;
+
+	private final int connectionIndex;
+
+	public ConnectionID(InstanceConnectionInfo connectionInfo, int connectionIndex) {
+		this(new InetSocketAddress(connectionInfo.address(), connectionInfo.dataPort()), connectionIndex);
+	}
+
+	public ConnectionID(InetSocketAddress address, int connectionIndex) {
+		this.address = checkNotNull(address);
+		checkArgument(connectionIndex >= 0);
+		this.connectionIndex = connectionIndex;
+	}
+
+	public InetSocketAddress getAddress() {
+		return address;
+	}
+
+	public int getConnectionIndex() {
+		return connectionIndex;
+	}
+
+	@Override
+	public int hashCode() {
+		return address.hashCode() + (31 * connectionIndex);
+	}
+
+	@Override
+	public boolean equals(Object other) {
+		if (other.getClass() != ConnectionID.class) {
+			return false;
+		}
+
+		final ConnectionID ra = (ConnectionID) other;
+		if (!ra.getAddress().equals(address) || ra.getConnectionIndex() != connectionIndex) {
+			return false;
+		}
+
+		return true;
+	}
+
+	@Override
+	public String toString() {
+		return address + " [" + connectionIndex + "]";
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ConnectionManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ConnectionManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ConnectionManager.java
index 76f8bbd..06dc151 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ConnectionManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ConnectionManager.java
@@ -18,8 +18,9 @@
 
 package org.apache.flink.runtime.io.network;
 
+import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool;
 import org.apache.flink.runtime.io.network.netty.PartitionRequestClient;
-import org.apache.flink.runtime.io.network.partition.IntermediateResultPartitionProvider;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider;
 
 import java.io.IOException;
 
@@ -29,20 +30,20 @@ import java.io.IOException;
  */
 public interface ConnectionManager {
 
-	void start(IntermediateResultPartitionProvider partitionProvider, TaskEventDispatcher taskEventDispatcher) throws IOException;
+	void start(ResultPartitionProvider partitionProvider, TaskEventDispatcher taskEventDispatcher, NetworkBufferPool networkbufferPool) throws IOException;
 
 	/**
-	 * Creates a {@link PartitionRequestClient} instance for the given {@link RemoteAddress}.
+	 * Creates a {@link PartitionRequestClient} instance for the given {@link ConnectionID}.
 	 */
-	PartitionRequestClient createPartitionRequestClient(RemoteAddress remoteAddress) throws IOException, InterruptedException;
+	PartitionRequestClient createPartitionRequestClient(ConnectionID connectionId) throws IOException, InterruptedException;
 
 	/**
 	 * Closes opened ChannelConnections in case of a resource release
-	 * @param remoteAddress
 	 */
-	void closeOpenChannelConnections(RemoteAddress remoteAddress);
+	void closeOpenChannelConnections(ConnectionID connectionId);
 
 	int getNumberOfActiveConnections();
 
 	void shutdown() throws IOException;
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/LocalConnectionManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/LocalConnectionManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/LocalConnectionManager.java
index 447f6e6..af6273e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/LocalConnectionManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/LocalConnectionManager.java
@@ -18,8 +18,9 @@
 
 package org.apache.flink.runtime.io.network;
 
+import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool;
 import org.apache.flink.runtime.io.network.netty.PartitionRequestClient;
-import org.apache.flink.runtime.io.network.partition.IntermediateResultPartitionProvider;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider;
 
 import java.io.IOException;
 
@@ -30,16 +31,16 @@ import java.io.IOException;
 public class LocalConnectionManager implements ConnectionManager {
 
 	@Override
-	public void start(IntermediateResultPartitionProvider partitionProvider, TaskEventDispatcher taskEventDispatcher) throws IOException {
+	public void start(ResultPartitionProvider partitionProvider, TaskEventDispatcher taskEventDispatcher, NetworkBufferPool networkbufferPool) throws IOException {
 	}
 
 	@Override
-	public PartitionRequestClient createPartitionRequestClient(RemoteAddress remoteAddress) throws IOException {
+	public PartitionRequestClient createPartitionRequestClient(ConnectionID connectionId) throws IOException {
 		return null;
 	}
 
 	@Override
-	public void closeOpenChannelConnections(RemoteAddress remoteAddress) {}
+	public void closeOpenChannelConnections(ConnectionID connectionId) {}
 
 	@Override
 	public int getNumberOfActiveConnections() {

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java
index 58b21e1..e02e744 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java
@@ -21,13 +21,14 @@ package org.apache.flink.runtime.io.network;
 import akka.actor.ActorRef;
 import akka.util.Timeout;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.io.network.api.writer.BufferWriter;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode;
+import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
 import org.apache.flink.runtime.io.network.buffer.BufferPool;
 import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool;
 import org.apache.flink.runtime.io.network.netty.NettyConfig;
 import org.apache.flink.runtime.io.network.netty.NettyConnectionManager;
-import org.apache.flink.runtime.io.network.partition.IntermediateResultPartition;
-import org.apache.flink.runtime.io.network.partition.IntermediateResultPartitionManager;
+import org.apache.flink.runtime.io.network.partition.ResultPartition;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionManager;
 import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
 import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration;
 import org.apache.flink.runtime.taskmanager.Task;
@@ -54,7 +55,7 @@ public class NetworkEnvironment {
 
 	private final FiniteDuration jobManagerTimeout;
 
-	private final IntermediateResultPartitionManager partitionManager;
+	private final ResultPartitionManager partitionManager;
 
 	private final TaskEventDispatcher taskEventDispatcher;
 
@@ -62,6 +63,8 @@ public class NetworkEnvironment {
 
 	private final ConnectionManager connectionManager;
 
+	private final NetworkEnvironmentConfiguration configuration;
+
 	private boolean isShutdown;
 
 	/**
@@ -74,8 +77,9 @@ public class NetworkEnvironment {
 		this.jobManager = checkNotNull(jobManager);
 		this.jobManagerTimeout = checkNotNull(jobManagerTimeout);
 
-		this.partitionManager = new IntermediateResultPartitionManager();
+		this.partitionManager = new ResultPartitionManager();
 		this.taskEventDispatcher = new TaskEventDispatcher();
+		this.configuration = checkNotNull(config);
 
 		// --------------------------------------------------------------------
 		// Network buffers
@@ -95,7 +99,7 @@ public class NetworkEnvironment {
 		connectionManager = nettyConfig.isDefined() ? new NettyConnectionManager(nettyConfig.get()) : new LocalConnectionManager();
 
 		try {
-			connectionManager.start(partitionManager, taskEventDispatcher);
+			connectionManager.start(partitionManager, taskEventDispatcher, networkBufferPool);
 		}
 		catch (Throwable t) {
 			throw new IOException("Failed to instantiate network connection manager: " + t.getMessage(), t);
@@ -115,30 +119,29 @@ public class NetworkEnvironment {
 	}
 
 	public void registerTask(Task task) throws IOException {
-		final ExecutionAttemptID executionId = task.getExecutionId();
-
-		final IntermediateResultPartition[] producedPartitions = task.getProducedPartitions();
-		final BufferWriter[] writers = task.getWriters();
+		final ResultPartition[] producedPartitions = task.getProducedPartitions();
+		final ResultPartitionWriter[] writers = task.getWriters();
 
 		if (writers.length != producedPartitions.length) {
 			throw new IllegalStateException("Unequal number of writers and partitions.");
 		}
 
 		for (int i = 0; i < producedPartitions.length; i++) {
-			final IntermediateResultPartition partition = producedPartitions[i];
-			final BufferWriter writer = writers[i];
+			final ResultPartition partition = producedPartitions[i];
+			final ResultPartitionWriter writer = writers[i];
 
 			// Buffer pool for the partition
 			BufferPool bufferPool = null;
 
 			try {
-				bufferPool = networkBufferPool.createBufferPool(partition.getNumberOfQueues(), false);
-				partition.setBufferPool(bufferPool);
+				bufferPool = networkBufferPool.createBufferPool(partition.getNumberOfSubpartitions(), false);
+				partition.registerBufferPool(bufferPool);
+
 				partitionManager.registerIntermediateResultPartition(partition);
 			}
 			catch (Throwable t) {
 				if (bufferPool != null) {
-					bufferPool.destroy();
+					bufferPool.lazyDestroy();
 				}
 
 				if (t instanceof IOException) {
@@ -150,7 +153,7 @@ public class NetworkEnvironment {
 			}
 
 			// Register writer with task event dispatcher
-			taskEventDispatcher.registerWriterForIncomingTaskEvents(executionId, writer.getPartitionId(), writer);
+			taskEventDispatcher.registerWriterForIncomingTaskEvents(writer.getPartitionId(), writer);
 		}
 
 		// Setup the buffer pool for each buffer reader
@@ -165,7 +168,7 @@ public class NetworkEnvironment {
 			}
 			catch (Throwable t) {
 				if (bufferPool != null) {
-					bufferPool.destroy();
+					bufferPool.lazyDestroy();
 				}
 
 				if (t instanceof IOException) {
@@ -185,10 +188,16 @@ public class NetworkEnvironment {
 		final ExecutionAttemptID executionId = task.getExecutionId();
 
 		if (task.isCanceledOrFailed()) {
-			partitionManager.failIntermediateResultPartitions(executionId);
+			partitionManager.releasePartitionsProducedBy(executionId);
 		}
 
-		taskEventDispatcher.unregisterWriters(executionId);
+		ResultPartitionWriter[] writers = task.getWriters();
+
+		if (writers != null) {
+			for (ResultPartitionWriter writer : task.getWriters()) {
+				taskEventDispatcher.unregisterWriter(writer);
+			}
+		}
 
 		final SingleInputGate[] inputGates = task.getInputGates();
 
@@ -206,7 +215,7 @@ public class NetworkEnvironment {
 		}
 	}
 
-	public IntermediateResultPartitionManager getPartitionManager() {
+	public ResultPartitionManager getPartitionManager() {
 		return partitionManager;
 	}
 
@@ -222,6 +231,10 @@ public class NetworkEnvironment {
 		return networkBufferPool;
 	}
 
+	public IOMode getDefaultIOMode() {
+		return configuration.ioMode();
+	}
+
 	public boolean hasReleasedAllResources() {
 		String msg = String.format("Network buffer pool: %d missing memory segments. %d registered buffer pools. Connection manager: %d active connections. Task event dispatcher: %d registered writers.",
 				networkBufferPool.getTotalNumberOfMemorySegments() - networkBufferPool.getNumberOfAvailableMemorySegments(), networkBufferPool.getNumberOfRegisteredBufferPools(), connectionManager.getNumberOfActiveConnections(), taskEventDispatcher.getNumberOfRegisteredWriters());

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/RemoteAddress.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/RemoteAddress.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/RemoteAddress.java
deleted file mode 100644
index 937055b..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/RemoteAddress.java
+++ /dev/null
@@ -1,122 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.io.network;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-
-import org.apache.flink.core.io.IOReadableWritable;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.executiongraph.IntermediateResult;
-import org.apache.flink.runtime.instance.InstanceConnectionInfo;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * A {@link RemoteAddress} identifies a connection to a remote task manager by
- * the socket address and a connection index. This allows multiple connections
- * to be distinguished by their connection index.
- * <p>
- * The connection index is assigned by the {@link IntermediateResult} and
- * ensures that it is safe to multiplex multiple data transfers over the same
- * physical TCP connection.
- */
-public class RemoteAddress implements IOReadableWritable, Serializable {
-
-	private InetSocketAddress address;
-
-	private int connectionIndex;
-
-	public RemoteAddress(InstanceConnectionInfo connectionInfo, int connectionIndex) {
-		this(new InetSocketAddress(connectionInfo.address(), connectionInfo.dataPort()), connectionIndex);
-	}
-
-	public RemoteAddress(InetSocketAddress address, int connectionIndex) {
-		this.address = checkNotNull(address);
-		checkArgument(connectionIndex >= 0);
-		this.connectionIndex = connectionIndex;
-	}
-
-	public InetSocketAddress getAddress() {
-		return address;
-	}
-
-	public int getConnectionIndex() {
-		return connectionIndex;
-	}
-
-	@Override
-	public int hashCode() {
-		return address.hashCode() + (31 * connectionIndex);
-	}
-
-	@Override
-	public boolean equals(Object other) {
-		if (other.getClass() != RemoteAddress.class) {
-			return false;
-		}
-
-		final RemoteAddress ra = (RemoteAddress) other;
-		if (!ra.getAddress().equals(address) || ra.getConnectionIndex() != connectionIndex) {
-			return false;
-		}
-
-		return true;
-	}
-
-	@Override
-	public String toString() {
-		return address + " [" + connectionIndex + "]";
-	}
-
-	// ------------------------------------------------------------------------
-	// Serialization
-	// ------------------------------------------------------------------------
-
-	public RemoteAddress() {
-		this.address = null;
-		this.connectionIndex = -1;
-	}
-
-	@Override
-	public void write(final DataOutputView out) throws IOException {
-		final InetAddress ia = address.getAddress();
-		out.writeInt(ia.getAddress().length);
-		out.write(ia.getAddress());
-		out.writeInt(address.getPort());
-
-		out.writeInt(connectionIndex);
-	}
-
-	@Override
-	public void read(final DataInputView in) throws IOException {
-		final byte[] addressBytes = new byte[in.readInt()];
-		in.readFully(addressBytes);
-
-		final InetAddress ia = InetAddress.getByAddress(addressBytes);
-		int port = in.readInt();
-
-		address = new InetSocketAddress(ia, port);
-		connectionIndex = in.readInt();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/TaskEventDispatcher.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/TaskEventDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/TaskEventDispatcher.java
index 7a529b9..845f72a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/TaskEventDispatcher.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/TaskEventDispatcher.java
@@ -18,59 +18,49 @@
 
 package org.apache.flink.runtime.io.network;
 
-import com.google.common.collect.HashBasedTable;
-import com.google.common.collect.Table;
+import com.google.common.collect.Maps;
 import org.apache.flink.runtime.event.task.TaskEvent;
-import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.io.network.api.writer.BufferWriter;
-import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
+import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.consumer.LocalInputChannel;
+import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel;
 import org.apache.flink.runtime.util.event.EventListener;
 
-import java.util.ArrayList;
-import java.util.List;
+import java.util.Map;
 
 /**
- * The task event dispatcher dispatches events flowing backwards from a consumer
- * to a producer. It only supports programs, where the producer and consumer
- * are running at the same time.
- * <p>
- * The publish method is either called from the local input channel or the
- * network I/O thread.
+ * The task event dispatcher dispatches events flowing backwards from a consuming task to the task
+ * producing the consumed result.
+ *
+ * <p> Backwards events only work for tasks, which produce pipelined results, where both the
+ * producing and consuming task are running at the same time.
  */
 public class TaskEventDispatcher {
 
-	Table<ExecutionAttemptID, IntermediateResultPartitionID, BufferWriter> registeredWriters = HashBasedTable.create();
+	private final Map<ResultPartitionID, ResultPartitionWriter> registeredWriters = Maps.newHashMap();
 
-	public void registerWriterForIncomingTaskEvents(ExecutionAttemptID executionId, IntermediateResultPartitionID partitionId, BufferWriter listener) {
+	public void registerWriterForIncomingTaskEvents(ResultPartitionID partitionId, ResultPartitionWriter writer) {
 		synchronized (registeredWriters) {
-			if (registeredWriters.put(executionId, partitionId, listener) != null) {
-				throw new IllegalStateException("Event dispatcher already contains buffer writer.");
+			if (registeredWriters.put(partitionId, writer) != null) {
+				throw new IllegalStateException("Already registered at task event dispatcher.");
 			}
 		}
 	}
 
-	public void unregisterWriters(ExecutionAttemptID executionId) {
+	public void unregisterWriter(ResultPartitionWriter writer) {
 		synchronized (registeredWriters) {
-			List<IntermediateResultPartitionID> writersToUnregister = new ArrayList<IntermediateResultPartitionID>();
-
-			for (IntermediateResultPartitionID partitionId : registeredWriters.row(executionId).keySet()) {
-				writersToUnregister.add(partitionId);
-			}
-
-			for(IntermediateResultPartitionID partitionId : writersToUnregister) {
-				registeredWriters.remove(executionId, partitionId);
-			}
+			registeredWriters.remove(writer.getPartitionId());
 		}
 	}
 
 	/**
-	 * Publishes the event to the registered {@link EventListener} instance.
+	 * Publishes the event to the registered {@link ResultPartitionWriter} instances.
 	 * <p>
-	 * This method is either called from a local input channel or the network
-	 * I/O thread on behalf of a remote input channel.
+	 * This method is either called directly from a {@link LocalInputChannel} or the network I/O
+	 * thread on behalf of a {@link RemoteInputChannel}.
 	 */
-	public boolean publish(ExecutionAttemptID executionId, IntermediateResultPartitionID partitionId, TaskEvent event) {
-		EventListener<TaskEvent> listener = registeredWriters.get(executionId, partitionId);
+	public boolean publish(ResultPartitionID partitionId, TaskEvent event) {
+		EventListener<TaskEvent> listener = registeredWriters.get(partitionId);
 
 		if (listener != null) {
 			listener.onEvent(event);
@@ -80,6 +70,9 @@ public class TaskEventDispatcher {
 		return false;
 	}
 
+	/**
+	 * Returns the number of currently registered writers.
+	 */
 	int getNumberOfRegisteredWriters() {
 		synchronized (registeredWriters) {
 			return registeredWriters.size();

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/AbstractRecordReader.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/AbstractRecordReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/AbstractRecordReader.java
index e70b6ee..4ee7fad 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/AbstractRecordReader.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/AbstractRecordReader.java
@@ -21,10 +21,10 @@ package org.apache.flink.runtime.io.network.api.reader;
 import org.apache.flink.core.io.IOReadableWritable;
 import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer;
 import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer.DeserializationResult;
+import org.apache.flink.runtime.io.network.api.serialization.SpillingAdaptiveSpanningRecordDeserializer;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
 import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
 import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
-import org.apache.flink.runtime.io.network.serialization.SpillingAdaptiveSpanningRecordDeserializer;
 
 import java.io.IOException;
 
@@ -43,6 +43,7 @@ abstract class AbstractRecordReader<T extends IOReadableWritable> extends Abstra
 
 	private boolean isFinished;
 
+	@SuppressWarnings("unchecked")
 	protected AbstractRecordReader(InputGate inputGate) {
 		super(inputGate);
 


[06/13] flink git commit: [FLINK-1350] [runtime] Add blocking result partition variant

Posted by uc...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/MutableHashTable.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/MutableHashTable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/MutableHashTable.java
index e69ef17..c0be611 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/MutableHashTable.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/MutableHashTable.java
@@ -300,7 +300,7 @@ public class MutableHashTable<BT, PT> implements MemorySegmentSource {
 	/**
 	 * The reader for the spilled-file of the probe partition that is currently read.
 	 */
-	private BlockChannelReader currentSpilledProbeSide;
+	private BlockChannelReader<MemorySegment> currentSpilledProbeSide;
 	
 	/**
 	 * The channel enumerator that is used while processing the current partition to create
@@ -802,7 +802,7 @@ public class MutableHashTable<BT, PT> implements MemorySegmentSource {
 			segments.add(getNextBuffer());
 			segments.add(getNextBuffer());
 			
-			final BlockChannelReader inReader = this.ioManager.createBlockChannelReader(p.getBuildSideChannel().getChannelID());
+			final BlockChannelReader<MemorySegment> inReader = this.ioManager.createBlockChannelReader(p.getBuildSideChannel().getChannelID());
 			final ChannelReaderInputView inView = new HeaderlessChannelReaderInputView(inReader, segments,
 						p.getBuildSideBlockCount(), p.getLastSegmentLimit(), false);
 			final ChannelReaderInputViewIterator<BT> inIter = new ChannelReaderInputViewIterator<BT>(inView, 

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReOpenableHashPartition.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReOpenableHashPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReOpenableHashPartition.java
index 56dcfae..84868ff 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReOpenableHashPartition.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReOpenableHashPartition.java
@@ -37,7 +37,7 @@ public class ReOpenableHashPartition<BT, PT> extends HashPartition<BT, PT> {
 
 	private FileIOChannel.ID initialBuildSideChannel = null;			// path to initial build side contents (only for in-memory partitions)
 	
-	private BlockChannelWriter initialBuildSideWriter = null;
+	private BlockChannelWriter<MemorySegment> initialBuildSideWriter = null;
 
 	private boolean isRestored = false;							// marks a restored partition
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java
index 3466024..d05bd9a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java
@@ -301,7 +301,7 @@ public class CombiningUnilateralSortMerger<E> extends UnilateralSortMerger<E> {
 				}
 
 				// create writer
-				final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channel);
+				final BlockChannelWriter<MemorySegment> writer = this.ioManager.createBlockChannelWriter(channel);
 				registerOpenChannelToBeRemovedAtShudown(writer);
 				final ChannelWriterOutputView output = new ChannelWriterOutputView(writer, this.writeMemory,
 																			this.memManager.getPageSize());
@@ -457,7 +457,7 @@ public class CombiningUnilateralSortMerger<E> extends UnilateralSortMerger<E> {
 			// create a new channel writer
 			final FileIOChannel.ID mergedChannelID = this.ioManager.createChannel();
 			registerChannelToBeRemovedAtShudown(mergedChannelID);
-			final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(mergedChannelID);
+			final BlockChannelWriter<MemorySegment> writer = this.ioManager.createBlockChannelWriter(mergedChannelID);
 			registerOpenChannelToBeRemovedAtShudown(writer);
 			final ChannelWriterOutputView output = new ChannelWriterOutputView(writer, writeBuffers, 
 																			this.memManager.getPageSize());

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java
index a938e98..388b7b9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java
@@ -1315,7 +1315,7 @@ public class UnilateralSortMerger<E> implements Sorter<E> {
 				registerChannelToBeRemovedAtShudown(channel);
 
 				// create writer
-				final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channel);
+				final BlockChannelWriter<MemorySegment> writer = this.ioManager.createBlockChannelWriter(channel);
 				registerOpenChannelToBeRemovedAtShudown(writer);
 				final ChannelWriterOutputView output = new ChannelWriterOutputView(writer, this.writeMemory,
 																			this.memManager.getPageSize());
@@ -1493,7 +1493,7 @@ public class UnilateralSortMerger<E> implements Sorter<E> {
 				final List<MemorySegment> segsForChannel = inputSegments.get(i);
 				
 				// create a reader. if there are multiple segments for the reader, issue multiple together per I/O request
-				final BlockChannelReader reader = this.ioManager.createBlockChannelReader(channel.getChannel());
+				final BlockChannelReader<MemorySegment> reader = this.ioManager.createBlockChannelReader(channel.getChannel());
 					
 				readerList.add(reader);
 				registerOpenChannelToBeRemovedAtShudown(reader);
@@ -1578,7 +1578,7 @@ public class UnilateralSortMerger<E> implements Sorter<E> {
 			// create a new channel writer
 			final FileIOChannel.ID mergedChannelID = this.ioManager.createChannel();
 			registerChannelToBeRemovedAtShudown(mergedChannelID);
-			final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(mergedChannelID);
+			final BlockChannelWriter<MemorySegment> writer = this.ioManager.createBlockChannelWriter(mergedChannelID);
 			registerOpenChannelToBeRemovedAtShudown(writer);
 			final ChannelWriterOutputView output = new ChannelWriterOutputView(writer, writeBuffers, 
 																			this.memManager.getPageSize());

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
index e3e4175..8cebc6c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
@@ -23,14 +23,15 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.execution.RuntimeEnvironment;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.io.network.api.writer.BufferWriter;
-import org.apache.flink.runtime.io.network.partition.IntermediateResultPartition;
+import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
+import org.apache.flink.runtime.io.network.partition.ResultPartition;
 import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.memorymanager.MemoryManager;
 import org.apache.flink.runtime.messages.ExecutionGraphMessages;
 import org.apache.flink.runtime.messages.JobManagerMessages;
+import org.apache.flink.runtime.messages.TaskManagerMessages.UnregisterTask;
 import org.apache.flink.runtime.profiling.TaskManagerProfiler;
 import org.apache.flink.util.ExceptionUtils;
 import org.slf4j.Logger;
@@ -73,6 +74,8 @@ public class Task {
 	/** The current execution state of the task */
 	private volatile ExecutionState executionState = ExecutionState.DEPLOYING;
 
+	private volatile Throwable failureCause;
+
 	// --------------------------------------------------------------------------------------------	
 
 	public Task(JobID jobId, JobVertexID vertexId, int taskIndex, int parallelism,
@@ -160,6 +163,10 @@ public class Task {
 		}
 	}
 
+	public Throwable getFailureCause() {
+		return failureCause;
+	}
+
 	// ----------------------------------------------------------------------------------------------------------------
 	//  States and Transitions
 	// ----------------------------------------------------------------------------------------------------------------
@@ -174,7 +181,7 @@ public class Task {
 	public boolean markAsFinished() {
 		if (STATE_UPDATER.compareAndSet(this, ExecutionState.RUNNING, ExecutionState.FINISHED)) {
 			notifyObservers(ExecutionState.FINISHED, null);
-			notifyExecutionStateChange(ExecutionState.FINISHED, null);
+			unregisterTask();
 			return true;
 		}
 		else {
@@ -195,8 +202,11 @@ public class Task {
 			// after all, we may have recognized our failure state before the cancelling and never sent a canceled
 			// message back
 			else if (STATE_UPDATER.compareAndSet(this, current, ExecutionState.FAILED)) {
+				this.failureCause = error;
+
 				notifyObservers(ExecutionState.FAILED, ExceptionUtils.stringifyException(error));
-				notifyExecutionStateChange(ExecutionState.FAILED, error);
+				unregisterTask();
+
 				return;
 			}
 		}
@@ -218,7 +228,7 @@ public class Task {
 				if (STATE_UPDATER.compareAndSet(this, current, ExecutionState.CANCELED)) {
 
 					notifyObservers(ExecutionState.CANCELED, null);
-					notifyExecutionStateChange(ExecutionState.CANCELED, null);
+					unregisterTask();
 					return;
 				}
 			}
@@ -268,9 +278,10 @@ public class Task {
 			if (current == ExecutionState.DEPLOYING) {
 				// directly set to canceled
 				if (STATE_UPDATER.compareAndSet(this, current, ExecutionState.FAILED)) {
+					this.failureCause = cause;
 
 					notifyObservers(ExecutionState.FAILED, null);
-					notifyExecutionStateChange(ExecutionState.FAILED, cause);
+					unregisterTask();
 					return;
 				}
 			}
@@ -284,8 +295,10 @@ public class Task {
 						LOG.error("Error while cancelling the task.", e);
 					}
 
+					this.failureCause = cause;
+
 					notifyObservers(ExecutionState.FAILED, null);
-					notifyExecutionStateChange(ExecutionState.FAILED, cause);
+					unregisterTask();
 
 					return;
 				}
@@ -309,7 +322,7 @@ public class Task {
 
 			if (STATE_UPDATER.compareAndSet(this, current, ExecutionState.CANCELED)) {
 				notifyObservers(ExecutionState.CANCELED, null);
-				notifyExecutionStateChange(ExecutionState.CANCELED, null);
+				unregisterTask();
 				return;
 			}
 		}
@@ -339,6 +352,10 @@ public class Task {
 		}
 	}
 
+	protected void unregisterTask() {
+		taskManager.tell(new UnregisterTask(executionId), ActorRef.noSender());
+	}
+
 	protected void notifyExecutionStateChange(ExecutionState executionState,
 											Throwable optionalError) {
 		LOG.info("Update execution state of {} ({}) to {}.", this.getTaskName(),
@@ -377,11 +394,11 @@ public class Task {
 		return environment != null ? environment.getAllInputGates() : null;
 	}
 
-	public BufferWriter[] getWriters() {
+	public ResultPartitionWriter[] getWriters() {
 		return environment != null ? environment.getAllWriters() : null;
 	}
 
-	public IntermediateResultPartition[] getProducedPartitions() {
+	public ResultPartition[] getProducedPartitions() {
 		return environment != null ? environment.getProducedPartitions() : null;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/util/AtomicDisposableReferenceCounter.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/AtomicDisposableReferenceCounter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/AtomicDisposableReferenceCounter.java
index ee5f281..3385ace 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/AtomicDisposableReferenceCounter.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/AtomicDisposableReferenceCounter.java
@@ -19,49 +19,61 @@
 package org.apache.flink.runtime.util;
 
 /**
- * Atomic reference counter, which enters a "disposed" state after the reference
- * count reaches 0.
+ * Atomic reference counter, which enters a "disposed" state after it reaches a configurable
+ * reference count (default 0).
  */
 public class AtomicDisposableReferenceCounter {
 
 	private final Object lock = new Object();
 
-	private int referenceCounter;
+	private int referenceCount;
 
 	private boolean isDisposed;
 
+	/** Enter the disposed state when the reference count reaches this number. */
+	private final int disposeOnReferenceCount;
+
+	public AtomicDisposableReferenceCounter() {
+		this.disposeOnReferenceCount = 0;
+	}
+
+	public AtomicDisposableReferenceCounter(int disposeOnReferenceCount) {
+		this.disposeOnReferenceCount = disposeOnReferenceCount;
+	}
+
 	/**
 	 * Increments the reference count and returns whether it was successful.
 	 * <p>
-	 * If the method returns <code>false</code>, the counter has already been
-	 * disposed. Otherwise it returns <code>true</code>.
+	 * If the method returns <code>false</code>, the counter has already been disposed. Otherwise it
+	 * returns <code>true</code>.
 	 */
-	public boolean incrementReferenceCounter() {
+	public boolean increment() {
 		synchronized (lock) {
 			if (isDisposed) {
 				return false;
 			}
 
-			referenceCounter++;
+			referenceCount++;
 			return true;
 		}
 	}
 
 	/**
-	 * Decrements the reference count.
+	 * Decrements the reference count and returns whether the reference counter entered the disposed
+	 * state.
 	 * <p>
-	 * If the method returns <code>true</code>, the decrement operation disposed
-	 * the counter. Otherwise it returns <code>false</code>.
+	 * If the method returns <code>true</code>, the decrement operation disposed the counter.
+	 * Otherwise it returns <code>false</code>.
 	 */
-	public boolean decrementReferenceCounter() {
+	public boolean decrement() {
 		synchronized (lock) {
 			if (isDisposed) {
 				return false;
 			}
 
-			referenceCounter--;
+			referenceCount--;
 
-			if (referenceCounter <= 0) {
+			if (referenceCount <= disposeOnReferenceCount) {
 				isDisposed = true;
 			}
 
@@ -69,9 +81,24 @@ public class AtomicDisposableReferenceCounter {
 		}
 	}
 
+	public int get() {
+		synchronized (lock) {
+			return referenceCount;
+		}
+	}
+
+	/**
+	 * Returns whether the reference count has reached the disposed state.
+	 */
+	public boolean isDisposed() {
+		synchronized (lock) {
+			return isDisposed;
+		}
+	}
+
 	public boolean disposeIfNotUsed() {
 		synchronized (lock) {
-			if(referenceCounter <= 0){
+			if (referenceCount <= disposeOnReferenceCount) {
 				isDisposed = true;
 			}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
index 98dd7eb..fe66b37 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
@@ -41,7 +41,7 @@ import org.apache.flink.runtime.ActorLogMessages
 import org.apache.flink.runtime.akka.AkkaUtils
 import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager
 import org.apache.flink.runtime.instance.InstanceManager
-import org.apache.flink.runtime.jobgraph.{ScheduleMode,JobGraph,JobStatus,JobID}
+import org.apache.flink.runtime.jobgraph.{JobGraph, JobStatus, JobID}
 import org.apache.flink.runtime.jobmanager.accumulators.AccumulatorManager
 import org.apache.flink.runtime.jobmanager.scheduler.{Scheduler => FlinkScheduler}
 import org.apache.flink.runtime.messages.JobManagerMessages._
@@ -97,7 +97,7 @@ class JobManager(val configuration: Configuration,
                  val delayBetweenRetries: Long,
                  val timeout: FiniteDuration)
   extends Actor with ActorLogMessages with ActorLogging {
-  
+
   /** Reference to the log, for debugging */
   val LOG = JobManager.LOG
 
@@ -283,20 +283,20 @@ class JobManager(val configuration: Configuration,
           if(newJobStatus.isTerminalState) {
             jobInfo.end = timeStamp
 
-            // is the client waiting for the job result?
+          // is the client waiting for the job result?
             newJobStatus match {
               case JobStatus.FINISHED =>
                 val accumulatorResults = accumulatorManager.getJobAccumulatorResults(jobID)
-                jobInfo.client ! JobResultSuccess(jobID,jobInfo.duration,accumulatorResults)
+                jobInfo.client ! JobResultSuccess(jobID, jobInfo.duration, accumulatorResults)
               case JobStatus.CANCELED =>
                 jobInfo.client ! Failure(new JobCancellationException(jobID,
-                  "Job was cancelled.",error))
+                  "Job was cancelled.", error))
               case JobStatus.FAILED =>
                 jobInfo.client ! Failure(new JobExecutionException(jobID,
-                  "Job execution failed.",error))
+                  "Job execution failed.", error))
               case x =>
-                val exception = new JobExecutionException(jobID,s"$x is not a " +
-                        "terminal state.")
+                val exception = new JobExecutionException(jobID, s"$x is not a " +
+                  "terminal state.")
                 jobInfo.client ! Failure(exception)
                 throw exception
             }
@@ -321,11 +321,11 @@ class JobManager(val configuration: Configuration,
         case None =>
       }
       
-    case ScheduleOrUpdateConsumers(jobId, executionId, partitionIndex) =>
+case ScheduleOrUpdateConsumers(jobId, partitionId) =>
       currentJobs.get(jobId) match {
         case Some((executionGraph, _)) =>
           sender ! Acknowledge
-          executionGraph.scheduleOrUpdateConsumers(executionId, partitionIndex)
+          executionGraph.scheduleOrUpdateConsumers(partitionId)
         case None =>
           log.error("Cannot find execution graph for job ID {} to schedule or update consumers",
             jobId)

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala
index 4630089..17e9138 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala
@@ -22,6 +22,7 @@ import org.apache.flink.runtime.accumulators.AccumulatorEvent
 import org.apache.flink.runtime.client.JobStatusMessage
 import org.apache.flink.runtime.executiongraph.{ExecutionAttemptID, ExecutionGraph}
 import org.apache.flink.runtime.instance.{InstanceID, Instance}
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID
 import org.apache.flink.runtime.jobgraph.{JobGraph, JobID, JobStatus, JobVertexID}
 import org.apache.flink.runtime.taskmanager.TaskExecutionState
 
@@ -76,18 +77,16 @@ object JobManagerMessages {
    * <p>
    * There is a call to this method for each
    * [[org.apache.flink.runtime.executiongraph.ExecutionVertex]] instance once per produced
-   * [[org.apache.flink.runtime.io.network.partition.IntermediateResultPartition]] instance,
+   * [[org.apache.flink.runtime.io.network.partition.ResultPartition]] instance,
    * either when first producing data (for pipelined executions) or when all data has been produced
    * (for staged executions).
    * <p>
    * The [[org.apache.flink.runtime.jobmanager.JobManager]] then can decide when to schedule the
    * partition consumers of the given session.
    *
-   * @see [[org.apache.flink.runtime.io.network.partition.IntermediateResultPartition]]
+   * @see [[org.apache.flink.runtime.io.network.partition.ResultPartition]]
    */
-  case class ScheduleOrUpdateConsumers(jobId: JobID,
-                                       executionId: ExecutionAttemptID,
-                                       partitionIndex: Int)
+  case class ScheduleOrUpdateConsumers(jobId: JobID, partitionId: ResultPartitionID)
 
   case class ConsumerNotificationResult(success: Boolean, error: Option[Throwable] = None)
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala
index b7bb060..4e6144a 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala
@@ -19,7 +19,7 @@
 package org.apache.flink.runtime.messages
 
 import org.apache.flink.core.io.InputSplit
-import org.apache.flink.runtime.deployment.{PartitionInfo, TaskDeploymentDescriptor}
+import org.apache.flink.runtime.deployment.{InputChannelDeploymentDescriptor, TaskDeploymentDescriptor}
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID
 import org.apache.flink.runtime.instance.InstanceID
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID
@@ -65,17 +65,21 @@ object TaskManagerMessages {
     def executionID: ExecutionAttemptID
   }
 
-  case class UpdateTaskSinglePartitionInfo(executionID: ExecutionAttemptID,
-                        resultId: IntermediateDataSetID,
-                        partitionInfo: PartitionInfo) extends UpdateTask
-
-  case class UpdateTaskMultiplePartitionInfos(executionID: ExecutionAttemptID,
-                                              partitionInfos: Seq[(IntermediateDataSetID,
-                                                PartitionInfo)]) extends UpdateTask
-
-  def createUpdateTaskMultiplePartitionInfos(executionID: ExecutionAttemptID,
-                                             resultIDs: java.util.List[IntermediateDataSetID],
-                                             partitionInfos: java.util.List[PartitionInfo]):
+  case class UpdateTaskSinglePartitionInfo(
+    executionID: ExecutionAttemptID,
+    resultId: IntermediateDataSetID,
+    partitionInfo: InputChannelDeploymentDescriptor)
+    extends UpdateTask
+
+  case class UpdateTaskMultiplePartitionInfos(
+    executionID: ExecutionAttemptID,
+    partitionInfos: Seq[(IntermediateDataSetID, InputChannelDeploymentDescriptor)])
+    extends UpdateTask
+
+  def createUpdateTaskMultiplePartitionInfos(
+    executionID: ExecutionAttemptID,
+    resultIDs: java.util.List[IntermediateDataSetID],
+    partitionInfos: java.util.List[InputChannelDeploymentDescriptor]):
   UpdateTaskMultiplePartitionInfos = {
     require(resultIDs.size() == partitionInfos.size(), "ResultIDs must have the same length as" +
       "partitionInfos.")

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/NetworkEnvironmentConfiguration.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/NetworkEnvironmentConfiguration.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/NetworkEnvironmentConfiguration.scala
index aeda6c4..f99aac0 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/NetworkEnvironmentConfiguration.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/NetworkEnvironmentConfiguration.scala
@@ -18,8 +18,10 @@
 
 package org.apache.flink.runtime.taskmanager
 
+import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode
 import org.apache.flink.runtime.io.network.netty.NettyConfig
 
 case class NetworkEnvironmentConfiguration(numNetworkBuffers: Int,
                                            networkBufferSize: Int,
+                                           ioMode: IOMode,
                                            nettyConfig: Option[NettyConfig] = None)

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
index 53c45ce..61cab6b 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
@@ -33,13 +33,14 @@ import org.apache.flink.runtime.ActorLogMessages
 import org.apache.flink.runtime.akka.AkkaUtils
 import org.apache.flink.runtime.blob.BlobCache
 import org.apache.flink.runtime.broadcast.BroadcastVariableManager
-import org.apache.flink.runtime.deployment.{PartitionInfo, TaskDeploymentDescriptor}
+import org.apache.flink.runtime.deployment.{InputChannelDeploymentDescriptor, TaskDeploymentDescriptor}
 import org.apache.flink.runtime.execution.librarycache.{BlobLibraryCacheManager, FallbackLibraryCacheManager, LibraryCacheManager}
 import org.apache.flink.runtime.execution.{CancelTaskException, ExecutionState, RuntimeEnvironment}
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID
 import org.apache.flink.runtime.filecache.FileCache
 import org.apache.flink.runtime.instance.{HardwareDescription, InstanceConnectionInfo, InstanceID}
-import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync
+import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode
+import org.apache.flink.runtime.io.disk.iomanager.{IOManager, IOManagerAsync}
 import org.apache.flink.runtime.io.network.NetworkEnvironment
 import org.apache.flink.runtime.io.network.netty.NettyConfig
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID
@@ -261,7 +262,7 @@ class TaskManager(val connectionInfo: InstanceConnectionInfo,
       }
 
     case UnregisterTask(executionID) =>
-      unregisterTask(executionID)
+      unregisterTaskAndNotifyFinalState(executionID)
 
     case updateMsg:UpdateTaskExecutionState =>
       currentJobManager foreach {
@@ -348,7 +349,7 @@ class TaskManager(val connectionInfo: InstanceConnectionInfo,
     case FailIntermediateResultPartitions(executionID) =>
       log.info("Fail intermediate result partitions associated with execution {}.", executionID)
       networkEnvironment foreach {
-        _.getPartitionManager.failIntermediateResultPartitions(executionID)
+        _.getPartitionManager.releasePartitionsProducedBy(executionID)
       }
 
     case BarrierReq(attemptID, checkpointID) =>
@@ -549,8 +550,9 @@ class TaskManager(val connectionInfo: InstanceConnectionInfo,
     registrationDuration = 0 seconds
   }
 
-  private def updateTask(executionId: ExecutionAttemptID,
-                         partitionInfos: Seq[(IntermediateDataSetID, PartitionInfo)]): Unit = {
+  private def updateTask(
+    executionId: ExecutionAttemptID,
+    partitionInfos: Seq[(IntermediateDataSetID, InputChannelDeploymentDescriptor)]): Unit = {
 
     runningTasks.get(executionId) match {
       case Some(task) =>
@@ -685,17 +687,24 @@ class TaskManager(val connectionInfo: InstanceConnectionInfo,
 
       for (t <- runningTasks.values) {
         t.failExternally(cause)
-        unregisterTask(t.getExecutionId)
+        unregisterTaskAndNotifyFinalState(t.getExecutionId)
       }
     }
   }
 
-  private def unregisterTask(executionID: ExecutionAttemptID): Unit = {
+  private def unregisterTaskAndNotifyFinalState(executionID: ExecutionAttemptID): Unit = {
     runningTasks.remove(executionID) match {
       case Some(task) =>
         log.info("Unregister task with execution ID {}.", executionID)
         removeAllTaskResources(task)
         libraryCacheManager foreach { _.unregisterTask(task.getJobID, executionID) }
+
+        log.info("Updating FINAL execution state of {} ({}) to {}.", task.getTaskName,
+          task.getExecutionId, task.getExecutionState);
+
+        self ! UpdateTaskExecutionState(new TaskExecutionState(
+          task.getJobID, task.getExecutionId, task.getExecutionState, task.getFailureCause))
+
       case None =>
         if (log.isDebugEnabled) {
           log.debug("Cannot find task with ID {} to unregister.", executionID)
@@ -1194,7 +1203,19 @@ object TaskManager {
         connectionInfo.address(), connectionInfo.dataPort(), pageSize, configuration))
     }
 
-    val networkConfig = NetworkEnvironmentConfiguration(numNetworkBuffers, pageSize, nettyConfig)
+    // Default spill I/O mode for intermediate results
+    val syncOrAsync = configuration.getString(ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE,
+      ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE)
+
+    val ioMode : IOMode = if (syncOrAsync == "async") {
+      IOMode.ASYNC
+    }
+    else {
+      IOMode.SYNC
+    }
+
+    val networkConfig = NetworkEnvironmentConfiguration(numNetworkBuffers, pageSize, ioMode,
+      nettyConfig)
 
     val networkBufferMem = numNetworkBuffers * pageSize
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java
index 1a274d0..5742fea 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java
@@ -48,8 +48,8 @@ public class TaskDeploymentDescriptorTest {
 			final Configuration jobConfiguration = new Configuration();
 			final Configuration taskConfiguration = new Configuration();
 			final Class<? extends AbstractInvokable> invokableClass = RegularPactTask.class;
-			final List<PartitionDeploymentDescriptor> producedResults = new ArrayList<PartitionDeploymentDescriptor>(0);
-			final List<PartitionConsumerDeploymentDescriptor> inputGates = new ArrayList<PartitionConsumerDeploymentDescriptor>(0);
+			final List<ResultPartitionDeploymentDescriptor> producedResults = new ArrayList<ResultPartitionDeploymentDescriptor>(0);
+			final List<InputGateDeploymentDescriptor> inputGates = new ArrayList<InputGateDeploymentDescriptor>(0);
 			final List<BlobKey> requiredJars = new ArrayList<BlobKey>(0);
 	
 			final TaskDeploymentDescriptor orig = new TaskDeploymentDescriptor(jobID, vertexID, execId, taskName,
@@ -70,7 +70,7 @@ public class TaskDeploymentDescriptorTest {
 			assertEquals(orig.getIndexInSubtaskGroup(), copy.getIndexInSubtaskGroup());
 			assertEquals(orig.getNumberOfSubtasks(), copy.getNumberOfSubtasks());
 			assertEquals(orig.getProducedPartitions(), copy.getProducedPartitions());
-			assertEquals(orig.getConsumedPartitions(), copy.getConsumedPartitions());
+			assertEquals(orig.getInputGates(), copy.getInputGates());
 
 			assertEquals(orig.getRequiredJarFiles(), copy.getRequiredJarFiles());
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
index 526ba7f..c979c42 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
@@ -38,8 +38,8 @@ import akka.testkit.JavaTestKit;
 import akka.testkit.TestActorRef;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.deployment.PartitionConsumerDeploymentDescriptor;
-import org.apache.flink.runtime.deployment.PartitionDeploymentDescriptor;
+import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor;
+import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor;
 import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
 import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.instance.Instance;
@@ -138,15 +138,15 @@ public class ExecutionGraphDeploymentTest {
 			assertEquals(RegularPactTask.class.getName(), descr.getInvokableClassName());
 			assertEquals("v2", descr.getTaskName());
 
-			List<PartitionDeploymentDescriptor> producedPartitions = descr.getProducedPartitions();
-			List<PartitionConsumerDeploymentDescriptor> consumedPartitions = descr.getConsumedPartitions();
+			List<ResultPartitionDeploymentDescriptor> producedPartitions = descr.getProducedPartitions();
+			List<InputGateDeploymentDescriptor> consumedPartitions = descr.getInputGates();
 
 			assertEquals(2, producedPartitions.size());
 			assertEquals(1, consumedPartitions.size());
 
-			assertEquals(10, producedPartitions.get(0).getNumberOfQueues());
-			assertEquals(10, producedPartitions.get(1).getNumberOfQueues());
-			assertEquals(10, consumedPartitions.get(0).getPartitions().length);
+			assertEquals(10, producedPartitions.get(0).getNumberOfSubpartitions());
+			assertEquals(10, producedPartitions.get(1).getNumberOfSubpartitions());
+			assertEquals(10, consumedPartitions.get(0).getInputChannelDeploymentDescriptors().length);
 		}
 		catch (Exception e) {
 			e.printStackTrace();

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java
index c05fcca..0462b3f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java
@@ -107,7 +107,7 @@ public class ChannelViewsTest
 		
 		// create the writer output view
 		List<MemorySegment> memory = this.memoryManager.allocatePages(this.parentTask, NUM_MEMORY_SEGMENTS);
-		final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channel);
+		final BlockChannelWriter<MemorySegment> writer = this.ioManager.createBlockChannelWriter(channel);
 		final ChannelWriterOutputView outView = new ChannelWriterOutputView(writer, memory, MEMORY_PAGE_SIZE);
 		
 		// write a number of pairs
@@ -120,7 +120,7 @@ public class ChannelViewsTest
 		
 		// create the reader input view
 		memory = this.memoryManager.allocatePages(this.parentTask, NUM_MEMORY_SEGMENTS);
-		final BlockChannelReader reader = this.ioManager.createBlockChannelReader(channel);
+		final BlockChannelReader<MemorySegment> reader = this.ioManager.createBlockChannelReader(channel);
 		final ChannelReaderInputView inView = new ChannelReaderInputView(reader, memory, outView.getBlockCount(), true);
 		generator.reset();
 		
@@ -151,7 +151,7 @@ public class ChannelViewsTest
 		
 		// create the writer output view
 		List<MemorySegment> memory = this.memoryManager.allocatePages(this.parentTask, NUM_MEMORY_SEGMENTS);
-		final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channel);
+		final BlockChannelWriter<MemorySegment> writer = this.ioManager.createBlockChannelWriter(channel);
 		final ChannelWriterOutputView outView = new ChannelWriterOutputView(writer, memory, MEMORY_PAGE_SIZE);
 		
 		// write a number of pairs
@@ -164,7 +164,7 @@ public class ChannelViewsTest
 		
 		// create the reader input view
 		memory = this.memoryManager.allocatePages(this.parentTask, NUM_MEMORY_SEGMENTS);
-		final BlockChannelReader reader = this.ioManager.createBlockChannelReader(channel);
+		final BlockChannelReader<MemorySegment> reader = this.ioManager.createBlockChannelReader(channel);
 		final ChannelReaderInputView inView = new ChannelReaderInputView(reader, memory, outView.getBlockCount(), true);
 		generator.reset();
 		
@@ -192,7 +192,7 @@ public class ChannelViewsTest
 		
 		// create the writer output view
 		List<MemorySegment> memory = this.memoryManager.allocatePages(this.parentTask, NUM_MEMORY_SEGMENTS);
-		final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channel);
+		final BlockChannelWriter<MemorySegment> writer = this.ioManager.createBlockChannelWriter(channel);
 		final ChannelWriterOutputView outView = new ChannelWriterOutputView(writer, memory, MEMORY_PAGE_SIZE);
 
 		// write a number of pairs
@@ -205,7 +205,7 @@ public class ChannelViewsTest
 
 		// create the reader input view
 		memory = this.memoryManager.allocatePages(this.parentTask, NUM_MEMORY_SEGMENTS);
-		final BlockChannelReader reader = this.ioManager.createBlockChannelReader(channel);
+		final BlockChannelReader<MemorySegment> reader = this.ioManager.createBlockChannelReader(channel);
 		final ChannelReaderInputView inView = new ChannelReaderInputView(reader, memory, outView.getBlockCount(), true);
 		generator.reset();
 
@@ -243,7 +243,7 @@ public class ChannelViewsTest
 		
 		// create the writer output view
 		List<MemorySegment> memory = this.memoryManager.allocatePages(this.parentTask, NUM_MEMORY_SEGMENTS);
-		final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channel);
+		final BlockChannelWriter<MemorySegment> writer = this.ioManager.createBlockChannelWriter(channel);
 		final ChannelWriterOutputView outView = new ChannelWriterOutputView(writer, memory, MEMORY_PAGE_SIZE);
 		
 		// write a number of pairs
@@ -256,7 +256,7 @@ public class ChannelViewsTest
 		
 		// create the reader input view
 		memory = this.memoryManager.allocatePages(this.parentTask, NUM_MEMORY_SEGMENTS);
-		final BlockChannelReader reader = this.ioManager.createBlockChannelReader(channel);
+		final BlockChannelReader<MemorySegment> reader = this.ioManager.createBlockChannelReader(channel);
 		final ChannelReaderInputView inView = new ChannelReaderInputView(reader, memory, true);
 		generator.reset();
 		
@@ -287,7 +287,7 @@ public class ChannelViewsTest
 		
 		// create the writer output view
 		List<MemorySegment> memory = this.memoryManager.allocatePages(this.parentTask, 1);
-		final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channel);
+		final BlockChannelWriter<MemorySegment> writer = this.ioManager.createBlockChannelWriter(channel);
 		final ChannelWriterOutputView outView = new ChannelWriterOutputView(writer, memory, MEMORY_PAGE_SIZE);
 		
 		// write a number of pairs
@@ -300,7 +300,7 @@ public class ChannelViewsTest
 		
 		// create the reader input view
 		memory = this.memoryManager.allocatePages(this.parentTask, 1);
-		final BlockChannelReader reader = this.ioManager.createBlockChannelReader(channel);
+		final BlockChannelReader<MemorySegment> reader = this.ioManager.createBlockChannelReader(channel);
 		final ChannelReaderInputView inView = new ChannelReaderInputView(reader, memory, outView.getBlockCount(), true);
 		generator.reset();
 		
@@ -331,7 +331,7 @@ public class ChannelViewsTest
 		
 		// create the writer output view
 		List<MemorySegment> memory = this.memoryManager.allocatePages(this.parentTask, NUM_MEMORY_SEGMENTS);
-		final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channel);
+		final BlockChannelWriter<MemorySegment> writer = this.ioManager.createBlockChannelWriter(channel);
 		final ChannelWriterOutputView outView = new ChannelWriterOutputView(writer, memory, MEMORY_PAGE_SIZE);
 		
 		// write a number of pairs
@@ -344,7 +344,7 @@ public class ChannelViewsTest
 		
 		// create the reader input view
 		memory = this.memoryManager.allocatePages(this.parentTask, NUM_MEMORY_SEGMENTS);
-		final BlockChannelReader reader = this.ioManager.createBlockChannelReader(channel);
+		final BlockChannelReader<MemorySegment> reader = this.ioManager.createBlockChannelReader(channel);
 		final ChannelReaderInputView inView = new ChannelReaderInputView(reader, memory, outView.getBlockCount(), true);
 		generator.reset();
 		

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java
index 27928a9..85d2113 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java
@@ -88,7 +88,7 @@ public class FileChannelStreamsITCase {
 			final FileIOChannel.ID channel = ioManager.createChannel();
 			
 			// create the writer output view
-			final BlockChannelWriter writer = ioManager.createBlockChannelWriter(channel);
+			final BlockChannelWriter<MemorySegment> writer = ioManager.createBlockChannelWriter(channel);
 			final FileChannelOutputView outView = new FileChannelOutputView(writer, memManager, memory, MEMORY_PAGE_SIZE);
 			
 			// write a number of pairs
@@ -102,7 +102,7 @@ public class FileChannelStreamsITCase {
 			// create the reader input view
 			List<MemorySegment> readMemory = memManager.allocatePages(new DummyInvokable(), NUM_MEMORY_SEGMENTS);
 			
-			final BlockChannelReader reader = ioManager.createBlockChannelReader(channel);
+			final BlockChannelReader<MemorySegment> reader = ioManager.createBlockChannelReader(channel);
 			final FileChannelInputView inView = new FileChannelInputView(reader, memManager, readMemory, outView.getBytesInLatestSegment());
 			generator.reset();
 			
@@ -132,7 +132,7 @@ public class FileChannelStreamsITCase {
 			final FileIOChannel.ID channel = this.ioManager.createChannel();
 			
 			// create the writer output view
-			final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channel);
+			final BlockChannelWriter<MemorySegment> writer = this.ioManager.createBlockChannelWriter(channel);
 			final FileChannelOutputView outView = new FileChannelOutputView(writer, memManager, memory, MEMORY_PAGE_SIZE);
 			
 			// write a number of pairs
@@ -146,7 +146,7 @@ public class FileChannelStreamsITCase {
 			// create the reader input view
 			List<MemorySegment> readMemory = memManager.allocatePages(new DummyInvokable(), NUM_MEMORY_SEGMENTS);
 			
-			final BlockChannelReader reader = ioManager.createBlockChannelReader(channel);
+			final BlockChannelReader<MemorySegment> reader = ioManager.createBlockChannelReader(channel);
 			final FileChannelInputView inView = new FileChannelInputView(reader, memManager, readMemory, outView.getBytesInLatestSegment());
 			generator.reset();
 			
@@ -176,7 +176,7 @@ public class FileChannelStreamsITCase {
 			final FileIOChannel.ID channel = this.ioManager.createChannel();
 			
 			// create the writer output view
-			final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channel);
+			final BlockChannelWriter<MemorySegment> writer = this.ioManager.createBlockChannelWriter(channel);
 			final FileChannelOutputView outView = new FileChannelOutputView(writer, memManager, memory, MEMORY_PAGE_SIZE);
 	
 			// write a number of pairs
@@ -190,7 +190,7 @@ public class FileChannelStreamsITCase {
 			// create the reader input view
 			List<MemorySegment> readMemory = memManager.allocatePages(new DummyInvokable(), NUM_MEMORY_SEGMENTS);
 			
-			final BlockChannelReader reader = ioManager.createBlockChannelReader(channel);
+			final BlockChannelReader<MemorySegment> reader = ioManager.createBlockChannelReader(channel);
 			final FileChannelInputView inView = new FileChannelInputView(reader, memManager, readMemory, outView.getBytesInLatestSegment());
 			generator.reset();
 	
@@ -226,7 +226,7 @@ public class FileChannelStreamsITCase {
 			final FileIOChannel.ID channel = this.ioManager.createChannel();
 			
 			// create the writer output view
-			final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channel);
+			final BlockChannelWriter<MemorySegment> writer = this.ioManager.createBlockChannelWriter(channel);
 			final FileChannelOutputView outView = new FileChannelOutputView(writer, memManager, memory, MEMORY_PAGE_SIZE);
 			
 			// write a number of pairs
@@ -240,7 +240,7 @@ public class FileChannelStreamsITCase {
 			// create the reader input view
 			List<MemorySegment> readMemory = memManager.allocatePages(new DummyInvokable(), 1);
 			
-			final BlockChannelReader reader = ioManager.createBlockChannelReader(channel);
+			final BlockChannelReader<MemorySegment> reader = ioManager.createBlockChannelReader(channel);
 			final FileChannelInputView inView = new FileChannelInputView(reader, memManager, readMemory, outView.getBytesInLatestSegment());
 			generator.reset();
 			
@@ -270,7 +270,7 @@ public class FileChannelStreamsITCase {
 			final FileIOChannel.ID channel = this.ioManager.createChannel();
 			
 			// create the writer output view
-			final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channel);
+			final BlockChannelWriter<MemorySegment> writer = this.ioManager.createBlockChannelWriter(channel);
 			final FileChannelOutputView outView = new FileChannelOutputView(writer, memManager, memory, MEMORY_PAGE_SIZE);
 			
 			// write a number of pairs
@@ -284,7 +284,7 @@ public class FileChannelStreamsITCase {
 			// create the reader input view
 			List<MemorySegment> readMemory = memManager.allocatePages(new DummyInvokable(), NUM_MEMORY_SEGMENTS);
 			
-			final BlockChannelReader reader = ioManager.createBlockChannelReader(channel);
+			final BlockChannelReader<MemorySegment> reader = ioManager.createBlockChannelReader(channel);
 			final FileChannelInputView inView = new FileChannelInputView(reader, memManager, readMemory, outView.getBytesInLatestSegment());
 			generator.reset();
 			

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java
index 1db2a6f..1f6899d 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java
@@ -49,7 +49,7 @@ public class FileChannelStreamsTest {
 			memMan.allocatePages(new DummyInvokable(), memory, 4);
 			
 			FileIOChannel.ID channel = ioManager.createChannel();
-			BlockChannelWriter writer = ioManager.createBlockChannelWriter(channel);
+			BlockChannelWriter<MemorySegment> writer = ioManager.createBlockChannelWriter(channel);
 			
 			FileChannelOutputView out = new FileChannelOutputView(writer, memMan, memory, memMan.getPageSize());
 			new StringValue("Some test text").write(out);
@@ -91,7 +91,7 @@ public class FileChannelStreamsTest {
 				wrt.close();
 			}
 			
-			BlockChannelReader reader = ioManager.createBlockChannelReader(channel);
+			BlockChannelReader<MemorySegment> reader = ioManager.createBlockChannelReader(channel);
 			FileChannelInputView in = new FileChannelInputView(reader, memMan, memory, 9);
 			
 			// read just something

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java
index 7e4d70d..f090ef1 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java
@@ -50,7 +50,7 @@ public class SeekableFileChannelInputViewTest {
 			memMan.allocatePages(new DummyInvokable(), memory, 4);
 			
 			FileIOChannel.ID channel = ioManager.createChannel();
-			BlockChannelWriter writer = ioManager.createBlockChannelWriter(channel);
+			BlockChannelWriter<MemorySegment> writer = ioManager.createBlockChannelWriter(channel);
 			FileChannelOutputView out = new FileChannelOutputView(writer, memMan, memory, memMan.getPageSize());
 			
 			// write some integers across 7.5 pages (7 pages = 114.688 bytes, 8 pages = 131.072 bytes)

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriterTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriterTest.java
new file mode 100644
index 0000000..0397de5
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriterTest.java
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.disk.iomanager;
+
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.util.TestNotificationListener;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+
+public class AsynchronousBufferFileWriterTest {
+
+	private static final IOManager ioManager = new IOManagerAsync();
+
+	private static final Buffer mockBuffer = mock(Buffer.class);
+
+	private AsynchronousBufferFileWriter writer;
+
+	@Before
+	public void setUp() throws IOException {
+		writer = new AsynchronousBufferFileWriter(ioManager.createChannel(), new RequestQueue<WriteRequest>());
+	}
+
+	@Test
+	public void testAddAndHandleRequest() throws Exception {
+		addRequest();
+		assertEquals("Didn't increment number of outstanding requests.", 1, writer.getNumberOfOutstandingRequests());
+
+		handleRequest();
+		assertEquals("Didn't decrement number of outstanding requests.", 0, writer.getNumberOfOutstandingRequests());
+	}
+
+	@Test
+	public void testSubscribe() throws Exception {
+		final TestNotificationListener listener = new TestNotificationListener();
+
+		// Unsuccessful subscription, because no outstanding requests
+		assertFalse("Allowed to subscribe w/o any outstanding requests.", writer.registerAllRequestsProcessedListener(listener));
+
+		// Successful subscription
+		addRequest();
+		assertTrue("Didn't allow to subscribe.", writer.registerAllRequestsProcessedListener(listener));
+
+		// Test notification
+		handleRequest();
+
+		assertEquals("Listener was not notified.", 1, listener.getNumberOfNotifications());
+	}
+
+	@Test
+	public void testSubscribeAndClose() throws IOException, InterruptedException {
+		final TestNotificationListener listener = new TestNotificationListener();
+
+		final AtomicReference<Throwable> error = new AtomicReference<Throwable>();
+
+		final CountDownLatch sync = new CountDownLatch(1);
+
+		addRequest();
+		addRequest();
+
+		writer.registerAllRequestsProcessedListener(listener);
+
+		final Thread asyncCloseThread = new Thread(new Runnable() {
+			@Override
+			public void run() {
+				try {
+					writer.close();
+				}
+				catch (Throwable t) {
+					error.set(t);
+				}
+				finally {
+					sync.countDown();
+				}
+			}
+		});
+
+		asyncCloseThread.start();
+
+		handleRequest();
+		handleRequest();
+
+		sync.await();
+
+		assertEquals("Listener was not notified.", 1, listener.getNumberOfNotifications());
+	}
+
+	@Test
+	public void testConcurrentSubscribeAndHandleRequest() throws Exception {
+		final ExecutorService executor = Executors.newFixedThreadPool(2);
+
+		final TestNotificationListener listener = new TestNotificationListener();
+
+		final Callable<Boolean> subscriber = new Callable<Boolean>() {
+			@Override
+			public Boolean call() throws Exception {
+				return writer.registerAllRequestsProcessedListener(listener);
+			}
+		};
+
+		final Callable<Void> requestHandler = new Callable<Void>() {
+			@Override
+			public Void call() throws Exception {
+				handleRequest();
+				return null;
+			}
+		};
+
+		try {
+			// Repeat this to provoke races
+			for (int i = 0; i < 50000; i++) {
+				listener.reset();
+
+				addRequest();
+
+				Future<Void> handleRequestFuture = executor.submit(requestHandler);
+				Future<Boolean> subscribeFuture = executor.submit(subscriber);
+
+				handleRequestFuture.get();
+
+				try {
+					if (subscribeFuture.get()) {
+						assertEquals("Race: Successfully subscribed, but was never notified.", 1, listener.getNumberOfNotifications());
+					}
+					else {
+						assertEquals("Race: Never subscribed successfully, but was notified.", 0, listener.getNumberOfNotifications());
+					}
+				}
+				catch (Throwable t) {
+					System.out.println(i);
+					Assert.fail(t.getMessage());
+				}
+			}
+		}
+		finally {
+			executor.shutdownNow();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	private void addRequest() throws IOException {
+		writer.writeBlock(mockBuffer);
+	}
+
+	private void handleRequest() {
+		writer.handleProcessedBuffer(mockBuffer, null);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelTest.java
new file mode 100644
index 0000000..49e93c6
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelTest.java
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.disk.iomanager;
+
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.util.TestNotificationListener;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.modules.junit4.PowerMockRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+
+@RunWith(PowerMockRunner.class)
+public class AsynchronousFileIOChannelTest {
+
+	private static final Logger LOG = LoggerFactory.getLogger(AsynchronousFileIOChannelTest.class);
+
+	@Test
+	public void testAllRequestsProcessedListenerNotification() throws Exception {
+		// -- Config ----------------------------------------------------------
+		final int numberOfRuns = 10;
+		final int numberOfRequests = 100;
+
+		// -- Setup -----------------------------------------------------------
+		final IOManagerAsync ioManager = new IOManagerAsync();
+
+		final ExecutorService executor = Executors.newFixedThreadPool(3);
+
+		final Random random = new Random();
+
+		final RequestQueue<WriteRequest> requestQueue = new RequestQueue<WriteRequest>();
+
+		final RequestDoneCallback<Buffer> ioChannelCallback = mock(RequestDoneCallback.class);
+
+		final TestNotificationListener listener = new TestNotificationListener();
+
+		// -- The Test --------------------------------------------------------
+		try {
+			// Repeatedly add requests and process them and have one thread try to register as a
+			// listener until the channel is closed and all requests are processed.
+
+			for (int run = 0; run < numberOfRuns; run++) {
+				final TestAsyncFileIOChannel ioChannel = new TestAsyncFileIOChannel(
+						ioManager.createChannel(), requestQueue, ioChannelCallback, true);
+
+				final CountDownLatch sync = new CountDownLatch(3);
+
+				// The mock requests
+				final Buffer buffer = mock(Buffer.class);
+				final WriteRequest request = mock(WriteRequest.class);
+
+				// Add requests task
+				Callable<Void> addRequestsTask = new Callable<Void>() {
+					@Override
+					public Void call() throws Exception {
+						for (int i = 0; i < numberOfRuns; i++) {
+							LOG.debug("Starting run {}.", i + 1);
+
+							for (int j = 0; j < numberOfRequests; j++) {
+								ioChannel.addRequest(request);
+							}
+
+							LOG.debug("Added all ({}) requests of run {}.", numberOfRequests, i + 1);
+
+							int sleep = random.nextInt(10);
+							LOG.debug("Sleeping for {} ms before next run.", sleep);
+
+							Thread.sleep(sleep);
+						}
+
+						LOG.debug("Done. Closing channel.");
+						ioChannel.close();
+
+						sync.countDown();
+
+						return null;
+					}
+				};
+
+				// Process requests task
+				Callable<Void> processRequestsTask = new Callable<Void>() {
+					@Override
+					public Void call() throws Exception {
+						int total = numberOfRequests * numberOfRuns;
+						for (int i = 0; i < total; i++) {
+							requestQueue.take();
+
+							ioChannel.handleProcessedBuffer(buffer, null);
+						}
+
+						LOG.debug("Processed all ({}) requests.", numberOfRequests);
+
+						sync.countDown();
+
+						return null;
+					}
+				};
+
+				// Listener
+				Callable<Void> registerListenerTask = new Callable<Void>() {
+					@Override
+					public Void call() throws Exception {
+						while (true) {
+							int current = listener.getNumberOfNotifications();
+
+							if (ioChannel.registerAllRequestsProcessedListener(listener)) {
+								listener.waitForNotification(current);
+							}
+							else if (ioChannel.isClosed()) {
+								break;
+							}
+						}
+
+						LOG.debug("Stopping listener. Channel closed.");
+
+						sync.countDown();
+
+						return null;
+					}
+				};
+
+				// Run tasks in random order
+				final List<Callable<?>> tasks = new LinkedList<Callable<?>>();
+				tasks.add(addRequestsTask);
+				tasks.add(processRequestsTask);
+				tasks.add(registerListenerTask);
+
+				Collections.shuffle(tasks);
+
+				for (Callable<?> task : tasks) {
+					executor.submit(task);
+				}
+
+				if (!sync.await(2, TimeUnit.MINUTES)) {
+					fail("Test failed due to a timeout. This indicates a deadlock due to the way" +
+							"that listeners are registered/notified in the asynchronous file I/O" +
+							"channel.");
+				}
+
+				listener.reset();
+			}
+		}
+		finally {
+			ioManager.shutdown();
+			executor.shutdown();
+		}
+	}
+
+	@Test
+	public void testClosedButAddRequestAndRegisterListenerRace() throws Exception {
+		// -- Config ----------------------------------------------------------
+		final int numberOfRuns = 1024;
+
+		// -- Setup -----------------------------------------------------------
+		final IOManagerAsync ioManager = new IOManagerAsync();
+
+		final ExecutorService executor = Executors.newFixedThreadPool(2);
+
+		final RequestQueue<WriteRequest> requestQueue = new RequestQueue<WriteRequest>();
+
+		@SuppressWarnings("unchecked")
+		final RequestDoneCallback<Buffer> ioChannelCallback = mock(RequestDoneCallback.class);
+
+		final TestNotificationListener listener = new TestNotificationListener();
+
+		// -- The Test --------------------------------------------------------
+		try {
+			// Repeatedly close the channel and add a request.
+			for (int i = 0; i < numberOfRuns; i++) {
+				final TestAsyncFileIOChannel ioChannel = new TestAsyncFileIOChannel(
+						ioManager.createChannel(), requestQueue, ioChannelCallback, true);
+
+				final CountDownLatch sync = new CountDownLatch(2);
+
+				final WriteRequest request = mock(WriteRequest.class);
+
+				ioChannel.close();
+
+				// Add request task
+				Callable<Void> addRequestTask = new Callable<Void>() {
+					@Override
+					public Void call() throws Exception {
+						try {
+							ioChannel.addRequest(request);
+						}
+						catch (Throwable expected) {
+						}
+						finally {
+							sync.countDown();
+						}
+
+						return null;
+					}
+				};
+
+				// Listener
+				Callable<Void> registerListenerTask = new Callable<Void>() {
+					@Override
+					public Void call() throws Exception {
+						try {
+							while (true) {
+								int current = listener.getNumberOfNotifications();
+
+								if (ioChannel.registerAllRequestsProcessedListener(listener)) {
+									listener.waitForNotification(current);
+								}
+								else if (ioChannel.isClosed()) {
+									break;
+								}
+							}
+						}
+						finally {
+							sync.countDown();
+						}
+
+						return null;
+					}
+				};
+
+				executor.submit(addRequestTask);
+				executor.submit(registerListenerTask);
+
+				if (!sync.await(2, TimeUnit.MINUTES)) {
+					fail("Test failed due to a timeout. This indicates a deadlock due to the way" +
+							"that listeners are registered/notified in the asynchronous file I/O" +
+							"channel.");
+				}
+			}
+		}
+		finally {
+			ioManager.shutdown();
+			executor.shutdown();
+		}
+	}
+
+	@Test
+	public void testClosingWaits() {
+		IOManagerAsync ioMan = new IOManagerAsync();
+		try {
+
+			final int NUM_BLOCKS = 100;
+			final MemorySegment seg = new MemorySegment(new byte[32 * 1024]);
+
+			final AtomicInteger callbackCounter = new AtomicInteger();
+			final AtomicBoolean exceptionOccurred = new AtomicBoolean();
+
+			final RequestDoneCallback<MemorySegment> callback = new RequestDoneCallback<MemorySegment>() {
+
+				@Override
+				public void requestSuccessful(MemorySegment buffer) {
+					// we do the non safe variant. the callbacks should come in order from
+					// the same thread, so it should always work
+					callbackCounter.set(callbackCounter.get() + 1);
+
+					if (buffer != seg) {
+						exceptionOccurred.set(true);
+					}
+				}
+
+				@Override
+				public void requestFailed(MemorySegment buffer, IOException e) {
+					exceptionOccurred.set(true);
+				}
+			};
+
+			BlockChannelWriterWithCallback<MemorySegment> writer = ioMan.createBlockChannelWriter(ioMan.createChannel(), callback);
+			try {
+				for (int i = 0; i < NUM_BLOCKS; i++) {
+					writer.writeBlock(seg);
+				}
+
+				writer.close();
+
+				assertEquals(NUM_BLOCKS, callbackCounter.get());
+				assertFalse(exceptionOccurred.get());
+			}
+			finally {
+				writer.closeAndDelete();
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+		finally {
+			ioMan.shutdown();
+		}
+	}
+
+	@Test
+	public void testExceptionForwardsToClose() {
+		IOManagerAsync ioMan = new IOManagerAsync();
+		try {
+			testExceptionForwardsToClose(ioMan, 100, 1);
+			testExceptionForwardsToClose(ioMan, 100, 50);
+			testExceptionForwardsToClose(ioMan, 100, 100);
+		} finally {
+			ioMan.shutdown();
+		}
+	}
+
+	private void testExceptionForwardsToClose(IOManagerAsync ioMan, final int numBlocks, final int failingBlock) {
+		try {
+			MemorySegment seg = new MemorySegment(new byte[32 * 1024]);
+			FileIOChannel.ID channelId = ioMan.createChannel();
+
+			BlockChannelWriterWithCallback<MemorySegment> writer = new AsynchronousBlockWriterWithCallback(channelId,
+					ioMan.getWriteRequestQueue(channelId), new NoOpCallback()) {
+
+				private int numBlocks;
+
+				@Override
+				public void writeBlock(MemorySegment segment) throws IOException {
+					numBlocks++;
+
+					if (numBlocks == failingBlock) {
+						this.requestsNotReturned.incrementAndGet();
+						this.requestQueue.add(new FailingWriteRequest(this, segment));
+					} else {
+						super.writeBlock(segment);
+					}
+				}
+			};
+
+			try {
+				for (int i = 0; i < numBlocks; i++) {
+					writer.writeBlock(seg);
+				}
+
+				writer.close();
+				fail("did not forward exception");
+			}
+			catch (IOException e) {
+				// expected
+			}
+			finally {
+				try {
+					writer.closeAndDelete();
+				} catch (Throwable t) {}
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	private static class NoOpCallback implements RequestDoneCallback<MemorySegment> {
+
+		@Override
+		public void requestSuccessful(MemorySegment buffer) {}
+
+		@Override
+		public void requestFailed(MemorySegment buffer, IOException e) {}
+	}
+
+	private static class FailingWriteRequest implements WriteRequest {
+
+		private final AsynchronousFileIOChannel<MemorySegment, WriteRequest> channel;
+
+		private final MemorySegment segment;
+
+		protected FailingWriteRequest(AsynchronousFileIOChannel<MemorySegment, WriteRequest> targetChannel, MemorySegment segment) {
+			this.channel = targetChannel;
+			this.segment = segment;
+		}
+
+		@Override
+		public void write() throws IOException {
+			throw new IOException();
+		}
+
+		@Override
+		public void requestDone(IOException ioex) {
+			this.channel.handleProcessedBuffer(this.segment, ioex);
+		}
+	}
+
+	private static class TestAsyncFileIOChannel extends AsynchronousFileIOChannel<Buffer, WriteRequest> {
+
+		protected TestAsyncFileIOChannel(
+				ID channelID,
+				RequestQueue<WriteRequest> requestQueue,
+				RequestDoneCallback<Buffer> callback,
+				boolean writeEnabled) throws IOException {
+
+			super(channelID, requestQueue, callback, writeEnabled);
+		}
+
+		int getNumberOfOutstandingRequests() {
+			return requestsNotReturned.get();
+		}
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelsTest.java
deleted file mode 100644
index 0ed6233..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelsTest.java
+++ /dev/null
@@ -1,176 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.io.disk.iomanager;
-
-import org.apache.flink.core.memory.MemorySegment;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.fail;
-
-public class AsynchronousFileIOChannelsTest {
-
-	@Test
-	public void testClosingWaits() {
-		IOManagerAsync ioMan = new IOManagerAsync();
-		try {
-			
-			final int NUM_BLOCKS = 100;
-			final MemorySegment seg = new MemorySegment(new byte[32 * 1024]);
-			
-			final AtomicInteger callbackCounter = new AtomicInteger();
-			final AtomicBoolean exceptionOccurred = new AtomicBoolean();
-			
-			final RequestDoneCallback<MemorySegment> callback = new RequestDoneCallback<MemorySegment>() {
-				
-				@Override
-				public void requestSuccessful(MemorySegment buffer) {
-					// we do the non safe variant. the callbacks should come in order from
-					// the same thread, so it should always work
-					callbackCounter.set(callbackCounter.get() + 1);
-					
-					if (buffer != seg) {
-						exceptionOccurred.set(true);
-					}
-				}
-				
-				@Override
-				public void requestFailed(MemorySegment buffer, IOException e) {
-					exceptionOccurred.set(true);
-				}
-			};
-			
-			BlockChannelWriterWithCallback writer = ioMan.createBlockChannelWriter(ioMan.createChannel(), callback);
-			try {
-				for (int i = 0; i < NUM_BLOCKS; i++) {
-					writer.writeBlock(seg);
-				}
-				
-				writer.close();
-				
-				assertEquals(NUM_BLOCKS, callbackCounter.get());
-				assertFalse(exceptionOccurred.get());
-			}
-			finally {
-				writer.closeAndDelete();
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-		finally {
-			ioMan.shutdown();
-		}
-	}
-	
-	@Test
-	public void testExceptionForwardsToClose() {
-		IOManagerAsync ioMan = new IOManagerAsync();
-		try {
-			testExceptionForwardsToClose(ioMan, 100, 1);
-			testExceptionForwardsToClose(ioMan, 100, 50);
-			testExceptionForwardsToClose(ioMan, 100, 100);
-		} finally {
-			ioMan.shutdown();
-		}
-	}
-	
-	private void testExceptionForwardsToClose(IOManagerAsync ioMan, final int numBlocks, final int failingBlock) {
-		try {
-			MemorySegment seg = new MemorySegment(new byte[32 * 1024]);
-			FileIOChannel.ID channelId = ioMan.createChannel();
-			
-			BlockChannelWriterWithCallback writer = new AsynchronousBlockWriterWithCallback(channelId, 
-					ioMan.getWriteRequestQueue(channelId), new NoOpCallback()) {
-				
-				private int numBlocks;
-				
-				@Override
-				public void writeBlock(MemorySegment segment) throws IOException {
-					numBlocks++;
-					
-					if (numBlocks == failingBlock) {
-						this.requestsNotReturned.incrementAndGet();
-						this.requestQueue.add(new FailingWriteRequest(this, segment));
-					} else {
-						super.writeBlock(segment);
-					}
-				}
-			};
-			
-			try {
-				for (int i = 0; i < numBlocks; i++) {
-					writer.writeBlock(seg);
-				}
-				
-				writer.close();
-				fail("did not forward exception");
-			}
-			catch (IOException e) {
-				// expected
-			}
-			finally {
-				try {
-					writer.closeAndDelete();
-				} catch (Throwable t) {}
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	private static class NoOpCallback implements RequestDoneCallback<MemorySegment> {
-
-		@Override
-		public void requestSuccessful(MemorySegment buffer) {}
-
-		@Override
-		public void requestFailed(MemorySegment buffer, IOException e) {}
-	}
-	
-	private static class FailingWriteRequest implements WriteRequest {
-		
-		private final AsynchronousFileIOChannel<MemorySegment, WriteRequest> channel;
-		
-		private final MemorySegment segment;
-		
-		protected FailingWriteRequest(AsynchronousFileIOChannel<MemorySegment, WriteRequest> targetChannel, MemorySegment segment) {
-			this.channel = targetChannel;
-			this.segment = segment;
-		}
-
-		@Override
-		public void write() throws IOException {
-			throw new IOException();
-		}
-
-		@Override
-		public void requestDone(IOException ioex) {
-			this.channel.handleProcessedBuffer(this.segment, ioex);
-		}
-	} 
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterFileSegmentReaderTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterFileSegmentReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterFileSegmentReaderTest.java
new file mode 100644
index 0000000..294a6e6
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterFileSegmentReaderTest.java
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.disk.iomanager;
+
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
+import org.apache.flink.runtime.testutils.DiscardingRecycler;
+import org.apache.flink.runtime.util.event.NotificationListener;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Random;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import static junit.framework.TestCase.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class BufferFileWriterFileSegmentReaderTest {
+
+	private static final int BUFFER_SIZE = 32 * 1024;
+
+	private static final BufferRecycler BUFFER_RECYCLER = new DiscardingRecycler();
+
+	private static final Random random = new Random();
+
+	private static final IOManager ioManager = new IOManagerAsync();
+
+	private BufferFileWriter writer;
+
+	private AsynchronousBufferFileSegmentReader reader;
+
+	private LinkedBlockingQueue<FileSegment> returnedFileSegments = new LinkedBlockingQueue<FileSegment>();
+
+	@Before
+	public void setUpWriterAndReader() {
+		final FileIOChannel.ID channel = ioManager.createChannel();
+
+		try {
+			writer = ioManager.createBufferFileWriter(channel);
+			reader = (AsynchronousBufferFileSegmentReader) ioManager.createBufferFileSegmentReader(channel, new QueuingCallback<FileSegment>(returnedFileSegments));
+		}
+		catch (IOException e) {
+			if (writer != null) {
+				writer.deleteChannel();
+			}
+
+			if (reader != null) {
+				reader.deleteChannel();
+			}
+
+			fail("Failed to setup writer and reader.");
+		}
+	}
+
+	@After
+	public void tearDownWriterAndReader() {
+		if (writer != null) {
+			writer.deleteChannel();
+		}
+
+		if (reader != null) {
+			reader.deleteChannel();
+		}
+
+		returnedFileSegments.clear();
+	}
+
+	@Test
+	public void testWriteRead() throws IOException, InterruptedException {
+		int numBuffers = 1024;
+		int currentNumber = 0;
+
+		final int minBufferSize = BUFFER_SIZE / 4;
+
+		// Write buffers filled with ascending numbers...
+		for (int i = 0; i < numBuffers; i++) {
+			final Buffer buffer = createBuffer();
+
+			int size = getNextMultipleOf(getRandomNumberInRange(minBufferSize, BUFFER_SIZE), 4);
+
+			buffer.setSize(size);
+
+			currentNumber = fillBufferWithAscendingNumbers(buffer, currentNumber);
+
+			writer.writeBlock(buffer);
+		}
+
+		// Make sure that the writes are finished
+		writer.close();
+
+		// Read buffers back in...
+		for (int i = 0; i < numBuffers; i++) {
+			assertFalse(reader.hasReachedEndOfFile());
+			reader.read();
+		}
+
+		// Wait for all requests to be finished
+		final CountDownLatch sync = new CountDownLatch(1);
+		final NotificationListener listener = new NotificationListener() {
+			@Override
+			public void onNotification() {
+				sync.countDown();
+			}
+		};
+
+		if (reader.registerAllRequestsProcessedListener(listener)) {
+			sync.await();
+		}
+
+		assertTrue(reader.hasReachedEndOfFile());
+
+		// Verify that the content is the same
+		assertEquals("Read less buffers than written.", numBuffers, returnedFileSegments.size());
+
+		currentNumber = 0;
+		FileSegment fileSegment;
+
+		ByteBuffer buffer = ByteBuffer.allocate(BUFFER_SIZE);
+
+		while ((fileSegment = returnedFileSegments.poll()) != null) {
+			buffer.position(0);
+			buffer.limit(fileSegment.getLength());
+
+			fileSegment.getFileChannel().read(buffer, fileSegment.getPosition());
+
+			currentNumber = verifyBufferFilledWithAscendingNumbers(new Buffer(new MemorySegment(buffer.array()), BUFFER_RECYCLER), currentNumber, fileSegment.getLength());
+		}
+
+		reader.close();
+	}
+
+	// ------------------------------------------------------------------------
+
+	private int getRandomNumberInRange(int min, int max) {
+		return random.nextInt((max - min) + 1) + min;
+	}
+
+	private int getNextMultipleOf(int number, int multiple) {
+		final int mod = number % multiple;
+
+		if (mod == 0) {
+			return number;
+		}
+
+		return number + multiple - mod;
+	}
+
+	private Buffer createBuffer() {
+		return new Buffer(new MemorySegment(new byte[BUFFER_SIZE]), BUFFER_RECYCLER);
+	}
+
+	public static int fillBufferWithAscendingNumbers(Buffer buffer, int currentNumber) {
+		MemorySegment segment = buffer.getMemorySegment();
+
+		final int size = buffer.getSize();
+
+		for (int i = 0; i < size; i += 4) {
+			segment.putInt(i, currentNumber++);
+		}
+
+		return currentNumber;
+	}
+
+	private int verifyBufferFilledWithAscendingNumbers(Buffer buffer, int currentNumber, int size) {
+		MemorySegment segment = buffer.getMemorySegment();
+
+		for (int i = 0; i < size; i += 4) {
+			if (segment.getInt(i) != currentNumber++) {
+				throw new IllegalStateException("Read unexpected number from buffer.");
+			}
+		}
+
+		return currentNumber;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterReaderTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterReaderTest.java
new file mode 100644
index 0000000..b0c702a
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterReaderTest.java
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.disk.iomanager;
+
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
+import org.apache.flink.runtime.testutils.DiscardingRecycler;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Random;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import static junit.framework.TestCase.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class BufferFileWriterReaderTest {
+
+	private static final int BUFFER_SIZE = 32 * 1024;
+
+	private static final BufferRecycler BUFFER_RECYCLER = new DiscardingRecycler();
+
+	private static final Random random = new Random();
+
+	private static final IOManager ioManager = new IOManagerAsync();
+
+	private BufferFileWriter writer;
+
+	private BufferFileReader reader;
+
+	private LinkedBlockingQueue<Buffer> returnedBuffers = new LinkedBlockingQueue<Buffer>();
+
+	@Before
+	public void setUpWriterAndReader() {
+		final FileIOChannel.ID channel = ioManager.createChannel();
+
+		try {
+			writer = ioManager.createBufferFileWriter(channel);
+			reader = ioManager.createBufferFileReader(channel, new QueuingCallback<Buffer>(returnedBuffers));
+		}
+		catch (IOException e) {
+			if (writer != null) {
+				writer.deleteChannel();
+			}
+
+			if (reader != null) {
+				reader.deleteChannel();
+			}
+
+			fail("Failed to setup writer and reader.");
+		}
+	}
+
+	@After
+	public void tearDownWriterAndReader() {
+		if (writer != null) {
+			writer.deleteChannel();
+		}
+
+		if (reader != null) {
+			reader.deleteChannel();
+		}
+
+		returnedBuffers.clear();
+	}
+
+	@Test
+	public void testWriteRead() throws IOException {
+		int numBuffers = 1024;
+		int currentNumber = 0;
+
+		final int minBufferSize = BUFFER_SIZE / 4;
+
+		// Write buffers filled with ascending numbers...
+		for (int i = 0; i < numBuffers; i++) {
+			final Buffer buffer = createBuffer();
+
+			int size = getNextMultipleOf(getRandomNumberInRange(minBufferSize, BUFFER_SIZE), 4);
+
+			buffer.setSize(size);
+
+			currentNumber = fillBufferWithAscendingNumbers(buffer, currentNumber);
+
+			writer.writeBlock(buffer);
+		}
+
+		// Make sure that the writes are finished
+		writer.close();
+
+		// Read buffers back in...
+		for (int i = 0; i < numBuffers; i++) {
+			assertFalse(reader.hasReachedEndOfFile());
+			reader.readInto(createBuffer());
+		}
+
+		reader.close();
+
+		assertTrue(reader.hasReachedEndOfFile());
+
+		// Verify that the content is the same
+		assertEquals("Read less buffers than written.", numBuffers, returnedBuffers.size());
+
+		currentNumber = 0;
+		Buffer buffer;
+
+		while ((buffer = returnedBuffers.poll()) != null) {
+			currentNumber = verifyBufferFilledWithAscendingNumbers(buffer, currentNumber);
+		}
+	}
+
+	@Test
+	public void testWriteSkipRead() throws IOException {
+		int numBuffers = 1024;
+		int currentNumber = 0;
+
+		final int minBufferSize = BUFFER_SIZE / 4;
+
+		// Write buffers filled with ascending numbers...
+		for (int i = 0; i < numBuffers; i++) {
+			final Buffer buffer = createBuffer();
+
+			currentNumber = fillBufferWithAscendingNumbers(buffer, currentNumber);
+
+			writer.writeBlock(buffer);
+		}
+
+		// Make sure that the writes are finished
+		writer.close();
+
+		final int toSkip = 32;
+
+		// Skip first buffers...
+		reader.seekToPosition((8 + BUFFER_SIZE) * toSkip);
+
+		numBuffers -= toSkip;
+
+		// Read buffers back in...
+		for (int i = 0; i < numBuffers; i++) {
+			assertFalse(reader.hasReachedEndOfFile());
+			reader.readInto(createBuffer());
+		}
+
+		reader.close();
+
+		assertTrue(reader.hasReachedEndOfFile());
+
+		// Verify that the content is the same
+		assertEquals("Read less buffers than written.", numBuffers, returnedBuffers.size());
+
+		// Start number after skipped buffers...
+		currentNumber = (BUFFER_SIZE / 4) * toSkip;
+
+		Buffer buffer;
+		while ((buffer = returnedBuffers.poll()) != null) {
+			currentNumber = verifyBufferFilledWithAscendingNumbers(buffer, currentNumber);
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	private int getRandomNumberInRange(int min, int max) {
+		return random.nextInt((max - min) + 1) + min;
+	}
+
+	private int getNextMultipleOf(int number, int multiple) {
+		final int mod = number % multiple;
+
+		if (mod == 0) {
+			return number;
+		}
+
+		return number + multiple - mod;
+	}
+
+	private Buffer createBuffer() {
+		return new Buffer(new MemorySegment(new byte[BUFFER_SIZE]), BUFFER_RECYCLER);
+	}
+
+	public static int fillBufferWithAscendingNumbers(Buffer buffer, int currentNumber) {
+		MemorySegment segment = buffer.getMemorySegment();
+
+		final int size = buffer.getSize();
+
+		for (int i = 0; i < size; i += 4) {
+			segment.putInt(i, currentNumber++);
+		}
+
+		return currentNumber;
+	}
+
+	private int verifyBufferFilledWithAscendingNumbers(Buffer buffer, int currentNumber) {
+		MemorySegment segment = buffer.getMemorySegment();
+
+		final int size = buffer.getSize();
+
+		for (int i = 0; i < size; i += 4) {
+			if (segment.getInt(i) != currentNumber++) {
+				throw new IllegalStateException("Read unexpected number from buffer.");
+			}
+		}
+
+		return currentNumber;
+	}
+}


[04/13] flink git commit: [FLINK-1350] [runtime] Add blocking result partition variant

Posted by uc...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/MockConsumer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/MockConsumer.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/MockConsumer.java
deleted file mode 100644
index 62375a6..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/MockConsumer.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.io.network.util;
-
-import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.partition.queue.IntermediateResultPartitionQueueIterator;
-
-import java.util.concurrent.Callable;
-import java.util.concurrent.atomic.AtomicReference;
-
-public class MockConsumer implements Callable<Boolean> {
-
-	private static final int SLEEP_TIME_MS = 20;
-
-	private final IntermediateResultPartitionQueueIterator iterator;
-
-	private final boolean slowConsumer;
-
-	private final AtomicReference<Throwable> error = new AtomicReference<Throwable>();
-
-	public MockConsumer(IntermediateResultPartitionQueueIterator iterator, boolean slowConsumer) {
-		this.iterator = iterator;
-		this.slowConsumer = slowConsumer;
-	}
-
-	@Override
-	public Boolean call() throws Exception {
-		MockNotificationListener listener = new MockNotificationListener();
-
-		int currentNumber = 0;
-
-		try {
-			while (true) {
-				Buffer buffer = iterator.getNextBuffer();
-
-				if (slowConsumer) {
-					Thread.sleep(SLEEP_TIME_MS);
-				}
-
-				if (buffer == null) {
-					if (iterator.subscribe(listener)) {
-						listener.waitForNotification();
-					}
-					else if (iterator.isConsumed()) {
-						break;
-					}
-				}
-				else {
-					try {
-						if (buffer.isBuffer()) {
-							currentNumber = verifyBufferFilledWithAscendingNumbers(buffer, currentNumber);
-						}
-					}
-					finally {
-						buffer.recycle();
-					}
-				}
-			}
-		}
-		catch (Throwable t) {
-			error.compareAndSet(null, t);
-			return false;
-		}
-
-		return true;
-	}
-
-	public Throwable getError() {
-		return error.get();
-	}
-
-	private int verifyBufferFilledWithAscendingNumbers(Buffer buffer, int currentNumber) {
-		MemorySegment segment = buffer.getMemorySegment();
-
-		for (int i = 4; i < segment.size(); i += 4) {
-			if (segment.getInt(i) != currentNumber++) {
-				throw new IllegalStateException("Read unexpected number from buffer.");
-			}
-		}
-
-		return currentNumber;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/MockInputChannel.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/MockInputChannel.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/MockInputChannel.java
deleted file mode 100644
index 301169a..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/MockInputChannel.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.io.network.util;
-
-import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
-import org.apache.flink.runtime.io.network.api.EndOfSuperstepEvent;
-import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.partition.consumer.InputChannel;
-import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
-import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-import org.mockito.stubbing.OngoingStubbing;
-
-import java.io.IOException;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-/**
- * A mocked input channel.
- */
-public class MockInputChannel {
-
-	private final InputChannel mock = Mockito.mock(InputChannel.class);
-
-	private final SingleInputGate inputGate;
-
-	// Abusing Mockito here... ;)
-	protected OngoingStubbing<Buffer> stubbing;
-
-	public MockInputChannel(SingleInputGate inputGate, int channelIndex) {
-		checkArgument(channelIndex >= 0);
-		this.inputGate = checkNotNull(inputGate);
-
-		when(mock.getChannelIndex()).thenReturn(channelIndex);
-	}
-
-	public MockInputChannel read(Buffer buffer) throws IOException {
-		if (stubbing == null) {
-			stubbing = when(mock.getNextBuffer()).thenReturn(buffer);
-		}
-		else {
-			stubbing = stubbing.thenReturn(buffer);
-		}
-
-		inputGate.onAvailableBuffer(mock);
-
-		return this;
-	}
-
-	public MockInputChannel readBuffer() throws IOException {
-		final Buffer buffer = mock(Buffer.class);
-		when(buffer.isBuffer()).thenReturn(true);
-
-		return read(buffer);
-	}
-
-	public MockInputChannel readEvent() throws IOException {
-		return read(EventSerializer.toBuffer(new TestTaskEvent()));
-	}
-
-	public MockInputChannel readEndOfSuperstepEvent() throws IOException {
-		return read(EventSerializer.toBuffer(EndOfSuperstepEvent.INSTANCE));
-	}
-
-	public MockInputChannel readEndOfPartitionEvent() throws IOException {
-		final Answer<Buffer> answer = new Answer<Buffer>() {
-			@Override
-			public Buffer answer(InvocationOnMock invocationOnMock) throws Throwable {
-				// Return true after finishing
-				when(mock.isReleased()).thenReturn(true);
-
-				return EventSerializer.toBuffer(EndOfPartitionEvent.INSTANCE);
-			}
-		};
-
-		if (stubbing == null) {
-			stubbing = when(mock.getNextBuffer()).thenAnswer(answer);
-		}
-		else {
-			stubbing = stubbing.thenAnswer(answer);
-		}
-
-		inputGate.onAvailableBuffer(mock);
-
-		return this;
-	}
-
-	public InputChannel getInputChannel() {
-		return mock;
-	}
-
-	// ------------------------------------------------------------------------
-
-	public static MockInputChannel[] createInputChannels(SingleInputGate inputGate, int numberOfInputChannels) {
-		checkNotNull(inputGate);
-		checkArgument(numberOfInputChannels > 0);
-
-		MockInputChannel[] mocks = new MockInputChannel[numberOfInputChannels];
-
-		for (int i = 0; i < numberOfInputChannels; i++) {
-			mocks[i] = new MockInputChannel(inputGate, i);
-
-			inputGate.setInputChannel(new IntermediateResultPartitionID(), mocks[i].getInputChannel());
-		}
-
-		return mocks;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/MockNotificationListener.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/MockNotificationListener.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/MockNotificationListener.java
deleted file mode 100644
index 56e0025..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/MockNotificationListener.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.io.network.util;
-
-import org.apache.flink.runtime.util.event.NotificationListener;
-
-import java.util.concurrent.atomic.AtomicInteger;
-
-public class MockNotificationListener implements NotificationListener {
-
-	final AtomicInteger numNotifications = new AtomicInteger();
-
-	@Override
-	public void onNotification() {
-		synchronized (numNotifications) {
-			numNotifications.incrementAndGet();
-
-			numNotifications.notifyAll();
-		}
-	}
-
-	public void waitForNotification() throws InterruptedException {
-
-		int current = numNotifications.get();
-
-		synchronized (numNotifications) {
-			while (current == numNotifications.get()) {
-				numNotifications.wait();
-			}
-		}
-	}
-
-	public int getNumberOfNotifications() {
-		return numNotifications.get();
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/MockProducer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/MockProducer.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/MockProducer.java
deleted file mode 100644
index 44d8ffe..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/MockProducer.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.io.network.util;
-
-import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.buffer.BufferPool;
-import org.apache.flink.runtime.io.network.partition.queue.IntermediateResultPartitionQueue;
-
-import java.util.concurrent.Callable;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-
-public class MockProducer implements Callable<Boolean> {
-
-	private static final int SLEEP_TIME_MS = 20;
-
-	private final IntermediateResultPartitionQueue queue;
-
-	private final BufferPool bufferPool;
-
-	private final int numBuffersToProduce;
-
-	private final boolean slowProducer;
-
-	private final AtomicInteger discardAfter = new AtomicInteger(Integer.MAX_VALUE);
-
-	private final AtomicReference<Throwable> error = new AtomicReference<Throwable>();
-
-	public MockProducer(IntermediateResultPartitionQueue queue, BufferPool bufferPool, int numBuffersToProduce, boolean slowProducer) {
-		this.queue = queue;
-		this.bufferPool = bufferPool;
-		this.numBuffersToProduce = numBuffersToProduce;
-		this.slowProducer = slowProducer;
-	}
-
-	@Override
-	public Boolean call() throws Exception {
-		try {
-			int currentNumber = 0;
-
-			for (int i = 0; i < numBuffersToProduce; i++) {
-				if (i >= discardAfter.get()) {
-					queue.discard();
-					return true;
-				}
-
-				Buffer buffer = bufferPool.requestBufferBlocking();
-
-				currentNumber = fillBufferWithAscendingNumbers(buffer, currentNumber);
-
-				queue.add(buffer);
-
-				if (slowProducer) {
-					Thread.sleep(SLEEP_TIME_MS);
-				}
-			}
-
-			queue.finish();
-		}
-		catch (Throwable t) {
-			error.compareAndSet(null, t);
-			return false;
-		}
-
-		return true;
-	}
-
-	void discard() {
-		discardAfter.set(0);
-	}
-
-	public void discardAfter(int numBuffers) {
-		discardAfter.set(numBuffers);
-	}
-
-	public Throwable getError() {
-		return error.get();
-	}
-
-	public static int fillBufferWithAscendingNumbers(Buffer buffer, int currentNumber) {
-		MemorySegment segment = buffer.getMemorySegment();
-
-		for (int i = 4; i < segment.size(); i += 4) {
-			segment.putInt(i, currentNumber++);
-		}
-
-		return currentNumber;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/MockSingleInputGate.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/MockSingleInputGate.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/MockSingleInputGate.java
deleted file mode 100644
index 3c708ac..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/MockSingleInputGate.java
+++ /dev/null
@@ -1,137 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.io.network.util;
-
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
-import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
-import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkElementIndex;
-import static org.mockito.Mockito.spy;
-
-public class MockSingleInputGate {
-
-	protected final SingleInputGate inputGate;
-
-	protected final MockInputChannel[] inputChannels;
-
-	public MockSingleInputGate(int numberOfInputChannels) {
-		this(numberOfInputChannels, true);
-	}
-
-	public MockSingleInputGate(int numberOfInputChannels, boolean initialize) {
-		checkArgument(numberOfInputChannels >= 1);
-
-		this.inputGate = spy(new SingleInputGate(new IntermediateDataSetID(), 0, numberOfInputChannels));
-
-		this.inputChannels = new MockInputChannel[numberOfInputChannels];
-
-		if (initialize) {
-			for (int i = 0; i < numberOfInputChannels; i++) {
-				inputChannels[i] = new MockInputChannel(inputGate, i);
-				inputGate.setInputChannel(new IntermediateResultPartitionID(), inputChannels[i].getInputChannel());
-			}
-		}
-	}
-
-	public MockSingleInputGate read(Buffer buffer, int channelIndex) throws IOException {
-		checkElementIndex(channelIndex, inputGate.getNumberOfInputChannels());
-
-		inputChannels[channelIndex].read(buffer);
-
-		return this;
-	}
-
-	public MockSingleInputGate readBuffer() throws IOException {
-		return readBuffer(0);
-	}
-
-	public MockSingleInputGate readBuffer(int channelIndex) throws IOException {
-		inputChannels[channelIndex].readBuffer();
-
-		return this;
-	}
-
-	public MockSingleInputGate readEvent() throws IOException {
-		return readEvent(0);
-	}
-
-	public MockSingleInputGate readEvent(int channelIndex) throws IOException {
-		inputChannels[channelIndex].readEvent();
-
-		return this;
-	}
-
-	public MockSingleInputGate readEndOfSuperstepEvent() throws IOException {
-		for (MockInputChannel inputChannel : inputChannels) {
-			inputChannel.readEndOfSuperstepEvent();
-		}
-
-		return this;
-	}
-
-	public MockSingleInputGate readEndOfSuperstepEvent(int channelIndex) throws IOException {
-		inputChannels[channelIndex].readEndOfSuperstepEvent();
-
-		return this;
-	}
-
-	public MockSingleInputGate readEndOfPartitionEvent() throws IOException {
-		for (MockInputChannel inputChannel : inputChannels) {
-			inputChannel.readEndOfPartitionEvent();
-		}
-
-		return this;
-	}
-
-	public MockSingleInputGate readEndOfPartitionEvent(int channelIndex) throws IOException {
-		inputChannels[channelIndex].readEndOfPartitionEvent();
-
-		return this;
-	}
-
-	public SingleInputGate getInputGate() {
-		return inputGate;
-	}
-
-	// ------------------------------------------------------------------------
-
-	public List<Integer> readAllChannels() throws IOException {
-		final List<Integer> readOrder = new ArrayList<Integer>(inputChannels.length);
-
-		for (int i = 0; i < inputChannels.length; i++) {
-			readOrder.add(i);
-		}
-
-		Collections.shuffle(readOrder);
-
-		for (int channelIndex : readOrder) {
-			inputChannels[channelIndex].readBuffer();
-		}
-
-		return readOrder;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestBufferFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestBufferFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestBufferFactory.java
new file mode 100644
index 0000000..d10bf0c
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestBufferFactory.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.util;
+
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
+import org.apache.flink.runtime.testutils.DiscardingRecycler;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class TestBufferFactory {
+
+	private static final int defaultSize = 32 * 1024;
+
+	private static final BufferRecycler discardingRecycler = new DiscardingRecycler();
+
+	private final int bufferSize;
+
+	private final BufferRecycler bufferRecycler;
+
+	private AtomicInteger numberOfCreatedBuffers = new AtomicInteger();
+
+	public TestBufferFactory() {
+		this(defaultSize, discardingRecycler);
+	}
+
+	public TestBufferFactory(int bufferSize) {
+		this(bufferSize, discardingRecycler);
+	}
+
+	public TestBufferFactory(int bufferSize, BufferRecycler bufferRecycler) {
+		checkArgument(bufferSize > 0);
+		this.bufferSize = bufferSize;
+		this.bufferRecycler = checkNotNull(bufferRecycler);
+	}
+
+	public Buffer create() {
+		numberOfCreatedBuffers.incrementAndGet();
+
+		return new Buffer(new MemorySegment(new byte[bufferSize]), bufferRecycler);
+	}
+
+	public Buffer createFrom(MemorySegment segment) {
+		return new Buffer(segment, bufferRecycler);
+	}
+
+	public int getNumberOfCreatedBuffers() {
+		return numberOfCreatedBuffers.get();
+	}
+
+	public int getBufferSize() {
+		return bufferSize;
+	}
+
+	// ------------------------------------------------------------------------
+	// Static test helpers
+	// ------------------------------------------------------------------------
+
+	public static Buffer createBuffer() {
+		return createBuffer(defaultSize);
+	}
+
+	public static Buffer createBuffer(int bufferSize) {
+		checkArgument(bufferSize > 0);
+
+		return new Buffer(new MemorySegment(new byte[bufferSize]), discardingRecycler);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestConsumerCallback.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestConsumerCallback.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestConsumerCallback.java
new file mode 100644
index 0000000..52083c4
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestConsumerCallback.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.util;
+
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.runtime.event.task.AbstractEvent;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.junit.Assert.assertEquals;
+
+public interface TestConsumerCallback {
+
+	void onBuffer(Buffer buffer);
+
+	void onEvent(AbstractEvent event);
+
+	public static class CountingCallback implements TestConsumerCallback {
+
+		private final AtomicInteger numberOfReadBuffers = new AtomicInteger();
+
+		private final AtomicInteger numberOfReadEvents = new AtomicInteger();
+
+		@Override
+		public void onBuffer(Buffer buffer) {
+			numberOfReadBuffers.incrementAndGet();
+		}
+
+		@Override
+		public void onEvent(AbstractEvent event) {
+			numberOfReadEvents.incrementAndGet();
+		}
+
+		/**
+		 * Returns the number of read buffers.
+		 */
+		public int getNumberOfReadBuffers() {
+			return numberOfReadBuffers.get();
+		}
+
+		/**
+		 * Returns the number of read events;
+		 */
+		public int getNumberOfReadEvents() {
+			return numberOfReadEvents.get();
+		}
+	}
+
+	public static class RecyclingCallback extends CountingCallback {
+
+		@Override
+		public void onBuffer(Buffer buffer) {
+			super.onBuffer(buffer);
+
+			buffer.recycle();
+		}
+
+		@Override
+		public void onEvent(AbstractEvent event) {
+			super.onEvent(event);
+		}
+	}
+
+	public class VerifyAscendingCallback extends RecyclingCallback {
+
+		@Override
+		public void onBuffer(Buffer buffer) {
+			final MemorySegment segment = buffer.getMemorySegment();
+
+			int expected = getNumberOfReadBuffers() * (segment.size() / 4);
+
+			for (int i = 0; i < segment.size(); i += 4) {
+				assertEquals(expected, segment.getInt(i));
+
+				expected++;
+			}
+
+			super.onBuffer(buffer);
+		}
+
+		@Override
+		public void onEvent(AbstractEvent event) {
+			super.onEvent(event);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestInfiniteBufferProvider.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestInfiniteBufferProvider.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestInfiniteBufferProvider.java
new file mode 100644
index 0000000..976e63d
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestInfiniteBufferProvider.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.util;
+
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferProvider;
+import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
+import org.apache.flink.runtime.util.event.EventListener;
+
+import java.io.IOException;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+public class TestInfiniteBufferProvider implements BufferProvider {
+
+	private final ConcurrentLinkedQueue<Buffer> buffers = new ConcurrentLinkedQueue<Buffer>();
+
+	private final TestBufferFactory bufferFactory = new TestBufferFactory(
+			32 * 1024, new InfiniteBufferProviderRecycler(buffers));
+
+	@Override
+	public Buffer requestBuffer() throws IOException {
+		Buffer buffer = buffers.poll();
+
+		if (buffer != null) {
+			return buffer;
+		}
+
+		return bufferFactory.create();
+	}
+
+	@Override
+	public Buffer requestBufferBlocking() throws IOException, InterruptedException {
+		return requestBuffer();
+	}
+
+	@Override
+	public boolean addListener(EventListener<Buffer> listener) {
+		return false;
+	}
+
+	@Override
+	public boolean isDestroyed() {
+		return false;
+	}
+
+	@Override
+	public int getMemorySegmentSize() {
+		return bufferFactory.getBufferSize();
+	}
+
+	private static class InfiniteBufferProviderRecycler implements BufferRecycler {
+
+		private final ConcurrentLinkedQueue<Buffer> buffers;
+
+		public InfiniteBufferProviderRecycler(ConcurrentLinkedQueue<Buffer> buffers) {
+			this.buffers = buffers;
+		}
+
+		@Override
+		public void recycle(MemorySegment segment) {
+			buffers.add(new Buffer(segment, this));
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestInputChannel.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestInputChannel.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestInputChannel.java
new file mode 100644
index 0000000..306de4c
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestInputChannel.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.util;
+
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.EndOfSuperstepEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.partition.consumer.InputChannel;
+import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
+import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import org.mockito.stubbing.OngoingStubbing;
+
+import java.io.IOException;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * A mocked input channel.
+ */
+public class TestInputChannel {
+
+	private final InputChannel mock = Mockito.mock(InputChannel.class);
+
+	private final SingleInputGate inputGate;
+
+	// Abusing Mockito here... ;)
+	protected OngoingStubbing<Buffer> stubbing;
+
+	public TestInputChannel(SingleInputGate inputGate, int channelIndex) {
+		checkArgument(channelIndex >= 0);
+		this.inputGate = checkNotNull(inputGate);
+
+		when(mock.getChannelIndex()).thenReturn(channelIndex);
+	}
+
+	public TestInputChannel read(Buffer buffer) throws IOException, InterruptedException {
+		if (stubbing == null) {
+			stubbing = when(mock.getNextBuffer()).thenReturn(buffer);
+		}
+		else {
+			stubbing = stubbing.thenReturn(buffer);
+		}
+
+		inputGate.onAvailableBuffer(mock);
+
+		return this;
+	}
+
+	public TestInputChannel readBuffer() throws IOException, InterruptedException {
+		final Buffer buffer = mock(Buffer.class);
+		when(buffer.isBuffer()).thenReturn(true);
+
+		return read(buffer);
+	}
+
+	public TestInputChannel readEvent() throws IOException, InterruptedException {
+		return read(EventSerializer.toBuffer(new TestTaskEvent()));
+	}
+
+	public TestInputChannel readEndOfSuperstepEvent() throws IOException, InterruptedException {
+		return read(EventSerializer.toBuffer(EndOfSuperstepEvent.INSTANCE));
+	}
+
+	public TestInputChannel readEndOfPartitionEvent() throws IOException, InterruptedException {
+		final Answer<Buffer> answer = new Answer<Buffer>() {
+			@Override
+			public Buffer answer(InvocationOnMock invocationOnMock) throws Throwable {
+				// Return true after finishing
+				when(mock.isReleased()).thenReturn(true);
+
+				return EventSerializer.toBuffer(EndOfPartitionEvent.INSTANCE);
+			}
+		};
+
+		if (stubbing == null) {
+			stubbing = when(mock.getNextBuffer()).thenAnswer(answer);
+		}
+		else {
+			stubbing = stubbing.thenAnswer(answer);
+		}
+
+		inputGate.onAvailableBuffer(mock);
+
+		return this;
+	}
+
+	public InputChannel getInputChannel() {
+		return mock;
+	}
+
+	// ------------------------------------------------------------------------
+
+	public static TestInputChannel[] createInputChannels(SingleInputGate inputGate, int numberOfInputChannels) {
+		checkNotNull(inputGate);
+		checkArgument(numberOfInputChannels > 0);
+
+		TestInputChannel[] mocks = new TestInputChannel[numberOfInputChannels];
+
+		for (int i = 0; i < numberOfInputChannels; i++) {
+			mocks[i] = new TestInputChannel(inputGate, i);
+
+			inputGate.setInputChannel(new IntermediateResultPartitionID(), mocks[i].getInputChannel());
+		}
+
+		return mocks;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestNotificationListener.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestNotificationListener.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestNotificationListener.java
new file mode 100644
index 0000000..1e943a6
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestNotificationListener.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.util;
+
+import org.apache.flink.runtime.util.event.NotificationListener;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * A mock notification listener.
+ */
+public class TestNotificationListener implements NotificationListener {
+
+	final AtomicInteger numberOfNotifications = new AtomicInteger();
+
+	@Override
+	public void onNotification() {
+		synchronized (numberOfNotifications) {
+			numberOfNotifications.incrementAndGet();
+
+			numberOfNotifications.notifyAll();
+		}
+	}
+
+	/**
+	 * Waits on a notification.
+	 *
+	 * <p> <strong>Important</strong>: It's necessary to get the current number of notifications
+	 * <em>before</em> registering the listener. Otherwise the wait call may block indefinitely.
+	 *
+	 * <pre>
+	 * MockNotificationListener listener = new MockNotificationListener();
+	 *
+	 * int current = listener.getNumberOfNotifications();
+	 *
+	 * // Register the listener
+	 * register(listener);
+	 *
+	 * listener.waitForNotification(current);
+	 * </pre>
+	 */
+	public void waitForNotification(int current) throws InterruptedException {
+		synchronized (numberOfNotifications) {
+			while (current == numberOfNotifications.get()) {
+				numberOfNotifications.wait();
+			}
+		}
+	}
+
+	public int getNumberOfNotifications() {
+		return numberOfNotifications.get();
+	}
+
+	public void reset() {
+		numberOfNotifications.set(0);
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestPartitionProducer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestPartitionProducer.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestPartitionProducer.java
new file mode 100644
index 0000000..31fd4a4
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestPartitionProducer.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.util;
+
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.partition.ResultPartition;
+import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
+
+import java.util.Random;
+import java.util.concurrent.Callable;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * A test partition producer.
+ *
+ * <p> The behaviour of the producer is customizable by specifying a source.
+ *
+ * @see TestProducerSource
+ */
+public class TestPartitionProducer implements Callable<Boolean> {
+
+	public static final int MAX_SLEEP_TIME_MS = 20;
+
+	/** The partition to add data to. */
+	private final ResultPartition partition;
+
+	/**
+	 * Flag indicating whether the consumer is slow. If true, the consumer will sleep a random
+	 * number of milliseconds between adding data.
+	 */
+	private final boolean isSlowProducer;
+
+	/** The source data. */
+	private final TestProducerSource source;
+
+	/** Random source for sleeps. */
+	private final Random random;
+
+	public TestPartitionProducer(
+			ResultPartition partition,
+			boolean isSlowProducer,
+			TestProducerSource source) {
+
+		this.partition = checkNotNull(partition);
+		this.isSlowProducer = isSlowProducer;
+		this.random = isSlowProducer ? new Random() : null;
+		this.source = checkNotNull(source);
+	}
+
+	@Override
+	public Boolean call() throws Exception {
+
+		try {
+			BufferOrEvent bufferOrEvent;
+
+			while ((bufferOrEvent = source.getNextBufferOrEvent()) != null) {
+				int targetChannelIndex = bufferOrEvent.getChannelIndex();
+
+				if (bufferOrEvent.isBuffer()) {
+					partition.add(bufferOrEvent.getBuffer(), targetChannelIndex);
+				}
+				else if (bufferOrEvent.isEvent()) {
+					final Buffer buffer = EventSerializer.toBuffer(bufferOrEvent.getEvent());
+
+					partition.add(buffer, targetChannelIndex);
+				}
+				else {
+					throw new IllegalStateException("BufferOrEvent instance w/o buffer nor event.");
+				}
+
+				// Check for interrupted flag after adding data to prevent resource leaks
+				if (Thread.interrupted()) {
+					throw new InterruptedException();
+				}
+
+				if (isSlowProducer) {
+					Thread.sleep(random.nextInt(MAX_SLEEP_TIME_MS + 1));
+				}
+			}
+
+			partition.finish();
+
+			return true;
+		}
+		finally {
+			partition.release();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestPooledBufferProvider.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestPooledBufferProvider.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestPooledBufferProvider.java
new file mode 100644
index 0000000..4893360
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestPooledBufferProvider.java
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.util;
+
+import com.google.common.collect.Queues;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferProvider;
+import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
+import org.apache.flink.runtime.util.event.EventListener;
+
+import java.io.IOException;
+import java.util.Queue;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+public class TestPooledBufferProvider implements BufferProvider {
+
+	private final Object bufferCreationLock = new Object();
+
+	private final ArrayBlockingQueue<Buffer> buffers;
+
+	private final TestBufferFactory bufferFactory;
+
+	private final PooledBufferProviderRecycler bufferRecycler;
+
+	private final int poolSize;
+
+	public TestPooledBufferProvider(int poolSize) {
+		checkArgument(poolSize > 0);
+		this.poolSize = poolSize;
+
+		this.buffers = new ArrayBlockingQueue<Buffer>(poolSize);
+		this.bufferRecycler = new PooledBufferProviderRecycler(buffers);
+		this.bufferFactory = new TestBufferFactory(32 * 1024, bufferRecycler);
+	}
+
+	@Override
+	public Buffer requestBuffer() throws IOException {
+		final Buffer buffer = buffers.poll();
+
+		if (buffer != null) {
+			return buffer;
+		}
+		else {
+			synchronized (bufferCreationLock) {
+				if (bufferFactory.getNumberOfCreatedBuffers() < poolSize) {
+					return bufferFactory.create();
+				}
+			}
+
+			return null;
+		}
+	}
+
+	@Override
+	public Buffer requestBufferBlocking() throws IOException, InterruptedException {
+		final Buffer buffer = buffers.poll();
+
+		if (buffer != null) {
+			return buffer;
+		}
+		else {
+			synchronized (bufferCreationLock) {
+				if (bufferFactory.getNumberOfCreatedBuffers() < poolSize) {
+					return bufferFactory.create();
+				}
+			}
+
+			return buffers.take();
+		}
+	}
+
+	@Override
+	public boolean addListener(EventListener<Buffer> listener) {
+		return bufferRecycler.registerListener(listener);
+	}
+
+	@Override
+	public boolean isDestroyed() {
+		return false;
+	}
+
+	@Override
+	public int getMemorySegmentSize() {
+		return bufferFactory.getBufferSize();
+	}
+
+	public int getNumberOfAvailableBuffers() {
+		return buffers.size();
+	}
+
+	private static class PooledBufferProviderRecycler implements BufferRecycler {
+
+		private final Object listenerRegistrationLock = new Object();
+
+		private final Queue<Buffer> buffers;
+
+		private final ConcurrentLinkedQueue<EventListener<Buffer>> registeredListeners =
+				Queues.newConcurrentLinkedQueue();
+
+		public PooledBufferProviderRecycler(Queue<Buffer> buffers) {
+			this.buffers = buffers;
+		}
+
+		@Override
+		public void recycle(MemorySegment segment) {
+			synchronized (listenerRegistrationLock) {
+				final Buffer buffer = new Buffer(segment, this);
+
+				EventListener<Buffer> listener = registeredListeners.poll();
+
+				if (listener == null) {
+					buffers.add(buffer);
+				}
+				else {
+					listener.onEvent(buffer);
+				}
+			}
+		}
+
+		boolean registerListener(EventListener<Buffer> listener) {
+			synchronized (listenerRegistrationLock) {
+				if (buffers.isEmpty()) {
+					registeredListeners.add(listener);
+
+					return true;
+				}
+
+				return false;
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestProducerSource.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestProducerSource.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestProducerSource.java
new file mode 100644
index 0000000..dea9df2
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestProducerSource.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.util;
+
+import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
+
+public interface TestProducerSource {
+
+	/**
+	 * Returns the next buffer or event instance.
+	 *
+	 * <p> The channel index specifies the subpartition add the data to.
+	 */
+	BufferOrEvent getNextBufferOrEvent() throws Exception;
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestSingleInputGate.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestSingleInputGate.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestSingleInputGate.java
new file mode 100644
index 0000000..d10e1a0
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestSingleInputGate.java
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.util;
+
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
+import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
+import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkElementIndex;
+import static org.mockito.Mockito.spy;
+
+/**
+ * A test input gate to mock reading data.
+ */
+public class TestSingleInputGate {
+
+	protected final SingleInputGate inputGate;
+
+	protected final TestInputChannel[] inputChannels;
+
+	public TestSingleInputGate(int numberOfInputChannels) {
+		this(numberOfInputChannels, true);
+	}
+
+	public TestSingleInputGate(int numberOfInputChannels, boolean initialize) {
+		checkArgument(numberOfInputChannels >= 1);
+
+		this.inputGate = spy(new SingleInputGate(new IntermediateDataSetID(), 0, numberOfInputChannels));
+
+		this.inputChannels = new TestInputChannel[numberOfInputChannels];
+
+		if (initialize) {
+			for (int i = 0; i < numberOfInputChannels; i++) {
+				inputChannels[i] = new TestInputChannel(inputGate, i);
+				inputGate.setInputChannel(new IntermediateResultPartitionID(), inputChannels[i].getInputChannel());
+			}
+		}
+	}
+
+	public TestSingleInputGate read(Buffer buffer, int channelIndex) throws IOException, InterruptedException {
+		checkElementIndex(channelIndex, inputGate.getNumberOfInputChannels());
+
+		inputChannels[channelIndex].read(buffer);
+
+		return this;
+	}
+
+	public TestSingleInputGate readBuffer() throws IOException, InterruptedException {
+		return readBuffer(0);
+	}
+
+	public TestSingleInputGate readBuffer(int channelIndex) throws IOException, InterruptedException {
+		inputChannels[channelIndex].readBuffer();
+
+		return this;
+	}
+
+	public TestSingleInputGate readEvent() throws IOException, InterruptedException {
+		return readEvent(0);
+	}
+
+	public TestSingleInputGate readEvent(int channelIndex) throws IOException, InterruptedException {
+		inputChannels[channelIndex].readEvent();
+
+		return this;
+	}
+
+	public TestSingleInputGate readEndOfSuperstepEvent() throws IOException, InterruptedException {
+		for (TestInputChannel inputChannel : inputChannels) {
+			inputChannel.readEndOfSuperstepEvent();
+		}
+
+		return this;
+	}
+
+	public TestSingleInputGate readEndOfSuperstepEvent(int channelIndex) throws IOException, InterruptedException {
+		inputChannels[channelIndex].readEndOfSuperstepEvent();
+
+		return this;
+	}
+
+	public TestSingleInputGate readEndOfPartitionEvent() throws IOException, InterruptedException {
+		for (TestInputChannel inputChannel : inputChannels) {
+			inputChannel.readEndOfPartitionEvent();
+		}
+
+		return this;
+	}
+
+	public TestSingleInputGate readEndOfPartitionEvent(int channelIndex) throws IOException, InterruptedException {
+		inputChannels[channelIndex].readEndOfPartitionEvent();
+
+		return this;
+	}
+
+	public SingleInputGate getInputGate() {
+		return inputGate;
+	}
+
+	// ------------------------------------------------------------------------
+
+	public List<Integer> readAllChannels() throws IOException, InterruptedException {
+		final List<Integer> readOrder = new ArrayList<Integer>(inputChannels.length);
+
+		for (int i = 0; i < inputChannels.length; i++) {
+			readOrder.add(i);
+		}
+
+		Collections.shuffle(readOrder);
+
+		for (int channelIndex : readOrder) {
+			inputChannels[channelIndex].readBuffer();
+		}
+
+		return readOrder;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestSubpartitionConsumer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestSubpartitionConsumer.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestSubpartitionConsumer.java
new file mode 100644
index 0000000..2766e53
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestSubpartitionConsumer.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.util;
+
+import org.apache.flink.runtime.event.task.AbstractEvent;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView;
+
+import java.util.Random;
+import java.util.concurrent.Callable;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * A test subpartition view consumer.
+ *
+ * <p> The behaviour of the consumer is customizable by specifying a callback.
+ *
+ * @see TestConsumerCallback
+ */
+public class TestSubpartitionConsumer implements Callable<Boolean> {
+
+	private static final int MAX_SLEEP_TIME_MS = 20;
+
+	/** The subpartition view to consume. */
+	private final ResultSubpartitionView subpartitionView;
+
+	/**
+	 * Flag indicating whether the consumer is slow. If true, the consumer will sleep a random
+	 * number of milliseconds between returned buffers.
+	 */
+	private final boolean isSlowConsumer;
+
+	/** The callback to handle a read buffer. */
+	private final TestConsumerCallback callback;
+
+	/** Random source for sleeps. */
+	private final Random random;
+
+	public TestSubpartitionConsumer(
+			ResultSubpartitionView subpartitionView,
+			boolean isSlowConsumer,
+			TestConsumerCallback callback) {
+
+		this.subpartitionView = checkNotNull(subpartitionView);
+		this.isSlowConsumer = isSlowConsumer;
+		this.random = isSlowConsumer ? new Random() : null;
+		this.callback = checkNotNull(callback);
+	}
+
+	@Override
+	public Boolean call() throws Exception {
+		final TestNotificationListener listener = new TestNotificationListener();
+
+		try {
+			while (true) {
+				if (Thread.interrupted()) {
+					throw new InterruptedException();
+				}
+
+				final Buffer buffer = subpartitionView.getNextBuffer();
+
+				if (isSlowConsumer) {
+					Thread.sleep(random.nextInt(MAX_SLEEP_TIME_MS + 1));
+				}
+
+				if (buffer != null) {
+					if (buffer.isBuffer()) {
+						callback.onBuffer(buffer);
+					}
+					else {
+						final AbstractEvent event = EventSerializer.fromBuffer(buffer,
+								getClass().getClassLoader());
+
+						callback.onEvent(event);
+
+						buffer.recycle();
+
+						if (event.getClass() == EndOfPartitionEvent.class) {
+							subpartitionView.notifySubpartitionConsumed();
+
+							return true;
+						}
+					}
+				}
+				else {
+					int current = listener.getNumberOfNotifications();
+
+					if (subpartitionView.registerListener(listener)) {
+						listener.waitForNotification(current);
+					}
+					else if (subpartitionView.isReleased()) {
+						return true;
+					}
+				}
+			}
+		}
+		finally {
+			subpartitionView.releaseAllResources();
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestSubpartitionProducer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestSubpartitionProducer.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestSubpartitionProducer.java
new file mode 100644
index 0000000..52c156e
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestSubpartitionProducer.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.util;
+
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.partition.ResultSubpartition;
+import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
+
+import java.util.Random;
+import java.util.concurrent.Callable;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * A test subpartition producer.
+ *
+ * <p> The behaviour of the producer is customizable by specifying a source.
+ *
+ * @see TestProducerSource
+ */
+public class TestSubpartitionProducer implements Callable<Boolean> {
+
+	public static final int MAX_SLEEP_TIME_MS = 20;
+
+	/** The subpartition to add data to. */
+	private final ResultSubpartition subpartition;
+
+	/**
+	 * Flag indicating whether the consumer is slow. If true, the consumer will sleep a random
+	 * number of milliseconds between adding data.
+	 */
+	private final boolean isSlowProducer;
+
+	/** The source data. */
+	private final TestProducerSource source;
+
+	/** Random source for sleeps. */
+	private final Random random;
+
+	public TestSubpartitionProducer(
+			ResultSubpartition subpartition,
+			boolean isSlowProducer,
+			TestProducerSource source) {
+
+		this.subpartition = checkNotNull(subpartition);
+		this.isSlowProducer = isSlowProducer;
+		this.random = isSlowProducer ? new Random() : null;
+		this.source = checkNotNull(source);
+	}
+
+	@Override
+	public Boolean call() throws Exception {
+
+		try {
+			BufferOrEvent bufferOrEvent;
+
+			while ((bufferOrEvent = source.getNextBufferOrEvent()) != null) {
+				if (bufferOrEvent.isBuffer()) {
+					subpartition.add(bufferOrEvent.getBuffer());
+				}
+				else if (bufferOrEvent.isEvent()) {
+					final Buffer buffer = EventSerializer.toBuffer(bufferOrEvent.getEvent());
+
+					subpartition.add(buffer);
+				}
+				else {
+					throw new IllegalStateException("BufferOrEvent instance w/o buffer nor event.");
+				}
+
+				// Check for interrupted flag after adding data to prevent resource leaks
+				if (Thread.interrupted()) {
+					throw new InterruptedException();
+				}
+
+				if (isSlowProducer) {
+					Thread.sleep(random.nextInt(MAX_SLEEP_TIME_MS + 1));
+				}
+			}
+
+			subpartition.finish();
+
+			return true;
+		}
+		finally {
+			subpartition.release();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestTaskEvent.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestTaskEvent.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestTaskEvent.java
index 4f547aa..0b29032 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestTaskEvent.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestTaskEvent.java
@@ -24,6 +24,9 @@ import org.apache.flink.runtime.event.task.TaskEvent;
 
 import java.io.IOException;
 
+/**
+ * A task event used in various tests.
+ */
 public class TestTaskEvent extends TaskEvent {
 
 	private double val0;

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSinkTaskTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSinkTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSinkTaskTest.java
index 84fc851..63ce5e2 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSinkTaskTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSinkTaskTest.java
@@ -21,8 +21,8 @@ package org.apache.flink.runtime.operators;
 import org.apache.flink.api.common.typeutils.record.RecordComparatorFactory;
 import org.apache.flink.api.java.record.io.DelimitedOutputFormat;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.io.network.api.reader.IteratorWrappingMockSingleInputGate;
-import org.apache.flink.runtime.io.network.api.writer.BufferWriter;
+import org.apache.flink.runtime.io.network.api.reader.IteratorWrappingTestSingleInputGate;
+import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
 import org.apache.flink.runtime.operators.testutils.InfiniteInputIterator;
 import org.apache.flink.runtime.operators.testutils.TaskCancelThread;
 import org.apache.flink.runtime.operators.testutils.TaskTestBase;
@@ -51,7 +51,7 @@ import java.util.HashSet;
 import java.util.Set;
 
 @RunWith(PowerMockRunner.class)
-@PrepareForTest({Task.class, BufferWriter.class})
+@PrepareForTest({Task.class, ResultPartitionWriter.class})
 public class DataSinkTaskTest extends TaskTestBase
 {
 	private static final Logger LOG = LoggerFactory.getLogger(DataSinkTaskTest.class);
@@ -138,7 +138,7 @@ public class DataSinkTaskTest extends TaskTestBase
 
 		super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
 
-		IteratorWrappingMockSingleInputGate<?>[] readers = new IteratorWrappingMockSingleInputGate[4];
+		IteratorWrappingTestSingleInputGate<?>[] readers = new IteratorWrappingTestSingleInputGate[4];
 		readers[0] = super.addInput(new UniformRecordGenerator(keyCnt, valCnt, 0, 0, false), 0, false);
 		readers[1] = super.addInput(new UniformRecordGenerator(keyCnt, valCnt, keyCnt, 0, false), 0, false);
 		readers[2] = super.addInput(new UniformRecordGenerator(keyCnt, valCnt, keyCnt * 2, 0, false), 0, false);
@@ -151,7 +151,7 @@ public class DataSinkTaskTest extends TaskTestBase
 		try {
 			// For the union reader to work, we need to start notifications *after* the union reader
 			// has been initialized.
-			for (IteratorWrappingMockSingleInputGate<?> reader : readers) {
+			for (IteratorWrappingTestSingleInputGate<?> reader : readers) {
 				reader.read();
 			}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSourceTaskTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSourceTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSourceTaskTest.java
index 90bb944..4548410 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSourceTaskTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSourceTaskTest.java
@@ -28,7 +28,7 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 
-import org.apache.flink.runtime.io.network.api.writer.BufferWriter;
+import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
 import org.apache.flink.runtime.taskmanager.Task;
 import org.junit.Assert;
 
@@ -47,7 +47,7 @@ import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.powermock.modules.junit4.PowerMockRunner;
 
 @RunWith(PowerMockRunner.class)
-@PrepareForTest({Task.class, BufferWriter.class})
+@PrepareForTest({Task.class, ResultPartitionWriter.class})
 public class DataSourceTaskTest extends TaskTestBase {
 
 	private static final int MEMORY_MANAGER_SIZE = 1024 * 1024;

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainTaskTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainTaskTest.java
index e22789c..88a71c4 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainTaskTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainTaskTest.java
@@ -27,7 +27,7 @@ import org.apache.flink.api.common.typeutils.record.RecordComparatorFactory;
 import org.apache.flink.api.common.typeutils.record.RecordSerializerFactory;
 import org.apache.flink.api.common.functions.RichGroupReduceFunction;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.io.network.api.writer.BufferWriter;
+import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
 import org.apache.flink.runtime.operators.CollectorMapDriver;
 import org.apache.flink.runtime.operators.DriverStrategy;
 import org.apache.flink.runtime.operators.RegularPactTask;
@@ -48,7 +48,7 @@ import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.powermock.modules.junit4.PowerMockRunner;
 
 @RunWith(PowerMockRunner.class)
-@PrepareForTest({Task.class, BufferWriter.class})
+@PrepareForTest({Task.class, ResultPartitionWriter.class})
 @SuppressWarnings("deprecation")
 public class ChainTaskTest extends TaskTestBase {
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java
index 7fb13e3..6625bbc 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java
@@ -27,10 +27,10 @@ import org.apache.flink.runtime.broadcast.BroadcastVariableManager;
 import org.apache.flink.runtime.execution.Environment;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
-import org.apache.flink.runtime.io.network.api.reader.IteratorWrappingMockSingleInputGate;
+import org.apache.flink.runtime.io.network.api.reader.IteratorWrappingTestSingleInputGate;
 import org.apache.flink.runtime.io.network.api.serialization.AdaptiveSpanningRecordDeserializer;
 import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer;
-import org.apache.flink.runtime.io.network.api.writer.BufferWriter;
+import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
 import org.apache.flink.runtime.io.network.buffer.BufferProvider;
 import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
@@ -71,7 +71,7 @@ public class MockEnvironment implements Environment {
 
 	private final List<InputGate> inputs;
 
-	private final List<BufferWriter> outputs;
+	private final List<ResultPartitionWriter> outputs;
 
 	private final JobID jobID = new JobID();
 
@@ -83,7 +83,7 @@ public class MockEnvironment implements Environment {
 		this.jobConfiguration = new Configuration();
 		this.taskConfiguration = new Configuration();
 		this.inputs = new LinkedList<InputGate>();
-		this.outputs = new LinkedList<BufferWriter>();
+		this.outputs = new LinkedList<ResultPartitionWriter>();
 
 		this.memManager = new DefaultMemoryManager(memorySize, 1);
 		this.ioManager = new IOManagerAsync();
@@ -91,9 +91,9 @@ public class MockEnvironment implements Environment {
 		this.bufferSize = bufferSize;
 	}
 
-	public IteratorWrappingMockSingleInputGate<Record> addInput(MutableObjectIterator<Record> inputIterator) {
+	public IteratorWrappingTestSingleInputGate<Record> addInput(MutableObjectIterator<Record> inputIterator) {
 		try {
-			final IteratorWrappingMockSingleInputGate<Record> reader = new IteratorWrappingMockSingleInputGate<Record>(bufferSize, Record.class, inputIterator);
+			final IteratorWrappingTestSingleInputGate<Record> reader = new IteratorWrappingTestSingleInputGate<Record>(bufferSize, Record.class, inputIterator);
 
 			inputs.add(reader.getInputGate());
 
@@ -118,7 +118,7 @@ public class MockEnvironment implements Environment {
 				}
 			});
 
-			BufferWriter mockWriter = mock(BufferWriter.class);
+			ResultPartitionWriter mockWriter = mock(ResultPartitionWriter.class);
 			when(mockWriter.getNumberOfOutputChannels()).thenReturn(1);
 			when(mockWriter.getBufferProvider()).thenReturn(mockBufferProvider);
 
@@ -225,13 +225,13 @@ public class MockEnvironment implements Environment {
 	}
 
 	@Override
-	public BufferWriter getWriter(int index) {
+	public ResultPartitionWriter getWriter(int index) {
 		return outputs.get(index);
 	}
 
 	@Override
-	public BufferWriter[] getAllWriters() {
-		return outputs.toArray(new BufferWriter[outputs.size()]);
+	public ResultPartitionWriter[] getAllWriters() {
+		return outputs.toArray(new ResultPartitionWriter[outputs.size()]);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java
index e0776d9..b93d37e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java
@@ -27,7 +27,7 @@ import org.apache.flink.api.java.record.io.FileOutputFormat;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.FileSystem.WriteMode;
 import org.apache.flink.core.fs.Path;
-import org.apache.flink.runtime.io.network.api.reader.IteratorWrappingMockSingleInputGate;
+import org.apache.flink.runtime.io.network.api.reader.IteratorWrappingTestSingleInputGate;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.memorymanager.MemoryManager;
 import org.apache.flink.runtime.operators.PactDriver;
@@ -55,14 +55,14 @@ public abstract class TaskTestBase {
 		this.mockEnv = new MockEnvironment(this.memorySize, this.inputSplitProvider, bufferSize);
 	}
 
-	public IteratorWrappingMockSingleInputGate<Record> addInput(MutableObjectIterator<Record> input, int groupId) {
-		final IteratorWrappingMockSingleInputGate<Record> reader = addInput(input, groupId, true);
+	public IteratorWrappingTestSingleInputGate<Record> addInput(MutableObjectIterator<Record> input, int groupId) {
+		final IteratorWrappingTestSingleInputGate<Record> reader = addInput(input, groupId, true);
 
 		return reader;
 	}
 
-	public IteratorWrappingMockSingleInputGate<Record> addInput(MutableObjectIterator<Record> input, int groupId, boolean read) {
-		final IteratorWrappingMockSingleInputGate<Record> reader = this.mockEnv.addInput(input);
+	public IteratorWrappingTestSingleInputGate<Record> addInput(MutableObjectIterator<Record> input, int groupId, boolean read) {
+		final IteratorWrappingTestSingleInputGate<Record> reader = this.mockEnv.addInput(input);
 		TaskConfig conf = new TaskConfig(this.mockEnv.getTaskConfiguration());
 		conf.addInputToGroup(groupId);
 		conf.setInputSerializer(RecordSerializerFactory.get(), groupId);

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerProcessReapingTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerProcessReapingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerProcessReapingTest.java
index a7173b4..f4ee52f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerProcessReapingTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerProcessReapingTest.java
@@ -196,7 +196,7 @@ public class TaskManagerProcessReapingTest {
 				cfg.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost");
 				cfg.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerPort);
 				cfg.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 4);
-				cfg.setInteger(ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, 100);
+				cfg.setInteger(ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, 256);
 
 				TaskManager.runTaskManager("localhost", taskManagerPort, cfg);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
index dbc6e9d..c41e6b9 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
@@ -29,16 +29,18 @@ import akka.util.Timeout;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.blob.BlobKey;
-import org.apache.flink.runtime.deployment.PartitionConsumerDeploymentDescriptor;
-import org.apache.flink.runtime.deployment.PartitionDeploymentDescriptor;
-import org.apache.flink.runtime.deployment.PartitionInfo;
+import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor;
+import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor;
+import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor;
+import org.apache.flink.runtime.deployment.ResultPartitionLocation;
 import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
 import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
-import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionType;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
@@ -101,8 +103,8 @@ public class TaskManagerTest {
 
 				final TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor(jid, vid, eid, "TestTask", 2, 7,
 						new Configuration(), new Configuration(), TestInvokableCorrect.class.getName(),
-						Collections.<PartitionDeploymentDescriptor>emptyList(),
-						Collections.<PartitionConsumerDeploymentDescriptor>emptyList(),
+						Collections.<ResultPartitionDeploymentDescriptor>emptyList(),
+						Collections.<InputGateDeploymentDescriptor>emptyList(),
 					new ArrayList<BlobKey>(), 0);
 
 				new Within(duration("1 seconds")){
@@ -140,14 +142,14 @@ public class TaskManagerTest {
 
 				final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid1, vid1, eid1, "TestTask1", 1, 5,
 						new Configuration(), new Configuration(), TestInvokableBlockingCancelable.class.getName(),
-						Collections.<PartitionDeploymentDescriptor>emptyList(),
-						Collections.<PartitionConsumerDeploymentDescriptor>emptyList(),
+						Collections.<ResultPartitionDeploymentDescriptor>emptyList(),
+						Collections.<InputGateDeploymentDescriptor>emptyList(),
 					new ArrayList<BlobKey>(), 0);
 
 				final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid2, vid2, eid2, "TestTask2", 2, 7,
 						new Configuration(), new Configuration(), TestInvokableBlockingCancelable.class.getName(),
-						Collections.<PartitionDeploymentDescriptor>emptyList(),
-						Collections.<PartitionConsumerDeploymentDescriptor>emptyList(),
+						Collections.<ResultPartitionDeploymentDescriptor>emptyList(),
+						Collections.<InputGateDeploymentDescriptor>emptyList(),
 					new ArrayList<BlobKey>(), 0);
 
 				final FiniteDuration d = duration("1 second");
@@ -243,14 +245,14 @@ public class TaskManagerTest {
 
 				final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid, vid1, eid1, "Sender", 0, 1,
 						new Configuration(), new Configuration(), Tasks.Sender.class.getName(),
-						Collections.<PartitionDeploymentDescriptor>emptyList(),
-						Collections.<PartitionConsumerDeploymentDescriptor>emptyList(),
+						Collections.<ResultPartitionDeploymentDescriptor>emptyList(),
+						Collections.<InputGateDeploymentDescriptor>emptyList(),
 					new ArrayList<BlobKey>(), 0);
 
 				final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid, vid2, eid2, "Receiver", 2, 7,
 						new Configuration(), new Configuration(), Tasks.Receiver.class.getName(),
-						Collections.<PartitionDeploymentDescriptor>emptyList(),
-						Collections.<PartitionConsumerDeploymentDescriptor>emptyList(),
+						Collections.<ResultPartitionDeploymentDescriptor>emptyList(),
+						Collections.<InputGateDeploymentDescriptor>emptyList(),
 					new ArrayList<BlobKey>(), 0);
 
 				new Within(duration("1 second")){
@@ -310,24 +312,24 @@ public class TaskManagerTest {
 
 			IntermediateResultPartitionID partitionId = new IntermediateResultPartitionID();
 
-			List<PartitionDeploymentDescriptor> irpdd = new ArrayList<PartitionDeploymentDescriptor>();
-			irpdd.add(new PartitionDeploymentDescriptor(new IntermediateDataSetID(), partitionId, IntermediateResultPartitionType.PIPELINED, 1));
+			List<ResultPartitionDeploymentDescriptor> irpdd = new ArrayList<ResultPartitionDeploymentDescriptor>();
+			irpdd.add(new ResultPartitionDeploymentDescriptor(new IntermediateDataSetID(), partitionId, ResultPartitionType.PIPELINED, 1));
 
-			PartitionConsumerDeploymentDescriptor ircdd =
-					new PartitionConsumerDeploymentDescriptor(
+			InputGateDeploymentDescriptor ircdd =
+					new InputGateDeploymentDescriptor(
 							new IntermediateDataSetID(),
-							new PartitionInfo[]{
-									new PartitionInfo(partitionId, eid1, PartitionInfo.PartitionLocation.LOCAL, null)
-							},
-							0);
+							0, new InputChannelDeploymentDescriptor[]{
+									new InputChannelDeploymentDescriptor(new ResultPartitionID(partitionId, eid1), ResultPartitionLocation.createLocal())
+							}
+					);
 
 			final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid, vid1, eid1, "Sender", 0, 1,
 					new Configuration(), new Configuration(), Tasks.Sender.class.getName(),
-					irpdd, Collections.<PartitionConsumerDeploymentDescriptor>emptyList(), new ArrayList<BlobKey>(), 0);
+					irpdd, Collections.<InputGateDeploymentDescriptor>emptyList(), new ArrayList<BlobKey>(), 0);
 
 			final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid, vid2, eid2, "Receiver", 2, 7,
 					new Configuration(), new Configuration(), Tasks.Receiver.class.getName(),
-					Collections.<PartitionDeploymentDescriptor>emptyList(),
+					Collections.<ResultPartitionDeploymentDescriptor>emptyList(),
 					Collections.singletonList(ircdd),
 					new ArrayList<BlobKey>(), 0);
 
@@ -402,25 +404,25 @@ public class TaskManagerTest {
 
 			IntermediateResultPartitionID partitionId = new IntermediateResultPartitionID();
 
-			List<PartitionDeploymentDescriptor> irpdd = new ArrayList<PartitionDeploymentDescriptor>();
-			irpdd.add(new PartitionDeploymentDescriptor(new IntermediateDataSetID(), partitionId, IntermediateResultPartitionType.PIPELINED, 1));
+			List<ResultPartitionDeploymentDescriptor> irpdd = new ArrayList<ResultPartitionDeploymentDescriptor>();
+			irpdd.add(new ResultPartitionDeploymentDescriptor(new IntermediateDataSetID(), partitionId, ResultPartitionType.PIPELINED, 1));
 
-			PartitionConsumerDeploymentDescriptor ircdd =
-					new PartitionConsumerDeploymentDescriptor(
+			InputGateDeploymentDescriptor ircdd =
+					new InputGateDeploymentDescriptor(
 							new IntermediateDataSetID(),
-							new PartitionInfo[]{
-									new PartitionInfo(partitionId, eid1, PartitionInfo.PartitionLocation.LOCAL, null)
-							},
-							0);
+							0, new InputChannelDeploymentDescriptor[]{
+									new InputChannelDeploymentDescriptor(new ResultPartitionID(partitionId, eid1), ResultPartitionLocation.createLocal())
+							}
+					);
 
 			final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid, vid1, eid1, "Sender", 0, 1,
 					new Configuration(), new Configuration(), Tasks.Sender.class.getName(),
-					irpdd, Collections.<PartitionConsumerDeploymentDescriptor>emptyList(),
+					irpdd, Collections.<InputGateDeploymentDescriptor>emptyList(),
 					new ArrayList<BlobKey>(), 0);
 
 			final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid, vid2, eid2, "Receiver", 2, 7,
 					new Configuration(), new Configuration(), Tasks.BlockingReceiver.class.getName(),
-					Collections.<PartitionDeploymentDescriptor>emptyList(),
+					Collections.<ResultPartitionDeploymentDescriptor>emptyList(),
 					Collections.singletonList(ircdd),
 					new ArrayList<BlobKey>(), 0);
 


[03/13] flink git commit: [FLINK-1350] [runtime] Add blocking result partition variant

Posted by uc...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
index 007ad5e..68b0e6f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
@@ -22,8 +22,8 @@ import akka.actor.ActorRef;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.blob.BlobKey;
 import org.apache.flink.runtime.broadcast.BroadcastVariableManager;
-import org.apache.flink.runtime.deployment.PartitionConsumerDeploymentDescriptor;
-import org.apache.flink.runtime.deployment.PartitionDeploymentDescriptor;
+import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor;
+import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor;
 import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
 import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.execution.RuntimeEnvironment;
@@ -37,7 +37,6 @@ import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
 import org.apache.flink.runtime.memorymanager.MemoryManager;
 import org.apache.flink.util.ExceptionUtils;
 import org.junit.Test;
-import org.mockito.Matchers;
 
 import java.util.ArrayList;
 import java.util.Collections;
@@ -51,7 +50,6 @@ import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.doNothing;
 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;
 
@@ -67,7 +65,7 @@ public class TaskTest {
 			final RuntimeEnvironment env = mock(RuntimeEnvironment.class);
 			
 			Task task = spy(new Task(jid, vid, 2, 7, eid, "TestTask", ActorRef.noSender()));
-			doNothing().when(task).notifyExecutionStateChange(any(ExecutionState.class), any(Throwable.class));
+			doNothing().when(task).unregisterTask();
 			task.setEnvironment(env);
 			
 			assertEquals(ExecutionState.DEPLOYING, task.getExecutionState());
@@ -86,8 +84,8 @@ public class TaskTest {
 			
 			task.markFailed(new Exception("test"));
 			assertTrue(ExecutionState.CANCELED == task.getExecutionState());
-			
-			verify(task, times(1)).notifyExecutionStateChange(ExecutionState.CANCELED, null);
+
+			verify(task).unregisterTask();
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -103,11 +101,12 @@ public class TaskTest {
 			final ExecutionAttemptID eid = new ExecutionAttemptID();
 			
 			final Task task = spy(new Task(jid, vid, 2, 7, eid, "TestTask", ActorRef.noSender()));
-			doNothing().when(task).notifyExecutionStateChange(any(ExecutionState.class), any(Throwable.class));
+			doNothing().when(task).unregisterTask();
 			
 			final AtomicReference<Throwable> error = new AtomicReference<Throwable>();
 			
 			Thread operation = new Thread() {
+				@Override
 				public void run() {
 					try {
 						assertTrue(task.markAsFinished());
@@ -135,8 +134,8 @@ public class TaskTest {
 			}
 			
 			assertEquals(ExecutionState.FINISHED, task.getExecutionState());
-			
-			verify(task).notifyExecutionStateChange(ExecutionState.FINISHED, null);
+
+			verify(task).unregisterTask();
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -152,11 +151,12 @@ public class TaskTest {
 			final ExecutionAttemptID eid = new ExecutionAttemptID();
 			
 			final Task task = spy(new Task(jid, vid, 2, 7, eid, "TestTask", ActorRef.noSender()));
-			doNothing().when(task).notifyExecutionStateChange(any(ExecutionState.class), any(Throwable.class));
-			
+			doNothing().when(task).unregisterTask();
+
 			final AtomicReference<Throwable> error = new AtomicReference<Throwable>();
 			
 			Thread operation = new Thread() {
+				@Override
 				public void run() {
 					try {
 						task.markFailed(new Exception("test exception message"));
@@ -185,7 +185,7 @@ public class TaskTest {
 			
 			// make sure the final state is correct and the task manager knows the changes
 			assertEquals(ExecutionState.FAILED, task.getExecutionState());
-			verify(task).notifyExecutionStateChange(Matchers.eq(ExecutionState.FAILED), any(Throwable.class));
+			verify(task).unregisterTask();
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -201,7 +201,7 @@ public class TaskTest {
 			final ExecutionAttemptID eid = new ExecutionAttemptID();
 
 			final Task task = spy(new Task(jid, vid, 2, 7, eid, "TestTask", ActorRef.noSender()));
-			doNothing().when(task).notifyExecutionStateChange(any(ExecutionState.class), any(Throwable.class));
+			doNothing().when(task).unregisterTask();
 			
 			final AtomicReference<Throwable> error = new AtomicReference<Throwable>();
 			
@@ -210,6 +210,7 @@ public class TaskTest {
 			final OneShotLatch afterCanceling = new OneShotLatch();
 			
 			Thread operation = new Thread() {
+				@Override
 				public void run() {
 					try {
 						toRunning.trigger();
@@ -245,7 +246,7 @@ public class TaskTest {
 			
 			// make sure the final state is correct and the task manager knows the changes
 			assertEquals(ExecutionState.CANCELED, task.getExecutionState());
-			verify(task).notifyExecutionStateChange(ExecutionState.CANCELED, null);
+			verify(task).unregisterTask();
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -262,12 +263,12 @@ public class TaskTest {
 			
 			TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor(jid, vid, eid, "TestTask", 2, 7,
 					new Configuration(), new Configuration(), TestInvokableCorrect.class.getName(),
-					Collections.<PartitionDeploymentDescriptor>emptyList(),
-					Collections.<PartitionConsumerDeploymentDescriptor>emptyList(),
+					Collections.<ResultPartitionDeploymentDescriptor>emptyList(),
+					Collections.<InputGateDeploymentDescriptor>emptyList(),
 					new ArrayList<BlobKey>(), 0);
 			
 			Task task = spy(new Task(jid, vid, 2, 7, eid, "TestTask", ActorRef.noSender()));
-			doNothing().when(task).notifyExecutionStateChange(any(ExecutionState.class), any(Throwable.class));
+			doNothing().when(task).unregisterTask();
 			
 			RuntimeEnvironment env = new RuntimeEnvironment(mock(ActorRef.class), task, tdd, getClass().getClassLoader(),
 					mock(MemoryManager.class), mock(IOManager.class), mock(InputSplitProvider.class),
@@ -281,8 +282,8 @@ public class TaskTest {
 			task.getEnvironment().getExecutingThread().join();
 			
 			assertEquals(ExecutionState.FINISHED, task.getExecutionState());
-			
-			verify(task).notifyExecutionStateChange(ExecutionState.FINISHED, null);
+
+			verify(task).unregisterTask();
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -299,12 +300,12 @@ public class TaskTest {
 			
 			TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor(jid, vid, eid, "TestTask", 2, 7,
 					new Configuration(), new Configuration(), TestInvokableWithException.class.getName(),
-					Collections.<PartitionDeploymentDescriptor>emptyList(),
-					Collections.<PartitionConsumerDeploymentDescriptor>emptyList(),
+					Collections.<ResultPartitionDeploymentDescriptor>emptyList(),
+					Collections.<InputGateDeploymentDescriptor>emptyList(),
 					new ArrayList<BlobKey>(), 0);
 			
 			Task task = spy(new Task(jid, vid, 2, 7, eid, "TestTask", ActorRef.noSender()));
-			doNothing().when(task).notifyExecutionStateChange(any(ExecutionState.class), any(Throwable.class));
+			doNothing().when(task).unregisterTask();
 			
 			RuntimeEnvironment env = new RuntimeEnvironment(mock(ActorRef.class), task, tdd, getClass().getClassLoader(),
 					mock(MemoryManager.class), mock(IOManager.class), mock(InputSplitProvider.class),
@@ -318,11 +319,8 @@ public class TaskTest {
 			task.getEnvironment().getExecutingThread().join();
 			
 			assertEquals(ExecutionState.FAILED, task.getExecutionState());
-			
-			verify(task).notifyExecutionStateChange(Matchers.eq(ExecutionState.FAILED), any(Throwable.class));
-			verify(task, times(0)).notifyExecutionStateChange(ExecutionState.CANCELING, null);
-			verify(task, times(0)).notifyExecutionStateChange(ExecutionState.CANCELED, null);
-			verify(task, times(0)).notifyExecutionStateChange(ExecutionState.FINISHED, null);
+
+			verify(task).unregisterTask();
 		}
 		catch (Exception e) {
 			e.printStackTrace();

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/DiscardingRecycler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/DiscardingRecycler.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/DiscardingRecycler.java
index d9bd232..466d1d6 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/DiscardingRecycler.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/DiscardingRecycler.java
@@ -24,5 +24,7 @@ import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
 public class DiscardingRecycler implements BufferRecycler {
 
 	@Override
-	public void recycle(MemorySegment memSeg) {}
+	public void recycle(MemorySegment memSeg) {
+		memSeg.free();
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/util/AtomicDisposableReferenceCounterTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/AtomicDisposableReferenceCounterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/AtomicDisposableReferenceCounterTest.java
index a2587a0..e016a50 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/AtomicDisposableReferenceCounterTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/AtomicDisposableReferenceCounterTest.java
@@ -37,13 +37,26 @@ public class AtomicDisposableReferenceCounterTest {
 	public void testSerialIncrementAndDecrement() {
 		AtomicDisposableReferenceCounter counter = new AtomicDisposableReferenceCounter();
 
-		assertTrue(counter.incrementReferenceCounter());
+		assertTrue(counter.increment());
 
-		assertTrue(counter.decrementReferenceCounter());
+		assertTrue(counter.decrement());
 
-		assertFalse(counter.incrementReferenceCounter());
+		assertFalse(counter.increment());
 
-		assertFalse(counter.decrementReferenceCounter());
+		assertFalse(counter.decrement());
+	}
+
+	@Test
+	public void testSerialIncrementAndDecrementWithCustomDisposeCount() {
+		AtomicDisposableReferenceCounter counter = new AtomicDisposableReferenceCounter(-2);
+
+		assertTrue(counter.increment());
+
+		assertFalse(counter.decrement());
+
+		assertFalse(counter.decrement());
+
+		assertTrue(counter.decrement());
 	}
 
 	@Test
@@ -63,7 +76,7 @@ public class AtomicDisposableReferenceCounterTest {
 				incrementer.setCounter(counter);
 				decrementer.setCounter(counter);
 
-				counter.incrementReferenceCounter();
+				counter.increment();
 
 				// Randomly decide which one should be first as the first task usually will win the race
 				boolean incrementFirst = random.nextBoolean();
@@ -90,7 +103,7 @@ public class AtomicDisposableReferenceCounterTest {
 
 		@Override
 		public Boolean call() throws Exception {
-			return counter.incrementReferenceCounter();
+			return counter.increment();
 		}
 	}
 
@@ -104,7 +117,7 @@ public class AtomicDisposableReferenceCounterTest {
 
 		@Override
 		public Boolean call() throws Exception {
-			return counter.decrementReferenceCounter();
+			return counter.decrement();
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/StreamRecordWriter.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/StreamRecordWriter.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/StreamRecordWriter.java
index b7af589..be9adf4 100755
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/StreamRecordWriter.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/StreamRecordWriter.java
@@ -18,7 +18,7 @@
 package org.apache.flink.streaming.io;
 
 import org.apache.flink.core.io.IOReadableWritable;
-import org.apache.flink.runtime.io.network.api.writer.BufferWriter;
+import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
 import org.apache.flink.runtime.io.network.api.writer.ChannelSelector;
 import org.apache.flink.runtime.io.network.api.writer.RecordWriter;
 import org.apache.flink.runtime.io.network.api.writer.RoundRobinChannelSelector;
@@ -31,15 +31,15 @@ public class StreamRecordWriter<T extends IOReadableWritable> extends RecordWrit
 
 	private OutputFlusher outputFlusher;
 
-	public StreamRecordWriter(BufferWriter writer) {
+	public StreamRecordWriter(ResultPartitionWriter writer) {
 		this(writer, new RoundRobinChannelSelector<T>(), 1000);
 	}
 
-	public StreamRecordWriter(BufferWriter writer, ChannelSelector<T> channelSelector) {
+	public StreamRecordWriter(ResultPartitionWriter writer, ChannelSelector<T> channelSelector) {
 		this(writer, channelSelector, 1000);
 	}
 
-	public StreamRecordWriter(BufferWriter writer, ChannelSelector<T> channelSelector, long timeout) {
+	public StreamRecordWriter(ResultPartitionWriter writer, ChannelSelector<T> channelSelector, long timeout) {
 		super(writer, channelSelector);
 
 		this.timeout = timeout;
@@ -70,7 +70,7 @@ public class StreamRecordWriter<T extends IOReadableWritable> extends RecordWrit
 		}
 		@Override
 		public void run() {
-			while (running && !writer.isFinished()) {
+			while (running) {
 				try {
 					flush();
 					Thread.sleep(timeout);

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/StreamingAbstractRecordReader.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/StreamingAbstractRecordReader.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/StreamingAbstractRecordReader.java
index d30c241..8296010 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/StreamingAbstractRecordReader.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/StreamingAbstractRecordReader.java
@@ -26,10 +26,10 @@ import org.apache.flink.runtime.io.network.api.reader.AbstractReader;
 import org.apache.flink.runtime.io.network.api.reader.ReaderBase;
 import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer;
 import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer.DeserializationResult;
+import org.apache.flink.runtime.io.network.api.serialization.SpillingAdaptiveSpanningRecordDeserializer;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
 import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
 import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
-import org.apache.flink.runtime.io.network.serialization.SpillingAdaptiveSpanningRecordDeserializer;
 import org.apache.flink.streaming.api.streamvertex.StreamingSuperstep;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java
----------------------------------------------------------------------
diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java b/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java
index 71b774f..6e86896 100644
--- a/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java
+++ b/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java
@@ -34,6 +34,14 @@ public abstract class JavaProgramTestBase extends AbstractTestBase {
 	private JobExecutionResult latestExecutionResult;
 	
 	private int degreeOfParallelism = DEFAULT_DEGREE_OF_PARALLELISM;
+
+	/**
+	 * The number of times a test should be repeated.
+	 *
+	 * <p> This is useful for runtime changes, which affect resource management. Running certain
+	 * tests repeatedly might help to discover resource leaks, race conditions etc.
+	 */
+	private int numberOfTestRepetitions = 1;
 	
 	private boolean isCollectionExecution;
 
@@ -50,6 +58,10 @@ public abstract class JavaProgramTestBase extends AbstractTestBase {
 		this.degreeOfParallelism = degreeOfParallelism;
 		setTaskManagerNumSlots(degreeOfParallelism);
 	}
+
+	public void setNumberOfTestRepetitions(int numberOfTestRepetitions) {
+		this.numberOfTestRepetitions = numberOfTestRepetitions;
+	}
 	
 	public int getDegreeOfParallelism() {
 		return isCollectionExecution ? 1 : degreeOfParallelism;
@@ -101,20 +113,24 @@ public abstract class JavaProgramTestBase extends AbstractTestBase {
 			TestEnvironment env = new TestEnvironment(this.executor, this.degreeOfParallelism);
 			env.getConfig().enableObjectReuse();
 			env.setAsContext();
-			
-			// call the test program
-			try {
-				testProgram();
-				this.latestExecutionResult = env.latestResult;
-			}
-			catch (Exception e) {
-				System.err.println(e.getMessage());
-				e.printStackTrace();
-				Assert.fail("Error while calling the test program: " + e.getMessage());
+
+			// Possibly run the test multiple times
+			for (int i = 0; i < numberOfTestRepetitions; i++) {
+				// call the test program
+				try {
+					testProgram();
+					this.latestExecutionResult = env.latestResult;
+				}
+				catch (Exception e) {
+					System.err.println(e.getMessage());
+					e.printStackTrace();
+					Assert.fail("Error while calling the test program: " + e.getMessage());
+				}
+
+				Assert.assertNotNull("The test program never triggered an execution.",
+						this.latestExecutionResult);
 			}
 			
-			Assert.assertNotNull("The test program never triggered an execution.", this.latestExecutionResult);
-			
 			// post-submit
 			try {
 				postSubmit();
@@ -150,18 +166,22 @@ public abstract class JavaProgramTestBase extends AbstractTestBase {
 			env.getConfig().disableObjectReuse();
 			env.setAsContext();
 
-			// call the test program
-			try {
-				testProgram();
-				this.latestExecutionResult = env.latestResult;
-			}
-			catch (Exception e) {
-				System.err.println(e.getMessage());
-				e.printStackTrace();
-				Assert.fail("Error while calling the test program: " + e.getMessage());
-			}
+			// Possibly run the test multiple times
+			for (int i = 0; i < numberOfTestRepetitions; i++) {
+				// call the test program
+				try {
+					testProgram();
+					this.latestExecutionResult = env.latestResult;
+				}
+				catch (Exception e) {
+					System.err.println(e.getMessage());
+					e.printStackTrace();
+					Assert.fail("Error while calling the test program: " + e.getMessage());
+				}
 
-			Assert.assertNotNull("The test program never triggered an execution.", this.latestExecutionResult);
+				Assert.assertNotNull("The test program never triggered an execution.",
+						this.latestExecutionResult);
+			}
 
 			// post-submit
 			try {
@@ -246,7 +266,7 @@ public abstract class JavaProgramTestBase extends AbstractTestBase {
 						Assert.fail("Cannot compare tuple fields");
 					}
 					
-					int cmp = ((Comparable<Object>) obj1).compareTo((Comparable<Object>) obj2);
+					int cmp = ((Comparable<Object>) obj1).compareTo(obj2);
 					if (cmp != 0) {
 						return cmp;
 					}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountITCase.java
index bd4e63a..b88eb4e 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountITCase.java
@@ -28,9 +28,9 @@ public class WordCountITCase extends JavaProgramTestBase {
 	protected String resultPath;
 
 	public WordCountITCase(){
-//		setDegreeOfParallelism(4);
-//		setNumTaskManagers(2);
-//		setTaskManagerNumSlots(2);
+		setDegreeOfParallelism(4);
+		setNumTaskManagers(2);
+		setTaskManagerNumSlots(2);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-tests/src/test/java/org/apache/flink/test/iterative/KMeansForTestITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/KMeansForTestITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/KMeansForTestITCase.java
new file mode 100644
index 0000000..732bd06
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/KMeansForTestITCase.java
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.test.iterative;
+
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichReduceFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.operators.IterativeDataSet;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.test.localDistributed.PackagedProgramEndToEndITCase;
+import org.apache.flink.test.testdata.KMeansData;
+import org.apache.flink.test.util.JavaProgramTestBase;
+import org.apache.flink.test.util.testjar.KMeansForTest;
+
+import java.io.Serializable;
+import java.util.Collection;
+
+/**
+ * This K-Means is a copy of {@link KMeansForTest} from the {@link PackagedProgramEndToEndITCase},
+ * which detected a problem with the wiring of blocking intermediate results reproducibly with
+ * multiple runs, whereas other tests didn't.
+ *
+ * <p> The code is copied here, because the packaged program test removes the classes from the
+ * classpath.
+ *
+ * <p> It's safe to remove this test in the future.
+ */
+public class KMeansForTestITCase extends JavaProgramTestBase {
+
+	protected String dataPath;
+	protected String clusterPath;
+	protected String resultPath;
+
+	public KMeansForTestITCase(){
+		setNumTaskManagers(2);
+		setTaskManagerNumSlots(2);
+		setNumberOfTestRepetitions(10);
+	}
+
+	@Override
+	protected void preSubmit() throws Exception {
+		dataPath = createTempFile("datapoints.txt", KMeansData.DATAPOINTS);
+		clusterPath = createTempFile("initial_centers.txt", KMeansData.INITIAL_CENTERS);
+		resultPath = getTempDirPath("result");
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		int numIterations = 20;
+
+		// get input data
+		DataSet<Point> points = env.readCsvFile(dataPath)
+				.fieldDelimiter("|")
+				.includeFields(true, true)
+				.types(Double.class, Double.class)
+				.map(new TuplePointConverter());
+
+		DataSet<Centroid> centroids = env.readCsvFile(clusterPath)
+				.fieldDelimiter("|")
+				.includeFields(true, true, true)
+				.types(Integer.class, Double.class, Double.class)
+				.map(new TupleCentroidConverter());
+
+		// set number of bulk iterations for KMeans algorithm
+		IterativeDataSet<Centroid> loop = centroids.iterate(numIterations);
+
+		DataSet<Centroid> newCentroids = points
+				// compute closest centroid for each point
+				.map(new SelectNearestCenter()).withBroadcastSet(loop, "centroids")
+						// count and sum point coordinates for each centroid
+				.map(new CountAppender())
+						// !test if key expressions are working!
+				.groupBy("field0").reduce(new CentroidAccumulator())
+						// compute new centroids from point counts and coordinate sums
+				.map(new CentroidAverager());
+
+		// feed new centroids back into next iteration
+		DataSet<Centroid> finalCentroids = loop.closeWith(newCentroids);
+
+		DataSet<Tuple2<Integer, Point>> clusteredPoints = points
+				// assign points to final clusters
+				.map(new SelectNearestCenter()).withBroadcastSet(finalCentroids, "centroids");
+
+		// emit result
+		clusteredPoints.writeAsCsv(resultPath, "\n", " ");
+
+		env.execute("KMeansForTest");
+	}
+
+	// *************************************************************************
+	//     DATA TYPES
+	// *************************************************************************
+
+	/**
+	 * A simple two-dimensional point.
+	 */
+	public static class Point implements Serializable {
+
+		public double x, y;
+
+		public Point() {}
+
+		public Point(double x, double y) {
+			this.x = x;
+			this.y = y;
+		}
+
+		public Point add(Point other) {
+			x += other.x;
+			y += other.y;
+			return this;
+		}
+
+		public Point div(long val) {
+			x /= val;
+			y /= val;
+			return this;
+		}
+
+		public double euclideanDistance(Point other) {
+			return Math.sqrt((x-other.x)*(x-other.x) + (y-other.y)*(y-other.y));
+		}
+
+		public void clear() {
+			x = y = 0.0;
+		}
+
+		@Override
+		public String toString() {
+			return x + " " + y;
+		}
+	}
+
+	/**
+	 * A simple two-dimensional centroid, basically a point with an ID.
+	 */
+	public static class Centroid extends Point {
+
+		public int id;
+
+		public Centroid() {}
+
+		public Centroid(int id, double x, double y) {
+			super(x,y);
+			this.id = id;
+		}
+
+		public Centroid(int id, Point p) {
+			super(p.x, p.y);
+			this.id = id;
+		}
+
+		@Override
+		public String toString() {
+			return id + " " + super.toString();
+		}
+	}
+
+	// *************************************************************************
+	//     USER FUNCTIONS
+	// *************************************************************************
+
+	/** Converts a Tuple2<Double,Double> into a Point. */
+	public static final class TuplePointConverter extends RichMapFunction<Tuple2<Double, Double>, Point> {
+
+		@Override
+		public Point map(Tuple2<Double, Double> t) throws Exception {
+			return new Point(t.f0, t.f1);
+		}
+	}
+
+	/** Converts a Tuple3<Integer, Double,Double> into a Centroid. */
+	public static final class TupleCentroidConverter extends RichMapFunction<Tuple3<Integer, Double, Double>, Centroid> {
+
+		@Override
+		public Centroid map(Tuple3<Integer, Double, Double> t) throws Exception {
+			return new Centroid(t.f0, t.f1, t.f2);
+		}
+	}
+
+	/** Determines the closest cluster center for a data point. */
+	public static final class SelectNearestCenter extends RichMapFunction<Point, Tuple2<Integer, Point>> {
+		private Collection<Centroid> centroids;
+
+		/** Reads the centroid values from a broadcast variable into a collection. */
+		@Override
+		public void open(Configuration parameters) throws Exception {
+			this.centroids = getRuntimeContext().getBroadcastVariable("centroids");
+		}
+
+		@Override
+		public Tuple2<Integer, Point> map(Point p) throws Exception {
+
+			double minDistance = Double.MAX_VALUE;
+			int closestCentroidId = -1;
+
+			// check all cluster centers
+			for (Centroid centroid : centroids) {
+				// compute distance
+				double distance = p.euclideanDistance(centroid);
+
+				// update nearest cluster if necessary
+				if (distance < minDistance) {
+					minDistance = distance;
+					closestCentroidId = centroid.id;
+				}
+			}
+
+			// emit a new record with the center id and the data point.
+			return new Tuple2<Integer, Point>(closestCentroidId, p);
+		}
+	}
+
+	// Use this so that we can check whether POJOs and the POJO comparator also work
+	public static final class DummyTuple3IntPointLong {
+		public Integer field0;
+		public Point field1;
+		public Long field2;
+
+		public DummyTuple3IntPointLong() {}
+
+		DummyTuple3IntPointLong(Integer f0, Point f1, Long f2) {
+			this.field0 = f0;
+			this.field1 = f1;
+			this.field2 = f2;
+		}
+	}
+
+	/** Appends a count variable to the tuple. */
+	public static final class CountAppender extends RichMapFunction<Tuple2<Integer, Point>, DummyTuple3IntPointLong> {
+
+		@Override
+		public DummyTuple3IntPointLong map(Tuple2<Integer, Point> t) {
+			return new DummyTuple3IntPointLong(t.f0, t.f1, 1L);
+		}
+	}
+
+	/** Sums and counts point coordinates. */
+	public static final class CentroidAccumulator extends RichReduceFunction<DummyTuple3IntPointLong> {
+
+		@Override
+		public DummyTuple3IntPointLong reduce(DummyTuple3IntPointLong val1, DummyTuple3IntPointLong val2) {
+			return new DummyTuple3IntPointLong(val1.field0, val1.field1.add(val2.field1), val1.field2 + val2.field2);
+		}
+	}
+
+	/** Computes new centroid from coordinate sum and count of points. */
+	public static final class CentroidAverager extends RichMapFunction<DummyTuple3IntPointLong, Centroid> {
+
+		@Override
+		public Centroid map(DummyTuple3IntPointLong value) {
+			return new Centroid(value.field0, value.field1.div(value.field2));
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureBatchRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureBatchRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureBatchRecoveryITCase.java
index 8cbd7ca..7bb094b 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureBatchRecoveryITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureBatchRecoveryITCase.java
@@ -409,7 +409,7 @@ public class ProcessFailureBatchRecoveryITCase {
 				cfg.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost");
 				cfg.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerPort);
 				cfg.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 4);
-				cfg.setInteger(ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, 100);
+				cfg.setInteger(ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, 256);
 				cfg.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 2);
 
 				TaskManager.runTaskManager(cfg, TaskManager.class);

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-tests/src/test/java/org/apache/flink/test/runtime/ConsumePipelinedAndBlockingResultITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/ConsumePipelinedAndBlockingResultITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/ConsumePipelinedAndBlockingResultITCase.java
new file mode 100644
index 0000000..007742b
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/ConsumePipelinedAndBlockingResultITCase.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.test.runtime;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.io.DiscardingOutputFormat;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+public class ConsumePipelinedAndBlockingResultITCase extends JavaProgramTestBase {
+
+	@Override
+	protected void testProgram() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setDegreeOfParallelism(1);
+
+		DataSet<Tuple1<Long>> pipelinedSource = env.fromElements(new Tuple1<Long>(1l));
+
+		DataSet<Tuple1<Long>> slowBlockingSource = env.generateSequence(0, 10).map(
+				new MapFunction<Long, Tuple1<Long>>() {
+					@Override
+					public Tuple1<Long> map(Long value) throws Exception {
+						Thread.sleep(200);
+
+						return new Tuple1<Long>(value);
+					}
+				}
+		);
+
+		slowBlockingSource.join(slowBlockingSource)
+				.where(0).equalTo(0).output(new DiscardingOutputFormat<Tuple2<Tuple1<Long>, Tuple1<Long>>>());
+
+		// Join the slow blocking and the pipelined source. This test should verify that this works
+		// w/o problems and the blocking result is not requested too early.
+		pipelinedSource.join(slowBlockingSource)
+				.where(0).equalTo(0)
+				.output(new DiscardingOutputFormat<Tuple2<Tuple1<Long>, Tuple1<Long>>>());
+
+		env.execute("Consume one pipelined and one blocking result test job");
+	}
+
+	@Override
+	protected boolean skipCollectionExecution() {
+		// Skip collection execution as it is independent of the runtime environment functionality,
+		// which is under test.
+		return true;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-tests/src/test/java/org/apache/flink/test/runtime/JoinDeadlockITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/JoinDeadlockITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/JoinDeadlockITCase.java
new file mode 100644
index 0000000..2c4dca3
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/JoinDeadlockITCase.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.test.runtime;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.test.util.JavaProgramTestBase;
+import org.junit.Rule;
+import org.junit.rules.Timeout;
+
+/**
+ * Tests a join, which leads to a deadlock with large data sizes and PIPELINED-only execution.
+ *
+ * @see <a href="https://issues.apache.org/jira/browse/FLINK-1343">FLINK-1343</a>
+ */
+public class JoinDeadlockITCase extends JavaProgramTestBase {
+
+	protected String resultPath;
+
+	@Rule
+	public Timeout globalTimeout = new Timeout(120 * 1000); // Set timeout for deadlocks
+
+	@Override
+	protected void preSubmit() throws Exception {
+		resultPath = getTempDirPath("result");
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Long> longs = env.generateSequence(0, 100000);
+
+		DataSet<Tuple1<Long>> longT1 = longs.map(new TupleWrapper());
+		DataSet<Tuple1<Long>> longT2 = longT1.project(0);
+		DataSet<Tuple1<Long>> longT3 = longs.map(new TupleWrapper());
+
+		longT2.join(longT3).where(0).equalTo(0).projectFirst(0)
+				.join(longT1).where(0).equalTo(0).projectFirst(0)
+				.writeAsText(resultPath);
+
+		env.execute();
+	}
+
+	public static class TupleWrapper implements MapFunction<Long, Tuple1<Long>> {
+
+		@Override
+		public Tuple1<Long> map(Long l) throws Exception {
+			return new Tuple1<Long>(l);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-tests/src/test/java/org/apache/flink/test/runtime/SelfJoinDeadlockITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/SelfJoinDeadlockITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/SelfJoinDeadlockITCase.java
new file mode 100644
index 0000000..7729138
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/SelfJoinDeadlockITCase.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.test.runtime;
+
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.io.GenericInputFormat;
+import org.apache.flink.api.common.io.NonParallelInput;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.test.util.JavaProgramTestBase;
+import org.apache.flink.util.Collector;
+import org.junit.Rule;
+import org.junit.rules.Timeout;
+
+import java.io.IOException;
+import java.util.Random;
+
+/**
+ * Tests a self-join, which leads to a deadlock with large data sizes and PIPELINED-only execution.
+ *
+ * @see <a href="https://issues.apache.org/jira/browse/FLINK-1141">FLINK-1141</a>
+ */
+public class SelfJoinDeadlockITCase extends JavaProgramTestBase {
+
+	protected String resultPath;
+
+	@Rule
+	public Timeout globalTimeout = new Timeout(120 * 1000); // Set timeout for deadlocks
+
+	@Override
+	protected void preSubmit() throws Exception {
+		resultPath = getTempDirPath("result");
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Integer, String>> ds = env.createInput(new LargeJoinDataGeneratorInputFormat(1000000));
+
+		ds.join(ds).where(0).equalTo(1).with(new Joiner()).writeAsText(resultPath);
+
+		env.execute("Local Selfjoin Test Job");
+	}
+
+	@SuppressWarnings("serial")
+	public static class Joiner implements FlatJoinFunction<Tuple3<Integer, Integer, String>, Tuple3<Integer, Integer, String>, Tuple5<Integer, Integer, Integer, String, String>> {
+
+		@Override
+		public void join(Tuple3<Integer, Integer, String> in1, Tuple3<Integer, Integer, String> in2, Collector<Tuple5<Integer, Integer, Integer, String, String>> out) throws Exception {
+			out.collect(new Tuple5<Integer, Integer, Integer, String, String>(in1.f0, in1.f1, in2.f1, in1.f2, in2.f2));
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	// Use custom input format to generate the data. Other available input formats (like collection
+	// input format) create data upfront and serialize it completely on the heap, which might
+	// break the test JVM heap sizes.
+	private static class LargeJoinDataGeneratorInputFormat extends GenericInputFormat<Tuple3<Integer, Integer, String>> implements NonParallelInput {
+
+		private static final long serialVersionUID = 1L;
+
+		private final Random rand = new Random(42);
+
+		private final int toProduce;
+
+		private int produced;
+
+		public LargeJoinDataGeneratorInputFormat(int toProduce) {
+			this.toProduce = toProduce;
+		}
+
+		@Override
+		public boolean reachedEnd() throws IOException {
+			return produced >= toProduce;
+		}
+
+		@Override
+		public Tuple3<Integer, Integer, String> nextRecord(Tuple3<Integer, Integer, String> reuse) throws IOException {
+			produced++;
+
+			return new Tuple3<Integer, Integer, String>(rand.nextInt(toProduce), rand.nextInt(toProduce), "aaa");
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-tests/src/test/java/org/apache/flink/test/util/testjar/KMeansForTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/util/testjar/KMeansForTest.java b/flink-tests/src/test/java/org/apache/flink/test/util/testjar/KMeansForTest.java
index 44447d8..784e824 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/util/testjar/KMeansForTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/util/testjar/KMeansForTest.java
@@ -16,27 +16,27 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.test.util.testjar;
 
-import java.io.Serializable;
-import java.util.Collection;
-
 import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.common.Program;
 import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.common.functions.RichReduceFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.operators.IterativeDataSet;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.test.localDistributed.PackagedProgramEndToEndITCase;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.operators.IterativeDataSet;
+
+import java.io.Serializable;
+import java.util.Collection;
 
 /**
- * This class belongs to the @see {@link PackagedProgramEndToEndITCase} test
+ * This class belongs to the {@link PackagedProgramEndToEndITCase} test.
  *
+ * <p> It's removed by Maven from classpath, so other tests must not depend on it.
  */
 @SuppressWarnings("serial")
 public class KMeansForTest implements Program {
@@ -45,8 +45,6 @@ public class KMeansForTest implements Program {
 	//     PROGRAM
 	// *************************************************************************
 
-
-
 	@Override
 	public Plan getPlan(String... args) {
 		if (args.length < 4) {
@@ -79,14 +77,14 @@ public class KMeansForTest implements Program {
 		IterativeDataSet<Centroid> loop = centroids.iterate(numIterations);
 
 		DataSet<Centroid> newCentroids = points
-			// compute closest centroid for each point
-			.map(new SelectNearestCenter()).withBroadcastSet(loop, "centroids")
-			// count and sum point coordinates for each centroid
-			.map(new CountAppender())
-			// !test if key expressions are working!
-			.groupBy("field0").reduce(new CentroidAccumulator())
-			// compute new centroids from point counts and coordinate sums
-			.map(new CentroidAverager());
+				// compute closest centroid for each point
+				.map(new SelectNearestCenter()).withBroadcastSet(loop, "centroids")
+						// count and sum point coordinates for each centroid
+				.map(new CountAppender())
+						// !test if key expressions are working!
+				.groupBy("field0").reduce(new CentroidAccumulator())
+						// compute new centroids from point counts and coordinate sums
+				.map(new CentroidAverager());
 
 		// feed new centroids back into next iteration
 		DataSet<Centroid> finalCentroids = loop.closeWith(newCentroids);


[13/13] flink git commit: [FLINK-1671] [optimizer] Fix comment typos

Posted by uc...@apache.org.
[FLINK-1671] [optimizer] Fix comment typos


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

Branch: refs/heads/master
Commit: 4a49a73a751822c0bca715e7b889c0a05d2a73bd
Parents: cf6c63c
Author: Ufuk Celebi <uc...@apache.org>
Authored: Tue Mar 17 10:47:27 2015 +0100
Committer: Ufuk Celebi <uc...@apache.org>
Committed: Wed Mar 18 17:44:41 2015 +0100

----------------------------------------------------------------------
 .../org/apache/flink/runtime/io/network/DataExchangeMode.java  | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/4a49a73a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/DataExchangeMode.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/DataExchangeMode.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/DataExchangeMode.java
index 7810c3e..c124383 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/DataExchangeMode.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/DataExchangeMode.java
@@ -34,12 +34,12 @@ public enum DataExchangeMode {
 
 	/**
 	 * The data exchange is decoupled. The sender first produces its entire result and finishes.
-	 * After that, the receiver is stated and may consume the data.
+	 * After that, the receiver is started and may consume the data.
 	 */
 	BATCH,
 
 	/**
-	 * The data exchange starts like ine {@link #PIPELINED} and falls back to {@link #BATCH}
+	 * The data exchange starts like in {@link #PIPELINED} and falls back to {@link #BATCH}
 	 * for recovery runs.
 	 */
 	PIPELINE_WITH_BATCH_FALLBACK;
@@ -59,7 +59,7 @@ public enum DataExchangeMode {
 	}
 
 	/**
-	 * Computes the mode of data exchange to be used for a, given an execution mode and ship strategy.
+	 * Computes the mode of data exchange to be used for a given execution mode and ship strategy.
 	 * The type of the data exchange depends also on whether this connection has been identified to require
 	 * pipeline breaking for deadlock avoidance.
 	 * <ul>


[02/13] flink git commit: [FLINK-1350] [runtime] Set result type to BLOCKING if data exchange mode is BATCH

Posted by uc...@apache.org.
[FLINK-1350] [runtime] Set result type to BLOCKING if data exchange mode is BATCH


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

Branch: refs/heads/master
Commit: 9c77f0785e43326521da5e535f9ab1f05a9c6280
Parents: 9d7acf3
Author: Ufuk Celebi <uc...@apache.org>
Authored: Tue Mar 17 10:42:19 2015 +0100
Committer: Ufuk Celebi <uc...@apache.org>
Committed: Wed Mar 18 17:44:40 2015 +0100

----------------------------------------------------------------------
 .../plantranslate/NepheleJobGraphGenerator.java | 52 +++++++++++++++-----
 .../executiongraph/ExecutionJobVertex.java      |  1 -
 .../runtime/jobgraph/AbstractJobVertex.java     | 27 +++++++---
 .../ExecutionGraphConstructionTest.java         | 15 +++---
 4 files changed, 67 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/9c77f078/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java b/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
index 052d439..9a47f79 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
@@ -18,16 +18,6 @@
 
 package org.apache.flink.compiler.plantranslate;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.aggregators.AggregatorRegistry;
 import org.apache.flink.api.common.aggregators.AggregatorWithName;
@@ -58,6 +48,8 @@ import org.apache.flink.compiler.plan.WorksetPlanNode;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.runtime.io.network.DataExchangeMode;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
 import org.apache.flink.runtime.iterative.convergence.WorksetEmptyConvergenceCriterion;
 import org.apache.flink.runtime.iterative.task.IterationHeadPactTask;
 import org.apache.flink.runtime.iterative.task.IterationIntermediatePactTask;
@@ -87,6 +79,16 @@ import org.apache.flink.runtime.operators.util.TaskConfig;
 import org.apache.flink.util.InstantiationUtil;
 import org.apache.flink.util.Visitor;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
 /**
  * This component translates the optimizer's resulting plan a nephele job graph. The
  * translation is a one to one mapping. All decisions are made by the optimizer, this class
@@ -1069,7 +1071,33 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 				throw new RuntimeException("Unknown runtime ship strategy: " + channel.getShipStrategy());
 		}
 
-		targetVertex.connectNewDataSetAsInput(sourceVertex, distributionPattern);
+		final ResultPartitionType resultType;
+
+		switch (channel.getDataExchangeMode()) {
+
+			case PIPELINED:
+				resultType = ResultPartitionType.PIPELINED;
+				break;
+
+			case BATCH:
+				// BLOCKING results are currently not supported in closed loop iterations
+				//
+				// See https://issues.apache.org/jira/browse/FLINK-1713 for details
+				resultType = channel.getSource().isOnDynamicPath()
+						? ResultPartitionType.PIPELINED
+						: ResultPartitionType.BLOCKING;
+				break;
+
+			case PIPELINE_WITH_BATCH_FALLBACK:
+				throw new UnsupportedOperationException("Data exchange mode " +
+						channel.getDataExchangeMode() + " currently not supported.");
+
+			default:
+				throw new UnsupportedOperationException("Unknown data exchange mode.");
+
+		}
+
+		targetVertex.connectNewDataSetAsInput(sourceVertex, distributionPattern, resultType);
 
 		// -------------- configure the source task's ship strategy strategies in task config --------------
 		final int outputIndex = sourceConfig.getNumOutputs();
@@ -1139,7 +1167,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 
 			boolean needsMemory = false;
 			// Don't add a pipeline breaker if the data exchange is already blocking.
-			if (tm.breaksPipeline()) {
+			if (tm.breaksPipeline() && channel.getDataExchangeMode() != DataExchangeMode.BATCH) {
 				config.setInputAsynchronouslyMaterialized(inputNum, true);
 				needsMemory = true;
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c77f078/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
index 6fdc628..ad72d13 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
@@ -45,7 +45,6 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-
 public class ExecutionJobVertex implements Serializable {
 	
 	private static final long serialVersionUID = 42L;

http://git-wip-us.apache.org/repos/asf/flink/blob/9c77f078/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
index 609ed3e..8816a69 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
@@ -18,9 +18,6 @@
 
 package org.apache.flink.runtime.jobgraph;
 
-import java.util.ArrayList;
-import java.util.List;
-
 import org.apache.commons.lang3.Validate;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.io.InputSplitSource;
@@ -29,6 +26,9 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
 
+import java.util.ArrayList;
+import java.util.List;
+
 /**
  * An abstract base class for a job vertex.
  */
@@ -316,12 +316,15 @@ public class AbstractJobVertex implements java.io.Serializable {
 	
 	// --------------------------------------------------------------------------------------------
 
-	public IntermediateDataSet createAndAddResultDataSet() {
-		return createAndAddResultDataSet(new IntermediateDataSetID());
+	public IntermediateDataSet createAndAddResultDataSet(ResultPartitionType partitionType) {
+		return createAndAddResultDataSet(new IntermediateDataSetID(), partitionType);
 	}
 
-	public IntermediateDataSet createAndAddResultDataSet(IntermediateDataSetID id) {
-		IntermediateDataSet result = new IntermediateDataSet(id, ResultPartitionType.PIPELINED, this);
+	public IntermediateDataSet createAndAddResultDataSet(
+			IntermediateDataSetID id,
+			ResultPartitionType partitionType) {
+
+		IntermediateDataSet result = new IntermediateDataSet(id, partitionType, this);
 		this.results.add(result);
 		return result;
 	}
@@ -333,7 +336,15 @@ public class AbstractJobVertex implements java.io.Serializable {
 	}
 
 	public void connectNewDataSetAsInput(AbstractJobVertex input, DistributionPattern distPattern) {
-		IntermediateDataSet dataSet = input.createAndAddResultDataSet();
+		connectNewDataSetAsInput(input, distPattern, ResultPartitionType.PIPELINED);
+	}
+
+	public void connectNewDataSetAsInput(
+			AbstractJobVertex input,
+			DistributionPattern distPattern,
+			ResultPartitionType partitionType) {
+
+		IntermediateDataSet dataSet = input.createAndAddResultDataSet(partitionType);
 		JobEdge edge = new JobEdge(dataSet, this, distPattern);
 		this.inputs.add(edge);
 		dataSet.addConsumer(edge);

http://git-wip-us.apache.org/repos/asf/flink/blob/9c77f078/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java
index b16109a..e0852c6 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java
@@ -27,6 +27,7 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
 import org.junit.Test;
 import org.mockito.Matchers;
 
@@ -129,9 +130,9 @@ public class ExecutionGraphConstructionTest {
 		v2.connectNewDataSetAsInput(v1, DistributionPattern.ALL_TO_ALL);
 		
 		// create results for v2 and v3
-		IntermediateDataSet v2result = v2.createAndAddResultDataSet();
-		IntermediateDataSet v3result_1 = v3.createAndAddResultDataSet();
-		IntermediateDataSet v3result_2 = v3.createAndAddResultDataSet();
+		IntermediateDataSet v2result = v2.createAndAddResultDataSet(ResultPartitionType.PIPELINED);
+		IntermediateDataSet v3result_1 = v3.createAndAddResultDataSet(ResultPartitionType.PIPELINED);
+		IntermediateDataSet v3result_2 = v3.createAndAddResultDataSet(ResultPartitionType.PIPELINED);
 		
 		
 		List<AbstractJobVertex> ordered = new ArrayList<AbstractJobVertex>(Arrays.asList(v1, v2, v3));
@@ -190,9 +191,9 @@ public class ExecutionGraphConstructionTest {
 		v2.connectNewDataSetAsInput(v1, DistributionPattern.ALL_TO_ALL);
 		
 		// create results for v2 and v3
-		IntermediateDataSet v2result = v2.createAndAddResultDataSet();
-		IntermediateDataSet v3result_1 = v3.createAndAddResultDataSet();
-		IntermediateDataSet v3result_2 = v3.createAndAddResultDataSet();
+		IntermediateDataSet v2result = v2.createAndAddResultDataSet(ResultPartitionType.PIPELINED);
+		IntermediateDataSet v3result_1 = v3.createAndAddResultDataSet(ResultPartitionType.PIPELINED);
+		IntermediateDataSet v3result_2 = v3.createAndAddResultDataSet(ResultPartitionType.PIPELINED);
 		
 		
 		List<AbstractJobVertex> ordered = new ArrayList<AbstractJobVertex>(Arrays.asList(v1, v2, v3));
@@ -584,7 +585,7 @@ public class ExecutionGraphConstructionTest {
 			v2.setParallelism(7);
 			v3.setParallelism(2);
 
-			IntermediateDataSet result = v1.createAndAddResultDataSet();
+			IntermediateDataSet result = v1.createAndAddResultDataSet(ResultPartitionType.PIPELINED);
 			v2.connectDataSetAsInput(result, DistributionPattern.ALL_TO_ALL);
 			v3.connectDataSetAsInput(result, DistributionPattern.ALL_TO_ALL);
 			


[09/13] flink git commit: [FLINK-1350] [runtime] Add blocking result partition variant

Posted by uc...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpillingAdaptiveSpanningRecordDeserializer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpillingAdaptiveSpanningRecordDeserializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpillingAdaptiveSpanningRecordDeserializer.java
new file mode 100644
index 0000000..eae3e65
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpillingAdaptiveSpanningRecordDeserializer.java
@@ -0,0 +1,637 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.api.serialization;
+
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.core.io.IOReadableWritable;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.InputViewDataInputStreamWrapper;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.util.DataInputDeserializer;
+import org.apache.flink.util.StringUtils;
+
+import java.io.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.EOFException;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.io.UTFDataFormatException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.channels.FileChannel;
+import java.util.Random;
+
+/**
+ * @param <T> The type of the record to be deserialized.
+ */
+public class SpillingAdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> implements RecordDeserializer<T> {
+	
+	private static final int THRESHOLD_FOR_SPILLING = 5 * 1024 * 1024; // 5 MiBytes
+	
+	private final NonSpanningWrapper nonSpanningWrapper;
+	
+	private final SpanningWrapper spanningWrapper;
+
+	private Buffer currentBuffer;
+
+	public SpillingAdaptiveSpanningRecordDeserializer() {
+		
+		String tempDirString = GlobalConfiguration.getString(
+				ConfigConstants.TASK_MANAGER_TMP_DIR_KEY,
+				ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH);
+		String[] directories = tempDirString.split(",|" + File.pathSeparator);
+		
+		this.nonSpanningWrapper = new NonSpanningWrapper();
+		this.spanningWrapper = new SpanningWrapper(directories);
+	}
+
+	@Override
+	public void setNextBuffer(Buffer buffer) throws IOException {
+		currentBuffer = buffer;
+
+		MemorySegment segment = buffer.getMemorySegment();
+		int numBytes = buffer.getSize();
+
+		setNextMemorySegment(segment, numBytes);
+	}
+
+	@Override
+	public Buffer getCurrentBuffer () {
+		Buffer tmp = currentBuffer;
+		currentBuffer = null;
+		return tmp;
+	}
+	
+	@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.getInputView());
+			
+			// 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 {
+			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 DataInputView {
+		
+		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;
+		}
+
+		@Override
+		public void skipBytesToRead(int numBytes) throws IOException {
+			int skippedBytes = skipBytes(numBytes);
+
+			if(skippedBytes < numBytes){
+				throw new EOFException("Could not skip " + numBytes + " bytes.");
+			}
+		}
+
+		@Override
+		public int read(byte[] b, int off, int len) throws IOException {
+			if(b == null){
+				throw new NullPointerException("Byte array b cannot be null.");
+			}
+
+			if(off < 0){
+				throw new IllegalArgumentException("The offset off cannot be negative.");
+			}
+
+			if(len < 0){
+				throw new IllegalArgumentException("The length len cannot be negative.");
+			}
+
+			int toRead = Math.min(len, remaining());
+			this.segment.get(this.position,b,off, toRead);
+			this.position += toRead;
+
+			return toRead;
+		}
+
+		@Override
+		public int read(byte[] b) throws IOException {
+			return read(b, 0, b.length);
+		}
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+	
+	private static final class SpanningWrapper {
+		
+		private final byte[] initialBuffer = new byte[1024];
+		
+		private final String[] tempDirs;
+		
+		private final Random rnd = new Random();
+
+		private final DataInputDeserializer serializationReadBuffer;
+
+		private final ByteBuffer lengthBuffer;
+		
+		private FileChannel spillingChannel;
+		
+		private byte[] buffer;
+
+		private int recordLength;
+		
+		private int accumulatedRecordBytes;
+
+		private MemorySegment leftOverData;
+
+		private int leftOverStart;
+
+		private int leftOverLimit;
+		
+		private File spillFile;
+		
+		private InputViewDataInputStreamWrapper spillFileReader;
+		
+		public SpanningWrapper(String[] tempDirs) {
+			this.tempDirs = tempDirs;
+			
+			this.lengthBuffer = ByteBuffer.allocate(4);
+			this.lengthBuffer.order(ByteOrder.BIG_ENDIAN);
+
+			this.recordLength = -1;
+
+			this.serializationReadBuffer = new DataInputDeserializer();
+			this.buffer = initialBuffer;
+		}
+		
+		private void initializeWithPartialRecord(NonSpanningWrapper partial, int nextRecordLength) throws IOException {
+			// set the length and copy what is available to the buffer
+			this.recordLength = nextRecordLength;
+			
+			final int numBytesChunk = partial.remaining();
+			
+			if (nextRecordLength > THRESHOLD_FOR_SPILLING) {
+				// create a spilling channel and put the data there
+				this.spillingChannel = createSpillingChannel();
+				
+				ByteBuffer toWrite = partial.segment.wrap(partial.position, numBytesChunk);
+				this.spillingChannel.write(toWrite);
+			}
+			else {
+				// collect in memory
+				ensureBufferCapacity(numBytesChunk);
+				partial.segment.get(partial.position, buffer, 0, numBytesChunk);
+			}
+			
+			this.accumulatedRecordBytes = numBytesChunk;
+		}
+		
+		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;
+					
+					if (this.recordLength > THRESHOLD_FOR_SPILLING) {
+						this.spillingChannel = createSpillingChannel();
+					}
+				}
+			}
+
+			// copy as much as we need or can for this next spanning record
+			int needed = this.recordLength - this.accumulatedRecordBytes;
+			int available = numBytesInSegment - segmentPosition;
+			int toCopy = Math.min(needed, available);
+
+			if (spillingChannel != null) {
+				// spill to file
+				ByteBuffer toWrite = segment.wrap(segmentPosition, toCopy);
+				this.spillingChannel.write(toWrite);
+			}
+			else {
+				ensureBufferCapacity(accumulatedRecordBytes + toCopy);
+				segment.get(segmentPosition, buffer, this.accumulatedRecordBytes, toCopy);
+			}
+			
+			this.accumulatedRecordBytes += toCopy;
+			
+			if (toCopy < available) {
+				// there is more data in the segment
+				this.leftOverData = segment;
+				this.leftOverStart = segmentPosition + toCopy;
+				this.leftOverLimit = numBytesInSegment;
+			}
+			
+			if (accumulatedRecordBytes == recordLength) {
+				// we have the full record
+				if (spillingChannel == null) {
+					this.serializationReadBuffer.setBuffer(buffer, 0, recordLength);
+				}
+				else {
+					spillingChannel.close();
+					
+					DataInputStream inStream = new DataInputStream(new BufferedInputStream(new FileInputStream(spillFile), 2 * 1024 * 1024));
+					this.spillFileReader = new InputViewDataInputStreamWrapper(inStream);
+				}
+			}
+		}
+		
+		private void moveRemainderToNonSpanningDeserializer(NonSpanningWrapper deserializer) {
+			deserializer.clear();
+			
+			if (leftOverData != null) {
+				deserializer.initializeFromMemorySegment(leftOverData, leftOverStart, leftOverLimit);
+			}
+		}
+		
+		private boolean hasFullRecord() {
+			return this.recordLength >= 0 && this.accumulatedRecordBytes >= this.recordLength;
+		}
+		
+		private int getNumGatheredBytes() {
+			return this.accumulatedRecordBytes + (this.recordLength >= 0 ? 4 : lengthBuffer.position());
+		}
+
+		public void clear() {
+			this.buffer = initialBuffer;
+			this.serializationReadBuffer.releaseArrays();
+
+			this.recordLength = -1;
+			this.lengthBuffer.clear();
+			this.leftOverData = null;
+			this.accumulatedRecordBytes = 0;
+			
+			if (spillingChannel != null) {
+				try {
+					spillingChannel.close();
+				}
+				catch (Throwable t) {
+					// ignore
+				}
+				spillingChannel = null;
+			}
+			if (spillFileReader != null) {
+				try {
+					spillFileReader.close();
+				}
+				catch (Throwable t) {
+					// ignore
+				}
+				spillFileReader = null;
+			}
+			if (spillFile != null) {
+				spillFile.delete();
+				spillFile = null;
+			}
+		}
+		
+		public DataInputView getInputView() {
+			if (spillFileReader == null) {
+				return serializationReadBuffer; 
+			}
+			else {
+				return spillFileReader;
+			}
+		}
+		
+		private void ensureBufferCapacity(int minLength) {
+			if (buffer.length < minLength) {
+				byte[] newBuffer = new byte[Math.max(minLength, buffer.length * 2)];
+				System.arraycopy(buffer, 0, newBuffer, 0, accumulatedRecordBytes);
+				buffer = newBuffer;
+			}
+		}
+		
+		@SuppressWarnings("resource")
+		private FileChannel createSpillingChannel() throws IOException {
+			if (spillFile != null) {
+				throw new IllegalStateException("Spilling file already exists.");
+			}
+			
+			String directory = tempDirs[rnd.nextInt(tempDirs.length)];
+			spillFile = new File(directory, randomString(rnd) + ".inputchannel");
+			
+			return new RandomAccessFile(spillFile, "rw").getChannel();
+		}
+		
+		private static String randomString(Random random) {
+			final byte[] bytes = new byte[20];
+			random.nextBytes(bytes);
+			return StringUtils.byteToHexString(bytes);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/BufferWriter.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/BufferWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/BufferWriter.java
deleted file mode 100644
index b9c6d33..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/BufferWriter.java
+++ /dev/null
@@ -1,119 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.io.network.api.writer;
-
-import org.apache.flink.runtime.event.task.AbstractEvent;
-import org.apache.flink.runtime.event.task.TaskEvent;
-import org.apache.flink.runtime.io.network.api.EndOfSuperstepEvent;
-import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.buffer.BufferProvider;
-import org.apache.flink.runtime.io.network.partition.IntermediateResultPartition;
-import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
-import org.apache.flink.runtime.util.event.EventListener;
-import org.apache.flink.runtime.io.network.api.TaskEventHandler;
-
-import java.io.IOException;
-
-/**
- * A buffer-oriented runtime result writer.
- * <p>
- * The {@link BufferWriter} is the runtime API for producing results. It
- * supports two kinds of data to be sent: buffers and events.
- * <p>
- * <strong>Important</strong>: When working directly with this API, it is
- * necessary to call {@link #finish()} after all data has been produced.
- */
-public final class BufferWriter implements EventListener<TaskEvent> {
-
-	private final IntermediateResultPartition partition;
-
-	private final TaskEventHandler taskEventHandler = new TaskEventHandler();
-
-	public BufferWriter(IntermediateResultPartition partition) {
-		this.partition = partition;
-	}
-
-	// ------------------------------------------------------------------------
-	// Attributes
-	// ------------------------------------------------------------------------
-
-	public IntermediateResultPartitionID getPartitionId() {
-		return partition.getPartitionId();
-	}
-
-	public BufferProvider getBufferProvider() {
-		return partition.getBufferProvider();
-	}
-
-	public int getNumberOfOutputChannels() {
-		return partition.getNumberOfQueues();
-	}
-
-	// ------------------------------------------------------------------------
-	// Data processing
-	// ------------------------------------------------------------------------
-
-	public void writeBuffer(Buffer buffer, int targetChannel) throws IOException {
-		partition.add(buffer, targetChannel);
-	}
-
-	public void writeEvent(AbstractEvent event, int targetChannel) throws IOException {
-		partition.add(EventSerializer.toBuffer(event), targetChannel);
-	}
-
-	public void writeEventToAllChannels(AbstractEvent event) throws IOException {
-		for (int i = 0; i < partition.getNumberOfQueues(); i++) {
-			Buffer buffer = EventSerializer.toBuffer(event);
-			partition.add(buffer, i);
-		}
-	}
-
-	public void writeEndOfSuperstep() throws IOException {
-		for (int i = 0; i < partition.getNumberOfQueues(); i++) {
-			Buffer buffer = EventSerializer.toBuffer(EndOfSuperstepEvent.INSTANCE);
-			partition.add(buffer, i);
-		}
-	}
-
-	public void finish() throws IOException, InterruptedException {
-		partition.finish();
-	}
-
-	public boolean isFinished() {
-		return partition.isFinished();
-	}
-
-	// ------------------------------------------------------------------------
-	// Event handling
-	// ------------------------------------------------------------------------
-
-	public TaskEventHandler getTaskEventHandler() {
-		return taskEventHandler;
-	}
-
-	public void subscribeToEvent(EventListener<TaskEvent> eventListener, Class<? extends TaskEvent> eventType) {
-		taskEventHandler.subscribe(eventListener, eventType);
-	}
-
-	@Override
-	public void onEvent(TaskEvent event) {
-		taskEventHandler.publish(event);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java
index b29d4da..7f84786 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java
@@ -31,7 +31,7 @@ import static org.apache.flink.runtime.io.network.api.serialization.RecordSerial
 /**
  * A record-oriented runtime result writer.
  * <p>
- * The RecordWriter wraps the runtime's {@link BufferWriter} and takes care of
+ * The RecordWriter wraps the runtime's {@link ResultPartitionWriter} and takes care of
  * serializing records into buffers.
  * <p>
  * <strong>Important</strong>: it is necessary to call {@link #flush()} after
@@ -43,7 +43,7 @@ import static org.apache.flink.runtime.io.network.api.serialization.RecordSerial
  */
 public class RecordWriter<T extends IOReadableWritable> {
 
-	protected final BufferWriter writer;
+	protected final ResultPartitionWriter writer;
 
 	private final ChannelSelector<T> channelSelector;
 
@@ -52,11 +52,12 @@ public class RecordWriter<T extends IOReadableWritable> {
 	/** {@link RecordSerializer} per outgoing channel */
 	private final RecordSerializer<T>[] serializers;
 
-	public RecordWriter(BufferWriter writer) {
+	public RecordWriter(ResultPartitionWriter writer) {
 		this(writer, new RoundRobinChannelSelector<T>());
 	}
 
-	public RecordWriter(BufferWriter writer, ChannelSelector<T> channelSelector) {
+	@SuppressWarnings("unchecked")
+	public RecordWriter(ResultPartitionWriter writer, ChannelSelector<T> channelSelector) {
 		this.writer = writer;
 		this.channelSelector = channelSelector;
 
@@ -73,10 +74,6 @@ public class RecordWriter<T extends IOReadableWritable> {
 		}
 	}
 
-	public boolean isFinished() {
-		return writer.isFinished();
-	}
-
 	public void emit(T record) throws IOException, InterruptedException {
 		for (int targetChannel : channelSelector.selectChannels(record, numChannels)) {
 			// serialize with corresponding serializer and send full buffer

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java
new file mode 100644
index 0000000..ac28519
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.api.writer;
+
+import org.apache.flink.runtime.event.task.AbstractEvent;
+import org.apache.flink.runtime.event.task.TaskEvent;
+import org.apache.flink.runtime.io.network.api.EndOfSuperstepEvent;
+import org.apache.flink.runtime.io.network.api.TaskEventHandler;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferProvider;
+import org.apache.flink.runtime.io.network.partition.ResultPartition;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.util.event.EventListener;
+
+import java.io.IOException;
+
+/**
+ * A buffer-oriented runtime result writer.
+ * <p>
+ * The {@link ResultPartitionWriter} is the runtime API for producing results. It
+ * supports two kinds of data to be sent: buffers and events.
+ * <p>
+ * <strong>Important</strong>: When working directly with this API, it is
+ * necessary to call {@link #finish()} after all data has been produced.
+ */
+public final class ResultPartitionWriter implements EventListener<TaskEvent> {
+
+	private final ResultPartition partition;
+
+	private final TaskEventHandler taskEventHandler = new TaskEventHandler();
+
+	public ResultPartitionWriter(ResultPartition partition) {
+		this.partition = partition;
+	}
+
+	// ------------------------------------------------------------------------
+	// Attributes
+	// ------------------------------------------------------------------------
+
+	public ResultPartitionID getPartitionId() {
+		return partition.getPartitionId();
+	}
+
+	public BufferProvider getBufferProvider() {
+		return partition.getBufferProvider();
+	}
+
+	public int getNumberOfOutputChannels() {
+		return partition.getNumberOfSubpartitions();
+	}
+
+	// ------------------------------------------------------------------------
+	// Data processing
+	// ------------------------------------------------------------------------
+
+	public void writeBuffer(Buffer buffer, int targetChannel) throws IOException {
+		partition.add(buffer, targetChannel);
+	}
+
+	public void writeEvent(AbstractEvent event, int targetChannel) throws IOException {
+		partition.add(EventSerializer.toBuffer(event), targetChannel);
+	}
+
+	public void writeEventToAllChannels(AbstractEvent event) throws IOException {
+		for (int i = 0; i < partition.getNumberOfSubpartitions(); i++) {
+			Buffer buffer = EventSerializer.toBuffer(event);
+			partition.add(buffer, i);
+		}
+	}
+
+	public void writeEndOfSuperstep() throws IOException {
+		for (int i = 0; i < partition.getNumberOfSubpartitions(); i++) {
+			Buffer buffer = EventSerializer.toBuffer(EndOfSuperstepEvent.INSTANCE);
+			partition.add(buffer, i);
+		}
+	}
+
+	public void finish() throws IOException, InterruptedException {
+		partition.finish();
+	}
+
+	// ------------------------------------------------------------------------
+	// Event handling
+	// ------------------------------------------------------------------------
+
+	public TaskEventHandler getTaskEventHandler() {
+		return taskEventHandler;
+	}
+
+	public void subscribeToEvent(EventListener<TaskEvent> eventListener, Class<? extends TaskEvent> eventType) {
+		taskEventHandler.subscribe(eventListener, eventType);
+	}
+
+	@Override
+	public void onEvent(TaskEvent event) {
+		taskEventHandler.publish(event);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/Buffer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/Buffer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/Buffer.java
index 0ca9562..2ed82fa 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/Buffer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/Buffer.java
@@ -39,7 +39,7 @@ public class Buffer {
 	/** The recycler for the backing {@link MemorySegment} */
 	private final BufferRecycler recycler;
 
-	private final boolean isBuffer;
+	private boolean isBuffer;
 
 	/** The current number of references to this buffer */
 	private int referenceCount = 1;
@@ -66,6 +66,14 @@ public class Buffer {
 		return isBuffer;
 	}
 
+	public void tagAsEvent() {
+		synchronized (recycleLock) {
+			ensureNotRecycled();
+		}
+
+		isBuffer = false;
+	}
+
 	public MemorySegment getMemorySegment() {
 		synchronized (recycleLock) {
 			ensureNotRecycled();
@@ -84,8 +92,6 @@ public class Buffer {
 
 	public int getSize() {
 		synchronized (recycleLock) {
-			ensureNotRecycled();
-
 			return currentSize;
 		}
 	}
@@ -103,8 +109,6 @@ public class Buffer {
 
 	public void recycle() {
 		synchronized (recycleLock) {
-			ensureNotRecycled();
-
 			if (--referenceCount == 0) {
 				recycler.recycle(memorySegment);
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferPool.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferPool.java
index 93fb7a8..c2a3c05 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferPool.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferPool.java
@@ -24,7 +24,7 @@ public interface BufferPool extends BufferProvider, BufferRecycler {
 
 	void setBufferPoolOwner(BufferPoolOwner owner);
 
-	void destroy() throws IOException;
+	void lazyDestroy();
 
 	@Override
 	boolean isDestroyed();

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferPoolOwner.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferPoolOwner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferPoolOwner.java
index 65c948a..66a6995 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferPoolOwner.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferPoolOwner.java
@@ -22,6 +22,6 @@ import java.io.IOException;
 
 public interface BufferPoolOwner {
 
-	void recycleBuffers(int numBuffersToRecycle) throws IOException;
+	void releaseMemory(int numBuffersToRecycle) throws IOException;
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferProvider.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferProvider.java
index e58a846..937ea28 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferProvider.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferProvider.java
@@ -52,4 +52,6 @@ public interface BufferProvider {
 
 	boolean isDestroyed();
 
+	int getMemorySegmentSize();
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java
index 9e4b5a1..4117e44 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java
@@ -83,6 +83,11 @@ class LocalBufferPool implements BufferPool {
 	}
 
 	@Override
+	public int getMemorySegmentSize() {
+		return networkBufferPool.getMemorySegmentSize();
+	}
+
+	@Override
 	public int getNumberOfRequiredMemorySegments() {
 		return numberOfRequiredMemorySegments;
 	}
@@ -124,15 +129,17 @@ class LocalBufferPool implements BufferPool {
 		return requestBuffer(true);
 	}
 
-	private Buffer requestBuffer(boolean isBlocking) throws InterruptedException {
+	private Buffer requestBuffer(boolean isBlocking) throws InterruptedException, IOException {
 		synchronized (availableMemorySegments) {
-			if (isDestroyed) {
-				return null;
-			}
-
 			returnExcessMemorySegments();
 
+			boolean askToRecycle = owner != null;
+
 			while (availableMemorySegments.isEmpty()) {
+				if (isDestroyed) {
+					return null;
+				}
+
 				if (numberOfRequestedMemorySegments < currentPoolSize) {
 					final MemorySegment segment = networkBufferPool.requestMemorySegment();
 
@@ -144,6 +151,10 @@ class LocalBufferPool implements BufferPool {
 					}
 				}
 
+				if (askToRecycle) {
+					owner.releaseMemory(1);
+				}
+
 				if (isBlocking) {
 					availableMemorySegments.wait(2000);
 				}
@@ -186,7 +197,7 @@ class LocalBufferPool implements BufferPool {
 	 * Destroy is called after the produce or consume phase of a task finishes.
 	 */
 	@Override
-	public void destroy() throws IOException {
+	public void lazyDestroy() {
 		synchronized (availableMemorySegments) {
 			if (!isDestroyed) {
 				MemorySegment segment;
@@ -230,7 +241,7 @@ class LocalBufferPool implements BufferPool {
 			// If there is a registered owner and we have still requested more buffers than our
 			// size, trigger a recycle via the owner.
 			if (owner != null && numberOfRequestedMemorySegments > currentPoolSize) {
-				owner.recycleBuffers(numberOfRequestedMemorySegments - numBuffers);
+				owner.releaseMemory(numberOfRequestedMemorySegments - numBuffers);
 			}
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java
index 3a6dbf5..c1e62da 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java
@@ -185,14 +185,18 @@ public class NetworkBufferPool implements BufferPoolFactory {
 	}
 
 	@Override
-	public void destroyBufferPool(BufferPool bufferPool) throws IOException {
+	public void destroyBufferPool(BufferPool bufferPool) {
 		synchronized (factoryLock) {
 			if (allBufferPools.remove(bufferPool)) {
 				managedBufferPools.remove(bufferPool);
 
 				numTotalRequiredBuffers -= bufferPool.getNumberOfRequiredMemorySegments();
 
-				redistributeBuffers();
+				try {
+					redistributeBuffers();
+				} catch (IOException e) {
+					throw new RuntimeException(e);
+				}
 			}
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyConfig.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyConfig.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyConfig.java
index 45ffb15..55057fe 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyConfig.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyConfig.java
@@ -63,7 +63,12 @@ public class NettyConfig {
 
 	private final Configuration config; // optional configuration
 
-	public NettyConfig(InetAddress serverAddress, int serverPort, int memorySegmentSize, Configuration config) {
+	public NettyConfig(
+			InetAddress serverAddress,
+			int serverPort,
+			int memorySegmentSize,
+			Configuration config) {
+
 		this.serverAddress = checkNotNull(serverAddress);
 
 		checkArgument(serverPort > 0 && serverPort <= 65536, "Invalid port number.");
@@ -93,42 +98,42 @@ public class NettyConfig {
 	// Setters
 	// ------------------------------------------------------------------------
 
-	NettyConfig setServerConnectBacklog(int connectBacklog) {
+	public NettyConfig setServerConnectBacklog(int connectBacklog) {
 		checkArgument(connectBacklog >= 0);
 		config.setInteger(CONNECT_BACKLOG, connectBacklog);
 
 		return this;
 	}
 
-	NettyConfig setServerNumThreads(int numThreads) {
+	public NettyConfig setServerNumThreads(int numThreads) {
 		checkArgument(numThreads >= 0);
 		config.setInteger(NUM_THREADS_SERVER, numThreads);
 
 		return this;
 	}
 
-	NettyConfig setClientNumThreads(int numThreads) {
+	public NettyConfig setClientNumThreads(int numThreads) {
 		checkArgument(numThreads >= 0);
 		config.setInteger(NUM_THREADS_CLIENT, numThreads);
 
 		return this;
 	}
 
-	NettyConfig setClientConnectTimeoutSeconds(int connectTimeoutSeconds) {
+	public NettyConfig setClientConnectTimeoutSeconds(int connectTimeoutSeconds) {
 		checkArgument(connectTimeoutSeconds >= 0);
 		config.setInteger(CLIENT_CONNECT_TIMEOUT_SECONDS, connectTimeoutSeconds);
 
 		return this;
 	}
 
-	NettyConfig setSendAndReceiveBufferSize(int bufferSize) {
+	public NettyConfig setSendAndReceiveBufferSize(int bufferSize) {
 		checkArgument(bufferSize >= 0);
 		config.setInteger(SEND_RECEIVE_BUFFER_SIZE, bufferSize);
 
 		return this;
 	}
 
-	NettyConfig setTransportType(String transport) {
+	public NettyConfig setTransportType(String transport) {
 		if (transport.equals("nio") || transport.equals("epoll") || transport.equals("auto")) {
 			config.setString(TRANSPORT_TYPE, transport);
 		}
@@ -143,32 +148,32 @@ public class NettyConfig {
 	// Getters
 	// ------------------------------------------------------------------------
 
-	int getServerConnectBacklog() {
+	public int getServerConnectBacklog() {
 		// default: 0 => Netty's default
 		return config.getInteger(CONNECT_BACKLOG, 0);
 	}
 
-	int getServerNumThreads() {
+	public int getServerNumThreads() {
 		// default: 0 => Netty's default: 2 * #cores
 		return config.getInteger(NUM_THREADS_SERVER, 0);
 	}
 
-	int getClientNumThreads() {
+	public int getClientNumThreads() {
 		// default: 0 => Netty's default: 2 * #cores
 		return config.getInteger(NUM_THREADS_CLIENT, 0);
 	}
 
-	int getClientConnectTimeoutSeconds() {
+	public int getClientConnectTimeoutSeconds() {
 		// default: 120s = 2min
 		return config.getInteger(CLIENT_CONNECT_TIMEOUT_SECONDS, 120);
 	}
 
-	int getSendAndReceiveBufferSize() {
+	public int getSendAndReceiveBufferSize() {
 		// default: 0 => Netty's default
 		return config.getInteger(SEND_RECEIVE_BUFFER_SIZE, 0);
 	}
 
-	TransportType getTransportType() {
+	public TransportType getTransportType() {
 		String transport = config.getString(TRANSPORT_TYPE, "nio");
 
 		if (transport.equals("nio")) {

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyConnectionManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyConnectionManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyConnectionManager.java
index 260ea7e..c097098 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyConnectionManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyConnectionManager.java
@@ -18,10 +18,11 @@
 
 package org.apache.flink.runtime.io.network.netty;
 
+import org.apache.flink.runtime.io.network.ConnectionID;
 import org.apache.flink.runtime.io.network.ConnectionManager;
-import org.apache.flink.runtime.io.network.RemoteAddress;
 import org.apache.flink.runtime.io.network.TaskEventDispatcher;
-import org.apache.flink.runtime.io.network.partition.IntermediateResultPartitionProvider;
+import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider;
 
 import java.io.IOException;
 
@@ -41,21 +42,21 @@ public class NettyConnectionManager implements ConnectionManager {
 	}
 
 	@Override
-	public void start(IntermediateResultPartitionProvider partitionProvider, TaskEventDispatcher taskEventDispatcher) throws IOException {
-		PartitionRequestProtocol partitionRequestProtocol = new PartitionRequestProtocol(partitionProvider, taskEventDispatcher);
+	public void start(ResultPartitionProvider partitionProvider, TaskEventDispatcher taskEventDispatcher, NetworkBufferPool networkbufferPool) throws IOException {
+		PartitionRequestProtocol partitionRequestProtocol = new PartitionRequestProtocol(partitionProvider, taskEventDispatcher, networkbufferPool);
 
 		client.init(partitionRequestProtocol);
 		server.init(partitionRequestProtocol);
 	}
 
 	@Override
-	public PartitionRequestClient createPartitionRequestClient(RemoteAddress remoteAddress) throws IOException, InterruptedException {
-		return partitionRequestClientFactory.createPartitionRequestClient(remoteAddress);
+	public PartitionRequestClient createPartitionRequestClient(ConnectionID connectionId) throws IOException, InterruptedException {
+		return partitionRequestClientFactory.createPartitionRequestClient(connectionId);
 	}
 
 	@Override
-	public void closeOpenChannelConnections(RemoteAddress remoteAddress) {
-		partitionRequestClientFactory.closeOpenChannelConnections(remoteAddress);
+	public void closeOpenChannelConnections(ConnectionID connectionId) {
+		partitionRequestClientFactory.closeOpenChannelConnections(connectionId);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java
index f6bf0e7..bef2740 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java
@@ -36,6 +36,7 @@ import org.apache.flink.runtime.event.task.TaskEvent;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
 import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
 
@@ -310,7 +311,6 @@ abstract class NettyMessage {
 
 			return result;
 		}
-
 		@Override
 		void readFrom(ByteBuf buffer) throws Exception {
 			DataInputView inputView = new ByteBufDataInputView(buffer);
@@ -347,9 +347,7 @@ abstract class NettyMessage {
 
 		final static byte ID = 2;
 
-		ExecutionAttemptID producerExecutionId;
-
-		IntermediateResultPartitionID partitionId;
+		ResultPartitionID partitionId;
 
 		int queueIndex;
 
@@ -358,8 +356,7 @@ abstract class NettyMessage {
 		public PartitionRequest() {
 		}
 
-		PartitionRequest(ExecutionAttemptID producerExecutionId, IntermediateResultPartitionID partitionId, int queueIndex, InputChannelID receiverId) {
-			this.producerExecutionId = producerExecutionId;
+		PartitionRequest(ResultPartitionID partitionId, int queueIndex, InputChannelID receiverId) {
 			this.partitionId = partitionId;
 			this.queueIndex = queueIndex;
 			this.receiverId = receiverId;
@@ -372,8 +369,8 @@ abstract class NettyMessage {
 			try {
 				result = allocateBuffer(allocator, ID, 16 + 16 + 4 + 16);
 
-				producerExecutionId.writeTo(result);
-				partitionId.writeTo(result);
+				partitionId.getPartitionId().writeTo(result);
+				partitionId.getProducerId().writeTo(result);
 				result.writeInt(queueIndex);
 				receiverId.writeTo(result);
 
@@ -390,16 +387,14 @@ abstract class NettyMessage {
 
 		@Override
 		public void readFrom(ByteBuf buffer) {
-			producerExecutionId = ExecutionAttemptID.fromByteBuf(buffer);
-			partitionId = IntermediateResultPartitionID.fromByteBuf(buffer);
+			partitionId = new ResultPartitionID(IntermediateResultPartitionID.fromByteBuf(buffer), ExecutionAttemptID.fromByteBuf(buffer));
 			queueIndex = buffer.readInt();
 			receiverId = InputChannelID.fromByteBuf(buffer);
 		}
 
 		@Override
 		public String toString() {
-			return String.format("PartitionRequest(ProducerID: %s, PartitionID: %s)",
-					producerExecutionId, partitionId);
+			return String.format("PartitionRequest(%s)", partitionId);
 		}
 	}
 
@@ -411,16 +406,13 @@ abstract class NettyMessage {
 
 		InputChannelID receiverId;
 
-		ExecutionAttemptID executionId;
-
-		IntermediateResultPartitionID partitionId;
+		ResultPartitionID partitionId;
 
 		public TaskEventRequest() {
 		}
 
-		TaskEventRequest(TaskEvent event, ExecutionAttemptID executionId, IntermediateResultPartitionID partitionId, InputChannelID receiverId) {
+		TaskEventRequest(TaskEvent event, ResultPartitionID partitionId, InputChannelID receiverId) {
 			this.event = event;
-			this.executionId = executionId;
 			this.receiverId = receiverId;
 			this.partitionId = partitionId;
 		}
@@ -437,9 +429,11 @@ abstract class NettyMessage {
 
 				result.writeInt(serializedEvent.remaining());
 				result.writeBytes(serializedEvent);
-				executionId.writeTo(result);
+
+				partitionId.getPartitionId().writeTo(result);
+				partitionId.getProducerId().writeTo(result);
+
 				receiverId.writeTo(result);
-				partitionId.writeTo(result);
 
 				return result;
 			}
@@ -463,9 +457,9 @@ abstract class NettyMessage {
 
 			event = (TaskEvent) EventSerializer.fromSerializedEvent(serializedEvent, getClass().getClassLoader());
 
-			executionId = ExecutionAttemptID.fromByteBuf(buffer);
+			partitionId = new ResultPartitionID(IntermediateResultPartitionID.fromByteBuf(buffer), ExecutionAttemptID.fromByteBuf(buffer));
+
 			receiverId = InputChannelID.fromByteBuf(buffer);
-			partitionId = IntermediateResultPartitionID.fromByteBuf(buffer);
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClient.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClient.java
index f26f15c..32e3951 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClient.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClient.java
@@ -22,10 +22,9 @@ import io.netty.channel.Channel;
 import io.netty.channel.ChannelFuture;
 import io.netty.channel.ChannelFutureListener;
 import org.apache.flink.runtime.event.task.TaskEvent;
-import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.io.network.RemoteAddress;
+import org.apache.flink.runtime.io.network.ConnectionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel;
-import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
 import org.apache.flink.runtime.util.AtomicDisposableReferenceCounter;
 
 import java.io.IOException;
@@ -38,7 +37,7 @@ import static org.apache.flink.runtime.io.network.netty.NettyMessage.TaskEventRe
  * Partition request client for remote partition requests.
  * <p>
  * This client is shared by all remote input channels, which request a partition
- * from the same {@link RemoteAddress}.
+ * from the same {@link ConnectionID}.
  */
 public class PartitionRequestClient {
 
@@ -46,17 +45,17 @@ public class PartitionRequestClient {
 
 	private final PartitionRequestClientHandler partitionRequestHandler;
 
-	private final RemoteAddress remoteAddress;
+	private final ConnectionID connectionId;
 
 	private final PartitionRequestClientFactory clientFactory;
 
 	// If zero, the underlying TCP channel can be safely closed
 	private final AtomicDisposableReferenceCounter closeReferenceCounter = new AtomicDisposableReferenceCounter();
 
-	PartitionRequestClient(Channel tcpChannel, PartitionRequestClientHandler partitionRequestHandler, RemoteAddress remoteAddress, PartitionRequestClientFactory clientFactory) {
+	PartitionRequestClient(Channel tcpChannel, PartitionRequestClientHandler partitionRequestHandler, ConnectionID connectionId, PartitionRequestClientFactory clientFactory) {
 		this.tcpChannel = checkNotNull(tcpChannel);
 		this.partitionRequestHandler = checkNotNull(partitionRequestHandler);
-		this.remoteAddress = checkNotNull(remoteAddress);
+		this.connectionId = checkNotNull(connectionId);
 		this.clientFactory = checkNotNull(clientFactory);
 	}
 
@@ -71,7 +70,7 @@ public class PartitionRequestClient {
 	 * instance of this client to ensure correct closing logic.
 	 */
 	boolean incrementReferenceCounter() {
-		return closeReferenceCounter.incrementReferenceCounter();
+		return closeReferenceCounter.increment();
 	}
 
 	/**
@@ -80,10 +79,10 @@ public class PartitionRequestClient {
 	 * The request goes to the remote producer, for which this partition
 	 * request client instance has been created.
 	 */
-	public void requestIntermediateResultPartition(ExecutionAttemptID producerExecutionId, final IntermediateResultPartitionID partitionId, final int requestedQueueIndex, final RemoteInputChannel inputChannel) throws IOException {
+	public void requestIntermediateResultPartition(final ResultPartitionID partitionId, int requestedQueueIndex, final RemoteInputChannel inputChannel) throws IOException {
 		partitionRequestHandler.addInputChannel(inputChannel);
 
-		tcpChannel.writeAndFlush(new PartitionRequest(producerExecutionId, partitionId, requestedQueueIndex, inputChannel.getInputChannelId()))
+		tcpChannel.writeAndFlush(new PartitionRequest(partitionId, requestedQueueIndex, inputChannel.getInputChannelId()))
 				.addListener(
 						new ChannelFutureListener() {
 							@Override
@@ -105,9 +104,9 @@ public class PartitionRequestClient {
 	 * guaranteed to be the case when both the respective producer and
 	 * consumer task run pipelined.
 	 */
-	public void sendTaskEvent(ExecutionAttemptID producerExecutionId, IntermediateResultPartitionID partitionId, TaskEvent event, final RemoteInputChannel inputChannel) throws IOException {
+	public void sendTaskEvent(ResultPartitionID partitionId, TaskEvent event, final RemoteInputChannel inputChannel) throws IOException {
 
-		tcpChannel.writeAndFlush(new TaskEventRequest(event, producerExecutionId, partitionId, inputChannel.getInputChannelId()))
+		tcpChannel.writeAndFlush(new TaskEventRequest(event, partitionId, inputChannel.getInputChannelId()))
 				.addListener(
 						new ChannelFutureListener() {
 							@Override
@@ -123,12 +122,12 @@ public class PartitionRequestClient {
 
 		partitionRequestHandler.removeInputChannel(inputChannel);
 
-		if (closeReferenceCounter.decrementReferenceCounter()) {
+		if (closeReferenceCounter.decrement()) {
 			// Close the TCP connection
 			tcpChannel.close();
 
 			// Make sure to remove the client from the factory
-			clientFactory.destroyPartitionRequestClient(remoteAddress, this);
+			clientFactory.destroyPartitionRequestClient(connectionId, this);
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactory.java
index d4c022b..f274b67 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactory.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactory.java
@@ -21,7 +21,7 @@ package org.apache.flink.runtime.io.network.netty;
 import io.netty.channel.Channel;
 import io.netty.channel.ChannelFuture;
 import io.netty.channel.ChannelFutureListener;
-import org.apache.flink.runtime.io.network.RemoteAddress;
+import org.apache.flink.runtime.io.network.ConnectionID;
 import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel;
 
 import java.io.IOException;
@@ -38,7 +38,7 @@ class PartitionRequestClientFactory {
 
 	private final NettyClient nettyClient;
 
-	private final ConcurrentMap<RemoteAddress, Object> clients = new ConcurrentHashMap<RemoteAddress, Object>();
+	private final ConcurrentMap<ConnectionID, Object> clients = new ConcurrentHashMap<ConnectionID, Object>();
 
 	PartitionRequestClientFactory(NettyClient nettyClient) {
 		this.nettyClient = nettyClient;
@@ -48,12 +48,12 @@ class PartitionRequestClientFactory {
 	 * Atomically establishes a TCP connection to the given remote address and
 	 * creates a {@link PartitionRequestClient} instance for this connection.
 	 */
-	PartitionRequestClient createPartitionRequestClient(RemoteAddress remoteAddress) throws IOException, InterruptedException {
+	PartitionRequestClient createPartitionRequestClient(ConnectionID connectionId) throws IOException, InterruptedException {
 		Object entry;
 		PartitionRequestClient client = null;
 
 		while (client == null) {
-			entry = clients.get(remoteAddress);
+			entry = clients.get(connectionId);
 
 			if (entry != null) {
 				// Existing channel or connecting channel
@@ -64,7 +64,7 @@ class PartitionRequestClientFactory {
 					ConnectingChannel future = (ConnectingChannel) entry;
 					client = future.waitForChannel();
 
-					clients.replace(remoteAddress, future, client);
+					clients.replace(connectionId, future, client);
 				}
 			}
 			else {
@@ -72,20 +72,20 @@ class PartitionRequestClientFactory {
 				// We create a "connecting 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.
-				ConnectingChannel connectingChannel = new ConnectingChannel(remoteAddress, this);
-				Object old = clients.putIfAbsent(remoteAddress, connectingChannel);
+				ConnectingChannel connectingChannel = new ConnectingChannel(connectionId, this);
+				Object old = clients.putIfAbsent(connectionId, connectingChannel);
 
 				if (old == null) {
-					nettyClient.connect(remoteAddress.getAddress()).addListener(connectingChannel);
+					nettyClient.connect(connectionId.getAddress()).addListener(connectingChannel);
 
 					client = connectingChannel.waitForChannel();
 
-					clients.replace(remoteAddress, connectingChannel, client);
+					clients.replace(connectionId, connectingChannel, client);
 				}
 				else if (old instanceof ConnectingChannel) {
 					client = ((ConnectingChannel) old).waitForChannel();
 
-					clients.replace(remoteAddress, old, client);
+					clients.replace(connectionId, old, client);
 				}
 				else {
 					client = (PartitionRequestClient) old;
@@ -95,7 +95,7 @@ class PartitionRequestClientFactory {
 			// Make sure to increment the reference count before handing a client
 			// out to ensure correct bookkeeping for channel closing.
 			if (!client.incrementReferenceCounter()) {
-				destroyPartitionRequestClient(remoteAddress, client);
+				destroyPartitionRequestClient(connectionId, client);
 				client = null;
 			}
 		}
@@ -103,14 +103,14 @@ class PartitionRequestClientFactory {
 		return client;
 	}
 
-	public void closeOpenChannelConnections(RemoteAddress remoteAddress) {
-		Object entry = clients.get(remoteAddress);
+	public void closeOpenChannelConnections(ConnectionID connectionId) {
+		Object entry = clients.get(connectionId);
 
 		if (entry instanceof ConnectingChannel) {
 			ConnectingChannel channel = (ConnectingChannel) entry;
 
 			if (channel.dispose()) {
-				clients.remove(remoteAddress, channel);
+				clients.remove(connectionId, channel);
 			}
 		}
 	}
@@ -120,24 +120,24 @@ class PartitionRequestClientFactory {
 	}
 
 	/**
-	 * Removes the client for the given {@link RemoteAddress}.
+	 * Removes the client for the given {@link ConnectionID}.
 	 */
-	void destroyPartitionRequestClient(RemoteAddress remoteAddress, PartitionRequestClient client) {
-		clients.remove(remoteAddress, client);
+	void destroyPartitionRequestClient(ConnectionID connectionId, PartitionRequestClient client) {
+		clients.remove(connectionId, client);
 	}
 
 	private static final class ConnectingChannel implements ChannelFutureListener {
 
 		private final Object connectLock = new Object();
 
-		private final RemoteAddress remoteAddress;
+		private final ConnectionID connectionId;
 
 		private final PartitionRequestClientFactory clientFactory;
 
 		private boolean disposeRequestClient = false;
 
-		public ConnectingChannel(RemoteAddress remoteAddress, PartitionRequestClientFactory clientFactory) {
-			this.remoteAddress = remoteAddress;
+		public ConnectingChannel(ConnectionID connectionId, PartitionRequestClientFactory clientFactory) {
+			this.connectionId = connectionId;
 			this.clientFactory = clientFactory;
 		}
 
@@ -164,7 +164,7 @@ class PartitionRequestClientFactory {
 					PartitionRequestClientHandler requestHandler =
 							(PartitionRequestClientHandler) channel.pipeline().get(PartitionRequestProtocol.CLIENT_REQUEST_HANDLER_NAME);
 
-					partitionRequestClient = new PartitionRequestClient(channel, requestHandler, remoteAddress, clientFactory);
+				partitionRequestClient = new PartitionRequestClient(channel, requestHandler, connectionId, clientFactory);
 
 					if (disposeRequestClient) {
 						partitionRequestClient.disposeIfNotUsed();

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestProtocol.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestProtocol.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestProtocol.java
index b8fbc10..61a0970 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestProtocol.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestProtocol.java
@@ -20,7 +20,8 @@ package org.apache.flink.runtime.io.network.netty;
 
 import io.netty.channel.ChannelPipeline;
 import org.apache.flink.runtime.io.network.TaskEventDispatcher;
-import org.apache.flink.runtime.io.network.partition.IntermediateResultPartitionProvider;
+import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider;
 
 import static org.apache.flink.runtime.io.network.netty.NettyMessage.NettyMessageEncoder;
 import static org.apache.flink.runtime.io.network.netty.NettyMessage.NettyMessageEncoder.createFrameLengthDecoder;
@@ -33,12 +34,14 @@ class PartitionRequestProtocol implements NettyProtocol {
 
 	private final NettyMessage.NettyMessageDecoder messageDecoder = new NettyMessage.NettyMessageDecoder();
 
-	private final IntermediateResultPartitionProvider partitionProvider;
+	private final ResultPartitionProvider partitionProvider;
 	private final TaskEventDispatcher taskEventDispatcher;
+	private final NetworkBufferPool networkbufferPool;
 
-	PartitionRequestProtocol(IntermediateResultPartitionProvider partitionProvider, TaskEventDispatcher taskEventDispatcher) {
+	PartitionRequestProtocol(ResultPartitionProvider partitionProvider, TaskEventDispatcher taskEventDispatcher, NetworkBufferPool networkbufferPool) {
 		this.partitionProvider = partitionProvider;
 		this.taskEventDispatcher = taskEventDispatcher;
+		this.networkbufferPool = networkbufferPool;
 	}
 
 	// +-------------------------------------------------------------------+
@@ -80,7 +83,7 @@ class PartitionRequestProtocol implements NettyProtocol {
 				.addLast("Message encoder", messageEncoder)
 				.addLast("Frame decoder", createFrameLengthDecoder())
 				.addLast("Client request decoder", messageDecoder)
-				.addLast("Server request handler", new PartitionRequestServerHandler(partitionProvider, taskEventDispatcher, queueOfPartitionQueues))
+				.addLast("Server request handler", new PartitionRequestServerHandler(partitionProvider, taskEventDispatcher, queueOfPartitionQueues, networkbufferPool))
 				.addLast("Queue of queues", queueOfPartitionQueues);
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java
index d107d95..420a276 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java
@@ -23,8 +23,10 @@ import io.netty.channel.ChannelFuture;
 import io.netty.channel.ChannelFutureListener;
 import io.netty.channel.ChannelHandlerContext;
 import io.netty.channel.ChannelInboundHandlerAdapter;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.partition.queue.IntermediateResultPartitionQueueIterator;
+import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView;
 import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
 import org.apache.flink.runtime.util.event.NotificationListener;
 import org.slf4j.Logger;
@@ -50,9 +52,9 @@ class PartitionRequestQueue extends ChannelInboundHandlerAdapter {
 
 	private final ChannelFutureListener writeListener = new WriteAndFlushNextMessageIfPossibleListener();
 
-	private final Queue<SequenceNumberingPartitionQueueIterator> queue = new ArrayDeque<SequenceNumberingPartitionQueueIterator>();
+	private final Queue<SequenceNumberingSubpartitionView> queue = new ArrayDeque<SequenceNumberingSubpartitionView>();
 
-	private SequenceNumberingPartitionQueueIterator currentPartitionQueue;
+	private SequenceNumberingSubpartitionView currentPartitionQueue;
 
 	private boolean fatalError;
 
@@ -97,18 +99,18 @@ class PartitionRequestQueue extends ChannelInboundHandlerAdapter {
 		super.channelRegistered(ctx);
 	}
 
-	public void enqueue(IntermediateResultPartitionQueueIterator partitionQueue, InputChannelID receiverId) throws Exception {
+	public void enqueue(ResultSubpartitionView partitionQueue, InputChannelID receiverId) throws Exception {
 		numEnqueueCalls.incrementAndGet();
-		ctx.pipeline().fireUserEventTriggered(new SequenceNumberingPartitionQueueIterator(partitionQueue, receiverId));
+		ctx.pipeline().fireUserEventTriggered(new SequenceNumberingSubpartitionView(partitionQueue, receiverId));
 	}
 
 	@Override
 	public void userEventTriggered(ChannelHandlerContext ctx, Object msg) throws Exception {
-		if (msg.getClass() == SequenceNumberingPartitionQueueIterator.class) {
+		if (msg.getClass() == SequenceNumberingSubpartitionView.class) {
 			boolean triggerWrite = queue.isEmpty();
 
 			numTotalEnqueueOperations++;
-			queue.add((SequenceNumberingPartitionQueueIterator) msg);
+			queue.add((SequenceNumberingSubpartitionView) msg);
 
 			if (triggerWrite) {
 				writeAndFlushNextMessageIfPossible(ctx.channel());
@@ -141,21 +143,28 @@ class PartitionRequestQueue extends ChannelInboundHandlerAdapter {
 					buffer = currentPartitionQueue.getNextBuffer();
 
 					if (buffer == null) {
-						if (currentPartitionQueue.subscribe(null)) {
+						if (currentPartitionQueue.registerListener(null)) {
 							numTotalSubscribeCalls++;
 							numOutstandingSubscribeCalls.incrementAndGet();
 
 							currentPartitionQueue = null;
 						}
-						else if (currentPartitionQueue.isConsumed()) {
-							numConsumedPartitions++;
-
+						else if (currentPartitionQueue.isReleased()) {
 							currentPartitionQueue = null;
 						}
 					}
 					else {
 						BufferResponse resp = new BufferResponse(buffer, currentPartitionQueue.getSequenceNumber(), currentPartitionQueue.getReceiverId());
 
+						if (!buffer.isBuffer() &&
+								EventSerializer.fromBuffer(buffer, getClass().getClassLoader()).getClass() == EndOfPartitionEvent.class) {
+
+							currentPartitionQueue.notifySubpartitionConsumed();
+							currentPartitionQueue.releaseAllResources();
+							currentPartitionQueue = null;
+
+						}
+
 						channel.writeAndFlush(resp).addListener(writeListener);
 
 						return;
@@ -164,13 +173,8 @@ class PartitionRequestQueue extends ChannelInboundHandlerAdapter {
 			}
 		}
 		catch (Throwable t) {
-			try {
-				if (buffer != null) {
-					buffer.recycle();
-				}
-			}
-			catch (Throwable ignored) {
-				// Make sure that this buffer is recycled in any case
+			if (buffer != null) {
+				buffer.recycle();
 			}
 
 			throw new IOException(t.getMessage(), t);
@@ -200,12 +204,12 @@ class PartitionRequestQueue extends ChannelInboundHandlerAdapter {
 
 	private void releaseAllResources() throws IOException {
 		if (currentPartitionQueue != null) {
-			currentPartitionQueue.discard();
+			currentPartitionQueue.releaseAllResources();
 			currentPartitionQueue = null;
 		}
 
 		while ((currentPartitionQueue = queue.poll()) != null) {
-			currentPartitionQueue.discard();
+			currentPartitionQueue.releaseAllResources();
 		}
 	}
 
@@ -232,15 +236,15 @@ class PartitionRequestQueue extends ChannelInboundHandlerAdapter {
 	 * Simple wrapper for the partition queue iterator, which increments a
 	 * sequence number for each returned buffer and remembers the receiver ID.
 	 */
-	private class SequenceNumberingPartitionQueueIterator implements IntermediateResultPartitionQueueIterator, NotificationListener {
+	private class SequenceNumberingSubpartitionView implements ResultSubpartitionView, NotificationListener {
 
-		private final IntermediateResultPartitionQueueIterator queueIterator;
+		private final ResultSubpartitionView queueIterator;
 
 		private final InputChannelID receiverId;
 
 		private int sequenceNumber = -1;
 
-		private SequenceNumberingPartitionQueueIterator(IntermediateResultPartitionQueueIterator queueIterator, InputChannelID receiverId) {
+		private SequenceNumberingSubpartitionView(ResultSubpartitionView queueIterator, InputChannelID receiverId) {
 			this.queueIterator = checkNotNull(queueIterator);
 			this.receiverId = checkNotNull(receiverId);
 		}
@@ -254,7 +258,7 @@ class PartitionRequestQueue extends ChannelInboundHandlerAdapter {
 		}
 
 		@Override
-		public Buffer getNextBuffer() throws IOException {
+		public Buffer getNextBuffer() throws IOException, InterruptedException {
 			Buffer buffer = queueIterator.getNextBuffer();
 
 			if (buffer != null) {
@@ -265,18 +269,23 @@ class PartitionRequestQueue extends ChannelInboundHandlerAdapter {
 		}
 
 		@Override
-		public void discard() throws IOException {
-			queueIterator.discard();
+		public void notifySubpartitionConsumed() throws IOException {
+			queueIterator.notifySubpartitionConsumed();
+		}
+
+		@Override
+		public boolean isReleased() {
+			return queueIterator.isReleased();
 		}
 
 		@Override
-		public boolean subscribe(NotificationListener ignored) throws AlreadySubscribedException {
-			return queueIterator.subscribe(this);
+		public boolean registerListener(NotificationListener ignored) throws IOException {
+			return queueIterator.registerListener(this);
 		}
 
 		@Override
-		public boolean isConsumed() {
-			return queueIterator.isConsumed();
+		public void releaseAllResources() throws IOException {
+			queueIterator.releaseAllResources();
 		}
 
 		/**

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandler.java
index b60256f..cb26e51 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandler.java
@@ -22,10 +22,12 @@ import com.google.common.base.Optional;
 import io.netty.channel.ChannelHandlerContext;
 import io.netty.channel.SimpleChannelInboundHandler;
 import org.apache.flink.runtime.io.network.TaskEventDispatcher;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
 import org.apache.flink.runtime.io.network.buffer.BufferProvider;
-import org.apache.flink.runtime.io.network.partition.IntermediateResultPartitionProvider;
+import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider;
+import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView;
 import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
-import org.apache.flink.runtime.io.network.partition.queue.IntermediateResultPartitionQueueIterator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -36,17 +38,42 @@ class PartitionRequestServerHandler extends SimpleChannelInboundHandler<NettyMes
 
 	private static final Logger LOG = LoggerFactory.getLogger(PartitionRequestServerHandler.class);
 
-	private final IntermediateResultPartitionProvider partitionProvider;
+	private final ResultPartitionProvider partitionProvider;
 
 	private final TaskEventDispatcher taskEventDispatcher;
 
 	private final PartitionRequestQueue outboundQueue;
 
-	PartitionRequestServerHandler(IntermediateResultPartitionProvider partitionProvider, TaskEventDispatcher taskEventDispatcher, PartitionRequestQueue outboundQueue) {
+	private final NetworkBufferPool networkBufferPool;
+
+	private BufferPool bufferPool;
+
+	PartitionRequestServerHandler(
+			ResultPartitionProvider partitionProvider,
+			TaskEventDispatcher taskEventDispatcher,
+			PartitionRequestQueue outboundQueue,
+			NetworkBufferPool networkBufferPool) {
 
 		this.partitionProvider = partitionProvider;
 		this.taskEventDispatcher = taskEventDispatcher;
 		this.outboundQueue = outboundQueue;
+		this.networkBufferPool = networkBufferPool;
+	}
+
+	@Override
+	public void channelRegistered(ChannelHandlerContext ctx) throws Exception {
+		super.channelRegistered(ctx);
+
+		bufferPool = networkBufferPool.createBufferPool(1, false);
+	}
+
+	@Override
+	public void channelUnregistered(ChannelHandlerContext ctx) throws Exception {
+		super.channelUnregistered(ctx);
+
+		if (bufferPool != null) {
+			bufferPool.lazyDestroy();
+		}
 	}
 
 	@Override
@@ -62,12 +89,11 @@ class PartitionRequestServerHandler extends SimpleChannelInboundHandler<NettyMes
 
 				LOG.debug("Read channel on {}: {}.",ctx.channel().localAddress(), request);
 
-				IntermediateResultPartitionQueueIterator queueIterator =
-						partitionProvider.getIntermediateResultPartitionIterator(
-								request.producerExecutionId,
+				ResultSubpartitionView queueIterator =
+						partitionProvider.getSubpartition(
 								request.partitionId,
 								request.queueIndex,
-								Optional.<BufferProvider>absent());
+								Optional.<BufferProvider>of(bufferPool));
 
 				if (queueIterator != null) {
 					outboundQueue.enqueue(queueIterator, request.receiverId);
@@ -82,7 +108,7 @@ class PartitionRequestServerHandler extends SimpleChannelInboundHandler<NettyMes
 			else if (msgClazz == TaskEventRequest.class) {
 				TaskEventRequest request = (TaskEventRequest) msg;
 
-				if (!taskEventDispatcher.publish(request.executionId, request.partitionId, request.event)) {
+				if (!taskEventDispatcher.publish(request.partitionId, request.event)) {
 					respondWithError(ctx, new IllegalArgumentException("Task event receiver not found."), request.receiverId);
 				}
 			}


[05/13] flink git commit: [FLINK-1350] [runtime] Add blocking result partition variant

Posted by uc...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java
index 297eeed..435588f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java
@@ -18,8 +18,7 @@
 
 package org.apache.flink.runtime.io.disk.iomanager;
 
-import static org.junit.Assert.*;
-
+import org.apache.flink.core.memory.MemorySegment;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -29,12 +28,8 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.io.disk.iomanager.BlockChannelReader;
-import org.apache.flink.runtime.io.disk.iomanager.BlockChannelWriter;
-import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel;
-import org.apache.flink.runtime.io.disk.iomanager.ReadRequest;
-import org.apache.flink.runtime.io.disk.iomanager.WriteRequest;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 public class IOManagerAsyncTest {
 	
@@ -65,7 +60,7 @@ public class IOManagerAsyncTest {
 		
 		try {
 			final FileIOChannel.ID channelID = this.ioManager.createChannel();
-			final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channelID);
+			final BlockChannelWriter<MemorySegment> writer = this.ioManager.createBlockChannelWriter(channelID);
 			
 			MemorySegment memSeg = new MemorySegment(new byte[32 * 1024]);
 			
@@ -75,15 +70,15 @@ public class IOManagerAsyncTest {
 				}
 				
 				writer.writeBlock(memSeg);
-				memSeg = writer.getNextReturnedSegment();
+				memSeg = writer.getNextReturnedBlock();
 			}
 			
 			writer.close();
 			
-			final BlockChannelReader reader = this.ioManager.createBlockChannelReader(channelID);
+			final BlockChannelReader<MemorySegment> reader = this.ioManager.createBlockChannelReader(channelID);
 			for (int i = 0; i < NUM_IOS; i++) {
 				reader.readBlock(memSeg);
-				memSeg = reader.getNextReturnedSegment();
+				memSeg = reader.getNextReturnedBlock();
 				
 				for (int pos = 0; pos < memSeg.size(); pos += 4) {
 					if (memSeg.getInt(pos) != i) {
@@ -112,10 +107,10 @@ public class IOManagerAsyncTest {
 			}
 			
 			final FileIOChannel.ID channelID = this.ioManager.createChannel();
-			final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channelID);
+			final BlockChannelWriter<MemorySegment> writer = this.ioManager.createBlockChannelWriter(channelID);
 			
 			for (int i = 0; i < NUM_IOS; i++) {
-				final MemorySegment memSeg = memSegs.isEmpty() ? writer.getNextReturnedSegment() : memSegs.remove(memSegs.size() - 1);
+				final MemorySegment memSeg = memSegs.isEmpty() ? writer.getNextReturnedBlock() : memSegs.remove(memSegs.size() - 1);
 				
 				for (int pos = 0; pos < memSeg.size(); pos += 4) {
 					memSeg.putInt(pos, i);
@@ -127,16 +122,16 @@ public class IOManagerAsyncTest {
 			
 			// get back the memory
 			while (memSegs.size() < NUM_SEGS) {
-				memSegs.add(writer.getNextReturnedSegment());
+				memSegs.add(writer.getNextReturnedBlock());
 			}
 			
-			final BlockChannelReader reader = this.ioManager.createBlockChannelReader(channelID);
+			final BlockChannelReader<MemorySegment> reader = this.ioManager.createBlockChannelReader(channelID);
 			while(!memSegs.isEmpty()) {
 				reader.readBlock(memSegs.remove(0));
 			}
 			
 			for (int i = 0; i < NUM_IOS; i++) {
-				final MemorySegment memSeg = reader.getNextReturnedSegment();
+				final MemorySegment memSeg = reader.getNextReturnedBlock();
 				
 				for (int pos = 0; pos < memSeg.size(); pos += 4) {
 					if (memSeg.getInt(pos) != i) {
@@ -150,7 +145,7 @@ public class IOManagerAsyncTest {
 			
 			// get back the memory
 			while (memSegs.size() < NUM_SEGS) {
-				memSegs.add(reader.getNextReturnedSegment());
+				memSegs.add(reader.getNextReturnedBlock());
 			}
 		}
 		catch (Exception ex) {

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java
index f1d5337..9abedb3 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java
@@ -79,13 +79,14 @@ public class IOManagerITCase {
 	 * parallel. It is designed to check the ability of the IO manager to correctly handle multiple threads.
 	 */
 	@Test
+	@SuppressWarnings("unchecked")
 	public void parallelChannelsTest() throws Exception {
 		final Random rnd = new Random(SEED);
 		final AbstractInvokable memOwner = new DefaultMemoryManagerTest.DummyInvokable();
 		
 		FileIOChannel.ID[] ids = new FileIOChannel.ID[NUM_CHANNELS];
-		BlockChannelWriter[] writers = new BlockChannelWriter[NUM_CHANNELS];
-		BlockChannelReader[] readers = new BlockChannelReader[NUM_CHANNELS];
+		BlockChannelWriter<MemorySegment>[] writers = new BlockChannelWriter[NUM_CHANNELS];
+		BlockChannelReader<MemorySegment>[] readers = new BlockChannelReader[NUM_CHANNELS];
 		ChannelWriterOutputView[] outs = new ChannelWriterOutputView[NUM_CHANNELS];
 		ChannelReaderInputView[] ins = new ChannelReaderInputView[NUM_CHANNELS];
 		
@@ -126,7 +127,7 @@ public class IOManagerITCase {
 			
 			List<MemorySegment> memSegs = this.memoryManager.allocatePages(memOwner, rnd.nextInt(MAXIMUM_NUMBER_OF_SEGMENTS_PER_CHANNEL - 1) + 1);
 				
-			final BlockChannelReader reader = this.ioManager.createBlockChannelReader(ids[i]);
+			final BlockChannelReader<MemorySegment> reader = this.ioManager.createBlockChannelReader(ids[i]);
 			final ChannelReaderInputView in = new ChannelReaderInputView(reader, memSegs, false);
 			int nextVal = 0;
 			

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java
index 9c129e6..89cc50d 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java
@@ -112,8 +112,8 @@ public class IOManagerPerformanceBenchmark {
 		final List<MemorySegment> memory = this.memManager.allocatePages(memoryOwner, numSegments);
 		final FileIOChannel.ID channel = this.ioManager.createChannel();
 		
-		BlockChannelWriter writer = null;
-		BlockChannelReader reader = null;
+		BlockChannelWriter<MemorySegment> writer = null;
+		BlockChannelReader<MemorySegment> reader = null;
 		
 		try {	
 			writer = this.ioManager.createBlockChannelWriter(channel);

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerTest.java
index 5f5bed3..39cb8ee 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerTest.java
@@ -20,9 +20,11 @@ package org.apache.flink.runtime.io.disk.iomanager;
 
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel.ID;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
 import org.junit.Test;
 
 import java.io.File;
+import java.io.IOException;
 import java.util.List;
 import java.util.concurrent.LinkedBlockingQueue;
 
@@ -90,17 +92,32 @@ public class IOManagerTest {
 		}
 
 		@Override
-		public BlockChannelWriter createBlockChannelWriter(ID channelID, LinkedBlockingQueue<MemorySegment> returnQueue) {
+		public BlockChannelWriter<MemorySegment> createBlockChannelWriter(ID channelID, LinkedBlockingQueue<MemorySegment> returnQueue) {
 			throw new UnsupportedOperationException();
 		}
 
 		@Override
-		public BlockChannelWriterWithCallback createBlockChannelWriter(ID channelID, RequestDoneCallback<MemorySegment> callback) {
+		public BlockChannelWriterWithCallback<MemorySegment> createBlockChannelWriter(ID channelID, RequestDoneCallback<MemorySegment> callback) {
 			throw new UnsupportedOperationException();
 		}
 
 		@Override
-		public BlockChannelReader createBlockChannelReader(ID channelID, LinkedBlockingQueue<MemorySegment> returnQueue) {
+		public BlockChannelReader<MemorySegment> createBlockChannelReader(ID channelID, LinkedBlockingQueue<MemorySegment> returnQueue) {
+			throw new UnsupportedOperationException();
+		}
+
+		@Override
+		public BufferFileWriter createBufferFileWriter(ID channelID) throws IOException {
+			throw new UnsupportedOperationException();
+		}
+
+		@Override
+		public BufferFileReader createBufferFileReader(ID channelID, RequestDoneCallback<Buffer> callback) throws IOException {
+			throw new UnsupportedOperationException();
+		}
+
+		@Override
+		public BufferFileSegmentReader createBufferFileSegmentReader(ID channelID, RequestDoneCallback<FileSegment> callback) throws IOException {
 			throw new UnsupportedOperationException();
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/AbstractReaderTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/AbstractReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/AbstractReaderTest.java
index 0d8183d..4e5fb40 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/AbstractReaderTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/AbstractReaderTest.java
@@ -44,6 +44,7 @@ import static org.mockito.Mockito.when;
 public class AbstractReaderTest {
 
 	@Test
+	@SuppressWarnings("unchecked")
 	public void testTaskEvent() throws Exception {
 		final AbstractReader reader = new MockReader(createInputGate(1));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/BufferReaderTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/BufferReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/BufferReaderTest.java
index f11c0db..7d3dbbe 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/BufferReaderTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/BufferReaderTest.java
@@ -20,7 +20,7 @@ package org.apache.flink.runtime.io.network.api.reader;
 
 import org.apache.flink.runtime.event.task.TaskEvent;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.util.MockSingleInputGate;
+import org.apache.flink.runtime.io.network.util.TestSingleInputGate;
 import org.apache.flink.runtime.io.network.util.TestTaskEvent;
 import org.apache.flink.runtime.taskmanager.Task;
 import org.apache.flink.runtime.util.event.EventListener;
@@ -39,12 +39,13 @@ import static org.mockito.Mockito.verify;
 
 @RunWith(PowerMockRunner.class)
 @PrepareForTest(Task.class)
+@SuppressWarnings("unchecked")
 public class BufferReaderTest {
 
 	@Test
 	public void testGetNextBufferOrEvent() throws IOException, InterruptedException {
 
-		final MockSingleInputGate inputGate = new MockSingleInputGate(1)
+		final TestSingleInputGate inputGate = new TestSingleInputGate(1)
 				.readBuffer().readBuffer().readEvent()
 				.readBuffer().readBuffer().readEvent()
 				.readBuffer().readEndOfPartitionEvent();
@@ -67,7 +68,7 @@ public class BufferReaderTest {
 	@Test
 	public void testIterativeGetNextBufferOrEvent() throws IOException, InterruptedException {
 
-		final MockSingleInputGate inputGate = new MockSingleInputGate(1)
+		final TestSingleInputGate inputGate = new TestSingleInputGate(1)
 				.readBuffer().readBuffer().readEvent()
 				.readBuffer().readBuffer().readEvent()
 				.readBuffer().readEndOfSuperstepEvent()

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/IteratorWrappingMockSingleInputGate.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/IteratorWrappingMockSingleInputGate.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/IteratorWrappingMockSingleInputGate.java
deleted file mode 100644
index 614fb8d..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/IteratorWrappingMockSingleInputGate.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.io.network.api.reader;
-
-import org.apache.flink.core.io.IOReadableWritable;
-import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
-import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
-import org.apache.flink.runtime.io.network.api.serialization.RecordSerializer;
-import org.apache.flink.runtime.io.network.api.serialization.SpanningRecordSerializer;
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
-import org.apache.flink.runtime.io.network.util.MockInputChannel;
-import org.apache.flink.runtime.io.network.util.MockSingleInputGate;
-import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
-import org.apache.flink.util.InstantiationUtil;
-import org.apache.flink.util.MutableObjectIterator;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import java.io.IOException;
-
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-public class IteratorWrappingMockSingleInputGate<T extends IOReadableWritable> extends MockSingleInputGate {
-
-	private final MockInputChannel inputChannel = new MockInputChannel(inputGate, 0);
-
-	private final int bufferSize;
-
-	private MutableObjectIterator<T> inputIterator;
-
-	private RecordSerializer<T> serializer;
-
-	private final T reuse;
-
-	public IteratorWrappingMockSingleInputGate(int bufferSize, Class<T> recordType, MutableObjectIterator<T> iterator) throws IOException, InterruptedException {
-		super(1, false);
-
-		this.bufferSize = bufferSize;
-		this.reuse = InstantiationUtil.instantiate(recordType);
-
-		wrapIterator(iterator);
-	}
-
-	private IteratorWrappingMockSingleInputGate<T> wrapIterator(MutableObjectIterator<T> iterator) throws IOException, InterruptedException {
-		inputIterator = iterator;
-		serializer = new SpanningRecordSerializer<T>();
-
-		// The input iterator can produce an infinite stream. That's why we have to serialize each
-		// record on demand and cannot do it upfront.
-		final Answer<Buffer> answer = new Answer<Buffer>() {
-			@Override
-			public Buffer answer(InvocationOnMock invocationOnMock) throws Throwable {
-				if (inputIterator.next(reuse) != null) {
-					final Buffer buffer = new Buffer(new MemorySegment(new byte[bufferSize]), mock(BufferRecycler.class));
-					serializer.setNextBuffer(buffer);
-					serializer.addRecord(reuse);
-
-					inputGate.onAvailableBuffer(inputChannel.getInputChannel());
-
-					// Call getCurrentBuffer to ensure size is set
-					return serializer.getCurrentBuffer();
-				}
-				else {
-
-					when(inputChannel.getInputChannel().isReleased()).thenReturn(true);
-
-					return EventSerializer.toBuffer(EndOfPartitionEvent.INSTANCE);
-				}
-			}
-		};
-
-		when(inputChannel.getInputChannel().getNextBuffer()).thenAnswer(answer);
-
-		inputGate.setInputChannel(new IntermediateResultPartitionID(), inputChannel.getInputChannel());
-
-		return this;
-	}
-
-	public IteratorWrappingMockSingleInputGate<T> read() {
-		inputGate.onAvailableBuffer(inputChannel.getInputChannel());
-
-		return this;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/IteratorWrappingTestSingleInputGate.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/IteratorWrappingTestSingleInputGate.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/IteratorWrappingTestSingleInputGate.java
new file mode 100644
index 0000000..3968eda
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/IteratorWrappingTestSingleInputGate.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.api.reader;
+
+import org.apache.flink.core.io.IOReadableWritable;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.api.serialization.RecordSerializer;
+import org.apache.flink.runtime.io.network.api.serialization.SpanningRecordSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
+import org.apache.flink.runtime.io.network.util.TestInputChannel;
+import org.apache.flink.runtime.io.network.util.TestSingleInputGate;
+import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.MutableObjectIterator;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.IOException;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class IteratorWrappingTestSingleInputGate<T extends IOReadableWritable> extends TestSingleInputGate {
+
+	private final TestInputChannel inputChannel = new TestInputChannel(inputGate, 0);
+
+	private final int bufferSize;
+
+	private MutableObjectIterator<T> inputIterator;
+
+	private RecordSerializer<T> serializer;
+
+	private final T reuse;
+
+	public IteratorWrappingTestSingleInputGate(int bufferSize, Class<T> recordType, MutableObjectIterator<T> iterator) throws IOException, InterruptedException {
+		super(1, false);
+
+		this.bufferSize = bufferSize;
+		this.reuse = InstantiationUtil.instantiate(recordType);
+
+		wrapIterator(iterator);
+	}
+
+	private IteratorWrappingTestSingleInputGate<T> wrapIterator(MutableObjectIterator<T> iterator) throws IOException, InterruptedException {
+		inputIterator = iterator;
+		serializer = new SpanningRecordSerializer<T>();
+
+		// The input iterator can produce an infinite stream. That's why we have to serialize each
+		// record on demand and cannot do it upfront.
+		final Answer<Buffer> answer = new Answer<Buffer>() {
+			@Override
+			public Buffer answer(InvocationOnMock invocationOnMock) throws Throwable {
+				if (inputIterator.next(reuse) != null) {
+					final Buffer buffer = new Buffer(new MemorySegment(new byte[bufferSize]), mock(BufferRecycler.class));
+					serializer.setNextBuffer(buffer);
+					serializer.addRecord(reuse);
+
+					inputGate.onAvailableBuffer(inputChannel.getInputChannel());
+
+					// Call getCurrentBuffer to ensure size is set
+					return serializer.getCurrentBuffer();
+				}
+				else {
+
+					when(inputChannel.getInputChannel().isReleased()).thenReturn(true);
+
+					return EventSerializer.toBuffer(EndOfPartitionEvent.INSTANCE);
+				}
+			}
+		};
+
+		when(inputChannel.getInputChannel().getNextBuffer()).thenAnswer(answer);
+
+		inputGate.setInputChannel(new IntermediateResultPartitionID(), inputChannel.getInputChannel());
+
+		return this;
+	}
+
+	public IteratorWrappingTestSingleInputGate<T> read() {
+		inputGate.onAvailableBuffer(inputChannel.getInputChannel());
+
+		return this;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializationTest.java
index be63fe5..cd6d580 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializationTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializationTest.java
@@ -24,7 +24,6 @@ import org.apache.flink.runtime.io.network.api.serialization.types.Serialization
 import org.apache.flink.runtime.io.network.api.serialization.types.Util;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
 import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
-import org.apache.flink.runtime.io.network.serialization.SpillingAdaptiveSpanningRecordDeserializer;
 import org.junit.Assert;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactoryTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactoryTest.java
index cf6eb9a..28862e8 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactoryTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactoryTest.java
@@ -124,7 +124,7 @@ public class BufferPoolFactoryTest {
 
 		assertEquals(networkBufferPool.getTotalNumberOfMemorySegments() / 2, second.getNumBuffers());
 
-		first.destroy();
+		first.lazyDestroy();
 
 		assertEquals(networkBufferPool.getTotalNumberOfMemorySegments(), second.getNumBuffers());
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferTest.java
index 17a079c..2630608 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferTest.java
@@ -21,13 +21,8 @@ package org.apache.flink.runtime.io.network.buffer;
 import org.apache.flink.core.memory.MemorySegment;
 import org.junit.Assert;
 import org.junit.Test;
-import org.mockito.Matchers;
 import org.mockito.Mockito;
 
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.lang.reflect.Modifier;
-
 public class BufferTest {
 
 	@Test
@@ -55,51 +50,4 @@ public class BufferTest {
 			// OK => expected exception
 		}
 	}
-
-	@Test
-	public void testExceptionAfterRecycle() throws Throwable {
-		final MemorySegment segment = new MemorySegment(new byte[1024]);
-		final BufferRecycler recycler = Mockito.mock(BufferRecycler.class);
-
-		final Buffer buffer = new Buffer(segment, recycler);
-
-		buffer.recycle();
-
-		// Verify that the buffer has been recycled
-		Mockito.verify(recycler, Mockito.times(1)).recycle(Matchers.any(MemorySegment.class));
-
-		final Buffer spyBuffer = Mockito.spy(buffer);
-
-		// Check that every method throws the appropriate exception after the
-		// buffer has been recycled.
-		//
-		// Note: We cannot directly work on the spied upon buffer to get the
-		// declared methods as Mockito adds some of its own.
-		for (final Method method : buffer.getClass().getDeclaredMethods()) {
-			if (Modifier.isPublic(method.getModifiers()) && !method.getName().equals("toString")
-					&& !method.getName().equals("isRecycled") && !method.getName().equals("isBuffer")) {
-				// Get method of the spied buffer to allow argument matchers
-				final Method spyMethod = spyBuffer.getClass().getDeclaredMethod(method.getName(), method.getParameterTypes());
-
-				final Class<?>[] paramTypes = spyMethod.getParameterTypes();
-				final Object[] params = new Object[paramTypes.length];
-
-				for (int i = 0; i < params.length; i++) {
-					params[i] = Matchers.any(paramTypes[i]);
-				}
-
-				try {
-					spyMethod.invoke(spyBuffer, params);
-					Assert.fail("Didn't throw expected exception for method: " + method.getName());
-				} catch (InvocationTargetException e) {
-					if (e.getTargetException() instanceof IllegalStateException) {
-						// OK => expected exception
-					}
-					else {
-						throw e.getTargetException();
-					}
-				}
-			}
-		}
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java
index 8609db0..c9a4b6d 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java
@@ -18,8 +18,10 @@
 
 package org.apache.flink.runtime.io.network.buffer;
 
+import com.google.common.collect.Lists;
 import org.apache.flink.runtime.util.event.EventListener;
 import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Matchers;
@@ -29,10 +31,12 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -52,6 +56,8 @@ public class LocalBufferPoolTest {
 
 	private BufferPool localBufferPool;
 
+	private final static ExecutorService executor = Executors.newCachedThreadPool();
+
 	@Before
 	public void setupLocalBufferPool() {
 		networkBufferPool = new NetworkBufferPool(numBuffers, memorySegmentSize);
@@ -63,13 +69,18 @@ public class LocalBufferPoolTest {
 	@After
 	public void destroyAndVerifyAllBuffersReturned() throws IOException {
 		if (!localBufferPool.isDestroyed()) {
-			localBufferPool.destroy();
+			localBufferPool.lazyDestroy();
 		}
 
 		String msg = "Did not return all buffers to memory segment pool after test.";
 		assertEquals(msg, numBuffers, networkBufferPool.getNumberOfAvailableMemorySegments());
 	}
 
+	@AfterClass
+	public static void shutdownExecutor() {
+		executor.shutdownNow();
+	}
+
 	@Test
 	public void testRequestMoreThanAvailable() throws IOException {
 		localBufferPool.setNumBuffers(numBuffers);
@@ -99,7 +110,7 @@ public class LocalBufferPoolTest {
 
 	@Test
 	public void testRequestAfterDestroy() throws IOException {
-		localBufferPool.destroy();
+		localBufferPool.lazyDestroy();
 
 		assertNull(localBufferPool.requestBuffer());
 	}
@@ -114,7 +125,7 @@ public class LocalBufferPoolTest {
 			requests.add(localBufferPool.requestBuffer());
 		}
 
-		localBufferPool.destroy();
+		localBufferPool.lazyDestroy();
 
 		// All buffers have been requested, but can not be returned yet.
 		assertEquals(numBuffers, getNumRequestedFromMemorySegmentPool());
@@ -212,6 +223,7 @@ public class LocalBufferPoolTest {
 	}
 
 	@Test
+	@SuppressWarnings("unchecked")
 	public void testCancelPendingRequestsAfterDestroy() throws IOException {
 		EventListener<Buffer> listener = Mockito.mock(EventListener.class);
 
@@ -224,7 +236,7 @@ public class LocalBufferPoolTest {
 
 		localBufferPool.addListener(listener);
 
-		localBufferPool.destroy();
+		localBufferPool.lazyDestroy();
 
 		available.recycle();
 
@@ -236,25 +248,69 @@ public class LocalBufferPoolTest {
 	// ------------------------------------------------------------------------
 
 	@Test
+	@SuppressWarnings("unchecked")
 	public void testConcurrentRequestRecycle() throws ExecutionException, InterruptedException, IOException {
 		int numConcurrentTasks = 128;
 		int numBuffersToRequestPerTask = 1024;
 
 		localBufferPool.setNumBuffers(numConcurrentTasks);
 
-		final ExecutorService executor = Executors.newCachedThreadPool();
+		Future<Boolean>[] taskResults = new Future[numConcurrentTasks];
+		for (int i = 0; i < numConcurrentTasks; i++) {
+			taskResults[i] = executor.submit(new BufferRequesterTask(localBufferPool, numBuffersToRequestPerTask));
+		}
 
-		try {
-			Future<Boolean>[] taskResults = new Future[numConcurrentTasks];
-			for (int i = 0; i < numConcurrentTasks; i++) {
-				taskResults[i] = executor.submit(new BufferRequesterTask(localBufferPool, numBuffersToRequestPerTask));
-			}
+		for (int i = 0; i < numConcurrentTasks; i++) {
+			assertTrue(taskResults[i].get());
+		}
+	}
 
-			for (int i = 0; i < numConcurrentTasks; i++) {
-				assertTrue(taskResults[i].get());
+	@Test
+	public void testDestroyDuringBlockingRequest() throws Exception {
+		// Config
+		final int numberOfBuffers = 1;
+
+		localBufferPool.setNumBuffers(numberOfBuffers);
+
+		final CountDownLatch sync = new CountDownLatch(1);
+
+		final Callable<List<Buffer>> requester = new Callable<List<Buffer>>() {
+
+			// Request all buffers in a blocking manner.
+			@Override
+			public List<Buffer> call() throws Exception {
+				final List<Buffer> requested = Lists.newArrayList();
+
+				// Request all available buffers
+				for (int i = 0; i < numberOfBuffers; i++) {
+					requested.add(localBufferPool.requestBufferBlocking());
+				}
+
+				// Notify that we've requested all buffers
+				sync.countDown();
+
+				// Try to request the next buffer (but pool should be destroyed either right before
+				// the request or more likely during the request).
+				assertNull(localBufferPool.requestBufferBlocking());
+
+				return requested;
 			}
-		} finally {
-			executor.shutdownNow();
+		};
+
+		Future<List<Buffer>> f = executor.submit(requester);
+
+		sync.await();
+
+		localBufferPool.lazyDestroy();
+
+		// Increase the likelihood that the requested is currently in the request call
+		Thread.sleep(50);
+
+		// This should return immediately if everything works as expected
+		List<Buffer> requestedBuffers = f.get(60, TimeUnit.SECONDS);
+
+		for (Buffer buffer : requestedBuffers) {
+			buffer.recycle();
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageSerializationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageSerializationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageSerializationTest.java
index 991af13..b2fe8a9 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageSerializationTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageSerializationTest.java
@@ -25,6 +25,7 @@ import org.apache.flink.runtime.event.task.IntegerTaskEvent;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
 import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
 import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
 import org.junit.Test;
@@ -123,33 +124,31 @@ public class NettyMessageSerializationTest {
 		}
 
 		{
-			NettyMessage.PartitionRequest expected = new NettyMessage.PartitionRequest(new ExecutionAttemptID(), new IntermediateResultPartitionID(), random.nextInt(), new InputChannelID());
+			NettyMessage.PartitionRequest expected = new NettyMessage.PartitionRequest(new ResultPartitionID(new IntermediateResultPartitionID(), new ExecutionAttemptID()), random.nextInt(), new InputChannelID());
 			NettyMessage.PartitionRequest actual = encodeAndDecode(expected);
 
-			assertEquals(expected.producerExecutionId, actual.producerExecutionId);
 			assertEquals(expected.partitionId, actual.partitionId);
 			assertEquals(expected.queueIndex, actual.queueIndex);
 			assertEquals(expected.receiverId, actual.receiverId);
 		}
 
 		{
-			NettyMessage.TaskEventRequest expected = new NettyMessage.TaskEventRequest(new IntegerTaskEvent(random.nextInt()), new ExecutionAttemptID(), new IntermediateResultPartitionID(), new InputChannelID());
+			NettyMessage.TaskEventRequest expected = new NettyMessage.TaskEventRequest(new IntegerTaskEvent(random.nextInt()), new ResultPartitionID(new IntermediateResultPartitionID(), new ExecutionAttemptID()), new InputChannelID());
 			NettyMessage.TaskEventRequest actual = encodeAndDecode(expected);
 
-			assertEquals(expected.executionId, actual.executionId);
 			assertEquals(expected.event, actual.event);
 			assertEquals(expected.partitionId, actual.partitionId);
 			assertEquals(expected.receiverId, actual.receiverId);
 		}
 	}
 
+	@SuppressWarnings("unchecked")
 	private <T extends NettyMessage> T encodeAndDecode(T msg) {
 		channel.writeOutbound(msg);
 		ByteBuf encoded = (ByteBuf) channel.readOutbound();
 
 		channel.writeInbound(encoded);
 
-
 		return (T) channel.readInbound();
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java
index 2f45d6b..3e7d3a3 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java
@@ -24,7 +24,7 @@ import io.netty.channel.ChannelPipeline;
 import io.netty.channel.ChannelPromise;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.io.network.RemoteAddress;
+import org.apache.flink.runtime.io.network.ConnectionID;
 import org.junit.Test;
 
 import java.io.IOException;
@@ -74,10 +74,10 @@ public class PartitionRequestClientFactoryTest {
 			final Thread connect = new Thread(new Runnable() {
 				@Override
 				public void run() {
-					RemoteAddress serverAddress = null;
+					ConnectionID serverAddress = null;
 
 					try {
-						serverAddress = createServerRemoteAddress(0);
+						serverAddress = createServerConnectionID(0);
 
 						// This triggers a connect
 						factory.createPartitionRequestClient(serverAddress);
@@ -176,7 +176,7 @@ public class PartitionRequestClientFactoryTest {
 		return new Tuple2<NettyServer, NettyClient>(server, client);
 	}
 
-	private static RemoteAddress createServerRemoteAddress(int connectionIndex) throws UnknownHostException {
-		return new RemoteAddress(new InetSocketAddress(InetAddress.getLocalHost(), SERVER_PORT), connectionIndex);
+	private static ConnectionID createServerConnectionID(int connectionIndex) throws UnknownHostException {
+		return new ConnectionID(new InetSocketAddress(InetAddress.getLocalHost(), SERVER_PORT), connectionIndex);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandlerTest.java
index d5df859..1c81db0 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandlerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandlerTest.java
@@ -92,6 +92,7 @@ public class PartitionRequestClientHandlerTest {
 	/**
 	 * Returns a mocked input channel in a state as it was released during a decode.
 	 */
+	@SuppressWarnings("unchecked")
 	private RemoteInputChannel createMockReleasedInputChannel(InputChannelID channelId) throws IOException {
 		final BufferProvider bufferProvider = mock(BufferProvider.class);
 		when(bufferProvider.requestBuffer()).thenReturn(null);

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java
new file mode 100644
index 0000000..68c12f4
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java
@@ -0,0 +1,278 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition;
+
+import com.google.common.base.Optional;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.runtime.event.task.AbstractEvent;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferProvider;
+import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
+import org.apache.flink.runtime.io.network.util.TestPooledBufferProvider;
+import org.apache.flink.runtime.io.network.util.TestConsumerCallback;
+import org.apache.flink.runtime.io.network.util.TestNotificationListener;
+import org.apache.flink.runtime.io.network.util.TestProducerSource;
+import org.apache.flink.runtime.io.network.util.TestSubpartitionConsumer;
+import org.apache.flink.runtime.io.network.util.TestSubpartitionProducer;
+import org.junit.AfterClass;
+import org.junit.Test;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import static org.apache.flink.runtime.io.network.util.TestBufferFactory.createBuffer;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+
+public class PipelinedSubpartitionTest extends SubpartitionTestBase {
+
+	/** Executor service for concurrent produce/consume tests */
+	private final static ExecutorService executorService = Executors.newCachedThreadPool();
+
+	@AfterClass
+	public static void shutdownExecutorService() throws Exception {
+		executorService.shutdownNow();
+	}
+
+	@Override
+	PipelinedSubpartition createSubpartition() {
+		final ResultPartition parent = mock(ResultPartition.class);
+
+		return new PipelinedSubpartition(0, parent);
+	}
+
+	@Test
+	public void testRegisterListener() throws Exception {
+		final PipelinedSubpartition subpartition = createSubpartition();
+
+		final TestNotificationListener listener = new TestNotificationListener();
+
+		// Register a listener
+		assertTrue(subpartition.registerListener(listener));
+
+		// Try to register another listener
+		try {
+			subpartition.registerListener(listener);
+
+			fail("Did not throw expected exception after duplicate listener registration.");
+		}
+		catch (IllegalStateException expected) {
+		}
+	}
+
+	@Test
+	public void testListenerNotification() throws Exception {
+		final TestNotificationListener listener = new TestNotificationListener();
+		assertEquals(0, listener.getNumberOfNotifications());
+
+		{
+			final PipelinedSubpartition subpartition = createSubpartition();
+
+			// Register a listener
+			assertTrue(subpartition.registerListener(listener));
+
+			// Notify on add and remove listener
+			subpartition.add(mock(Buffer.class));
+			assertEquals(1, listener.getNumberOfNotifications());
+
+			// No notification, should have removed listener after first notification
+			subpartition.add(mock(Buffer.class));
+			assertEquals(1, listener.getNumberOfNotifications());
+		}
+
+		{
+			final PipelinedSubpartition subpartition = createSubpartition();
+
+			// Register a listener
+			assertTrue(subpartition.registerListener(listener));
+
+			// Notify on finish
+			subpartition.finish();
+			assertEquals(2, listener.getNumberOfNotifications());
+		}
+
+		{
+			final PipelinedSubpartition subpartition = createSubpartition();
+
+			// Register a listener
+			assertTrue(subpartition.registerListener(listener));
+
+			// Notify on release
+			subpartition.release();
+			assertEquals(3, listener.getNumberOfNotifications());
+		}
+	}
+
+	@Test
+	public void testIllegalReadViewRequest() throws Exception {
+		final PipelinedSubpartition subpartition = createSubpartition();
+
+		// Successful request
+		assertNotNull(subpartition.getReadView(Optional.<BufferProvider>absent()));
+
+		try {
+			subpartition.getReadView(Optional.<BufferProvider>absent());
+
+			fail("Did not throw expected exception after duplicate read view request.");
+		}
+		catch (IllegalStateException expected) {
+		}
+	}
+
+	@Test
+	public void testBasicPipelinedProduceConsumeLogic() throws Exception {
+		final PipelinedSubpartition subpartition = createSubpartition();
+
+		TestNotificationListener listener = new TestNotificationListener();
+
+		ResultSubpartitionView view = subpartition.getReadView(Optional.<BufferProvider>absent());
+
+		// Empty => should return null
+		assertNull(view.getNextBuffer());
+
+		// Register listener for notifications
+		assertTrue(view.registerListener(listener));
+
+		assertEquals(0, listener.getNumberOfNotifications());
+
+		// Add data to the queue...
+		subpartition.add(createBuffer());
+
+		// ...should have resulted in a notification
+		assertEquals(1, listener.getNumberOfNotifications());
+
+		// ...and one available result
+		assertNotNull(view.getNextBuffer());
+		assertNull(view.getNextBuffer());
+
+		// Add data to the queue...
+		subpartition.add(createBuffer());
+		// ...don't allow to subscribe, if data is available
+		assertFalse(view.registerListener(listener));
+
+		assertEquals(1, listener.getNumberOfNotifications());
+	}
+
+	@Test
+	public void testConcurrentFastProduceAndFastConsume() throws Exception {
+		testProduceConsume(false, false);
+	}
+
+	@Test
+	public void testConcurrentFastProduceAndSlowConsume() throws Exception {
+		testProduceConsume(false, true);
+	}
+
+	@Test
+	public void testConcurrentSlowProduceAndFastConsume() throws Exception {
+		testProduceConsume(true, false);
+	}
+
+	@Test
+	public void testConcurrentSlowProduceAndSlowConsume() throws Exception {
+		testProduceConsume(true, true);
+	}
+
+	private void testProduceConsume(boolean isSlowProducer, boolean isSlowConsumer) throws Exception {
+		// Config
+		final int producerBufferPoolSize = 8;
+		final int producerNumberOfBuffersToProduce = 128;
+
+		// Producer behaviour
+		final TestProducerSource producerSource = new TestProducerSource() {
+
+			private BufferProvider bufferProvider = new TestPooledBufferProvider(producerBufferPoolSize);
+
+			private int numberOfBuffers;
+
+			@Override
+			public BufferOrEvent getNextBufferOrEvent() throws Exception {
+
+				if (numberOfBuffers == producerNumberOfBuffersToProduce) {
+					return null;
+				}
+
+				final Buffer buffer = bufferProvider.requestBufferBlocking();
+
+				final MemorySegment segment = buffer.getMemorySegment();
+
+				int next = numberOfBuffers * (segment.size() / 4);
+
+				for (int i = 0; i < segment.size(); i += 4) {
+					segment.putInt(i, next);
+
+					next++;
+				}
+
+				numberOfBuffers++;
+
+				return new BufferOrEvent(buffer, 0);
+			}
+		};
+
+		// Consumer behaviour
+		final TestConsumerCallback consumerCallback = new TestConsumerCallback() {
+
+			private int numberOfBuffers;
+
+			@Override
+			public void onBuffer(Buffer buffer) {
+				final MemorySegment segment = buffer.getMemorySegment();
+
+				int expected = numberOfBuffers * (segment.size() / 4);
+
+				for (int i = 0; i < segment.size(); i += 4) {
+					assertEquals(expected, segment.getInt(i));
+
+					expected++;
+				}
+
+				numberOfBuffers++;
+
+				buffer.recycle();
+			}
+
+			@Override
+			public void onEvent(AbstractEvent event) {
+				// Nothing to do in this test
+			}
+		};
+
+		final PipelinedSubpartition subpartition = createSubpartition();
+
+		final PipelinedSubpartitionView view = subpartition.getReadView(
+				Optional.<BufferProvider>absent());
+
+		Future<Boolean> producer = executorService.submit(
+				new TestSubpartitionProducer(subpartition, isSlowProducer, producerSource));
+
+		Future<Boolean> consumer = executorService.submit(
+				new TestSubpartitionConsumer(view, isSlowConsumer, consumerCallback));
+
+		// Wait for producer and consumer to finish
+		producer.get();
+		consumer.get();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java
new file mode 100644
index 0000000..8c8692d
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition;
+
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
+import org.junit.AfterClass;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import static org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode.*;
+import static org.mockito.Mockito.mock;
+
+public class SpillableSubpartitionTest extends SubpartitionTestBase {
+
+	/** Executor service for concurrent produce/consume tests */
+	private final static ExecutorService executorService = Executors.newCachedThreadPool();
+
+	/** Asynchronous I/O manager */
+	private static final IOManager ioManager = new IOManagerAsync();
+
+	@AfterClass
+	public static void shutdown() {
+		executorService.shutdownNow();
+		ioManager.shutdown();
+	}
+
+	@Override
+	ResultSubpartition createSubpartition() {
+		return new SpillableSubpartition(0, mock(ResultPartition.class), ioManager, SYNC);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewAsyncIOTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewAsyncIOTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewAsyncIOTest.java
new file mode 100644
index 0000000..981c8ee
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewAsyncIOTest.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition;
+
+import org.apache.flink.runtime.io.disk.iomanager.BufferFileWriter;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
+import org.apache.flink.runtime.io.network.util.TestConsumerCallback;
+import org.apache.flink.runtime.io.network.util.TestInfiniteBufferProvider;
+import org.apache.flink.runtime.io.network.util.TestPooledBufferProvider;
+import org.apache.flink.runtime.io.network.util.TestSubpartitionConsumer;
+import org.junit.AfterClass;
+import org.junit.Test;
+
+import static org.mockito.Mockito.mock;
+
+public class SpilledSubpartitionViewAsyncIOTest {
+
+	private static final IOManager ioManager = new IOManagerAsync();
+
+	@AfterClass
+	public static void shutdown() {
+		ioManager.shutdown();
+	}
+
+	@Test
+	public void testWriteConsume() throws Exception {
+		// Config
+		final int numberOfBuffersToWrite = 1024;
+
+		// Setup
+		final BufferFileWriter writer = SpilledSubpartitionViewTest
+				.createWriterAndWriteBuffers(ioManager, new TestInfiniteBufferProvider(), numberOfBuffersToWrite);
+
+		writer.close();
+
+		final TestPooledBufferProvider viewBufferPool = new TestPooledBufferProvider(1);
+
+		final SpilledSubpartitionViewAsyncIO view = new SpilledSubpartitionViewAsyncIO(
+				mock(ResultSubpartition.class), viewBufferPool, ioManager,
+				writer.getChannelID(), 0);
+
+		final TestSubpartitionConsumer consumer = new TestSubpartitionConsumer(view, false,
+				new TestConsumerCallback.RecyclingCallback());
+
+		// Consume subpartition
+		consumer.call();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewSyncIOTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewSyncIOTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewSyncIOTest.java
new file mode 100644
index 0000000..f8baae4
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewSyncIOTest.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition;
+
+import org.apache.flink.runtime.io.disk.iomanager.BufferFileWriter;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
+import org.apache.flink.runtime.io.network.util.TestConsumerCallback;
+import org.apache.flink.runtime.io.network.util.TestInfiniteBufferProvider;
+import org.apache.flink.runtime.io.network.util.TestPooledBufferProvider;
+import org.apache.flink.runtime.io.network.util.TestSubpartitionConsumer;
+import org.junit.AfterClass;
+import org.junit.Test;
+
+import static org.mockito.Mockito.mock;
+
+public class SpilledSubpartitionViewSyncIOTest {
+
+	private static final IOManager ioManager = new IOManagerAsync();
+
+	private static final TestInfiniteBufferProvider writerBufferPool =
+			new TestInfiniteBufferProvider();
+
+	@AfterClass
+	public static void shutdown() {
+		ioManager.shutdown();
+	}
+
+	@Test
+	public void testWriteConsume() throws Exception {
+		// Config
+		final int numberOfBuffersToWrite = 512;
+
+		// Setup
+		final BufferFileWriter writer = SpilledSubpartitionViewTest
+				.createWriterAndWriteBuffers(ioManager, writerBufferPool, numberOfBuffersToWrite);
+
+		writer.close();
+
+		final TestPooledBufferProvider viewBufferPool = new TestPooledBufferProvider(1);
+
+		final SpilledSubpartitionViewSyncIO view = new SpilledSubpartitionViewSyncIO(
+				mock(ResultSubpartition.class),
+				viewBufferPool.getMemorySegmentSize(),
+				writer.getChannelID(),
+				0);
+
+		final TestSubpartitionConsumer consumer = new TestSubpartitionConsumer(view, false,
+				new TestConsumerCallback.RecyclingCallback());
+
+		// Consume subpartition
+		consumer.call();
+	}
+
+	@Test
+	public void testConsumeWithFewBuffers() throws Exception {
+		// Config
+		final int numberOfBuffersToWrite = 512;
+
+		// Setup
+		final BufferFileWriter writer = SpilledSubpartitionViewTest
+				.createWriterAndWriteBuffers(ioManager, writerBufferPool, numberOfBuffersToWrite);
+
+		writer.close();
+
+		final SpilledSubpartitionViewSyncIO view = new SpilledSubpartitionViewSyncIO(
+				mock(ResultSubpartition.class),
+				32 * 1024,
+				writer.getChannelID(),
+				0);
+
+		// No buffer available, don't deadlock. We need to make progress in situations when the view
+		// is consumed at an input gate with local and remote channels. The remote channels might
+		// eat up all the buffers, at which point the spilled view will not have any buffers
+		// available and the input gate can't make any progress if we don't return immediately.
+		//
+		// The current solution is straight-forward with a separate buffer per spilled subpartition,
+		// but introduces memory-overhead.
+		//
+		// TODO Replace with asynchronous buffer pool request as this introduces extra buffers per
+		// consumed subpartition.
+		final TestSubpartitionConsumer consumer = new TestSubpartitionConsumer(view, false,
+				new TestConsumerCallback.RecyclingCallback());
+
+		consumer.call();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewTest.java
new file mode 100644
index 0000000..11037dc
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewTest.java
@@ -0,0 +1,181 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition;
+
+import com.google.common.collect.Lists;
+import org.apache.flink.runtime.io.disk.iomanager.BufferFileWriter;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode;
+import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.BufferProvider;
+import org.apache.flink.runtime.io.network.util.TestConsumerCallback.RecyclingCallback;
+import org.apache.flink.runtime.io.network.util.TestInfiniteBufferProvider;
+import org.apache.flink.runtime.io.network.util.TestPooledBufferProvider;
+import org.apache.flink.runtime.io.network.util.TestSubpartitionConsumer;
+import org.junit.AfterClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.mockito.Mockito.mock;
+
+/**
+ * Test for both the asynchronous and synchronous spilled subpartition view implementation.
+ */
+@RunWith(Parameterized.class)
+public class SpilledSubpartitionViewTest {
+
+	private static final IOManager ioManager = new IOManagerAsync();
+
+	private static final ExecutorService executor = Executors.newCachedThreadPool();
+
+	private static final TestInfiniteBufferProvider writerBufferPool =
+			new TestInfiniteBufferProvider();
+
+	private IOMode ioMode;
+
+	public SpilledSubpartitionViewTest(IOMode ioMode) {
+		this.ioMode = ioMode;
+	}
+
+	@AfterClass
+	public static void shutdown() {
+		ioManager.shutdown();
+		executor.shutdown();
+	}
+
+	@Parameterized.Parameters
+	public static Collection<Object[]> ioMode() {
+		return Arrays.asList(new Object[][]{
+				{IOMode.SYNC},
+				{IOMode.ASYNC}
+		});
+	}
+
+	@Test
+	public void testReadMultipleFilesWithSingleBufferPool() throws Exception {
+		// Setup
+		BufferFileWriter[] writers = new BufferFileWriter[]{
+				createWriterAndWriteBuffers(ioManager, writerBufferPool, 512),
+				createWriterAndWriteBuffers(ioManager, writerBufferPool, 512)
+		};
+
+		final ResultSubpartitionView[] readers = new ResultSubpartitionView[writers.length];
+
+		// Make this buffer pool small so that we can test the behaviour of the asynchronous view
+		// with few  buffers.
+		final BufferProvider inputBuffers = new TestPooledBufferProvider(2);
+
+		final ResultSubpartition parent = mock(ResultSubpartition.class);
+
+		try {
+			// Wait for writers to finish
+			for (BufferFileWriter writer : writers) {
+				writer.close();
+			}
+
+			// Create the views depending on the test configuration
+			for (int i = 0; i < readers.length; i++) {
+				if (ioMode.isSynchronous()) {
+					readers[i] = new SpilledSubpartitionViewSyncIO(
+							parent,
+							inputBuffers.getMemorySegmentSize(),
+							writers[i].getChannelID(),
+							0);
+				}
+				else {
+					// For the asynchronous view, it is important that a registered listener will
+					// eventually be notified even if the view never got a buffer to read data into.
+					//
+					// At runtime, multiple threads never share the same buffer pool as in test. We
+					// do it here to provoke the erroneous behaviour.
+					readers[i] = new SpilledSubpartitionViewAsyncIO(
+							parent, inputBuffers, ioManager, writers[i].getChannelID(), 0);
+				}
+			}
+
+			final List<Future<Boolean>> results = Lists.newArrayList();
+
+			// Submit the consuming tasks
+			for (ResultSubpartitionView view : readers) {
+				results.add(executor.submit(new TestSubpartitionConsumer(
+						view, false, new RecyclingCallback())));
+			}
+
+			// Wait for the results
+			for (Future<Boolean> res : results) {
+				try {
+					res.get(2, TimeUnit.MINUTES);
+				}
+				catch (TimeoutException e) {
+					throw new TimeoutException("There has been a timeout in the test. This " +
+							"indicates that there is a bug/deadlock in the tested subpartition " +
+							"view. The timed out test was in " + ioMode + " mode.");
+				}
+			}
+		}
+		finally {
+			for (BufferFileWriter writer : writers) {
+				if (writer != null) {
+					writer.deleteChannel();
+				}
+			}
+
+			for (ResultSubpartitionView reader : readers) {
+				if (reader != null) {
+					reader.releaseAllResources();
+				}
+			}
+		}
+	}
+
+	/**
+	 * Returns a buffer file writer, to which the specified number of buffer write requests have
+	 * been issued (including an end of partition event).
+	 *
+	 * <p> Call {@link BufferFileWriter#close()} to ensure that all buffers have been written.
+	 */
+	static BufferFileWriter createWriterAndWriteBuffers(
+			IOManager ioManager,
+			BufferProvider bufferProvider,
+			int numberOfBuffers) throws IOException {
+
+		final BufferFileWriter writer = ioManager.createBufferFileWriter(ioManager.createChannel());
+
+		for (int i = 0; i < numberOfBuffers; i++) {
+			writer.writeBlock(bufferProvider.requestBuffer());
+		}
+
+		writer.writeBlock(EventSerializer.toBuffer(EndOfPartitionEvent.INSTANCE));
+
+		return writer;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java
new file mode 100644
index 0000000..a41a013
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition;
+
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.junit.Test;
+
+import static org.junit.Assert.assertFalse;
+import static org.mockito.Mockito.mock;
+
+/**
+ * Basic subpartition behaviour tests.
+ */
+public abstract class SubpartitionTestBase {
+
+	/**
+	 * Return the subpartition to be tested.
+	 */
+	abstract ResultSubpartition createSubpartition();
+
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testAddAfterFinish() throws Exception {
+		final ResultSubpartition subpartition = createSubpartition();
+
+		try {
+			subpartition.finish();
+
+			assertFalse(subpartition.add(mock(Buffer.class)));
+		}
+		finally {
+			if (subpartition != null) {
+				subpartition.release();
+			}
+		}
+	}
+
+	@Test
+	public void testAddAfterRelease() throws Exception {
+		final ResultSubpartition subpartition = createSubpartition();
+
+		try {
+			subpartition.release();
+
+			assertFalse(subpartition.add(mock(Buffer.class)));
+		}
+		finally {
+			if (subpartition != null) {
+				subpartition.release();
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java
index e5c87e9..6cd5469 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java
@@ -20,7 +20,8 @@ package org.apache.flink.runtime.io.network.partition.consumer;
 
 import com.google.common.base.Optional;
 import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.deployment.PartitionInfo;
+import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor;
+import org.apache.flink.runtime.deployment.ResultPartitionLocation;
 import org.apache.flink.runtime.event.task.TaskEvent;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.io.network.ConnectionManager;
@@ -28,8 +29,9 @@ import org.apache.flink.runtime.io.network.TaskEventDispatcher;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
 import org.apache.flink.runtime.io.network.buffer.BufferPool;
 import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
-import org.apache.flink.runtime.io.network.partition.IntermediateResultPartitionManager;
-import org.apache.flink.runtime.io.network.partition.queue.IntermediateResultPartitionQueueIterator;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionManager;
+import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView;
 import org.apache.flink.runtime.io.network.util.TestTaskEvent;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
@@ -45,16 +47,17 @@ import static org.mockito.Mockito.when;
 public class SingleInputGateTest {
 
 	@Test
+	@SuppressWarnings("unchecked")
 	public void testBackwardsEventWithUninitializedChannel() throws Exception {
 		// Setup environment
 		final TaskEventDispatcher taskEventDispatcher = mock(TaskEventDispatcher.class);
-		when(taskEventDispatcher.publish(any(ExecutionAttemptID.class), any(IntermediateResultPartitionID.class), any(TaskEvent.class))).thenReturn(true);
+		when(taskEventDispatcher.publish(any(ResultPartitionID.class), any(TaskEvent.class))).thenReturn(true);
 
-		final IntermediateResultPartitionQueueIterator iterator = mock(IntermediateResultPartitionQueueIterator.class);
+		final ResultSubpartitionView iterator = mock(ResultSubpartitionView.class);
 		when(iterator.getNextBuffer()).thenReturn(new Buffer(new MemorySegment(new byte[1024]), mock(BufferRecycler.class)));
 
-		final IntermediateResultPartitionManager partitionManager = mock(IntermediateResultPartitionManager.class);
-		when(partitionManager.getIntermediateResultPartitionIterator(any(ExecutionAttemptID.class), any(IntermediateResultPartitionID.class), anyInt(), any(Optional.class))).thenReturn(iterator);
+		final ResultPartitionManager partitionManager = mock(ResultPartitionManager.class);
+		when(partitionManager.getSubpartition(any(ResultPartitionID.class), anyInt(), any(Optional.class))).thenReturn(iterator);
 
 		// Setup reader with one local and one unknown input channel
 		final IntermediateDataSetID resultId = new IntermediateDataSetID();
@@ -66,38 +69,36 @@ public class SingleInputGateTest {
 		inputGate.setBufferPool(bufferPool);
 
 		// Local
-		ExecutionAttemptID localProducer = new ExecutionAttemptID();
-		IntermediateResultPartitionID localPartitionId = new IntermediateResultPartitionID();
+		ResultPartitionID localPartitionId = new ResultPartitionID(new IntermediateResultPartitionID(), new ExecutionAttemptID());
 
-		InputChannel local = new LocalInputChannel(inputGate, 0, localProducer, localPartitionId, partitionManager, taskEventDispatcher);
+		InputChannel local = new LocalInputChannel(inputGate, 0, localPartitionId, partitionManager, taskEventDispatcher);
 
 		// Unknown
-		ExecutionAttemptID unknownProducer = new ExecutionAttemptID();
-		IntermediateResultPartitionID unknownPartitionId = new IntermediateResultPartitionID();
+		ResultPartitionID unknownPartitionId = new ResultPartitionID(new IntermediateResultPartitionID(), new ExecutionAttemptID());
 
-		InputChannel unknown = new UnknownInputChannel(inputGate, 1, unknownProducer, unknownPartitionId, partitionManager, taskEventDispatcher, mock(ConnectionManager.class));
+		InputChannel unknown = new UnknownInputChannel(inputGate, 1, unknownPartitionId, partitionManager, taskEventDispatcher, mock(ConnectionManager.class));
 
 		// Set channels
-		inputGate.setInputChannel(localPartitionId, local);
-		inputGate.setInputChannel(unknownPartitionId, unknown);
+		inputGate.setInputChannel(localPartitionId.getPartitionId(), local);
+		inputGate.setInputChannel(unknownPartitionId.getPartitionId(), unknown);
 
 		// Request partitions
 		inputGate.requestPartitions();
 
 		// Only the local channel can request
-		verify(partitionManager, times(1)).getIntermediateResultPartitionIterator(any(ExecutionAttemptID.class), any(IntermediateResultPartitionID.class), anyInt(), any(Optional.class));
+		verify(partitionManager, times(1)).getSubpartition(any(ResultPartitionID.class), anyInt(), any(Optional.class));
 
 		// Send event backwards and initialize unknown channel afterwards
 		final TaskEvent event = new TestTaskEvent();
 		inputGate.sendTaskEvent(event);
 
 		// Only the local channel can send out the event
-		verify(taskEventDispatcher, times(1)).publish(any(ExecutionAttemptID.class), any(IntermediateResultPartitionID.class), any(TaskEvent.class));
+		verify(taskEventDispatcher, times(1)).publish(any(ResultPartitionID.class), any(TaskEvent.class));
 
 		// After the update, the pending event should be send to local channel
-		inputGate.updateInputChannel(new PartitionInfo(unknownPartitionId, unknownProducer, PartitionInfo.PartitionLocation.LOCAL, null));
+		inputGate.updateInputChannel(new InputChannelDeploymentDescriptor(new ResultPartitionID(unknownPartitionId.getPartitionId(), unknownPartitionId.getProducerId()), ResultPartitionLocation.createLocal()));
 
-		verify(partitionManager, times(2)).getIntermediateResultPartitionIterator(any(ExecutionAttemptID.class), any(IntermediateResultPartitionID.class), anyInt(), any(Optional.class));
-		verify(taskEventDispatcher, times(2)).publish(any(ExecutionAttemptID.class), any(IntermediateResultPartitionID.class), any(TaskEvent.class));
+		verify(partitionManager, times(2)).getSubpartition(any(ResultPartitionID.class), anyInt(), any(Optional.class));
+		verify(taskEventDispatcher, times(2)).publish(any(ResultPartitionID.class), any(TaskEvent.class));
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java
index c1db44d..131c4f6 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.runtime.io.network.partition.consumer;
 
-import org.apache.flink.runtime.io.network.util.MockInputChannel;
+import org.apache.flink.runtime.io.network.util.TestInputChannel;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.junit.Test;
 
@@ -36,9 +36,9 @@ public class UnionInputGateTest {
 
 		assertEquals(ig1.getNumberOfInputChannels() + ig2.getNumberOfInputChannels(), union.getNumberOfInputChannels());
 
-		final MockInputChannel[][] inputChannels = new MockInputChannel[][]{
-				MockInputChannel.createInputChannels(ig1, 3),
-				MockInputChannel.createInputChannels(ig2, 5)
+		final TestInputChannel[][] inputChannels = new TestInputChannel[][]{
+				TestInputChannel.createInputChannels(ig1, 3),
+				TestInputChannel.createInputChannels(ig2, 5)
 		};
 
 		inputChannels[0][0].readBuffer(); // 0 => 0

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/queue/PipelinedPartitionQueueTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/queue/PipelinedPartitionQueueTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/queue/PipelinedPartitionQueueTest.java
deleted file mode 100644
index 4c70378..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/queue/PipelinedPartitionQueueTest.java
+++ /dev/null
@@ -1,224 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.io.network.partition.queue;
-
-import com.google.common.base.Optional;
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.buffer.BufferPool;
-import org.apache.flink.runtime.io.network.buffer.BufferProvider;
-import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool;
-import org.apache.flink.runtime.io.network.partition.queue.IntermediateResultPartitionQueueIterator.AlreadySubscribedException;
-import org.apache.flink.runtime.io.network.util.MockConsumer;
-import org.apache.flink.runtime.io.network.util.MockNotificationListener;
-import org.apache.flink.runtime.io.network.util.MockProducer;
-import org.apache.flink.runtime.util.event.NotificationListener;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.Random;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-
-public class PipelinedPartitionQueueTest {
-
-	private static final int NUM_BUFFERS = 1024;
-
-	private static final int BUFFER_SIZE = 32 * 1024;
-
-	private static final NetworkBufferPool networkBuffers = new NetworkBufferPool(NUM_BUFFERS, BUFFER_SIZE);
-
-	private PipelinedPartitionQueue queue;
-
-	@Before
-	public void setup() {
-		this.queue = new PipelinedPartitionQueue();
-	}
-
-	@Test(expected = IllegalQueueIteratorRequestException.class)
-	public void testExceptionWhenMultipleConsumers() throws IOException {
-		queue.getQueueIterator(Optional.<BufferProvider>absent());
-
-		// This queue is only consumable once, so this should throw an Exception
-		queue.getQueueIterator(Optional.<BufferProvider>absent());
-	}
-
-	@Test(expected = AlreadySubscribedException.class)
-	public void testExceptionWhenMultipleSubscribers() throws IOException {
-		IntermediateResultPartitionQueueIterator iterator = queue.getQueueIterator(Optional.<BufferProvider>absent());
-
-		NotificationListener listener = mock(NotificationListener.class);
-
-		// First subscribe should be fine
-		assertTrue(iterator.subscribe(listener));
-
-		// This should throw an already subscribed exception
-		iterator.subscribe(listener);
-	}
-
-	@Test
-	public void testProduceConsume() throws Exception {
-		Buffer boe = mock(Buffer.class);
-
-		MockNotificationListener listener = new MockNotificationListener();
-
-		IntermediateResultPartitionQueueIterator iterator = queue.getQueueIterator(Optional.<BufferProvider>absent());
-
-		// Empty queue => should return null
-		assertNull(iterator.getNextBuffer());
-
-		// But iterator should not be consumed yet...
-		assertFalse(iterator.isConsumed());
-
-		// Subscribe for notifications
-		assertTrue(iterator.subscribe(listener));
-
-		assertEquals(0, listener.getNumberOfNotifications());
-
-		// Add data to the queue...
-		queue.add(boe);
-
-		// ...should result in a notification
-		assertEquals(1, listener.getNumberOfNotifications());
-
-		// ...and one available result
-		assertNotNull(iterator.getNextBuffer());
-		assertNull(iterator.getNextBuffer());
-		assertFalse(iterator.isConsumed());
-
-		// Add data to the queue...
-		queue.add(boe);
-		// ...don't allow to subscribe, if data is available
-		assertFalse(iterator.subscribe(listener));
-
-		assertEquals(1, listener.getNumberOfNotifications());
-	}
-
-	@Test
-	public void testDiscardingProduceWhileSubscribedConsumer() throws IOException {
-		IntermediateResultPartitionQueueIterator iterator = queue.getQueueIterator(Optional.<BufferProvider>absent());
-
-		NotificationListener listener = mock(NotificationListener.class);
-
-		assertTrue(iterator.subscribe(listener));
-
-		queue.discard();
-
-		verify(listener, times(1)).onNotification();
-
-		assertTrue(iterator.isConsumed());
-	}
-
-	@Test
-	public void testConcurrentProduceConsume() throws Exception {
-		doTestConcurrentProduceConsume(false, false);
-	}
-
-	@Test
-	public void testConcurrentSlowProduceConsume() throws Exception {
-		doTestConcurrentProduceConsume(true, false);
-	}
-
-	@Test
-	public void testConcurrentProduceSlowConsume() throws Exception {
-		doTestConcurrentProduceConsume(true, false);
-	}
-
-	@Test
-	public void testConcurrentDiscardingProduceConsume() throws Exception {
-		doTestConcurrentProduceConsume(false, false, true);
-	}
-
-	@Test
-	public void testConcurrentDiscardingSlowProduceConsume() throws Exception {
-		doTestConcurrentProduceConsume(true, false, true);
-	}
-
-	@Test
-	public void testConcurrentDiscardingProduceSlowConsume() throws Exception {
-		doTestConcurrentProduceConsume(false, true, true);
-	}
-
-	private void doTestConcurrentProduceConsume(boolean slowProducer, boolean slowConsumer) throws Exception {
-		doTestConcurrentProduceConsume(slowProducer, slowConsumer, false);
-	}
-
-	private void doTestConcurrentProduceConsume(boolean slowProducer, boolean slowConsumer, boolean discardProduce) throws Exception {
-
-		final int bufferPoolSize = 8;
-
-		final int numBuffersToProduce = 64;
-
-		BufferPool producerBufferPool = networkBuffers.createBufferPool(bufferPoolSize, true);
-
-		MockProducer producer = new MockProducer(queue, producerBufferPool, numBuffersToProduce, slowProducer);
-
-		if (discardProduce) {
-			producer.discardAfter(new Random().nextInt(numBuffersToProduce));
-		}
-
-		MockConsumer consumer = new MockConsumer(queue.getQueueIterator(Optional.<BufferProvider>absent()), slowConsumer);
-
-		ExecutorService executorService = Executors.newCachedThreadPool();
-
-		try {
-			Future<Boolean> producerSuccess = executorService.submit(producer);
-			Future<Boolean> consumerSuccess = executorService.submit(consumer);
-
-			boolean success = false;
-			try {
-				success = producerSuccess.get();
-				success &= consumerSuccess.get();
-			}
-			catch (Throwable t) {
-				t.printStackTrace();
-
-				if (producer.getError() != null) {
-					System.err.println("Producer error:");
-					producer.getError().printStackTrace();
-				}
-
-				if (consumer.getError() != null) {
-					System.err.println("Consumer error:");
-					consumer.getError().printStackTrace();
-				}
-
-				fail("Unexpected failure during test: " + t.getMessage() + ". Producer error: " + producer.getError() + ", consumer error: " + consumer.getError());
-			}
-
-			producerBufferPool.destroy();
-
-			assertTrue(success);
-		} finally {
-			executorService.shutdownNow();
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/serialization/LargeRecordsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/serialization/LargeRecordsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/serialization/LargeRecordsTest.java
index 5ce145b..d9e3562 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/serialization/LargeRecordsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/serialization/LargeRecordsTest.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.io.network.serialization;
 
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.network.api.serialization.SpillingAdaptiveSpanningRecordDeserializer;
 import org.apache.flink.runtime.io.network.api.serialization.AdaptiveSpanningRecordDeserializer;
 import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer;
 import org.apache.flink.runtime.io.network.api.serialization.RecordSerializer;


[12/13] flink git commit: [FLINK-1350] [runtime] Add blocking result partition variant

Posted by uc...@apache.org.
[FLINK-1350] [runtime] Add blocking result partition variant

- Renames runtime intermediate result classes:
  a) Removes "Intermediate" prefix
  b) Queue => Subpartition
  c) Iterator => View

- [FLINK-1350] Adds a spillable result subpartition variant for BLOCKING
  results, which writes data to memory first and starts to spill
  (asynchronously) if not enough memory is available to produce the
  result in-memory only.

  Receiving tasks of BLOCKING results are only deployed after *all*
  partitions have been fully produced. PIPELINED and BLOCKING results can not
  be mixed.

- [FLINK-1359] Adds simple state tracking to result partitions with
  notifications after partitions/subpartitions have been consumed. Each
  partition has to be consumed at least once before it can be released.

  Currently there is no notion of historic intermediate results, i.e. results
  are released as soon as they are consumed.


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

Branch: refs/heads/master
Commit: 9d7acf3657cbd3fb0b238b20ba864b6a74774e40
Parents: 1930678
Author: Ufuk Celebi <uc...@apache.org>
Authored: Tue Jan 6 17:11:08 2015 +0100
Committer: Ufuk Celebi <uc...@apache.org>
Committed: Wed Mar 18 17:44:40 2015 +0100

----------------------------------------------------------------------
 .../flink/compiler/dag/OptimizerNode.java       |   7 +-
 .../org/apache/flink/compiler/plan/Channel.java |   1 +
 .../apache/flink/compiler/plan/PlanNode.java    |   3 +-
 .../plantranslate/NepheleJobGraphGenerator.java |   5 +-
 .../flink/compiler/PipelineBreakerTest.java     | 176 ++---
 .../flink/configuration/ConfigConstants.java    |  15 +-
 .../java/org/apache/flink/util/AbstractID.java  |   6 +
 .../org/apache/flink/api/java/tuple/Tuple.java  |   2 +
 flink-runtime/pom.xml                           |   2 +-
 .../InputChannelDeploymentDescriptor.java       | 138 ++++
 .../InputGateDeploymentDescriptor.java          |  89 +++
 ...PartialInputChannelDeploymentDescriptor.java | 116 ++++
 .../deployment/PartialPartitionInfo.java        | 102 ---
 .../PartitionConsumerDeploymentDescriptor.java  | 102 ---
 .../PartitionDeploymentDescriptor.java          | 122 ----
 .../flink/runtime/deployment/PartitionInfo.java | 176 -----
 .../ResultPartitionDeploymentDescriptor.java    | 114 ++++
 .../deployment/ResultPartitionLocation.java     | 101 +++
 .../deployment/TaskDeploymentDescriptor.java    | 103 ++-
 .../flink/runtime/execution/Environment.java    |   6 +-
 .../runtime/execution/RuntimeEnvironment.java   |  40 +-
 .../flink/runtime/executiongraph/Execution.java | 215 ++++---
 .../runtime/executiongraph/ExecutionEdge.java   |   5 +
 .../runtime/executiongraph/ExecutionGraph.java  |  27 +-
 .../executiongraph/ExecutionJobVertex.java      |  32 +-
 .../runtime/executiongraph/ExecutionVertex.java | 214 ++++---
 .../executiongraph/IntermediateResult.java      |  52 +-
 .../IntermediateResultPartition.java            |  29 +
 .../io/disk/ChannelReaderInputViewIterator.java |   2 +-
 .../runtime/io/disk/FileChannelInputView.java   |   6 +-
 .../runtime/io/disk/FileChannelOutputView.java  |   6 +-
 .../io/disk/SeekableFileChannelInputView.java   |   6 +-
 .../flink/runtime/io/disk/SpillingBuffer.java   |  12 +-
 .../disk/iomanager/AbstractFileIOChannel.java   |   7 +-
 .../disk/iomanager/AsynchronousBlockReader.java |  22 +-
 .../disk/iomanager/AsynchronousBlockWriter.java |   6 +-
 .../AsynchronousBlockWriterWithCallback.java    |   2 +-
 .../iomanager/AsynchronousBufferFileReader.java |  48 ++
 .../AsynchronousBufferFileSegmentReader.java    |  46 ++
 .../iomanager/AsynchronousBufferFileWriter.java |  64 ++
 .../iomanager/AsynchronousFileIOChannel.java    | 271 ++++++--
 .../io/disk/iomanager/BlockChannelReader.java   |  22 +-
 .../io/disk/iomanager/BlockChannelWriter.java   |  10 +-
 .../BlockChannelWriterWithCallback.java         |  12 +-
 .../io/disk/iomanager/BufferFileReader.java     |  33 +
 .../disk/iomanager/BufferFileSegmentReader.java |  31 +
 .../io/disk/iomanager/BufferFileWriter.java     |  38 ++
 .../disk/iomanager/ChannelReaderInputView.java  |  10 +-
 .../disk/iomanager/ChannelWriterOutputView.java |   8 +-
 .../io/disk/iomanager/FileIOChannel.java        |   3 +
 .../runtime/io/disk/iomanager/FileSegment.java  |  52 ++
 .../HeaderlessChannelReaderInputView.java       |   4 +-
 .../runtime/io/disk/iomanager/IOManager.java    |  32 +-
 .../io/disk/iomanager/IOManagerAsync.java       |  32 +-
 .../io/disk/iomanager/QueuingCallback.java      |  14 +-
 .../iomanager/SynchronousBufferFileReader.java  |  85 +++
 .../iomanager/SynchronousFileIOChannel.java     |   2 +-
 .../flink/runtime/io/network/ConnectionID.java  |  85 +++
 .../runtime/io/network/ConnectionManager.java   |  13 +-
 .../io/network/LocalConnectionManager.java      |   9 +-
 .../runtime/io/network/NetworkEnvironment.java  |  53 +-
 .../flink/runtime/io/network/RemoteAddress.java | 122 ----
 .../runtime/io/network/TaskEventDispatcher.java |  57 +-
 .../api/reader/AbstractRecordReader.java        |   3 +-
 ...llingAdaptiveSpanningRecordDeserializer.java | 637 +++++++++++++++++++
 .../io/network/api/writer/BufferWriter.java     | 119 ----
 .../io/network/api/writer/RecordWriter.java     |  13 +-
 .../api/writer/ResultPartitionWriter.java       | 115 ++++
 .../flink/runtime/io/network/buffer/Buffer.java |  14 +-
 .../runtime/io/network/buffer/BufferPool.java   |   2 +-
 .../io/network/buffer/BufferPoolOwner.java      |   2 +-
 .../io/network/buffer/BufferProvider.java       |   2 +
 .../io/network/buffer/LocalBufferPool.java      |  25 +-
 .../io/network/buffer/NetworkBufferPool.java    |   8 +-
 .../runtime/io/network/netty/NettyConfig.java   |  31 +-
 .../network/netty/NettyConnectionManager.java   |  17 +-
 .../runtime/io/network/netty/NettyMessage.java  |  36 +-
 .../network/netty/PartitionRequestClient.java   |  27 +-
 .../netty/PartitionRequestClientFactory.java    |  42 +-
 .../network/netty/PartitionRequestProtocol.java |  11 +-
 .../io/network/netty/PartitionRequestQueue.java |  69 +-
 .../netty/PartitionRequestServerHandler.java    |  44 +-
 .../partition/IntermediateResultPartition.java  | 319 ----------
 .../IntermediateResultPartitionManager.java     | 139 ----
 .../IntermediateResultPartitionProvider.java    |  37 --
 .../partition/PipelinedSubpartition.java        | 218 +++++++
 .../partition/PipelinedSubpartitionView.java    |  74 +++
 .../io/network/partition/ResultPartition.java   | 421 ++++++++++++
 .../io/network/partition/ResultPartitionID.java |  77 +++
 .../partition/ResultPartitionManager.java       | 146 +++++
 .../partition/ResultPartitionProvider.java      |  33 +
 .../network/partition/ResultPartitionType.java  |  62 ++
 .../network/partition/ResultSubpartition.java   |  81 +++
 .../partition/ResultSubpartitionView.java       |  59 ++
 .../partition/SpillableSubpartition.java        | 222 +++++++
 .../partition/SpillableSubpartitionView.java    | 163 +++++
 .../SpilledSubpartitionViewAsyncIO.java         | 365 +++++++++++
 .../SpilledSubpartitionViewSyncIO.java          | 177 ++++++
 .../partition/consumer/InputChannel.java        |  38 +-
 .../partition/consumer/LocalInputChannel.java   |  75 ++-
 .../partition/consumer/RemoteInputChannel.java  |  29 +-
 .../partition/consumer/SingleInputGate.java     | 176 ++---
 .../partition/consumer/UnknownInputChannel.java |  33 +-
 .../IllegalQueueIteratorRequestException.java   |  33 -
 .../queue/IntermediateResultPartitionQueue.java |  53 --
 ...ntermediateResultPartitionQueueIterator.java |  74 ---
 .../queue/PipelinedPartitionQueue.java          | 185 ------
 ...llingAdaptiveSpanningRecordDeserializer.java | 637 -------------------
 .../iterative/io/SerializedUpdateBuffer.java    |   8 +-
 .../iterative/task/IterationHeadPactTask.java   |   4 +-
 .../runtime/jobgraph/AbstractJobVertex.java     |  13 +-
 .../runtime/jobgraph/IntermediateDataSet.java   |  24 +-
 .../IntermediateResultPartitionType.java        |  51 --
 .../flink/runtime/operators/TempBarrier.java    |   1 +
 .../runtime/operators/hash/HashPartition.java   |  12 +-
 .../operators/hash/MutableHashTable.java        |   4 +-
 .../operators/hash/ReOpenableHashPartition.java |   2 +-
 .../sort/CombiningUnilateralSortMerger.java     |   4 +-
 .../operators/sort/UnilateralSortMerger.java    |   6 +-
 .../apache/flink/runtime/taskmanager/Task.java  |  37 +-
 .../util/AtomicDisposableReferenceCounter.java  |  55 +-
 .../flink/runtime/jobmanager/JobManager.scala   |  20 +-
 .../runtime/messages/JobManagerMessages.scala   |   9 +-
 .../runtime/messages/TaskManagerMessages.scala  |  28 +-
 .../NetworkEnvironmentConfiguration.scala       |   2 +
 .../flink/runtime/taskmanager/TaskManager.scala |  39 +-
 .../TaskDeploymentDescriptorTest.java           |   6 +-
 .../ExecutionGraphDeploymentTest.java           |  14 +-
 .../flink/runtime/io/disk/ChannelViewsTest.java |  24 +-
 .../io/disk/FileChannelStreamsITCase.java       |  20 +-
 .../runtime/io/disk/FileChannelStreamsTest.java |   4 +-
 .../disk/SeekableFileChannelInputViewTest.java  |   2 +-
 .../AsynchronousBufferFileWriterTest.java       | 178 ++++++
 .../AsynchronousFileIOChannelTest.java          | 429 +++++++++++++
 .../AsynchronousFileIOChannelsTest.java         | 176 -----
 .../BufferFileWriterFileSegmentReaderTest.java  | 198 ++++++
 .../iomanager/BufferFileWriterReaderTest.java   | 225 +++++++
 .../io/disk/iomanager/IOManagerAsyncTest.java   |  31 +-
 .../io/disk/iomanager/IOManagerITCase.java      |   7 +-
 .../IOManagerPerformanceBenchmark.java          |   4 +-
 .../io/disk/iomanager/IOManagerTest.java        |  23 +-
 .../network/api/reader/AbstractReaderTest.java  |   1 +
 .../io/network/api/reader/BufferReaderTest.java |   7 +-
 .../IteratorWrappingMockSingleInputGate.java    | 103 ---
 .../IteratorWrappingTestSingleInputGate.java    | 103 +++
 .../SpanningRecordSerializationTest.java        |   1 -
 .../network/buffer/BufferPoolFactoryTest.java   |   2 +-
 .../runtime/io/network/buffer/BufferTest.java   |  52 --
 .../io/network/buffer/LocalBufferPoolTest.java  |  84 ++-
 .../netty/NettyMessageSerializationTest.java    |   9 +-
 .../PartitionRequestClientFactoryTest.java      |  10 +-
 .../PartitionRequestClientHandlerTest.java      |   1 +
 .../partition/PipelinedSubpartitionTest.java    | 278 ++++++++
 .../partition/SpillableSubpartitionTest.java    |  49 ++
 .../SpilledSubpartitionViewAsyncIOTest.java     |  65 ++
 .../SpilledSubpartitionViewSyncIOTest.java      | 103 +++
 .../partition/SpilledSubpartitionViewTest.java  | 181 ++++++
 .../network/partition/SubpartitionTestBase.java |  70 ++
 .../partition/consumer/SingleInputGateTest.java |  41 +-
 .../partition/consumer/UnionInputGateTest.java  |   8 +-
 .../queue/PipelinedPartitionQueueTest.java      | 224 -------
 .../network/serialization/LargeRecordsTest.java |   1 +
 .../runtime/io/network/util/MockConsumer.java   | 100 ---
 .../io/network/util/MockInputChannel.java       | 130 ----
 .../network/util/MockNotificationListener.java  |  52 --
 .../runtime/io/network/util/MockProducer.java   | 106 ---
 .../io/network/util/MockSingleInputGate.java    | 137 ----
 .../io/network/util/TestBufferFactory.java      |  88 +++
 .../io/network/util/TestConsumerCallback.java   | 103 +++
 .../util/TestInfiniteBufferProvider.java        |  81 +++
 .../io/network/util/TestInputChannel.java       | 130 ++++
 .../network/util/TestNotificationListener.java  |  73 +++
 .../io/network/util/TestPartitionProducer.java  | 107 ++++
 .../network/util/TestPooledBufferProvider.java  | 152 +++++
 .../io/network/util/TestProducerSource.java     |  32 +
 .../io/network/util/TestSingleInputGate.java    | 140 ++++
 .../network/util/TestSubpartitionConsumer.java  | 121 ++++
 .../network/util/TestSubpartitionProducer.java  | 105 +++
 .../runtime/io/network/util/TestTaskEvent.java  |   3 +
 .../runtime/operators/DataSinkTaskTest.java     |  10 +-
 .../runtime/operators/DataSourceTaskTest.java   |   4 +-
 .../operators/chaining/ChainTaskTest.java       |   4 +-
 .../operators/testutils/MockEnvironment.java    |  20 +-
 .../operators/testutils/TaskTestBase.java       |  10 +-
 .../TaskManagerProcessReapingTest.java          |   2 +-
 .../runtime/taskmanager/TaskManagerTest.java    |  70 +-
 .../flink/runtime/taskmanager/TaskTest.java     |  54 +-
 .../runtime/testutils/DiscardingRecycler.java   |   4 +-
 .../AtomicDisposableReferenceCounterTest.java   |  27 +-
 .../flink/streaming/io/StreamRecordWriter.java  |  10 +-
 .../io/StreamingAbstractRecordReader.java       |   2 +-
 .../flink/test/util/JavaProgramTestBase.java    |  68 +-
 .../exampleJavaPrograms/WordCountITCase.java    |   6 +-
 .../test/iterative/KMeansForTestITCase.java     | 276 ++++++++
 .../ProcessFailureBatchRecoveryITCase.java      |   2 +-
 ...ConsumePipelinedAndBlockingResultITCase.java |  67 ++
 .../flink/test/runtime/JoinDeadlockITCase.java  |  70 ++
 .../test/runtime/SelfJoinDeadlockITCase.java    | 104 +++
 .../flink/test/util/testjar/KMeansForTest.java  |  34 +-
 199 files changed, 9363 insertions(+), 4562 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-compiler/src/main/java/org/apache/flink/compiler/dag/OptimizerNode.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/dag/OptimizerNode.java b/flink-compiler/src/main/java/org/apache/flink/compiler/dag/OptimizerNode.java
index 16c60a4..5aaf114 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/dag/OptimizerNode.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/dag/OptimizerNode.java
@@ -460,12 +460,13 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
 	public InterestingProperties getInterestingProperties() {
 		return this.intProps;
 	}
-	
-	
+
+	@Override
 	public long getEstimatedOutputSize() {
 		return this.estimatedOutputSize;
 	}
 
+	@Override
 	public long getEstimatedNumRecords() {
 		return this.estimatedNumRecords;
 	}
@@ -478,6 +479,7 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
 		this.estimatedNumRecords = estimatedNumRecords;
 	}
 	
+	@Override
 	public float getEstimatedAvgWidthPerOutputRecord() {
 		if (this.estimatedOutputSize > 0 && this.estimatedNumRecords > 0) {
 			return ((float) this.estimatedOutputSize) / this.estimatedNumRecords;
@@ -941,6 +943,7 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
 		if (this.closedBranchingNodes == null) { 
 			this.closedBranchingNodes = new HashSet<OptimizerNode>();
 		}
+
 		this.closedBranchingNodes.add(alreadyClosed);
 	}
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-compiler/src/main/java/org/apache/flink/compiler/plan/Channel.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/plan/Channel.java b/flink-compiler/src/main/java/org/apache/flink/compiler/plan/Channel.java
index 3903c84..454cf30 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/plan/Channel.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/plan/Channel.java
@@ -105,6 +105,7 @@ public class Channel implements EstimateProvider, Cloneable, DumpableConnection<
 	 *
 	 * @return The source.
 	 */
+	@Override
 	public PlanNode getSource() {
 		return this.source;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-compiler/src/main/java/org/apache/flink/compiler/plan/PlanNode.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/plan/PlanNode.java b/flink-compiler/src/main/java/org/apache/flink/compiler/plan/PlanNode.java
index 4f72144..0b74add 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/plan/PlanNode.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/plan/PlanNode.java
@@ -381,8 +381,7 @@ public abstract class PlanNode implements Visitable<PlanNode>, DumpableNode<Plan
 	public List<Channel> getOutgoingChannels() {
 		return this.outChannels;
 	}
-	
-	
+
 	// --------------------------------------------------------------------------------------------
 	//                                Miscellaneous
 	// --------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java b/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
index 4681d0d..052d439 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
@@ -1068,10 +1068,8 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 			default:
 				throw new RuntimeException("Unknown runtime ship strategy: " + channel.getShipStrategy());
 		}
-		
+
 		targetVertex.connectNewDataSetAsInput(sourceVertex, distributionPattern);
-		
-//		sourceVertex.conn/ectTo(targetVertex, channelType, distributionPattern);
 
 		// -------------- configure the source task's ship strategy strategies in task config --------------
 		final int outputIndex = sourceConfig.getNumOutputs();
@@ -1140,6 +1138,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 			final TempMode tm = channel.getTempMode();
 
 			boolean needsMemory = false;
+			// Don't add a pipeline breaker if the data exchange is already blocking.
 			if (tm.breaksPipeline()) {
 				config.setInputAsynchronouslyMaterialized(inputNum, true);
 				needsMemory = true;

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-compiler/src/test/java/org/apache/flink/compiler/PipelineBreakerTest.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/PipelineBreakerTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/PipelineBreakerTest.java
index e034696..36eb85b 100644
--- a/flink-compiler/src/test/java/org/apache/flink/compiler/PipelineBreakerTest.java
+++ b/flink-compiler/src/test/java/org/apache/flink/compiler/PipelineBreakerTest.java
@@ -18,21 +18,29 @@
 
 package org.apache.flink.compiler;
 
-import static org.junit.Assert.*;
-
-import org.apache.flink.compiler.testfunctions.IdentityMapper;
-import org.apache.flink.compiler.testfunctions.SelectOneReducer;
-import org.junit.Test;
 import org.apache.flink.api.common.Plan;
+import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.io.DiscardingOutputFormat;
 import org.apache.flink.api.java.operators.IterativeDataSet;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.compiler.plan.BulkIterationPlanNode;
 import org.apache.flink.compiler.plan.DualInputPlanNode;
 import org.apache.flink.compiler.plan.OptimizedPlan;
+import org.apache.flink.compiler.plan.PlanNode;
 import org.apache.flink.compiler.plan.SingleInputPlanNode;
 import org.apache.flink.compiler.plan.SinkPlanNode;
+import org.apache.flink.compiler.plan.SourcePlanNode;
+import org.apache.flink.compiler.testfunctions.IdentityMapper;
+import org.apache.flink.compiler.testfunctions.SelectOneReducer;
 import org.apache.flink.configuration.Configuration;
+import org.junit.Test;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 @SuppressWarnings("serial")
 public class PipelineBreakerTest extends CompilerTestBase {
@@ -42,21 +50,21 @@ public class PipelineBreakerTest extends CompilerTestBase {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 			env.setDegreeOfParallelism(64);
-			
+
 			DataSet<Long> source = env.generateSequence(1, 10).map(new IdentityMapper<Long>());
-			
+
 			DataSet<Long> result = source.map(new IdentityMapper<Long>())
-										.map(new IdentityMapper<Long>())
-											.withBroadcastSet(source, "bc");
-			
+					.map(new IdentityMapper<Long>())
+					.withBroadcastSet(source, "bc");
+
 			result.print();
-			
+
 			Plan p = env.createProgramPlan();
 			OptimizedPlan op = compileNoStats(p);
-			
+
 			SinkPlanNode sink = op.getDataSinks().iterator().next();
 			SingleInputPlanNode mapper = (SingleInputPlanNode) sink.getInput().getSource();
-			
+
 			assertTrue(mapper.getInput().getTempMode().breaksPipeline());
 		}
 		catch (Exception e) {
@@ -64,33 +72,33 @@ public class PipelineBreakerTest extends CompilerTestBase {
 			fail(e.getMessage());
 		}
 	}
-	
+
 	@Test
 	public void testPipelineBreakerBroadcastedAllReduce() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 			env.setDegreeOfParallelism(64);
-			
+
 			DataSet<Long> sourceWithMapper = env.generateSequence(1, 10).map(new IdentityMapper<Long>());
-			
+
 			DataSet<Long> bcInput1 = sourceWithMapper
-										.map(new IdentityMapper<Long>())
-										.reduce(new SelectOneReducer<Long>());
+					.map(new IdentityMapper<Long>())
+					.reduce(new SelectOneReducer<Long>());
 			DataSet<Long> bcInput2 = env.generateSequence(1, 10);
-			
+
 			DataSet<Long> result = sourceWithMapper
 					.map(new IdentityMapper<Long>())
-							.withBroadcastSet(bcInput1, "bc1")
-							.withBroadcastSet(bcInput2, "bc2");
-			
+					.withBroadcastSet(bcInput1, "bc1")
+					.withBroadcastSet(bcInput2, "bc2");
+
 			result.print();
-			
+
 			Plan p = env.createProgramPlan();
 			OptimizedPlan op = compileNoStats(p);
-			
+
 			SinkPlanNode sink = op.getDataSinks().iterator().next();
 			SingleInputPlanNode mapper = (SingleInputPlanNode) sink.getInput().getSource();
-			
+
 			assertTrue(mapper.getInput().getTempMode().breaksPipeline());
 		}
 		catch (Exception e) {
@@ -98,39 +106,39 @@ public class PipelineBreakerTest extends CompilerTestBase {
 			fail(e.getMessage());
 		}
 	}
-	
+
 	@Test
 	public void testPipelineBreakerBroadcastedPartialSolution() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 			env.setDegreeOfParallelism(64);
-			
-			
+
+
 			DataSet<Long> initialSource = env.generateSequence(1, 10);
 			IterativeDataSet<Long> iteration = initialSource.iterate(100);
-			
-			
+
+
 			DataSet<Long> sourceWithMapper = env.generateSequence(1, 10).map(new IdentityMapper<Long>());
-			
+
 			DataSet<Long> bcInput1 = sourceWithMapper
-										.map(new IdentityMapper<Long>())
-										.reduce(new SelectOneReducer<Long>());
-			
+					.map(new IdentityMapper<Long>())
+					.reduce(new SelectOneReducer<Long>());
+
 			DataSet<Long> result = sourceWithMapper
 					.map(new IdentityMapper<Long>())
-							.withBroadcastSet(iteration, "bc2")
-							.withBroadcastSet(bcInput1, "bc1");
-							
-			
+					.withBroadcastSet(iteration, "bc2")
+					.withBroadcastSet(bcInput1, "bc1");
+
+
 			iteration.closeWith(result).print();
-			
+
 			Plan p = env.createProgramPlan();
 			OptimizedPlan op = compileNoStats(p);
-			
+
 			SinkPlanNode sink = op.getDataSinks().iterator().next();
 			BulkIterationPlanNode iterationPlanNode = (BulkIterationPlanNode) sink.getInput().getSource();
 			SingleInputPlanNode mapper = (SingleInputPlanNode) iterationPlanNode.getRootOfStepFunction();
-			
+
 			assertTrue(mapper.getInput().getTempMode().breaksPipeline());
 		}
 		catch (Exception e) {
@@ -138,98 +146,98 @@ public class PipelineBreakerTest extends CompilerTestBase {
 			fail(e.getMessage());
 		}
 	}
-	
+
 	@Test
 	public void testPilelineBreakerWithCross() {
 		try {
 			{
 				ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 				env.setDegreeOfParallelism(64);
-				
+
 				DataSet<Long> initialSource = env.generateSequence(1, 10);
-				
-				Configuration conf= new Configuration();
+
+				Configuration conf = new Configuration();
 				conf.setString(PactCompiler.HINT_LOCAL_STRATEGY, PactCompiler.HINT_LOCAL_STRATEGY_NESTEDLOOP_BLOCKED_OUTER_FIRST);
 				initialSource
-					.map(new IdentityMapper<Long>())
-					.cross(initialSource).withParameters(conf)
-					.print();
-				
-				
+						.map(new IdentityMapper<Long>())
+						.cross(initialSource).withParameters(conf)
+						.print();
+
+
 				Plan p = env.createProgramPlan();
 				OptimizedPlan op = compileNoStats(p);
 				SinkPlanNode sink = op.getDataSinks().iterator().next();
 				DualInputPlanNode mapper = (DualInputPlanNode) sink.getInput().getSource();
-				
+
 				assertTrue(mapper.getInput1().getTempMode().breaksPipeline());
 			}
-			
+
 			{
 				ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 				env.setDegreeOfParallelism(64);
-				
+
 				DataSet<Long> initialSource = env.generateSequence(1, 10);
-				
-				Configuration conf= new Configuration();
+
+				Configuration conf = new Configuration();
 				conf.setString(PactCompiler.HINT_LOCAL_STRATEGY, PactCompiler.HINT_LOCAL_STRATEGY_NESTEDLOOP_BLOCKED_OUTER_SECOND);
 				initialSource
-					.map(new IdentityMapper<Long>())
-					.cross(initialSource).withParameters(conf)
-					.print();
-				
-				
+						.map(new IdentityMapper<Long>())
+						.cross(initialSource).withParameters(conf)
+						.print();
+
+
 				Plan p = env.createProgramPlan();
 				OptimizedPlan op = compileNoStats(p);
-				
+
 				SinkPlanNode sink = op.getDataSinks().iterator().next();
 				DualInputPlanNode mapper = (DualInputPlanNode) sink.getInput().getSource();
-				
+
 				assertTrue(mapper.getInput2().getTempMode().breaksPipeline());
 			}
-			
+
 			{
 				ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 				env.setDegreeOfParallelism(64);
-				
+
 				DataSet<Long> initialSource = env.generateSequence(1, 10);
-				
-				Configuration conf= new Configuration();
+
+				Configuration conf = new Configuration();
 				conf.setString(PactCompiler.HINT_LOCAL_STRATEGY, PactCompiler.HINT_LOCAL_STRATEGY_NESTEDLOOP_STREAMED_OUTER_FIRST);
 				initialSource
-					.map(new IdentityMapper<Long>())
-					.cross(initialSource).withParameters(conf)
-					.print();
-				
-				
+						.map(new IdentityMapper<Long>())
+						.cross(initialSource).withParameters(conf)
+						.print();
+
+
 				Plan p = env.createProgramPlan();
 				OptimizedPlan op = compileNoStats(p);
-				
+
 				SinkPlanNode sink = op.getDataSinks().iterator().next();
 				DualInputPlanNode mapper = (DualInputPlanNode) sink.getInput().getSource();
-				
+
 				assertTrue(mapper.getInput1().getTempMode().breaksPipeline());
 			}
-			
+
 			{
 				ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 				env.setDegreeOfParallelism(64);
-				
+
 				DataSet<Long> initialSource = env.generateSequence(1, 10);
-				
-				Configuration conf= new Configuration();
+
+				Configuration conf = new Configuration();
 				conf.setString(PactCompiler.HINT_LOCAL_STRATEGY, PactCompiler.HINT_LOCAL_STRATEGY_NESTEDLOOP_STREAMED_OUTER_SECOND);
 				initialSource
-					.map(new IdentityMapper<Long>())
-					.cross(initialSource).withParameters(conf)
-					.print();
-				
-				
+						.map(new IdentityMapper<Long>())
+						.cross(initialSource).withParameters(conf)
+						.print();
+
+
 				Plan p = env.createProgramPlan();
 				OptimizedPlan op = compileNoStats(p);
-				
+
 				SinkPlanNode sink = op.getDataSinks().iterator().next();
 				DualInputPlanNode mapper = (DualInputPlanNode) sink.getInput().getSource();
-				
+
 				assertTrue(mapper.getInput2().getTempMode().breaksPipeline());
 			}
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
index dd920d7..4fc89c3 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
@@ -122,8 +122,7 @@ public final class ConfigConstants {
 	 * The key for the config parameter defining whether the memory manager allocates memory lazy.
 	 */
 	public static final String TASK_MANAGER_MEMORY_LAZY_ALLOCATION_KEY = "taskmanager.memory.lazyalloc";
-	
-	
+
 	/**
 	 * The config parameter defining the number of buffers used in the network stack. This defines the
 	 * number of possible tasks and shuffles.
@@ -136,6 +135,12 @@ public final class ConfigConstants {
 	public static final String TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY = "taskmanager.network.bufferSizeInBytes";
 
 	/**
+	 * The implementation to use for spillable/spilled intermediate results, which have both
+	 * synchronous and asynchronous implementations: "sync" or "async".
+	 */
+	public static final String TASK_MANAGER_NETWORK_DEFAULT_IO_MODE = "taskmanager.network.defaultIOMode";
+
+	/**
 	 * The config parameter defining the number of task slots of a task manager.
 	 */
 	public static final String TASK_MANAGER_NUM_TASK_SLOTS = "taskmanager.numberOfTaskSlots";
@@ -463,6 +468,12 @@ public final class ConfigConstants {
 	public static final int DEFAULT_TASK_MANAGER_NETWORK_BUFFER_SIZE = 32768;
 
 	/**
+	 * The implementation to use for spillable/spilled intermediate results, which have both
+	 * synchronous and asynchronous implementations: "sync" or "async".
+	 */
+	public static final String DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE = "sync";
+
+	/**
 	 * Flag indicating whether to start a thread, which repeatedly logs the memory usage of the JVM.
 	 */
 	public static final boolean DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD = false;

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-core/src/main/java/org/apache/flink/util/AbstractID.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/AbstractID.java b/flink-core/src/main/java/org/apache/flink/util/AbstractID.java
index a93c7ed..250869f 100644
--- a/flink-core/src/main/java/org/apache/flink/util/AbstractID.java
+++ b/flink-core/src/main/java/org/apache/flink/util/AbstractID.java
@@ -174,6 +174,12 @@ public class AbstractID implements IOReadableWritable, Comparable<AbstractID>, j
 		longToByteArray(this.upperPart, ba, SIZE_OF_LONG);
 		return StringUtils.byteToHexString(ba);
 	}
+
+	public String toShortString() {
+		final byte[] ba = new byte[SIZE_OF_LONG];
+		longToByteArray(upperPart, ba, 0);
+		return StringUtils.byteToHexString(ba);
+	}
 	
 	@Override
 	public int compareTo(AbstractID o) {

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-java/src/main/java/org/apache/flink/api/java/tuple/Tuple.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/tuple/Tuple.java b/flink-java/src/main/java/org/apache/flink/api/java/tuple/Tuple.java
index 145d215..9da50c7 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/tuple/Tuple.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/tuple/Tuple.java
@@ -108,5 +108,7 @@ public abstract class Tuple implements java.io.Serializable {
 	private static final Class<?>[] CLASSES = new Class<?>[] {
 		Tuple1.class, Tuple2.class, Tuple3.class, Tuple4.class, Tuple5.class, Tuple6.class, Tuple7.class, Tuple8.class, Tuple9.class, Tuple10.class, Tuple11.class, Tuple12.class, Tuple13.class, Tuple14.class, Tuple15.class, Tuple16.class, Tuple17.class, Tuple18.class, Tuple19.class, Tuple20.class, Tuple21.class, Tuple22.class, Tuple23.class, Tuple24.class, Tuple25.class
 	};
+
+
 	// END_OF_TUPLE_DEPENDENT_CODE
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/pom.xml
----------------------------------------------------------------------
diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml
index 14e27aa..ad570b8 100644
--- a/flink-runtime/pom.xml
+++ b/flink-runtime/pom.xml
@@ -90,7 +90,7 @@ under the License.
 		<dependency>
 			<groupId>io.netty</groupId>
 			<artifactId>netty-all</artifactId>
-			<version>4.0.24.Final</version>
+			<version>4.0.26.Final</version>
 		</dependency>
 
 		<dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptor.java
new file mode 100644
index 0000000..7592231
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptor.java
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.deployment;
+
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionEdge;
+import org.apache.flink.runtime.executiongraph.IntermediateResultPartition;
+import org.apache.flink.runtime.instance.Instance;
+import org.apache.flink.runtime.instance.SimpleSlot;
+import org.apache.flink.runtime.io.network.ConnectionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.consumer.InputChannel;
+import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+import java.util.Arrays;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * Deployment descriptor for a single input channel instance.
+ *
+ * <p> Each input channel consumes a single subpartition. The index of the subpartition to consume
+ * is part of the {@link InputGateDeploymentDescriptor} as it is the same for each input channel of
+ * the respective input gate.
+ *
+ * @see InputChannel
+ * @see SingleInputGate
+ */
+public class InputChannelDeploymentDescriptor implements Serializable {
+
+	private static Logger LOG = LoggerFactory.getLogger(InputChannelDeploymentDescriptor.class);
+
+	/** The ID of the partition the input channel is going to consume. */
+	private final ResultPartitionID consumedPartitionId;
+
+	/** The location of the partition the input channel is going to consume. */
+	private final ResultPartitionLocation consumedPartitionLocation;
+
+	public InputChannelDeploymentDescriptor(
+			ResultPartitionID consumedPartitionId,
+			ResultPartitionLocation consumedPartitionLocation) {
+
+		this.consumedPartitionId = checkNotNull(consumedPartitionId);
+		this.consumedPartitionLocation = checkNotNull(consumedPartitionLocation);
+	}
+
+	public ResultPartitionID getConsumedPartitionId() {
+		return consumedPartitionId;
+	}
+
+	public ResultPartitionLocation getConsumedPartitionLocation() {
+		return consumedPartitionLocation;
+	}
+
+	@Override
+	public String toString() {
+		return String.format("InputChannelDeploymentDescriptor [consumed partition id: %s, " +
+						"consumed partition location: %s]",
+				consumedPartitionId, consumedPartitionLocation);
+	}
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates an input channel deployment descriptor for each partition.
+	 */
+	public static InputChannelDeploymentDescriptor[] fromEdges(
+			ExecutionEdge[] edges, SimpleSlot consumerSlot) {
+
+		final InputChannelDeploymentDescriptor[] icdd = new InputChannelDeploymentDescriptor[edges.length];
+
+		// Each edge is connected to a different result partition
+		for (int i = 0; i < edges.length; i++) {
+			final IntermediateResultPartition consumedPartition = edges[i].getSource();
+			final Execution producer = consumedPartition.getProducer().getCurrentExecutionAttempt();
+
+			final ExecutionState producerState = producer.getState();
+			final SimpleSlot producerSlot = producer.getAssignedResource();
+
+			final ResultPartitionLocation partitionLocation;
+
+			// The producing task needs to be RUNNING or already FINISHED
+			if (consumedPartition.isConsumable() && producerSlot != null &&
+					(producerState == ExecutionState.RUNNING
+							|| producerState == ExecutionState.FINISHED)) {
+
+				final Instance partitionInstance = producerSlot.getInstance();
+
+				if (partitionInstance.equals(consumerSlot.getInstance())) {
+					// Consuming task is deployed to the same instance as the partition => local
+					partitionLocation = ResultPartitionLocation.createLocal();
+				}
+				else {
+					// Different instances => remote
+					final ConnectionID connectionId = new ConnectionID(
+							partitionInstance.getInstanceConnectionInfo(),
+							consumedPartition.getIntermediateResult().getConnectionIndex());
+
+					partitionLocation = ResultPartitionLocation.createRemote(connectionId);
+				}
+			}
+			else {
+				// The producing task might not have registered the partition yet
+				partitionLocation = ResultPartitionLocation.createUnknown();
+			}
+
+			final ResultPartitionID consumedPartitionId = new ResultPartitionID(
+					consumedPartition.getPartitionId(), producer.getAttemptId());
+
+			icdd[i] = new InputChannelDeploymentDescriptor(
+					consumedPartitionId, partitionLocation);
+		}
+
+		LOG.debug("Created {} from edges {}.", Arrays.toString(icdd), Arrays.toString(edges));
+
+		return icdd;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputGateDeploymentDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputGateDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputGateDeploymentDescriptor.java
new file mode 100644
index 0000000..77b072a
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputGateDeploymentDescriptor.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.deployment;
+
+import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
+
+import java.io.Serializable;
+import java.util.Arrays;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * Deployment descriptor for a single input gate instance.
+ *
+ * <p> Each input gate consumes partitions of a single intermediate result. The consumed
+ * subpartition index is the same for each consumed partition.
+ *
+ * @see SingleInputGate
+ */
+public class InputGateDeploymentDescriptor implements Serializable {
+
+	/**
+	 * The ID of the consumed intermediate result. Each input gate consumes partitions of the
+	 * intermediate result specified by this ID. This ID also identifies the input gate at the
+	 * consuming task.
+	 */
+	private final IntermediateDataSetID consumedResultId;
+
+	/**
+	 * The index of the consumed subpartition of each consumed partition. This index depends on the
+	 * {@link DistributionPattern} and the subtask indices of the producing and consuming task.
+	 */
+	private final int consumedSubpartitionIndex;
+
+	/** An input channel for each consumed subpartition. */
+	private final InputChannelDeploymentDescriptor[] inputChannels;
+
+	public InputGateDeploymentDescriptor(
+			IntermediateDataSetID consumedResultId,
+			int consumedSubpartitionIndex,
+			InputChannelDeploymentDescriptor[] inputChannels) {
+
+		this.consumedResultId = checkNotNull(consumedResultId);
+
+		checkArgument(consumedSubpartitionIndex >= 0);
+		this.consumedSubpartitionIndex = consumedSubpartitionIndex;
+
+		this.inputChannels = checkNotNull(inputChannels);
+	}
+
+	public IntermediateDataSetID getConsumedResultId() {
+		return consumedResultId;
+	}
+
+	public int getConsumedSubpartitionIndex() {
+		return consumedSubpartitionIndex;
+	}
+
+	public InputChannelDeploymentDescriptor[] getInputChannelDeploymentDescriptors() {
+		return inputChannels;
+	}
+
+	@Override
+	public String toString() {
+		return String.format("InputGateDeploymentDescriptor [result id: %s, " +
+						"consumed subpartition index: %d, input channels: %s]",
+				consumedResultId.toShortString(), consumedSubpartitionIndex,
+				Arrays.toString(inputChannels));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartialInputChannelDeploymentDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartialInputChannelDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartialInputChannelDeploymentDescriptor.java
new file mode 100644
index 0000000..adea18d
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartialInputChannelDeploymentDescriptor.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.deployment;
+
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.IntermediateResult;
+import org.apache.flink.runtime.executiongraph.IntermediateResultPartition;
+import org.apache.flink.runtime.instance.InstanceConnectionInfo;
+import org.apache.flink.runtime.io.network.ConnectionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * Partial deployment descriptor for a single input channel instance.
+ *
+ * <p> This deployment descriptor is created in {@link Execution#scheduleOrUpdateConsumers(java.util.List)},
+ * if the consumer instance is not yet clear. Once the instance on which the consumer runs is known,
+ * the deployment descriptor is updated by completing the partition location.
+ */
+public class PartialInputChannelDeploymentDescriptor {
+
+	/** The result ID identifies the input gate to update. */
+	private final IntermediateDataSetID resultId;
+
+	/** The partition ID identifies the input channel to update. */
+	private final ResultPartitionID partitionID;
+
+	/** The partition connection info. */
+	private final InstanceConnectionInfo partitionConnectionInfo;
+
+	/** The partition connection index. */
+	private final int partitionConnectionIndex;
+
+	public PartialInputChannelDeploymentDescriptor(
+			IntermediateDataSetID resultId,
+			ResultPartitionID partitionID,
+			InstanceConnectionInfo partitionConnectionInfo,
+			int partitionConnectionIndex) {
+
+		this.resultId = checkNotNull(resultId);
+		this.partitionID = checkNotNull(partitionID);
+		this.partitionConnectionInfo = checkNotNull(partitionConnectionInfo);
+		this.partitionConnectionIndex = partitionConnectionIndex;
+	}
+
+	/**
+	 * Creates a channel deployment descriptor by completing the partition location.
+	 *
+	 * @see InputChannelDeploymentDescriptor
+	 */
+	public InputChannelDeploymentDescriptor createInputChannelDeploymentDescriptor(
+			Execution consumerExecution) {
+
+		checkNotNull(consumerExecution, "Consumer execution null");
+
+		InstanceConnectionInfo consumerConnectionInfo = consumerExecution.getAssignedResourceLocation();
+
+		checkNotNull(consumerConnectionInfo, "Consumer connection info null");
+
+		final ResultPartitionLocation partitionLocation;
+
+		if (consumerConnectionInfo.equals(partitionConnectionInfo)) {
+			partitionLocation = ResultPartitionLocation.createLocal();
+		}
+		else {
+			partitionLocation = ResultPartitionLocation.createRemote(
+					new ConnectionID(partitionConnectionInfo, partitionConnectionIndex));
+		}
+
+		return new InputChannelDeploymentDescriptor(partitionID, partitionLocation);
+	}
+
+	public IntermediateDataSetID getResultId() {
+		return resultId;
+	}
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates a partial input channel for the given partition and producing task.
+	 */
+	public static PartialInputChannelDeploymentDescriptor fromEdge(
+			IntermediateResultPartition partition,
+			Execution producer) {
+
+		final ResultPartitionID partitionId = new ResultPartitionID(
+				partition.getPartitionId(), producer.getAttemptId());
+
+		final IntermediateResult result = partition.getIntermediateResult();
+
+		final IntermediateDataSetID resultId = result.getId();
+		final InstanceConnectionInfo partitionConnectionInfo = producer.getAssignedResourceLocation();
+		final int partitionConnectionIndex = result.getConnectionIndex();
+
+		return new PartialInputChannelDeploymentDescriptor(
+				resultId, partitionId, partitionConnectionInfo, partitionConnectionIndex);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartialPartitionInfo.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartialPartitionInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartialPartitionInfo.java
deleted file mode 100644
index a27c976..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartialPartitionInfo.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.deployment;
-
-import org.apache.flink.runtime.executiongraph.Execution;
-import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.executiongraph.ExecutionEdge;
-import org.apache.flink.runtime.executiongraph.IntermediateResultPartition;
-import org.apache.flink.runtime.instance.InstanceConnectionInfo;
-import org.apache.flink.runtime.io.network.RemoteAddress;
-import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
-import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
-
-/**
- * This class contains the partial partition info which is created if the consumer instance is not
- * yet clear. Once the instance on which the consumer runs is known, the complete partition info
- * can be computed.
- */
-public class PartialPartitionInfo {
-	private final IntermediateDataSetID intermediateDataSetID;
-
-	private final IntermediateResultPartitionID partitionID;
-
-	private final ExecutionAttemptID producerExecutionID;
-
-	private final InstanceConnectionInfo producerInstanceConnectionInfo;
-
-	private final int partitionConnectionIndex;
-
-	public PartialPartitionInfo(IntermediateDataSetID intermediateDataSetID,
-								IntermediateResultPartitionID partitionID,
-								ExecutionAttemptID executionID,
-								InstanceConnectionInfo producerInstanceConnectionInfo,
-								int partitionConnectionIndex) {
-		this.intermediateDataSetID = intermediateDataSetID;
-		this.partitionID = partitionID;
-		this.producerExecutionID = executionID;
-		this.producerInstanceConnectionInfo = producerInstanceConnectionInfo;
-		this.partitionConnectionIndex = partitionConnectionIndex;
-	}
-
-	public PartitionInfo createPartitionInfo(Execution consumerExecution) throws IllegalStateException {
-		if(consumerExecution != null){
-			PartitionInfo.PartitionLocation producerLocation;
-
-			RemoteAddress resolvedProducerAddress;
-
-			if(consumerExecution.getAssignedResourceLocation().equals(
-					producerInstanceConnectionInfo)) {
-				resolvedProducerAddress = null;
-				producerLocation = PartitionInfo.PartitionLocation.LOCAL;
-			} else {
-				resolvedProducerAddress = new RemoteAddress(producerInstanceConnectionInfo,
-						partitionConnectionIndex);
-
-				producerLocation = PartitionInfo.PartitionLocation.REMOTE;
-			}
-
-			return new PartitionInfo(partitionID, producerExecutionID, producerLocation,
-					resolvedProducerAddress);
-
-		} else {
-			throw new RuntimeException("Cannot create partition info, because consumer execution " +
-					"is null.");
-		}
-	}
-
-	public IntermediateDataSetID getIntermediateDataSetID() {
-		return intermediateDataSetID;
-	}
-
-	public static PartialPartitionInfo fromEdge(final ExecutionEdge edge){
-		IntermediateResultPartition partition = edge.getSource();
-		IntermediateResultPartitionID partitionID = edge.getSource().getPartitionId();
-
-		IntermediateDataSetID intermediateDataSetID = partition.getIntermediateResult().getId();
-
-		Execution producer = partition.getProducer().getCurrentExecutionAttempt();
-		ExecutionAttemptID producerExecutionID = producer.getAttemptId();
-
-		return new PartialPartitionInfo(intermediateDataSetID, partitionID, producerExecutionID,
-				producer.getAssignedResourceLocation(),
-				partition.getIntermediateResult().getConnectionIndex());
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartitionConsumerDeploymentDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartitionConsumerDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartitionConsumerDeploymentDescriptor.java
deleted file mode 100644
index 7300da4..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartitionConsumerDeploymentDescriptor.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.deployment;
-
-import org.apache.flink.core.io.IOReadableWritable;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.Arrays;
-
-/**
- * A partition consumer deployment descriptor combines information of all partitions, which are
- * consumed by a single reader.
- */
-public class PartitionConsumerDeploymentDescriptor implements IOReadableWritable, Serializable {
-
-	private IntermediateDataSetID resultId;
-
-	private PartitionInfo[] partitions;
-
-	private int queueIndex;
-
-	public PartitionConsumerDeploymentDescriptor() {
-	}
-
-	public PartitionConsumerDeploymentDescriptor(IntermediateDataSetID resultId, PartitionInfo[] partitions, int queueIndex) {
-		this.resultId = resultId;
-		this.partitions = partitions;
-		this.queueIndex = queueIndex;
-	}
-
-	// ------------------------------------------------------------------------
-	// Properties
-	// ------------------------------------------------------------------------
-
-	public IntermediateDataSetID getResultId() {
-		return resultId;
-	}
-
-	public PartitionInfo[] getPartitions() {
-		return partitions;
-	}
-
-	public int getQueueIndex() {
-		return queueIndex;
-	}
-
-	// ------------------------------------------------------------------------
-	// Serialization
-	// ------------------------------------------------------------------------
-
-	@Override
-	public void write(DataOutputView out) throws IOException {
-		resultId.write(out);
-		out.writeInt(partitions.length);
-		for (PartitionInfo partition : partitions) {
-			partition.write(out);
-		}
-
-		out.writeInt(queueIndex);
-	}
-
-	@Override
-	public void read(DataInputView in) throws IOException {
-		resultId = new IntermediateDataSetID();
-		resultId.read(in);
-
-		partitions = new PartitionInfo[in.readInt()];
-		for (int i = 0; i < partitions.length; i++) {
-			partitions[i] = new PartitionInfo();
-			partitions[i].read(in);
-		}
-
-		this.queueIndex = in.readInt();
-	}
-
-	@Override
-	public String toString() {
-		return String.format("PartitionConsumerDeploymentDescriptor(ResultID: %s, " +
-				"Queue index: %d, Partitions: %s)", resultId, queueIndex,
-				Arrays.toString(partitions));
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartitionDeploymentDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartitionDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartitionDeploymentDescriptor.java
deleted file mode 100644
index 37651c9..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartitionDeploymentDescriptor.java
+++ /dev/null
@@ -1,122 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.deployment;
-
-import org.apache.flink.core.io.IOReadableWritable;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.executiongraph.IntermediateResultPartition;
-import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
-import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
-import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionType;
-
-import java.io.IOException;
-import java.io.Serializable;
-
-/**
- * A partition deployment descriptor combines information for a produced intermediate result
- * partition.
- */
-public class PartitionDeploymentDescriptor implements IOReadableWritable, Serializable {
-
-	private final IntermediateDataSetID resultId;
-
-	private final IntermediateResultPartitionID partitionId;
-
-	private IntermediateResultPartitionType partitionType;
-
-	private int numberOfQueues;
-
-	public PartitionDeploymentDescriptor() {
-		this.resultId = new IntermediateDataSetID();
-		this.partitionId = new IntermediateResultPartitionID();
-		this.numberOfQueues = -1;
-	}
-
-	public PartitionDeploymentDescriptor(IntermediateDataSetID resultId, IntermediateResultPartitionID partitionId, IntermediateResultPartitionType partitionType, int numberOfQueues) {
-		this.resultId = resultId;
-		this.partitionId = partitionId;
-		this.partitionType = partitionType;
-		this.numberOfQueues = numberOfQueues;
-	}
-
-	// ------------------------------------------------------------------------
-	// Properties
-	// ------------------------------------------------------------------------
-
-	public IntermediateDataSetID getResultId() {
-		return resultId;
-	}
-
-	public IntermediateResultPartitionID getPartitionId() {
-		return partitionId;
-	}
-
-	public IntermediateResultPartitionType getPartitionType() {
-		return partitionType;
-	}
-
-	public int getNumberOfQueues() {
-		return numberOfQueues;
-	}
-
-	// ------------------------------------------------------------------------
-	// Serialization
-	// ------------------------------------------------------------------------
-
-	@Override
-	public void write(DataOutputView out) throws IOException {
-		resultId.write(out);
-		partitionId.write(out);
-		out.writeInt(partitionType.ordinal());
-		out.writeInt(numberOfQueues);
-	}
-
-	@Override
-	public void read(DataInputView in) throws IOException {
-		resultId.read(in);
-		partitionId.read(in);
-		partitionType = IntermediateResultPartitionType.values()[in.readInt()];
-		numberOfQueues = in.readInt();
-	}
-
-	// ------------------------------------------------------------------------
-
-	public static PartitionDeploymentDescriptor fromIntermediateResultPartition(IntermediateResultPartition partition) {
-
-		IntermediateResultPartitionID partitionId = partition.getPartitionId();
-
-		// The produced data is partitioned at runtime among a number of queues.
-		// If no consumers are known at this point, we use a single queue,
-		// otherwise we have a queue for each consumer sub task.
-		int numberOfQueues = 1;
-
-		if (!partition.getConsumers().isEmpty() && !partition.getConsumers().get(0).isEmpty()) {
-			numberOfQueues = partition.getConsumers().get(0).size();
-		}
-
-		return new PartitionDeploymentDescriptor(partition.getIntermediateResult().getId(), partitionId, partition.getIntermediateResult().getResultType(), numberOfQueues);
-	}
-
-	@Override
-	public String toString() {
-		return String.format("PartitionDeploymentDescriptor(ResultID: %s, partitionID: %s, " +
-				"Partition type: %s)", resultId, partitionId, partitionType);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartitionInfo.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartitionInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartitionInfo.java
deleted file mode 100644
index 6a30853..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartitionInfo.java
+++ /dev/null
@@ -1,176 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.deployment;
-
-import org.apache.flink.core.io.IOReadableWritable;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.execution.ExecutionState;
-import org.apache.flink.runtime.executiongraph.Execution;
-import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.executiongraph.ExecutionEdge;
-import org.apache.flink.runtime.executiongraph.IntermediateResultPartition;
-import org.apache.flink.runtime.instance.SimpleSlot;
-import org.apache.flink.runtime.io.network.RemoteAddress;
-import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.Serializable;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * A partition info instance contains all information necessary for a reader to create an input
- * channel to request a partition at runtime.
- */
-public class PartitionInfo implements IOReadableWritable, Serializable {
-
-	private static Logger LOG = LoggerFactory.getLogger(PartitionInfo.class);
-
-	public enum PartitionLocation {
-		LOCAL, REMOTE, UNKNOWN
-	}
-
-	private final IntermediateResultPartitionID partitionId;
-
-	private ExecutionAttemptID producerExecutionId;
-
-	private PartitionLocation producerLocation;
-
-	private RemoteAddress producerAddress; // != null, iff known remote producer
-
-	public PartitionInfo(IntermediateResultPartitionID partitionId, ExecutionAttemptID producerExecutionId, PartitionLocation producerLocation, RemoteAddress producerAddress) {
-		this.partitionId = checkNotNull(partitionId);
-		this.producerExecutionId = checkNotNull(producerExecutionId);
-		this.producerLocation = checkNotNull(producerLocation);
-		this.producerAddress = producerAddress;
-	}
-
-	public PartitionInfo() {
-		this.partitionId = new IntermediateResultPartitionID();
-		this.producerExecutionId = new ExecutionAttemptID();
-		this.producerLocation = PartitionLocation.UNKNOWN;
-		this.producerAddress = null;
-	}
-
-	// ------------------------------------------------------------------------
-	// Properties
-	// ------------------------------------------------------------------------
-
-	public IntermediateResultPartitionID getPartitionId() {
-		return partitionId;
-	}
-
-	public ExecutionAttemptID getProducerExecutionId() {
-		return producerExecutionId;
-	}
-
-	public PartitionLocation getProducerLocation() {
-		return producerLocation;
-	}
-
-	public RemoteAddress getProducerAddress() {
-		return producerAddress;
-	}
-
-	// ------------------------------------------------------------------------
-	// Serialization
-	// ------------------------------------------------------------------------
-
-	@Override
-	public void write(DataOutputView out) throws IOException {
-		partitionId.write(out);
-		producerExecutionId.write(out);
-		out.writeInt(producerLocation.ordinal());
-		if (producerLocation == PartitionLocation.REMOTE) {
-			producerAddress.write(out);
-		}
-	}
-
-	@Override
-	public void read(DataInputView in) throws IOException {
-		partitionId.read(in);
-		producerExecutionId.read(in);
-		producerLocation = PartitionLocation.values()[in.readInt()];
-		if (producerLocation == PartitionLocation.REMOTE) {
-			producerAddress = new RemoteAddress();
-			producerAddress.read(in);
-		}
-	}
-
-	// ------------------------------------------------------------------------
-
-	public static PartitionInfo fromEdge(ExecutionEdge edge, SimpleSlot consumerSlot) {
-		IntermediateResultPartition partition = edge.getSource();
-		IntermediateResultPartitionID partitionId = partition.getPartitionId();
-
-		// Intermediate result partition producer
-		Execution producer = partition.getProducer().getCurrentExecutionAttempt();
-
-		ExecutionAttemptID producerExecutionId = producer.getAttemptId();
-		RemoteAddress producerAddress = null;
-		PartitionLocation producerLocation = PartitionLocation.UNKNOWN;
-
-		SimpleSlot producerSlot = producer.getAssignedResource();
-		ExecutionState producerState = producer.getState();
-
-		// The producer needs to be running, otherwise the consumer might request a partition,
-		// which has not been registered yet.
-		if (producerSlot != null && (producerState == ExecutionState.RUNNING ||
-			producerState == ExecutionState.FINISHED)) {
-			if (producerSlot.getInstance().equals(consumerSlot.getInstance())) {
-				producerLocation = PartitionLocation.LOCAL;
-			}
-			else {
-				producerAddress = new RemoteAddress(producerSlot.getInstance().getInstanceConnectionInfo(),
-						partition.getIntermediateResult().getConnectionIndex());
-
-				producerLocation = PartitionLocation.REMOTE;
-			}
-		}
-
-		PartitionInfo partitionInfo = new PartitionInfo(partitionId, producerExecutionId,
-				producerLocation, producerAddress);
-
-		LOG.debug("Create partition info {}.", partitionInfo);
-
-		return partitionInfo;
-	}
-
-	public static PartitionInfo[] fromEdges(ExecutionEdge[] edges, SimpleSlot consumerSlot) {
-		// Every edge consumes a different result partition, which might be of
-		// local, remote, or unknown location.
-		PartitionInfo[] partitions = new PartitionInfo[edges.length];
-
-		for (int i = 0; i < edges.length; i++) {
-			partitions[i] = fromEdge(edges[i], consumerSlot);
-		}
-
-		return partitions;
-	}
-
-	@Override
-	public String toString() {
-		return String.format("PartitionInfo(PartitionID: %s, ProducerID: %s, " +
-				"ProducerLocation: %s, ProducerAddress: %s)", partitionId, producerExecutionId,
-				producerLocation, producerAddress);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptor.java
new file mode 100644
index 0000000..4a88f18
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptor.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.deployment;
+
+import org.apache.flink.runtime.executiongraph.IntermediateResultPartition;
+import org.apache.flink.runtime.io.network.partition.ResultPartition;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
+import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
+
+import java.io.Serializable;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * Deployment descriptor for a result partition.
+ *
+ * @see ResultPartition
+ */
+public class ResultPartitionDeploymentDescriptor implements Serializable {
+
+	/** The ID of the result this partition belongs to. */
+	private final IntermediateDataSetID resultId;
+
+	/** The ID of the partition. */
+	private final IntermediateResultPartitionID partitionId;
+
+	/** The type of the partition. */
+	private final ResultPartitionType partitionType;
+
+	/** The number of subpartitions. */
+	private final int numberOfSubpartitions;
+
+	public ResultPartitionDeploymentDescriptor(
+			IntermediateDataSetID resultId,
+			IntermediateResultPartitionID partitionId,
+			ResultPartitionType partitionType,
+			int numberOfSubpartitions) {
+
+		this.resultId = checkNotNull(resultId);
+		this.partitionId = checkNotNull(partitionId);
+		this.partitionType = checkNotNull(partitionType);
+
+		checkArgument(numberOfSubpartitions >= 1);
+		this.numberOfSubpartitions = numberOfSubpartitions;
+	}
+
+	public IntermediateDataSetID getResultId() {
+		return resultId;
+	}
+
+	public IntermediateResultPartitionID getPartitionId() {
+		return partitionId;
+	}
+
+	public ResultPartitionType getPartitionType() {
+		return partitionType;
+	}
+
+	public int getNumberOfSubpartitions() {
+		return numberOfSubpartitions;
+	}
+
+	@Override
+	public String toString() {
+		return String.format("ResultPartitionDeploymentDescriptor [result id: %s," +
+						"partition id: %s,partition type: %s]",
+				resultId, partitionId, partitionType);
+	}
+
+	// ------------------------------------------------------------------------
+
+	public static ResultPartitionDeploymentDescriptor from(IntermediateResultPartition partition) {
+
+		final IntermediateDataSetID resultId = partition.getIntermediateResult().getId();
+		final IntermediateResultPartitionID partitionId = partition.getPartitionId();
+		final ResultPartitionType partitionType = partition.getIntermediateResult().getResultType();
+
+		// The produced data is partitioned among a number of subpartitions.
+		//
+		// If no consumers are known at this point, we use a single subpartition, otherwise we have
+		// one for each consuming sub task.
+		int numberOfSubpartitions = 1;
+
+		if (!partition.getConsumers().isEmpty() && !partition.getConsumers().get(0).isEmpty()) {
+
+			if (partition.getConsumers().size() > 1) {
+				new IllegalStateException("Currently, only a single consumer group per partition is supported.");
+			}
+
+			numberOfSubpartitions = partition.getConsumers().get(0).size();
+		}
+
+		return new ResultPartitionDeploymentDescriptor(
+				resultId, partitionId, partitionType, numberOfSubpartitions);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ResultPartitionLocation.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ResultPartitionLocation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ResultPartitionLocation.java
new file mode 100644
index 0000000..3922c22
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ResultPartitionLocation.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.deployment;
+
+import org.apache.flink.runtime.io.network.ConnectionID;
+
+import java.io.Serializable;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * Location of a result partition from the perspective of the consuming task.
+ *
+ * <p> The location indicates both the instance, on which the partition is produced and the state of
+ * the producing task. There are three possibilities:
+ *
+ * <ol>
+ * <li><strong>Local:</strong> The partition is available at the same instance on which the
+ * consuming task is (being) deployed and the producing task has registered the result partition.
+ *
+ * <li><strong>Remote:</strong> The result partition is available at a different instance from the
+ * one, on which the consuming task is (being) deployed and the producing task has registered the
+ * result partition.
+ *
+ * <li><strong>Unknown:</strong> The producing task has not yet registered the result partition.
+ * When deploying the consuming task, the instance might be known or unknown. In any case, the
+ * consuming task cannot request it yet. Instead, it will be updated at runtime after the
+ * producing task is guaranteed to have registered the partition. A producing task is guaranteed
+ * to have registered the partition after its state has switched to running.
+ * </ol>
+ */
+public class ResultPartitionLocation implements Serializable {
+
+	/** The type of location for the result partition. */
+	private final LocationType locationType;
+
+	/** The connection ID of a remote result partition. */
+	private final ConnectionID connectionId;
+
+	private enum LocationType {
+		LOCAL,
+		REMOTE,
+		UNKNOWN
+	}
+
+	private ResultPartitionLocation(LocationType locationType, ConnectionID connectionId) {
+		this.locationType = checkNotNull(locationType);
+		this.connectionId = connectionId;
+	}
+
+	public static ResultPartitionLocation createRemote(ConnectionID connectionId) {
+		return new ResultPartitionLocation(LocationType.REMOTE, checkNotNull(connectionId));
+	}
+
+	public static ResultPartitionLocation createLocal() {
+		return new ResultPartitionLocation(LocationType.LOCAL, null);
+	}
+
+	public static ResultPartitionLocation createUnknown() {
+		return new ResultPartitionLocation(LocationType.UNKNOWN, null);
+	}
+
+	// ------------------------------------------------------------------------
+
+	public boolean isLocal() {
+		return locationType == LocationType.LOCAL;
+	}
+
+	public boolean isRemote() {
+		return locationType == LocationType.REMOTE;
+	}
+
+	public boolean isUnknown() {
+		return locationType == LocationType.UNKNOWN;
+	}
+
+	public ConnectionID getConnectionId() {
+		return connectionId;
+	}
+
+	@Override
+	public String toString() {
+		return "ResultPartitionLocation [" + locationType + (isRemote() ? " [" + connectionId + "]]" : "]");
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java
index 6993248..f7518bd 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java
@@ -26,7 +26,7 @@ import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.state.StateHandle;
 
 import java.io.Serializable;
-import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 
 import static com.google.common.base.Preconditions.checkArgument;
@@ -49,47 +49,47 @@ public final class TaskDeploymentDescriptor implements Serializable {
 	private final ExecutionAttemptID executionId;
 
 	/** The task's name. */
-	private String taskName;
+	private final String taskName;
 
 	/** The task's index in the subtask group. */
-	private int indexInSubtaskGroup;
+	private final int indexInSubtaskGroup;
 
 	/** The number of sub tasks. */
-	private int numberOfSubtasks;
+	private final int numberOfSubtasks;
 
 	/** The configuration of the job the task belongs to. */
-	private Configuration jobConfiguration;
+	private final Configuration jobConfiguration;
 
 	/** The task's configuration object. */
-	private Configuration taskConfiguration;
+	private final Configuration taskConfiguration;
 
 	/** The name of the class containing the task code to be executed. */
-	private String invokableClassName;
-
+	private final String invokableClassName;
 
 	/** The list of produced intermediate result partition deployment descriptors. */
-	private List<PartitionDeploymentDescriptor> producedPartitions;
+	private final List<ResultPartitionDeploymentDescriptor> producedPartitions;
 
 	/** The list of consumed intermediate result partitions. */
-	private List<PartitionConsumerDeploymentDescriptor> consumedPartitions;
+	private final List<InputGateDeploymentDescriptor> inputGates;
 
-	private int targetSlotNumber;
+	private final int targetSlotNumber;
 
 	/** The list of JAR files required to run this task. */
 	private final List<BlobKey> requiredJarFiles;
-	
+
 	private StateHandle operatorStates;
 
+
 	/**
 	 * Constructs a task deployment descriptor.
 	 */
 	public TaskDeploymentDescriptor(
-			JobID jobID, JobVertexID vertexID,  ExecutionAttemptID executionId,  String taskName,
-			int indexInSubtaskGroup,  int numberOfSubtasks, Configuration jobConfiguration,
+			JobID jobID, JobVertexID vertexID, ExecutionAttemptID executionId, String taskName,
+			int indexInSubtaskGroup, int numberOfSubtasks, Configuration jobConfiguration,
 			Configuration taskConfiguration, String invokableClassName,
-			List<PartitionDeploymentDescriptor> producedPartitions,
-			List<PartitionConsumerDeploymentDescriptor> consumedPartitions,
-			List<BlobKey> requiredJarFiles, int targetSlotNumber){
+			List<ResultPartitionDeploymentDescriptor> producedPartitions,
+			List<InputGateDeploymentDescriptor> inputGates,
+			List<BlobKey> requiredJarFiles, int targetSlotNumber) {
 
 		this.jobID = checkNotNull(jobID);
 		this.vertexID = checkNotNull(vertexID);
@@ -103,37 +103,25 @@ public final class TaskDeploymentDescriptor implements Serializable {
 		this.taskConfiguration = checkNotNull(taskConfiguration);
 		this.invokableClassName = checkNotNull(invokableClassName);
 		this.producedPartitions = checkNotNull(producedPartitions);
-		this.consumedPartitions = checkNotNull(consumedPartitions);
+		this.inputGates = checkNotNull(inputGates);
 		this.requiredJarFiles = checkNotNull(requiredJarFiles);
+		checkArgument(targetSlotNumber >= 0);
 		this.targetSlotNumber = targetSlotNumber;
 	}
 
-	/**
-	 * Default constructor for serialization/deserialization.
-	 */
-	public TaskDeploymentDescriptor() {
-		this.jobID = new JobID();
-		this.vertexID = new JobVertexID();
-		this.executionId = new ExecutionAttemptID();
-		this.jobConfiguration = new Configuration();
-		this.taskConfiguration = new Configuration();
-		this.producedPartitions = new ArrayList<PartitionDeploymentDescriptor>();
-		this.consumedPartitions = new ArrayList<PartitionConsumerDeploymentDescriptor>();
-		this.requiredJarFiles = new ArrayList<BlobKey>();
-	}
-
 	public TaskDeploymentDescriptor(
 			JobID jobID, JobVertexID vertexID, ExecutionAttemptID executionId, String taskName,
 			int indexInSubtaskGroup, int numberOfSubtasks, Configuration jobConfiguration,
 			Configuration taskConfiguration, String invokableClassName,
-			List<PartitionDeploymentDescriptor> producedPartitions,
-			List<PartitionConsumerDeploymentDescriptor> consumedPartitions,
-			List<BlobKey> requiredJarFiles, int targetSlotNumber, StateHandle operatorStates) {
+			List<ResultPartitionDeploymentDescriptor> producedPartitions,
+			List<InputGateDeploymentDescriptor> inputGates,
+			List<BlobKey> requiredJarFiles, int targetSlotNumber,
+			StateHandle operatorStates) {
 
 		this(jobID, vertexID, executionId, taskName, indexInSubtaskGroup, numberOfSubtasks,
 				jobConfiguration, taskConfiguration, invokableClassName, producedPartitions,
-				consumedPartitions, requiredJarFiles, targetSlotNumber);
-		
+				inputGates, requiredJarFiles, targetSlotNumber);
+
 		setOperatorState(operatorStates);
 	}
 
@@ -164,7 +152,7 @@ public final class TaskDeploymentDescriptor implements Serializable {
 
 	/**
 	 * Returns the task's index in the subtask group.
-	 * 
+	 *
 	 * @return the task's index in the subtask group
 	 */
 	public int getIndexInSubtaskGroup() {
@@ -177,10 +165,10 @@ public final class TaskDeploymentDescriptor implements Serializable {
 	public int getNumberOfSubtasks() {
 		return numberOfSubtasks;
 	}
-	
+
 	/**
 	 * Gets the number of the slot into which the task is to be deployed.
-	 * 
+	 *
 	 * @return The number of the target slot.
 	 */
 	public int getTargetSlotNumber() {
@@ -208,12 +196,12 @@ public final class TaskDeploymentDescriptor implements Serializable {
 		return invokableClassName;
 	}
 
-	public List<PartitionDeploymentDescriptor> getProducedPartitions() {
+	public List<ResultPartitionDeploymentDescriptor> getProducedPartitions() {
 		return producedPartitions;
 	}
 
-	public List<PartitionConsumerDeploymentDescriptor> getConsumedPartitions() {
-		return consumedPartitions;
+	public List<InputGateDeploymentDescriptor> getInputGates() {
+		return inputGates;
 	}
 
 	public List<BlobKey> getRequiredJarFiles() {
@@ -222,25 +210,26 @@ public final class TaskDeploymentDescriptor implements Serializable {
 
 	@Override
 	public String toString() {
-		final StringBuilder pddBuilder = new StringBuilder("");
-		final StringBuilder pcddBuilder = new StringBuilder("");
+		return String.format("TaskDeploymentDescriptor [job id: %s, job vertex id: %s, " +
+						"execution id: %s, task name: %s (%d/%d), invokable: %s, " +
+						"produced partitions: %s, input gates: %s]",
+				jobID, vertexID, executionId, taskName, indexInSubtaskGroup, numberOfSubtasks,
+				invokableClassName, collectionToString(producedPartitions),
+				collectionToString(inputGates));
+	}
 
-		for(PartitionDeploymentDescriptor pdd: producedPartitions) {
-			pddBuilder.append(pdd);
-		}
+	private String collectionToString(Collection<?> collection) {
+		final StringBuilder strBuilder = new StringBuilder();
+
+		strBuilder.append("[");
 
-		for(PartitionConsumerDeploymentDescriptor pcdd: consumedPartitions) {
-			pcddBuilder.append(pcdd);
+		for (Object elem : collection) {
+			strBuilder.append(elem.toString());
 		}
 
-		final String strProducedPartitions = pddBuilder.toString();
-		final String strConsumedPartitions = pcddBuilder.toString();
+		strBuilder.append("]");
 
-		return String.format("TaskDeploymentDescriptor(JobID: %s, JobVertexID: %s, " +
-				"ExecutionID: %s, Task name: %s, (%d/%d), Invokable: %s, " +
-				"Produced partitions: %s, Consumed partitions: %s", jobID, vertexID, executionId,
-				taskName, indexInSubtaskGroup, numberOfSubtasks, invokableClassName,
-				strProducedPartitions, strConsumedPartitions);
+		return strBuilder.toString();
 	}
 
 	public void setOperatorState(StateHandle operatorStates) {

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java
index fc2d2c8..556bb11 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java
@@ -23,7 +23,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.broadcast.BroadcastVariableManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.io.network.api.writer.BufferWriter;
+import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
 import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobID;
@@ -130,9 +130,9 @@ public interface Environment {
 
 	BroadcastVariableManager getBroadcastVariableManager();
 
-	BufferWriter getWriter(int index);
+	ResultPartitionWriter getWriter(int index);
 
-	BufferWriter[] getAllWriters();
+	ResultPartitionWriter[] getAllWriters();
 
 	InputGate getInputGate(int index);
 


[07/13] flink git commit: [FLINK-1350] [runtime] Add blocking result partition variant

Posted by uc...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewSyncIO.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewSyncIO.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewSyncIO.java
new file mode 100644
index 0000000..62ebddf
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewSyncIO.java
@@ -0,0 +1,177 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition;
+
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.disk.iomanager.BufferFileReader;
+import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel;
+import org.apache.flink.runtime.io.disk.iomanager.SynchronousBufferFileReader;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
+import org.apache.flink.runtime.util.event.NotificationListener;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.Queue;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * View over a spilled subpartition.
+ *
+ * <p> Reads are done synchronously.
+ */
+class SpilledSubpartitionViewSyncIO implements ResultSubpartitionView {
+
+	/** The subpartition this view belongs to. */
+	private final ResultSubpartition parent;
+
+	/** The synchronous file reader to do the actual I/O. */
+	private final BufferFileReader fileReader;
+
+	/** The buffer pool to read data into. */
+	private final SpillReadBufferPool bufferPool;
+
+	/** Flag indicating whether all resources have been released. */
+	private AtomicBoolean isReleased = new AtomicBoolean();
+
+	SpilledSubpartitionViewSyncIO(
+			ResultSubpartition parent,
+			int memorySegmentSize,
+			FileIOChannel.ID channelId,
+			long initialSeekPosition) throws IOException {
+
+		checkArgument(initialSeekPosition >= 0, "Initial seek position is < 0.");
+
+		this.parent = checkNotNull(parent);
+
+		this.bufferPool = new SpillReadBufferPool(2, memorySegmentSize);
+
+		this.fileReader = new SynchronousBufferFileReader(channelId, false);
+
+		if (initialSeekPosition > 0) {
+			fileReader.seekToPosition(initialSeekPosition);
+		}
+	}
+
+	@Override
+	public Buffer getNextBuffer() throws IOException, InterruptedException {
+
+		if (fileReader.hasReachedEndOfFile()) {
+			return null;
+		}
+
+		// It's OK to request the buffer in a blocking fashion as the buffer pool is NOT shared
+		// among all consumed subpartitions.
+		final Buffer buffer = bufferPool.requestBufferBlocking();
+
+		fileReader.readInto(buffer);
+
+		return buffer;
+	}
+
+	@Override
+	public boolean registerListener(NotificationListener listener) throws IOException {
+		return false;
+	}
+
+	@Override
+	public void notifySubpartitionConsumed() throws IOException {
+		parent.onConsumedSubpartition();
+	}
+
+	@Override
+	public void releaseAllResources() throws IOException {
+		if (isReleased.compareAndSet(false, true)) {
+			fileReader.close();
+			bufferPool.destroy();
+		}
+	}
+
+	@Override
+	public boolean isReleased() {
+		return isReleased.get();
+	}
+
+	/**
+	 * A buffer pool to provide buffer to read the file into.
+	 *
+	 * <p> This pool ensures that a consuming input gate makes progress in all cases, even when all
+	 * buffers of the input gate buffer pool have been requested by remote input channels.
+	 *
+	 * TODO Replace with asynchronous buffer pool request as this introduces extra buffers per
+	 * consumed subpartition.
+	 */
+	private static class SpillReadBufferPool implements BufferRecycler {
+
+		private final Queue<Buffer> buffers;
+
+		private boolean isDestroyed;
+
+		public SpillReadBufferPool(int numberOfBuffers, int memorySegmentSize) {
+			this.buffers = new ArrayDeque<Buffer>(numberOfBuffers);
+
+			synchronized (buffers) {
+				for (int i = 0; i < numberOfBuffers; i++) {
+					buffers.add(new Buffer(new MemorySegment(new byte[memorySegmentSize]), this));
+				}
+			}
+		}
+
+		@Override
+		public void recycle(MemorySegment memorySegment) {
+			synchronized (buffers) {
+				if (isDestroyed) {
+					memorySegment.free();
+				}
+				else {
+					buffers.add(new Buffer(memorySegment, this));
+					buffers.notifyAll();
+				}
+			}
+		}
+
+		private Buffer requestBufferBlocking() throws InterruptedException {
+			synchronized (buffers) {
+				while (true) {
+					if (isDestroyed) {
+						return null;
+					}
+
+					Buffer buffer = buffers.poll();
+
+					if (buffer != null) {
+						return buffer;
+					}
+					// Else: wait for a buffer
+					buffers.wait();
+				}
+			}
+		}
+
+		private void destroy() {
+			synchronized (buffers) {
+				isDestroyed = true;
+				buffers.notifyAll();
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java
index fb41549..855509c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java
@@ -19,20 +19,19 @@
 package org.apache.flink.runtime.io.network.partition.consumer;
 
 import org.apache.flink.runtime.event.task.TaskEvent;
-import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.partition.queue.IntermediateResultPartitionQueue;
-import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView;
 
 import java.io.IOException;
 
 /**
- * An input channel is the consumer of a single subpartition of an {@link IntermediateResultPartitionQueue}.
+ * An input channel consumes a single {@link ResultSubpartitionView}.
  * <p>
  * For each channel, the consumption life cycle is as follows:
  * <ol>
- * <li>{@link #requestIntermediateResultPartition(int)}</li>
- * <li>{@link #getNextBuffer()} until {@link #isReleased()}</li>
+ * <li>{@link #requestSubpartition(int)}</li>
+ * <li>{@link #getNextBuffer()}</li>
  * <li>{@link #releaseAllResources()}</li>
  * </ol>
  */
@@ -40,17 +39,13 @@ public abstract class InputChannel {
 
 	protected final int channelIndex;
 
-	protected final ExecutionAttemptID producerExecutionId;
-
-	protected final IntermediateResultPartitionID partitionId;
+	protected final ResultPartitionID partitionId;
 
 	protected final SingleInputGate inputGate;
 
-	protected InputChannel(SingleInputGate inputGate, int channelIndex, ExecutionAttemptID producerExecutionId,
-			IntermediateResultPartitionID partitionId) {
+	protected InputChannel(SingleInputGate inputGate, int channelIndex, ResultPartitionID partitionId) {
 		this.inputGate = inputGate;
 		this.channelIndex = channelIndex;
-		this.producerExecutionId = producerExecutionId;
 		this.partitionId = partitionId;
 	}
 
@@ -62,19 +57,6 @@ public abstract class InputChannel {
 		return channelIndex;
 	}
 
-	public ExecutionAttemptID getProducerExecutionId() {
-		return producerExecutionId;
-	}
-
-	public IntermediateResultPartitionID getPartitionId() {
-		return partitionId;
-	}
-
-	@Override
-	public String toString() {
-		return String.format("[%s:%s]", producerExecutionId, partitionId);
-	}
-
 	/**
 	 * Notifies the owning {@link SingleInputGate} about an available {@link Buffer} instance.
 	 */
@@ -93,12 +75,12 @@ public abstract class InputChannel {
 	 * The queue index to request depends on which sub task the channel belongs
 	 * to and is specified by the consumer of this channel.
 	 */
-	public abstract void requestIntermediateResultPartition(int queueIndex) throws IOException, InterruptedException;
+	public abstract void requestSubpartition(int subpartitionIndex) throws IOException, InterruptedException;
 
 	/**
 	 * Returns the next buffer from the consumed subpartition.
 	 */
-	public abstract Buffer getNextBuffer() throws IOException;
+	public abstract Buffer getNextBuffer() throws IOException, InterruptedException;
 
 	// ------------------------------------------------------------------------
 	// Task events
@@ -120,6 +102,8 @@ public abstract class InputChannel {
 
 	public abstract boolean isReleased();
 
+	public abstract void notifySubpartitionConsumed() throws IOException;
+
 	/**
 	 * Releases all resources of the channel.
 	 */

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java
index c05952f..65f2627 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java
@@ -20,12 +20,13 @@ package org.apache.flink.runtime.io.network.partition.consumer;
 
 import com.google.common.base.Optional;
 import org.apache.flink.runtime.event.task.TaskEvent;
-import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.io.network.TaskEventDispatcher;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.partition.IntermediateResultPartitionManager;
-import org.apache.flink.runtime.io.network.partition.queue.IntermediateResultPartitionQueueIterator;
-import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionManager;
+import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView;
 import org.apache.flink.runtime.util.event.NotificationListener;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -42,24 +43,24 @@ public class LocalInputChannel extends InputChannel implements NotificationListe
 
 	private static final Logger LOG = LoggerFactory.getLogger(LocalInputChannel.class);
 
-	private final IntermediateResultPartitionManager partitionManager;
+	private final ResultPartitionManager partitionManager;
 
 	private final TaskEventDispatcher taskEventDispatcher;
 
-	private IntermediateResultPartitionQueueIterator queueIterator;
+	private ResultSubpartitionView queueIterator;
 
-	private boolean isReleased;
+	private volatile boolean isReleased;
 
 	private volatile Buffer lookAhead;
 
 	public LocalInputChannel(
-			SingleInputGate gate, int channelIndex,
-			ExecutionAttemptID producerExecutionId,
-			IntermediateResultPartitionID partitionId,
-			IntermediateResultPartitionManager partitionManager,
+			SingleInputGate gate,
+			int channelIndex,
+			ResultPartitionID partitionId,
+			ResultPartitionManager partitionManager,
 			TaskEventDispatcher taskEventDispatcher) {
 
-		super(gate, channelIndex, producerExecutionId, partitionId);
+		super(gate, channelIndex, partitionId);
 
 		this.partitionManager = checkNotNull(partitionManager);
 		this.taskEventDispatcher = checkNotNull(taskEventDispatcher);
@@ -70,20 +71,23 @@ public class LocalInputChannel extends InputChannel implements NotificationListe
 	// ------------------------------------------------------------------------
 
 	@Override
-	public void requestIntermediateResultPartition(int queueIndex) throws IOException {
+	public void requestSubpartition(int subpartitionIndex) throws IOException, InterruptedException {
 		if (queueIterator == null) {
-			LOG.debug("Requesting LOCAL queue {} from partition {} produced by {}.", queueIndex, partitionId,
-						producerExecutionId);
+			LOG.debug("Requesting LOCAL queue {} of partition {}.", subpartitionIndex, partitionId);
 
-			queueIterator = partitionManager.getIntermediateResultPartitionIterator(
-					producerExecutionId, partitionId, queueIndex, Optional.of(inputGate.getBufferProvider()));
+			queueIterator = partitionManager
+					.getSubpartition(partitionId, subpartitionIndex, Optional.of(inputGate.getBufferProvider()));
+
+			if (queueIterator == null) {
+				throw new IOException("Error requesting sub partition.");
+			}
 
 			getNextLookAhead();
 		}
 	}
 
 	@Override
-	public Buffer getNextBuffer() throws IOException {
+	public Buffer getNextBuffer() throws IOException, InterruptedException {
 		checkState(queueIterator != null, "Queried for a buffer before requesting a queue.");
 
 		// After subscribe notification
@@ -94,6 +98,13 @@ public class LocalInputChannel extends InputChannel implements NotificationListe
 		Buffer next = lookAhead;
 		lookAhead = null;
 
+		if (!next.isBuffer() && EventSerializer
+				.fromBuffer(next, getClass().getClassLoader())
+				.getClass() == EndOfPartitionEvent.class) {
+
+				return next;
+		}
+
 		getNextLookAhead();
 
 		return next;
@@ -107,7 +118,7 @@ public class LocalInputChannel extends InputChannel implements NotificationListe
 	public void sendTaskEvent(TaskEvent event) throws IOException {
 		checkState(queueIterator != null, "Tried to send task event to producer before requesting a queue.");
 
-		if (!taskEventDispatcher.publish(producerExecutionId, partitionId, event)) {
+		if (!taskEventDispatcher.publish(partitionId, event)) {
 			throw new IOException("Error while publishing event " + event + " to producer. The producer could not be found.");
 		}
 	}
@@ -121,6 +132,13 @@ public class LocalInputChannel extends InputChannel implements NotificationListe
 		return isReleased;
 	}
 
+	@Override
+	public void notifySubpartitionConsumed() throws IOException {
+		if (queueIterator != null) {
+			queueIterator.notifySubpartitionConsumed();
+		}
+	}
+
 	/**
 	 * Releases the look ahead {@link Buffer} instance and discards the queue
 	 * iterator.
@@ -134,7 +152,7 @@ public class LocalInputChannel extends InputChannel implements NotificationListe
 			}
 
 			if (queueIterator != null) {
-				queueIterator.discard();
+				queueIterator.releaseAllResources();
 				queueIterator = null;
 			}
 
@@ -144,7 +162,7 @@ public class LocalInputChannel extends InputChannel implements NotificationListe
 
 	@Override
 	public String toString() {
-		return "LOCAL " + super.toString();
+		return "LocalInputChannel [" + partitionId + "]";
 	}
 
 	// ------------------------------------------------------------------------
@@ -153,12 +171,21 @@ public class LocalInputChannel extends InputChannel implements NotificationListe
 
 	@Override
 	public void onNotification() {
-		notifyAvailableBuffer();
+		if (isReleased) {
+			return;
+		}
+
+		try {
+			getNextLookAhead();
+		}
+		catch (Exception e) {
+			throw new RuntimeException(e);
+		}
 	}
 
 	// ------------------------------------------------------------------------
 
-	private void getNextLookAhead() throws IOException {
+	private void getNextLookAhead() throws IOException, InterruptedException {
 		while (true) {
 			lookAhead = queueIterator.getNextBuffer();
 
@@ -167,7 +194,7 @@ public class LocalInputChannel extends InputChannel implements NotificationListe
 				break;
 			}
 
-			if (queueIterator.subscribe(this) || queueIterator.isConsumed()) {
+			if (queueIterator.registerListener(this) || queueIterator.isReleased()) {
 				return;
 			}
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
index d50ddc2..2ca2ff7 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
@@ -19,13 +19,12 @@
 package org.apache.flink.runtime.io.network.partition.consumer;
 
 import org.apache.flink.runtime.event.task.TaskEvent;
-import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.io.network.ConnectionID;
 import org.apache.flink.runtime.io.network.ConnectionManager;
-import org.apache.flink.runtime.io.network.RemoteAddress;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
 import org.apache.flink.runtime.io.network.buffer.BufferProvider;
 import org.apache.flink.runtime.io.network.netty.PartitionRequestClient;
-import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -47,7 +46,7 @@ public class RemoteInputChannel extends InputChannel {
 
 	private final InputChannelID id;
 
-	private final RemoteAddress producerAddress;
+	private final ConnectionID producerAddress;
 
 	private final Queue<Buffer> receivedBuffers = new ArrayDeque<Buffer>();
 
@@ -64,12 +63,11 @@ public class RemoteInputChannel extends InputChannel {
 	public RemoteInputChannel(
 			SingleInputGate gate,
 			int channelIndex,
-			ExecutionAttemptID producerExecutionId,
-			IntermediateResultPartitionID partitionId,
-			RemoteAddress producerAddress,
+			ResultPartitionID partitionId,
+			ConnectionID producerAddress,
 			ConnectionManager connectionManager) {
 
-		super(gate, channelIndex, producerExecutionId, partitionId);
+		super(gate, channelIndex, partitionId);
 
 		/**
 		 * This ID is used by the {@link PartitionRequestClient} to distinguish
@@ -85,13 +83,13 @@ public class RemoteInputChannel extends InputChannel {
 	// ------------------------------------------------------------------------
 
 	@Override
-	public void requestIntermediateResultPartition(int queueIndex) throws IOException, InterruptedException {
+	public void requestSubpartition(int subpartitionIndex) throws IOException, InterruptedException {
 		if (partitionRequestClient == null) {
-			LOG.debug("Requesting REMOTE queue {} from partition {} produced by {}.", queueIndex, partitionId, producerExecutionId);
+			LOG.debug("Requesting REMOTE queue {} from of partition {}.", subpartitionIndex, partitionId);
 
 			partitionRequestClient = connectionManager.createPartitionRequestClient(producerAddress);
 
-			partitionRequestClient.requestIntermediateResultPartition(producerExecutionId, partitionId, queueIndex, this);
+			partitionRequestClient.requestIntermediateResultPartition(partitionId, subpartitionIndex, this);
 		}
 	}
 
@@ -125,7 +123,7 @@ public class RemoteInputChannel extends InputChannel {
 
 		checkIoError();
 
-		partitionRequestClient.sendTaskEvent(producerExecutionId, partitionId, event, this);
+		partitionRequestClient.sendTaskEvent(partitionId, event, this);
 	}
 
 	// ------------------------------------------------------------------------
@@ -137,6 +135,11 @@ public class RemoteInputChannel extends InputChannel {
 		return isReleased.get();
 	}
 
+	@Override
+	public void notifySubpartitionConsumed() {
+		// Nothing to do
+	}
+
 	/**
 	 * Releases all received buffers and closes the partition request client.
 	 */
@@ -160,7 +163,7 @@ public class RemoteInputChannel extends InputChannel {
 
 	@Override
 	public String toString() {
-		return "REMOTE " + id + " " + producerAddress + " " + super.toString();
+		return "RemoteInputChannel [" + partitionId + " at " + producerAddress + "]";
 	}
 
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
index d981451..5b97d26 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
@@ -19,18 +19,19 @@
 package org.apache.flink.runtime.io.network.partition.consumer;
 
 import com.google.common.collect.Maps;
-import org.apache.flink.runtime.deployment.PartitionConsumerDeploymentDescriptor;
-import org.apache.flink.runtime.deployment.PartitionInfo;
+import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor;
+import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor;
+import org.apache.flink.runtime.deployment.ResultPartitionLocation;
 import org.apache.flink.runtime.event.task.AbstractEvent;
 import org.apache.flink.runtime.event.task.TaskEvent;
-import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.io.network.NetworkEnvironment;
-import org.apache.flink.runtime.io.network.RemoteAddress;
 import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
 import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
 import org.apache.flink.runtime.io.network.buffer.BufferPool;
 import org.apache.flink.runtime.io.network.buffer.BufferProvider;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
 import org.apache.flink.runtime.util.event.EventListener;
@@ -39,11 +40,13 @@ import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
@@ -51,20 +54,23 @@ import static com.google.common.base.Preconditions.checkState;
 
 /**
  * An input gate consumes one or more partitions of a single produced intermediate result.
- * <p>
- * Each intermediate result is partitioned over its producing parallel subtasks; each of these
+ *
+ * <p> Each intermediate result is partitioned over its producing parallel subtasks; each of these
  * partitions is furthermore partitioned into one or more subpartitions.
- * <p>
- * As an example, consider a map-reduce program, where the map operator produces data and the reduce
- * operator consumes the produced data.
+ *
+ * <p> As an example, consider a map-reduce program, where the map operator produces data and the
+ * reduce operator consumes the produced data.
+ *
  * <pre>
  * +-----+              +---------------------+              +--------+
  * | Map | = produce => | Intermediate Result | <= consume = | Reduce |
  * +-----+              +---------------------+              +--------+
  * </pre>
- * When deploying such a program in parallel, the intermediate result will be partitioned over its
+ *
+ * <p> When deploying such a program in parallel, the intermediate result will be partitioned over its
  * producing parallel subtasks; each of these partitions is furthermore partitioned into one or more
  * subpartitions.
+ *
  * <pre>
  *                            Intermediate result
  *               +-----------------------------------------+
@@ -81,10 +87,10 @@ import static com.google.common.base.Preconditions.checkState;
  *               |                      +----------------+ |              +-----------------------+
  *               +-----------------------------------------+
  * </pre>
- * In the above example, two map subtasks produce the intermediate result in parallel, resulting
- * in two partitions (Partition 1 and 2). Each of these partitions is subpartitioned into two
+ *
+ * <p> In the above example, two map subtasks produce the intermediate result in parallel, resulting
+ * in two partitions (Partition 1 and 2). Each of these partitions is further partitioned into two
  * subpartitions -- one for each parallel reduce subtask.
- * <p>
  */
 public class SingleInputGate implements InputGate {
 
@@ -102,7 +108,7 @@ public class SingleInputGate implements InputGate {
 
 	/**
 	 * The index of the consumed subpartition of each consumed partition. This index depends on the
-	 * distribution pattern and both subtask indices of the producing and consuming task.
+	 * {@link DistributionPattern} and the subtask indices of the producing and consuming task.
 	 */
 	private final int consumedSubpartitionIndex;
 
@@ -128,7 +134,7 @@ public class SingleInputGate implements InputGate {
 	private boolean requestedPartitionsFlag;
 
 	/** Flag indicating whether all resources have been released. */
-	private boolean releasedResourcesFlag;
+	private volatile boolean isReleased;
 
 	/** Registered listener to forward buffer notifications to. */
 	private final List<EventListener<InputGate>> registeredListeners = new CopyOnWriteArrayList<EventListener<InputGate>>();
@@ -185,42 +191,48 @@ public class SingleInputGate implements InputGate {
 
 	public void setInputChannel(IntermediateResultPartitionID partitionId, InputChannel inputChannel) {
 		synchronized (requestLock) {
-			if (inputChannels.put(checkNotNull(partitionId), checkNotNull(inputChannel)) == null &&
-					inputChannel.getClass() == UnknownInputChannel.class) {
+			if (inputChannels.put(checkNotNull(partitionId), checkNotNull(inputChannel)) == null
+					&& inputChannel.getClass() == UnknownInputChannel.class) {
+
 				numberOfUninitializedChannels++;
 			}
 		}
 	}
 
-	public void updateInputChannel(PartitionInfo partitionInfo) throws IOException, InterruptedException {
+	public void updateInputChannel(InputChannelDeploymentDescriptor icdd) throws IOException, InterruptedException {
 		synchronized (requestLock) {
-			if (releasedResourcesFlag) {
+			if (isReleased) {
 				// There was a race with a task failure/cancel
 				return;
 			}
 
-			final IntermediateResultPartitionID partitionId = partitionInfo.getPartitionId();
+			final IntermediateResultPartitionID partitionId = icdd.getConsumedPartitionId().getPartitionId();
 
 			InputChannel current = inputChannels.get(partitionId);
 
 			if (current.getClass() == UnknownInputChannel.class) {
+
 				UnknownInputChannel unknownChannel = (UnknownInputChannel) current;
 
 				InputChannel newChannel;
 
-				if (partitionInfo.getProducerLocation() == PartitionInfo.PartitionLocation.REMOTE) {
-					newChannel = unknownChannel.toRemoteInputChannel(partitionInfo.getProducerAddress());
-				}
-				else if (partitionInfo.getProducerLocation() == PartitionInfo.PartitionLocation.LOCAL) {
+				ResultPartitionLocation partitionLocation = icdd.getConsumedPartitionLocation();
+
+				if (partitionLocation.isLocal()) {
 					newChannel = unknownChannel.toLocalInputChannel();
 				}
+				else if (partitionLocation.isRemote()) {
+					newChannel = unknownChannel.toRemoteInputChannel(partitionLocation.getConnectionId());
+				}
 				else {
 					throw new IllegalStateException("Tried to update unknown channel with unknown channel.");
 				}
 
+				LOG.debug("Updated unknown input channel to {}.", newChannel);
+
 				inputChannels.put(partitionId, newChannel);
 
-				newChannel.requestIntermediateResultPartition(consumedSubpartitionIndex);
+				newChannel.requestSubpartition(consumedSubpartitionIndex);
 
 				for (TaskEvent event : pendingEvents) {
 					newChannel.sendTaskEvent(event);
@@ -235,7 +247,7 @@ public class SingleInputGate implements InputGate {
 
 	public void releaseAllResources() throws IOException {
 		synchronized (requestLock) {
-			if (!releasedResourcesFlag) {
+			if (!isReleased) {
 				try {
 					for (InputChannel inputChannel : inputChannels.values()) {
 						try {
@@ -249,11 +261,11 @@ public class SingleInputGate implements InputGate {
 					// The buffer pool can actually be destroyed immediately after the
 					// reader received all of the data from the input channels.
 					if (bufferPool != null) {
-						bufferPool.destroy();
+						bufferPool.lazyDestroy();
 					}
 				}
 				finally {
-					releasedResourcesFlag = true;
+					isReleased = true;
 				}
 			}
 		}
@@ -284,7 +296,7 @@ public class SingleInputGate implements InputGate {
 
 			synchronized (requestLock) {
 				for (InputChannel inputChannel : inputChannels.values()) {
-					inputChannel.requestIntermediateResultPartition(consumedSubpartitionIndex);
+					inputChannel.requestSubpartition(consumedSubpartitionIndex);
 				}
 			}
 
@@ -299,13 +311,16 @@ public class SingleInputGate implements InputGate {
 	@Override
 	public BufferOrEvent getNextBufferOrEvent() throws IOException, InterruptedException {
 
-		if (releasedResourcesFlag) {
+		if (isReleased) {
 			throw new IllegalStateException("The input has already been consumed. This indicates misuse of the input gate.");
 		}
 
 		requestPartitions();
 
-		final InputChannel currentChannel = inputChannelsWithData.take();
+		InputChannel currentChannel = null;
+		while (currentChannel == null) {
+			currentChannel = inputChannelsWithData.poll(2, TimeUnit.SECONDS);
+		}
 
 		final Buffer buffer = currentChannel.getNextBuffer();
 
@@ -322,6 +337,8 @@ public class SingleInputGate implements InputGate {
 			final AbstractEvent event = EventSerializer.fromBuffer(buffer, getClass().getClassLoader());
 
 			if (event.getClass() == EndOfPartitionEvent.class) {
+				currentChannel.notifySubpartitionConsumed();
+
 				currentChannel.releaseAllResources();
 			}
 
@@ -354,58 +371,63 @@ public class SingleInputGate implements InputGate {
 	public void onAvailableBuffer(InputChannel channel) {
 		inputChannelsWithData.add(channel);
 
-		for (int i = 0; i < registeredListeners.size(); i++) {
-			registeredListeners.get(i).onEvent(this);
+		for (EventListener<InputGate> registeredListener : registeredListeners) {
+			registeredListener.onEvent(this);
 		}
 	}
 
 	// ------------------------------------------------------------------------
 
-	public static SingleInputGate create(NetworkEnvironment networkEnvironment, PartitionConsumerDeploymentDescriptor desc) {
-		// The consumed intermediate data set (all partitions are part of this data set)
-		final IntermediateDataSetID resultId = desc.getResultId();
-		// The queue to request from each consumed partition
-		final int queueIndex = desc.getQueueIndex();
-		// There is one input channel for each consumed partition
-		final PartitionInfo[] partitions = desc.getPartitions();
-		final int numberOfInputChannels = partitions.length;
-		final SingleInputGate reader = new SingleInputGate(resultId, queueIndex, numberOfInputChannels);
-		// Create input channels
-		final InputChannel[] inputChannels = new InputChannel[numberOfInputChannels];
-		int channelIndex = 0;
-		for (PartitionInfo partition : partitions) {
-			final ExecutionAttemptID producerExecutionId = partition.getProducerExecutionId();
-			final IntermediateResultPartitionID partitionId = partition.getPartitionId();
-			final PartitionInfo.PartitionLocation producerLocation = partition.getProducerLocation();
-			switch (producerLocation) {
-				case LOCAL:
-					LOG.debug("Create LocalInputChannel for {}.", partition);
-
-					inputChannels[channelIndex] = new LocalInputChannel(reader, channelIndex, producerExecutionId, partitionId,
-							networkEnvironment.getPartitionManager(), networkEnvironment.getTaskEventDispatcher());
-
-					break;
-				case REMOTE:
-					LOG.debug("Create RemoteInputChannel for {}.", partition);
-
-					final RemoteAddress producerAddress = checkNotNull(partition.getProducerAddress(),
-							"Missing producer address for remote intermediate result partition.");
-
-					inputChannels[channelIndex] = new RemoteInputChannel(reader, channelIndex, producerExecutionId, partitionId,
-							producerAddress, networkEnvironment.getConnectionManager());
-
-					break;
-				case UNKNOWN:
-					LOG.debug("Create UnknownInputChannel for {}.", partition);
-
-					inputChannels[channelIndex] = new UnknownInputChannel(reader, channelIndex, producerExecutionId, partitionId,
-							networkEnvironment.getPartitionManager(), networkEnvironment.getTaskEventDispatcher(), networkEnvironment.getConnectionManager());
-
-					break;
+	/**
+	 * Creates an input gate and all of its input channels.
+	 */
+	public static SingleInputGate create(
+			InputGateDeploymentDescriptor igdd,
+			NetworkEnvironment networkEnvironment) {
+
+		final IntermediateDataSetID consumedResultId = checkNotNull(igdd.getConsumedResultId());
+
+		final int consumedSubpartitionIndex = igdd.getConsumedSubpartitionIndex();
+		checkArgument(consumedSubpartitionIndex >= 0);
+
+		final InputChannelDeploymentDescriptor[] icdd = checkNotNull(igdd.getInputChannelDeploymentDescriptors());
+
+		final SingleInputGate inputGate = new SingleInputGate(
+				consumedResultId, consumedSubpartitionIndex, icdd.length);
+
+		// Create the input channels. There is one input channel for each consumed partition.
+		final InputChannel[] inputChannels = new InputChannel[icdd.length];
+
+		for (int i = 0; i < inputChannels.length; i++) {
+
+			final ResultPartitionID partitionId = icdd[i].getConsumedPartitionId();
+			final ResultPartitionLocation partitionLocation = icdd[i].getConsumedPartitionLocation();
+
+			if (partitionLocation.isLocal()) {
+				inputChannels[i] = new LocalInputChannel(inputGate, i, partitionId,
+						networkEnvironment.getPartitionManager(),
+						networkEnvironment.getTaskEventDispatcher());
+			}
+			else if (partitionLocation.isRemote()) {
+				inputChannels[i] = new RemoteInputChannel(inputGate, i, partitionId,
+						partitionLocation.getConnectionId(),
+						networkEnvironment.getConnectionManager());
+			}
+			else if (partitionLocation.isUnknown()) {
+				inputChannels[i] = new UnknownInputChannel(inputGate, i, partitionId,
+						networkEnvironment.getPartitionManager(),
+						networkEnvironment.getTaskEventDispatcher(),
+						networkEnvironment.getConnectionManager());
+			}
+			else {
+				throw new IllegalStateException("Unexpected partition location.");
 			}
-			reader.setInputChannel(partitionId, inputChannels[channelIndex]);
-			channelIndex++;
+
+			inputGate.setInputChannel(partitionId.getPartitionId(), inputChannels[i]);
 		}
-		return reader;
+
+		LOG.debug("Created input channels {} from {}.", Arrays.toString(inputChannels), igdd);
+
+		return inputGate;
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java
index f8e42ba..4bde292 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java
@@ -19,14 +19,13 @@
 package org.apache.flink.runtime.io.network.partition.consumer;
 
 import org.apache.flink.runtime.event.task.TaskEvent;
-import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.io.network.ConnectionID;
 import org.apache.flink.runtime.io.network.ConnectionManager;
-import org.apache.flink.runtime.io.network.RemoteAddress;
 import org.apache.flink.runtime.io.network.TaskEventDispatcher;
 import org.apache.flink.runtime.io.network.api.reader.BufferReader;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.partition.IntermediateResultPartitionManager;
-import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionManager;
 
 import java.io.IOException;
 
@@ -38,21 +37,21 @@ import static com.google.common.base.Preconditions.checkNotNull;
  */
 public class UnknownInputChannel extends InputChannel {
 
-	private final IntermediateResultPartitionManager partitionManager;
+	private final ResultPartitionManager partitionManager;
 
 	private final TaskEventDispatcher taskEventDispatcher;
 
 	private final ConnectionManager connectionManager;
 
 	public UnknownInputChannel(
-			SingleInputGate gate, int channelIndex,
-			ExecutionAttemptID producerExecutionId,
-			IntermediateResultPartitionID partitionId,
-			IntermediateResultPartitionManager partitionManager,
+			SingleInputGate gate,
+			int channelIndex,
+			ResultPartitionID partitionId,
+			ResultPartitionManager partitionManager,
 			TaskEventDispatcher taskEventDispatcher,
 			ConnectionManager connectionManager) {
 
-		super(gate, channelIndex, producerExecutionId, partitionId);
+		super(gate, channelIndex, partitionId);
 
 		this.partitionManager = partitionManager;
 		this.taskEventDispatcher = taskEventDispatcher;
@@ -60,7 +59,7 @@ public class UnknownInputChannel extends InputChannel {
 	}
 
 	@Override
-	public void requestIntermediateResultPartition(int queueIndex) throws IOException {
+	public void requestSubpartition(int subpartitionIndex) throws IOException {
 		// Nothing to do here
 	}
 
@@ -89,24 +88,28 @@ public class UnknownInputChannel extends InputChannel {
 	}
 
 	@Override
+	public void notifySubpartitionConsumed() {
+	}
+
+	@Override
 	public void releaseAllResources() throws IOException {
 		// Nothing to do here
 	}
 
 	@Override
 	public String toString() {
-		return "UNKNOWN " + super.toString();
+		return "UnknownInputChannel [" + partitionId + "]";
 	}
 
 	// ------------------------------------------------------------------------
 	// Graduation to a local or remote input channel at runtime
 	// ------------------------------------------------------------------------
 
-	public RemoteInputChannel toRemoteInputChannel(RemoteAddress producerAddress) {
-		return new RemoteInputChannel(inputGate, channelIndex, producerExecutionId, partitionId, checkNotNull(producerAddress), connectionManager);
+	public RemoteInputChannel toRemoteInputChannel(ConnectionID producerAddress) {
+		return new RemoteInputChannel(inputGate, channelIndex, partitionId, checkNotNull(producerAddress), connectionManager);
 	}
 
 	public LocalInputChannel toLocalInputChannel() {
-		return new LocalInputChannel(inputGate, channelIndex, producerExecutionId, partitionId, partitionManager, taskEventDispatcher);
+		return new LocalInputChannel(inputGate, channelIndex, partitionId, partitionManager, taskEventDispatcher);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/queue/IllegalQueueIteratorRequestException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/queue/IllegalQueueIteratorRequestException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/queue/IllegalQueueIteratorRequestException.java
deleted file mode 100644
index 80284c4..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/queue/IllegalQueueIteratorRequestException.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.io.network.partition.queue;
-
-import java.io.IOException;
-
-public class IllegalQueueIteratorRequestException extends IOException {
-
-	private static final long serialVersionUID = 8381253563445306324L;
-
-	public IllegalQueueIteratorRequestException() {
-	}
-
-	public IllegalQueueIteratorRequestException(String message) {
-		super(message);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/queue/IntermediateResultPartitionQueue.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/queue/IntermediateResultPartitionQueue.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/queue/IntermediateResultPartitionQueue.java
deleted file mode 100644
index 278e2a1..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/queue/IntermediateResultPartitionQueue.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.io.network.partition.queue;
-
-import com.google.common.base.Optional;
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.buffer.BufferProvider;
-
-import java.io.IOException;
-
-public interface IntermediateResultPartitionQueue {
-
-	// ------------------------------------------------------------------------
-	// Produce
-	// ------------------------------------------------------------------------
-
-	void add(Buffer buffer) throws IOException;
-
-	void finish() throws IOException;
-
-	void discard() throws IOException;
-
-	// ------------------------------------------------------------------------
-	// Consume
-	// ------------------------------------------------------------------------
-
-	IntermediateResultPartitionQueueIterator getQueueIterator(Optional<BufferProvider> bufferProvider) throws IllegalQueueIteratorRequestException, IOException;
-
-	// ------------------------------------------------------------------------
-	// Properties
-	// ------------------------------------------------------------------------
-
-	boolean isFinished();
-
-	int recycleBuffers() throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/queue/IntermediateResultPartitionQueueIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/queue/IntermediateResultPartitionQueueIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/queue/IntermediateResultPartitionQueueIterator.java
deleted file mode 100644
index 9fbb331..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/queue/IntermediateResultPartitionQueueIterator.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.io.network.partition.queue;
-
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.util.event.NotificationListener;
-
-import java.io.IOException;
-
-public interface IntermediateResultPartitionQueueIterator {
-
-	// ------------------------------------------------------------------------
-	// Properties
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Returns whether this iterator has been fully consumed, e.g. no more data
-	 * or queue has been discarded.
-	 */
-	boolean isConsumed();
-
-	// ------------------------------------------------------------------------
-	// Consume
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Returns the next {@link Buffer} instance of this queue iterator.
-	 * <p>
-	 * If there is currently no instance available, it will return <code>null</code>.
-	 * This might happen for example when a pipelined queue producer is slower
-	 * than the consumer or a spilled queue needs to read in more data.
-	 * <p>
-	 * <strong>Important</strong>: The consumer has to make sure that each
-	 * buffer instance will eventually be recycled with {@link Buffer#recycle()}
-	 * after it has been consumed.
-	 */
-	Buffer getNextBuffer() throws IOException;
-
-	/**
-	 * Discards the consumption of this queue iterator.
-	 */
-	void discard() throws IOException;
-
-	/**
-	 * Subscribes to data availability notifications.
-	 * <p>
-	 * Returns whether the subscription was successful. A subscription fails,
-	 * if there is data available.
-	 */
-	boolean subscribe(NotificationListener listener) throws AlreadySubscribedException;
-
-	// ------------------------------------------------------------------------
-
-	public class AlreadySubscribedException extends IOException {
-
-		private static final long serialVersionUID = -5583394817361970668L;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/queue/PipelinedPartitionQueue.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/queue/PipelinedPartitionQueue.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/queue/PipelinedPartitionQueue.java
deleted file mode 100644
index 5d562e4..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/queue/PipelinedPartitionQueue.java
+++ /dev/null
@@ -1,185 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.io.network.partition.queue;
-
-import com.google.common.base.Optional;
-import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
-import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.buffer.BufferProvider;
-import org.apache.flink.runtime.util.event.NotificationListener;
-
-import java.util.ArrayDeque;
-import java.util.Queue;
-
-/**
- * An in-memory only queue, which can be consumed once by a single consumer.
- */
-public class PipelinedPartitionQueue implements IntermediateResultPartitionQueue {
-
-	final Queue<Buffer> queue = new ArrayDeque<Buffer>();
-
-	boolean hasFinishedProduce;
-
-	private NotificationListener listener;
-
-	private boolean hasBeenDiscarded;
-
-	private boolean hasConsumer;
-
-	@Override
-	public void add(Buffer buffer) {
-		synchronized (queue) {
-			if (!hasFinishedProduce) {
-				queue.add(buffer);
-
-				maybeNotifyListener();
-			}
-			else if (!buffer.isRecycled()) {
-					buffer.recycle();
-			}
-		}
-	}
-
-	@Override
-	public boolean isFinished() {
-		synchronized (queue) {
-			return hasFinishedProduce;
-		}
-	}
-
-	@Override
-	public void finish() {
-		synchronized (queue) {
-			if (hasFinishedProduce) {
-				return;
-			}
-
-			queue.add(EventSerializer.toBuffer(EndOfPartitionEvent.INSTANCE));
-
-			hasFinishedProduce = true;
-
-			maybeNotifyListener();
-		}
-	}
-
-	@Override
-	public int recycleBuffers() {
-		// Nothing to do here. Buffers are recycled when they are consumed.
-		return 0;
-	}
-
-	@Override
-	public void discard() {
-		synchronized (queue) {
-			Buffer buffer;
-			while ((buffer = queue.poll()) != null) {
-				if (!buffer.isRecycled()) {
-					buffer.recycle();
-				}
-			}
-
-			hasFinishedProduce = true;
-			hasBeenDiscarded = true;
-
-			maybeNotifyListener();
-		}
-	}
-
-	@Override
-	public IntermediateResultPartitionQueueIterator getQueueIterator(Optional<BufferProvider> bufferProvider)
-			throws IllegalQueueIteratorRequestException {
-		synchronized (queue) {
-			if (hasBeenDiscarded) {
-				throw new IllegalQueueIteratorRequestException("Queue has been discarded during produce phase.");
-			}
-
-			if (hasConsumer) {
-				throw new IllegalQueueIteratorRequestException("Consumable once queue has been consumed/is being consumed.");
-			}
-
-			hasConsumer = true;
-
-			return new PipelinedPartitionQueueIterator(this);
-		}
-	}
-
-	// Call in synchronized scope
-	private void maybeNotifyListener() {
-		NotificationListener consumer = listener;
-		if (consumer != null) {
-			listener = null;
-
-			// TODO This is dangerous with the locks. Every listener needs to make sure not to query the queue again :S
-			consumer.onNotification();
-		}
-	}
-
-	private static class PipelinedPartitionQueueIterator implements IntermediateResultPartitionQueueIterator {
-
-		private final PipelinedPartitionQueue partitionQueue;
-
-		private boolean isDiscarded;
-
-		private PipelinedPartitionQueueIterator(PipelinedPartitionQueue partitionQueue) {
-			this.partitionQueue = partitionQueue;
-		}
-
-		@Override
-		public boolean isConsumed() {
-			synchronized (partitionQueue.queue) {
-				return (partitionQueue.isFinished() && partitionQueue.queue.isEmpty()) || partitionQueue.hasBeenDiscarded;
-			}
-		}
-
-		@Override
-		public Buffer getNextBuffer() {
-			synchronized (partitionQueue.queue) {
-				return partitionQueue.queue.poll();
-			}
-		}
-
-		@Override
-		public void discard() {
-			synchronized (partitionQueue.queue) {
-				if (!isDiscarded) {
-					partitionQueue.discard();
-
-					isDiscarded = true;
-				}
-			}
-		}
-
-		@Override
-		public boolean subscribe(NotificationListener listener) throws AlreadySubscribedException {
-			synchronized (partitionQueue.queue) {
-				if (isConsumed() || !partitionQueue.queue.isEmpty()) {
-					return false;
-				}
-
-				if (partitionQueue.listener == null) {
-					partitionQueue.listener = listener;
-					return true;
-				}
-
-				throw new AlreadySubscribedException();
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/serialization/SpillingAdaptiveSpanningRecordDeserializer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/serialization/SpillingAdaptiveSpanningRecordDeserializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/serialization/SpillingAdaptiveSpanningRecordDeserializer.java
deleted file mode 100644
index 5de42b1..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/serialization/SpillingAdaptiveSpanningRecordDeserializer.java
+++ /dev/null
@@ -1,637 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.io.network.serialization;
-
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.GlobalConfiguration;
-import org.apache.flink.core.io.IOReadableWritable;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.InputViewDataInputStreamWrapper;
-import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer;
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.util.DataInputDeserializer;
-import org.apache.flink.util.StringUtils;
-
-import java.io.BufferedInputStream;
-import java.io.DataInputStream;
-import java.io.EOFException;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.io.UTFDataFormatException;
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-import java.nio.channels.FileChannel;
-import java.util.Random;
-
-/**
- * @param <T> The type of the record to be deserialized.
- */
-public class SpillingAdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> implements RecordDeserializer<T> {
-	
-	private static final int THRESHOLD_FOR_SPILLING = 5 * 1024 * 1024; // 5 MiBytes
-	
-	private final NonSpanningWrapper nonSpanningWrapper;
-	
-	private final SpanningWrapper spanningWrapper;
-
-	private Buffer currentBuffer;
-
-	public SpillingAdaptiveSpanningRecordDeserializer() {
-		
-		String tempDirString = GlobalConfiguration.getString(
-				ConfigConstants.TASK_MANAGER_TMP_DIR_KEY,
-				ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH);
-		String[] directories = tempDirString.split(",|" + File.pathSeparator);
-		
-		this.nonSpanningWrapper = new NonSpanningWrapper();
-		this.spanningWrapper = new SpanningWrapper(directories);
-	}
-
-	@Override
-	public void setNextBuffer(Buffer buffer) throws IOException {
-		currentBuffer = buffer;
-
-		MemorySegment segment = buffer.getMemorySegment();
-		int numBytes = buffer.getSize();
-
-		setNextMemorySegment(segment, numBytes);
-	}
-
-	@Override
-	public Buffer getCurrentBuffer () {
-		Buffer tmp = currentBuffer;
-		currentBuffer = null;
-		return tmp;
-	}
-	
-	@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.getInputView());
-			
-			// 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 {
-			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 DataInputView {
-		
-		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;
-		}
-
-		@Override
-		public void skipBytesToRead(int numBytes) throws IOException {
-			int skippedBytes = skipBytes(numBytes);
-
-			if(skippedBytes < numBytes){
-				throw new EOFException("Could not skip " + numBytes + " bytes.");
-			}
-		}
-
-		@Override
-		public int read(byte[] b, int off, int len) throws IOException {
-			if(b == null){
-				throw new NullPointerException("Byte array b cannot be null.");
-			}
-
-			if(off < 0){
-				throw new IllegalArgumentException("The offset off cannot be negative.");
-			}
-
-			if(len < 0){
-				throw new IllegalArgumentException("The length len cannot be negative.");
-			}
-
-			int toRead = Math.min(len, remaining());
-			this.segment.get(this.position,b,off, toRead);
-			this.position += toRead;
-
-			return toRead;
-		}
-
-		@Override
-		public int read(byte[] b) throws IOException {
-			return read(b, 0, b.length);
-		}
-	}
-
-	// -----------------------------------------------------------------------------------------------------------------
-	
-	private static final class SpanningWrapper {
-		
-		private final byte[] initialBuffer = new byte[1024];
-		
-		private final String[] tempDirs;
-		
-		private final Random rnd = new Random();
-
-		private final DataInputDeserializer serializationReadBuffer;
-
-		private final ByteBuffer lengthBuffer;
-		
-		private FileChannel spillingChannel;
-		
-		private byte[] buffer;
-
-		private int recordLength;
-		
-		private int accumulatedRecordBytes;
-
-		private MemorySegment leftOverData;
-
-		private int leftOverStart;
-
-		private int leftOverLimit;
-		
-		private File spillFile;
-		
-		private InputViewDataInputStreamWrapper spillFileReader;
-		
-		public SpanningWrapper(String[] tempDirs) {
-			this.tempDirs = tempDirs;
-			
-			this.lengthBuffer = ByteBuffer.allocate(4);
-			this.lengthBuffer.order(ByteOrder.BIG_ENDIAN);
-
-			this.recordLength = -1;
-
-			this.serializationReadBuffer = new DataInputDeserializer();
-			this.buffer = initialBuffer;
-		}
-		
-		private void initializeWithPartialRecord(NonSpanningWrapper partial, int nextRecordLength) throws IOException {
-			// set the length and copy what is available to the buffer
-			this.recordLength = nextRecordLength;
-			
-			final int numBytesChunk = partial.remaining();
-			
-			if (nextRecordLength > THRESHOLD_FOR_SPILLING) {
-				// create a spilling channel and put the data there
-				this.spillingChannel = createSpillingChannel();
-				
-				ByteBuffer toWrite = partial.segment.wrap(partial.position, numBytesChunk);
-				this.spillingChannel.write(toWrite);
-			}
-			else {
-				// collect in memory
-				ensureBufferCapacity(numBytesChunk);
-				partial.segment.get(partial.position, buffer, 0, numBytesChunk);
-			}
-			
-			this.accumulatedRecordBytes = numBytesChunk;
-		}
-		
-		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;
-					
-					if (this.recordLength > THRESHOLD_FOR_SPILLING) {
-						this.spillingChannel = createSpillingChannel();
-					}
-				}
-			}
-
-			// copy as much as we need or can for this next spanning record
-			int needed = this.recordLength - this.accumulatedRecordBytes;
-			int available = numBytesInSegment - segmentPosition;
-			int toCopy = Math.min(needed, available);
-
-			if (spillingChannel != null) {
-				// spill to file
-				ByteBuffer toWrite = segment.wrap(segmentPosition, toCopy);
-				this.spillingChannel.write(toWrite);
-			}
-			else {
-				ensureBufferCapacity(accumulatedRecordBytes + toCopy);
-				segment.get(segmentPosition, buffer, this.accumulatedRecordBytes, toCopy);
-			}
-			
-			this.accumulatedRecordBytes += toCopy;
-			
-			if (toCopy < available) {
-				// there is more data in the segment
-				this.leftOverData = segment;
-				this.leftOverStart = segmentPosition + toCopy;
-				this.leftOverLimit = numBytesInSegment;
-			}
-			
-			if (accumulatedRecordBytes == recordLength) {
-				// we have the full record
-				if (spillingChannel == null) {
-					this.serializationReadBuffer.setBuffer(buffer, 0, recordLength);
-				}
-				else {
-					spillingChannel.close();
-					
-					DataInputStream inStream = new DataInputStream(new BufferedInputStream(new FileInputStream(spillFile), 2 * 1024 * 1024));
-					this.spillFileReader = new InputViewDataInputStreamWrapper(inStream);
-				}
-			}
-		}
-		
-		private void moveRemainderToNonSpanningDeserializer(NonSpanningWrapper deserializer) {
-			deserializer.clear();
-			
-			if (leftOverData != null) {
-				deserializer.initializeFromMemorySegment(leftOverData, leftOverStart, leftOverLimit);
-			}
-		}
-		
-		private boolean hasFullRecord() {
-			return this.recordLength >= 0 && this.accumulatedRecordBytes >= this.recordLength;
-		}
-		
-		private int getNumGatheredBytes() {
-			return this.accumulatedRecordBytes + (this.recordLength >= 0 ? 4 : lengthBuffer.position());
-		}
-
-		public void clear() {
-			this.buffer = initialBuffer;
-			this.serializationReadBuffer.releaseArrays();
-
-			this.recordLength = -1;
-			this.lengthBuffer.clear();
-			this.leftOverData = null;
-			this.accumulatedRecordBytes = 0;
-			
-			if (spillingChannel != null) {
-				try {
-					spillingChannel.close();
-				}
-				catch (Throwable t) {
-					// ignore
-				}
-				spillingChannel = null;
-			}
-			if (spillFileReader != null) {
-				try {
-					spillFileReader.close();
-				}
-				catch (Throwable t) {
-					// ignore
-				}
-				spillFileReader = null;
-			}
-			if (spillFile != null) {
-				spillFile.delete();
-				spillFile = null;
-			}
-		}
-		
-		public DataInputView getInputView() {
-			if (spillFileReader == null) {
-				return serializationReadBuffer; 
-			}
-			else {
-				return spillFileReader;
-			}
-		}
-		
-		private void ensureBufferCapacity(int minLength) {
-			if (buffer.length < minLength) {
-				byte[] newBuffer = new byte[Math.max(minLength, buffer.length * 2)];
-				System.arraycopy(buffer, 0, newBuffer, 0, accumulatedRecordBytes);
-				buffer = newBuffer;
-			}
-		}
-		
-		@SuppressWarnings("resource")
-		private FileChannel createSpillingChannel() throws IOException {
-			if (spillFile != null) {
-				throw new IllegalStateException("Spilling file already exists.");
-			}
-			
-			String directory = tempDirs[rnd.nextInt(tempDirs.length)];
-			spillFile = new File(directory, randomString(rnd) + ".inputchannel");
-			
-			return new RandomAccessFile(spillFile, "rw").getChannel();
-		}
-		
-		private static String randomString(Random random) {
-			final byte[] bytes = new byte[20];
-			random.nextBytes(bytes);
-			return StringUtils.byteToHexString(bytes);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/SerializedUpdateBuffer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/SerializedUpdateBuffer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/SerializedUpdateBuffer.java
index 5896dcf..ca1ade9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/SerializedUpdateBuffer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/SerializedUpdateBuffer.java
@@ -46,7 +46,7 @@ public class SerializedUpdateBuffer extends AbstractPagedOutputView {
 
 	private ArrayDeque<MemorySegment> fullBuffers;
 
-	private BlockChannelWriter currentWriter;
+	private BlockChannelWriter<MemorySegment> currentWriter;
 
 	private final IOManager ioManager;
 
@@ -182,7 +182,7 @@ public class SerializedUpdateBuffer extends AbstractPagedOutputView {
 
 			// now close the writer and create the reader
 			currentWriter.close();
-			final BlockChannelReader reader = ioManager.createBlockChannelReader(currentWriter.getChannelID());
+			final BlockChannelReader<MemorySegment> reader = ioManager.createBlockChannelReader(currentWriter.getChannelID());
 
 			// gather some memory segments to circulate while reading back the data
 			final List<MemorySegment> readSegments = new ArrayList<MemorySegment>();
@@ -263,14 +263,14 @@ public class SerializedUpdateBuffer extends AbstractPagedOutputView {
 
 		private final Deque<MemorySegment> fullBufferSource;
 
-		private final BlockChannelReader spilledBufferSource;
+		private final BlockChannelReader<MemorySegment> spilledBufferSource;
 
 		private int spilledBuffersRemaining;
 
 		private int requestsRemaining;
 
 		private ReadEnd(MemorySegment firstMemSegment, LinkedBlockingQueue<MemorySegment> emptyBufferTarget,
-										Deque<MemorySegment> fullBufferSource, BlockChannelReader spilledBufferSource,
+										Deque<MemorySegment> fullBufferSource, BlockChannelReader<MemorySegment> spilledBufferSource,
 										List<MemorySegment> emptyBuffers, int numBuffersSpilled)
 			throws IOException {
 			super(firstMemSegment, firstMemSegment.getInt(0), HEADER_LENGTH);

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationHeadPactTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationHeadPactTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationHeadPactTask.java
index b8e4857..abaf311 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationHeadPactTask.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationHeadPactTask.java
@@ -22,7 +22,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.flink.runtime.io.network.api.writer.BufferWriter;
+import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
 import org.apache.flink.runtime.io.network.api.writer.RecordWriter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -91,7 +91,7 @@ public class IterationHeadPactTask<X, Y, S extends Function, OT> extends Abstrac
 
 	private TypeSerializerFactory<X> solutionTypeSerializer;
 
-	private BufferWriter toSync;
+	private ResultPartitionWriter toSync;
 
 	private int initialSolutionSetInput; // undefined for bulk iterations
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
index b9e1eac..609ed3e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
@@ -24,6 +24,7 @@ import java.util.List;
 import org.apache.commons.lang3.Validate;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.io.InputSplitSource;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
@@ -318,31 +319,31 @@ public class AbstractJobVertex implements java.io.Serializable {
 	public IntermediateDataSet createAndAddResultDataSet() {
 		return createAndAddResultDataSet(new IntermediateDataSetID());
 	}
-	
+
 	public IntermediateDataSet createAndAddResultDataSet(IntermediateDataSetID id) {
-		IntermediateDataSet result = new IntermediateDataSet(id, this);
+		IntermediateDataSet result = new IntermediateDataSet(id, ResultPartitionType.PIPELINED, this);
 		this.results.add(result);
 		return result;
 	}
-	
+
 	public void connectDataSetAsInput(IntermediateDataSet dataSet, DistributionPattern distPattern) {
 		JobEdge edge = new JobEdge(dataSet, this, distPattern);
 		this.inputs.add(edge);
 		dataSet.addConsumer(edge);
 	}
-	
+
 	public void connectNewDataSetAsInput(AbstractJobVertex input, DistributionPattern distPattern) {
 		IntermediateDataSet dataSet = input.createAndAddResultDataSet();
 		JobEdge edge = new JobEdge(dataSet, this, distPattern);
 		this.inputs.add(edge);
 		dataSet.addConsumer(edge);
 	}
-	
+
 	public void connectIdInput(IntermediateDataSetID dataSetId, DistributionPattern distPattern) {
 		JobEdge edge = new JobEdge(dataSetId, this, distPattern);
 		this.inputs.add(edge);
 	}
-	
+
 	// --------------------------------------------------------------------------------------------
 	
 	public boolean isInputVertex() {

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSet.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSet.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSet.java
index c8415de..86888e2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSet.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSet.java
@@ -18,9 +18,13 @@
 
 package org.apache.flink.runtime.jobgraph;
 
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+
 import java.util.ArrayList;
 import java.util.List;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 /**
  * An intermediate data set is the data set produced by an operator - either a
  * source or any intermediate operation.
@@ -38,6 +42,9 @@ public class IntermediateDataSet implements java.io.Serializable {
 	private final AbstractJobVertex producer;		// the operation that produced this data set
 	
 	private final List<JobEdge> consumers = new ArrayList<JobEdge>();
+
+	// The type of partition to use at runtime
+	private final ResultPartitionType resultType;
 	
 	// --------------------------------------------------------------------------------------------
 	
@@ -46,12 +53,13 @@ public class IntermediateDataSet implements java.io.Serializable {
 	}
 	
 	public IntermediateDataSet(IntermediateDataSetID id, AbstractJobVertex producer) {
-		if (id == null || producer == null) {
-			throw new NullPointerException();
-		}
-		
-		this.id = id;
-		this.producer = producer;
+		this(id, ResultPartitionType.PIPELINED, producer);
+	}
+
+	public IntermediateDataSet(IntermediateDataSetID id, ResultPartitionType resultType, AbstractJobVertex producer) {
+		this.id = checkNotNull(id);
+		this.producer = checkNotNull(producer);
+		this.resultType = checkNotNull(resultType);
 	}
 
 	// --------------------------------------------------------------------------------------------
@@ -67,6 +75,10 @@ public class IntermediateDataSet implements java.io.Serializable {
 	public List<JobEdge> getConsumers() {
 		return this.consumers;
 	}
+
+	public ResultPartitionType getResultType() {
+		return resultType;
+	}
 	
 	// --------------------------------------------------------------------------------------------
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateResultPartitionType.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateResultPartitionType.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateResultPartitionType.java
deleted file mode 100644
index 25d057e..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateResultPartitionType.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.jobgraph;
-
-public enum IntermediateResultPartitionType {
-
-	BLOCKING(true, false, false),
-
-	PIPELINED(false, true, true),
-	PIPELINED_PERSISTENT(true, true, true);
-
-	private final boolean isPersistent;
-
-	private final boolean isPipelined;
-
-	private final boolean hasBackPressure;
-
-	IntermediateResultPartitionType(boolean isPersistent, boolean isPipelined, boolean hasBackPressure) {
-		this.isPersistent = isPersistent;
-		this.isPipelined = isPipelined;
-		this.hasBackPressure = hasBackPressure;
-	}
-
-	public boolean hasBackPressure() {
-		return hasBackPressure;
-	}
-
-	public boolean isPipelined() {
-		return isPipelined;
-	}
-
-	public boolean isPersistent() {
-		return isPersistent;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/operators/TempBarrier.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/TempBarrier.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/TempBarrier.java
index 013a733..63f2fd5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/TempBarrier.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/TempBarrier.java
@@ -166,6 +166,7 @@ public class TempBarrier<T> implements CloseableInputProvider<T> {
 			this.buffer = buffer;
 		}
 		
+		@Override
 		public void run() {
 			final MutableObjectIterator<T> input = this.input;
 			final TypeSerializer<T> serializer = this.serializer;

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashPartition.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashPartition.java
index 14e4ae6..cee9ebb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashPartition.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashPartition.java
@@ -93,9 +93,9 @@ public class HashPartition<BT, PT> extends AbstractPagedInputView implements See
 	
 	// ------------------------------------------ Spilling ----------------------------------------------
 	
-	private BlockChannelWriter buildSideChannel;		// the channel writer for the build side, if partition is spilled
+	private BlockChannelWriter<MemorySegment> buildSideChannel;		// the channel writer for the build side, if partition is spilled
 	
-	protected BlockChannelWriter probeSideChannel;		// the channel writer from the probe side, if partition is spilled
+	protected BlockChannelWriter<MemorySegment> probeSideChannel;		// the channel writer from the probe side, if partition is spilled
 	
 	// ------------------------------------------ Restoring ----------------------------------------------
 	
@@ -219,12 +219,12 @@ public class HashPartition<BT, PT> extends AbstractPagedInputView implements See
 		return this.probeSideRecordCounter;
 	}
 
-	public BlockChannelWriter getBuildSideChannel() {
+	public BlockChannelWriter<MemorySegment> getBuildSideChannel() {
 		return this.buildSideChannel;
 	}
 	
 	
-	public BlockChannelWriter getProbeSideChannel() {
+	public BlockChannelWriter<MemorySegment> getProbeSideChannel() {
 		return this.probeSideChannel;
 	}
 	
@@ -506,7 +506,7 @@ public class HashPartition<BT, PT> extends AbstractPagedInputView implements See
 		
 		private final MemorySegmentSource memSource;
 		
-		private BlockChannelWriter writer;
+		private BlockChannelWriter<MemorySegment> writer;
 		
 		private int currentBlockNumber;
 		
@@ -553,7 +553,7 @@ public class HashPartition<BT, PT> extends AbstractPagedInputView implements See
 			return this.currentBlockNumber + 1;
 		}
 		
-		int spill(BlockChannelWriter writer) throws IOException
+		int spill(BlockChannelWriter<MemorySegment> writer) throws IOException
 		{
 			this.writer = writer;
 			final int numSegments = this.targetList.size();


[11/13] flink git commit: [FLINK-1350] [runtime] Add blocking result partition variant

Posted by uc...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java
index 8daee7c..96b2f55 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java
@@ -22,13 +22,14 @@ import akka.actor.ActorRef;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.broadcast.BroadcastVariableManager;
-import org.apache.flink.runtime.deployment.PartitionConsumerDeploymentDescriptor;
-import org.apache.flink.runtime.deployment.PartitionDeploymentDescriptor;
+import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor;
+import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor;
 import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.network.NetworkEnvironment;
-import org.apache.flink.runtime.io.network.api.writer.BufferWriter;
-import org.apache.flink.runtime.io.network.partition.IntermediateResultPartition;
+import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
+import org.apache.flink.runtime.io.network.partition.ResultPartition;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
 import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
@@ -92,9 +93,8 @@ public class RuntimeEnvironment implements Environment, Runnable {
 
 	private final AtomicBoolean canceled = new AtomicBoolean();
 
-	private final IntermediateResultPartition[] producedPartitions;
-
-	private final BufferWriter[] writers;
+	private final ResultPartition[] producedPartitions;
+	private final ResultPartitionWriter[] writers;
 
 	private final SingleInputGate[] inputGates;
 
@@ -116,23 +116,27 @@ public class RuntimeEnvironment implements Environment, Runnable {
 
 		try {
 			// Produced intermediate result partitions
-			final List<PartitionDeploymentDescriptor> partitions = tdd.getProducedPartitions();
+			final List<ResultPartitionDeploymentDescriptor> partitions = tdd.getProducedPartitions();
 
-			this.producedPartitions = new IntermediateResultPartition[partitions.size()];
-			this.writers = new BufferWriter[partitions.size()];
+			this.producedPartitions = new ResultPartition[partitions.size()];
+			this.writers = new ResultPartitionWriter[partitions.size()];
 
 			for (int i = 0; i < this.producedPartitions.length; i++) {
-				this.producedPartitions[i] = IntermediateResultPartition.create(this, i, owner.getJobID(), owner.getExecutionId(), networkEnvironment, partitions.get(i));
-				writers[i] = new BufferWriter(this.producedPartitions[i]);
+				ResultPartitionDeploymentDescriptor desc = partitions.get(i);
+				ResultPartitionID partitionId = new ResultPartitionID(desc.getPartitionId(), owner.getExecutionId());
+
+				this.producedPartitions[i] = new ResultPartition(owner.getJobID(), partitionId, desc.getPartitionType(), desc.getNumberOfSubpartitions(), networkEnvironment, ioManager);
+
+				writers[i] = new ResultPartitionWriter(this.producedPartitions[i]);
 			}
 
 			// Consumed intermediate result partitions
-			final List<PartitionConsumerDeploymentDescriptor> consumedPartitions = tdd.getConsumedPartitions();
+			final List<InputGateDeploymentDescriptor> consumedPartitions = tdd.getInputGates();
 
 			this.inputGates = new SingleInputGate[consumedPartitions.size()];
 
 			for (int i = 0; i < inputGates.length; i++) {
-				inputGates[i] = SingleInputGate.create(networkEnvironment, consumedPartitions.get(i));
+				inputGates[i] = SingleInputGate.create(consumedPartitions.get(i), networkEnvironment);
 
 				// The input gates are organized by key for task updates/channel updates at runtime
 				inputGatesById.put(inputGates[i].getConsumedResultId(), inputGates[i]);
@@ -211,7 +215,7 @@ public class RuntimeEnvironment implements Environment, Runnable {
 
 			// Finish the produced partitions
 			if (producedPartitions != null) {
-				for (IntermediateResultPartition partition : producedPartitions) {
+				for (ResultPartition partition : producedPartitions) {
 					if (partition != null) {
 						partition.finish();
 					}
@@ -340,14 +344,14 @@ public class RuntimeEnvironment implements Environment, Runnable {
 	}
 
 	@Override
-	public BufferWriter getWriter(int index) {
+	public ResultPartitionWriter getWriter(int index) {
 		checkElementIndex(index, writers.length, "Illegal environment writer request.");
 
 		return writers[checkElementIndex(index, writers.length)];
 	}
 
 	@Override
-	public BufferWriter[] getAllWriters() {
+	public ResultPartitionWriter[] getAllWriters() {
 		return writers;
 	}
 
@@ -363,7 +367,7 @@ public class RuntimeEnvironment implements Environment, Runnable {
 		return inputGates;
 	}
 
-	public IntermediateResultPartition[] getProducedPartitions() {
+	public ResultPartition[] getProducedPartitions() {
 		return producedPartitions;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
index cf24b20..eda5bdf 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
@@ -20,23 +20,22 @@ package org.apache.flink.runtime.executiongraph;
 
 import akka.actor.ActorRef;
 import akka.dispatch.OnComplete;
-import static akka.dispatch.Futures.future;
-
 import akka.dispatch.OnFailure;
 import akka.pattern.Patterns;
 import akka.util.Timeout;
 import org.apache.flink.runtime.JobException;
 import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.deployment.PartialPartitionInfo;
-import org.apache.flink.runtime.deployment.PartitionInfo;
+import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor;
+import org.apache.flink.runtime.deployment.PartialInputChannelDeploymentDescriptor;
+import org.apache.flink.runtime.deployment.ResultPartitionLocation;
 import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
 import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.instance.Instance;
 import org.apache.flink.runtime.instance.InstanceConnectionInfo;
 import org.apache.flink.runtime.instance.SimpleSlot;
-import org.apache.flink.runtime.instance.Instance;
-import org.apache.flink.runtime.io.network.RemoteAddress;
+import org.apache.flink.runtime.io.network.ConnectionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
-import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
 import org.apache.flink.runtime.jobmanager.scheduler.CoLocationConstraint;
 import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
 import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit;
@@ -44,24 +43,23 @@ import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotAllocationFuture;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotAllocationFutureAction;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
-import org.apache.flink.runtime.messages.TaskManagerMessages;
 import org.apache.flink.runtime.messages.TaskManagerMessages.TaskOperationResult;
 import org.apache.flink.runtime.state.StateHandle;
 import org.apache.flink.util.ExceptionUtils;
 import org.slf4j.Logger;
-
 import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
 
 import java.io.Serializable;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 
-import static com.google.common.base.Preconditions.checkArgument;
+import static akka.dispatch.Futures.future;
 import static com.google.common.base.Preconditions.checkNotNull;
 import static org.apache.flink.runtime.execution.ExecutionState.CANCELED;
 import static org.apache.flink.runtime.execution.ExecutionState.CANCELING;
@@ -71,6 +69,12 @@ import static org.apache.flink.runtime.execution.ExecutionState.FAILED;
 import static org.apache.flink.runtime.execution.ExecutionState.FINISHED;
 import static org.apache.flink.runtime.execution.ExecutionState.RUNNING;
 import static org.apache.flink.runtime.execution.ExecutionState.SCHEDULED;
+import static org.apache.flink.runtime.messages.TaskManagerMessages.CancelTask;
+import static org.apache.flink.runtime.messages.TaskManagerMessages.FailIntermediateResultPartitions;
+import static org.apache.flink.runtime.messages.TaskManagerMessages.SubmitTask;
+import static org.apache.flink.runtime.messages.TaskManagerMessages.UpdateTask;
+import static org.apache.flink.runtime.messages.TaskManagerMessages.UpdateTaskSinglePartitionInfo;
+import static org.apache.flink.runtime.messages.TaskManagerMessages.createUpdateTaskMultiplePartitionInfos;
 
 /**
  * A single execution of a vertex. While an {@link ExecutionVertex} can be executed multiple times (for recovery,
@@ -113,7 +117,7 @@ public class Execution implements Serializable {
 
 	private final FiniteDuration timeout;
 
-	private ConcurrentLinkedQueue<PartialPartitionInfo> partialPartitionInfos;
+	private ConcurrentLinkedQueue<PartialInputChannelDeploymentDescriptor> partialInputChannelDeploymentDescriptors;
 
 	private volatile ExecutionState state = CREATED;
 	
@@ -128,8 +132,6 @@ public class Execution implements Serializable {
 	// --------------------------------------------------------------------------------------------
 	
 	public Execution(ExecutionVertex vertex, int attemptNumber, long startTimestamp, FiniteDuration timeout) {
-		checkArgument(attemptNumber >= 0);
-		
 		this.vertex = checkNotNull(vertex);
 		this.attemptId = new ExecutionAttemptID();
 		
@@ -140,17 +142,17 @@ public class Execution implements Serializable {
 
 		this.timeout = timeout;
 
-		this.partialPartitionInfos = new ConcurrentLinkedQueue<PartialPartitionInfo>();
+		this.partialInputChannelDeploymentDescriptors = new ConcurrentLinkedQueue<PartialInputChannelDeploymentDescriptor>();
 	}
 	
 	// --------------------------------------------------------------------------------------------
 	//   Properties
 	// --------------------------------------------------------------------------------------------
-	
+
 	public ExecutionVertex getVertex() {
 		return vertex;
 	}
-	
+
 	public ExecutionAttemptID getAttemptId() {
 		return attemptId;
 	}
@@ -158,15 +160,15 @@ public class Execution implements Serializable {
 	public int getAttemptNumber() {
 		return attemptNumber;
 	}
-	
+
 	public ExecutionState getState() {
 		return state;
 	}
-	
+
 	public SimpleSlot getAssignedResource() {
 		return assignedResource;
 	}
-	
+
 	public InstanceConnectionInfo getAssignedResourceLocation() {
 		return assignedResourceLocation;
 	}
@@ -196,8 +198,8 @@ public class Execution implements Serializable {
 		}
 		assignedResource = null;
 
-		partialPartitionInfos.clear();
-		partialPartitionInfos = null;
+		partialInputChannelDeploymentDescriptors.clear();
+		partialInputChannelDeploymentDescriptors = null;
 	}
 	
 	// --------------------------------------------------------------------------------------------
@@ -226,7 +228,7 @@ public class Execution implements Serializable {
 
 		// sanity check
 		if (locationConstraint != null && sharingGroup == null) {
-			throw new RuntimeException("Trying to schedule with co-location constraint but without slot sharing not allowed.");
+			throw new RuntimeException("Trying to schedule with co-location constraint but without slot sharing allowed.");
 		}
 
 		if (transitionState(CREATED, SCHEDULED)) {
@@ -328,7 +330,7 @@ public class Execution implements Serializable {
 
 			Instance instance = slot.getInstance();
 			Future<Object> deployAction = Patterns.ask(instance.getTaskManager(),
-					new TaskManagerMessages.SubmitTask(deployment), new Timeout(timeout));
+					new SubmitTask(deployment), new Timeout(timeout));
 
 			deployAction.onComplete(new OnComplete<Object>(){
 
@@ -432,22 +434,38 @@ public class Execution implements Serializable {
 		}
 	}
 
-	// TODO This leads to many unnecessary RPC calls in most cases
-	void scheduleOrUpdateConsumers(List<List<ExecutionEdge>> consumers) {
-		if (consumers.size() != 1) {
-			fail(new IllegalStateException("Only one consumer is supported currently."));
+	void scheduleOrUpdateConsumers(List<List<ExecutionEdge>> allConsumers) {
+		if (allConsumers.size() != 1) {
+			fail(new IllegalStateException("Currently, only a single consumer group per partition is supported."));
 		}
 
-		final List<ExecutionEdge> consumer = consumers.get(0);
-
-		for (ExecutionEdge edge : consumer) {
+		for (ExecutionEdge edge : allConsumers.get(0)) {
 			final ExecutionVertex consumerVertex = edge.getTarget();
 
-			final ExecutionState consumerState = consumerVertex.getExecutionState();
+			final Execution consumer = consumerVertex.getCurrentExecutionAttempt();
+			final ExecutionState consumerState = consumer.getState();
 
-			if (consumerState == CREATED) {
-				consumerVertex.cachePartitionInfo(PartialPartitionInfo.fromEdge(edge));
+			final IntermediateResultPartition partition = edge.getSource();
 
+			// ----------------------------------------------------------------
+			// Consumer is created => try to deploy and cache input channel
+			// descriptors if there is a deployment race
+			// ----------------------------------------------------------------
+			if (consumerState == CREATED) {
+				final Execution partitionExecution = partition.getProducer()
+						.getCurrentExecutionAttempt();
+
+				consumerVertex.cachePartitionInfo(PartialInputChannelDeploymentDescriptor.fromEdge(
+						partition, partitionExecution));
+
+				// When deploying a consuming task, its task deployment descriptor will contain all
+				// deployment information available at the respective time. It is possible that some
+				// of the partitions to be consumed have not been created yet. These are updated
+				// runtime via the update messages.
+				//
+				// TODO The current approach may send many update messages even though the consuming
+				// task has already been deployed with all necessary information. We have to check
+				// whether this is a problem and fix it, if it is.
 				future(new Callable<Boolean>(){
 					@Override
 					public Boolean call() throws Exception {
@@ -468,41 +486,64 @@ public class Execution implements Serializable {
 					consumerVertex.sendPartitionInfos();
 				}
 			}
-			else if (consumerState == RUNNING) {
-				SimpleSlot consumerSlot = consumerVertex.getCurrentAssignedResource();
-				ExecutionAttemptID consumerExecutionId = consumerVertex.
-						getCurrentExecutionAttempt().getAttemptId();
-
-				IntermediateResultPartitionID partitionID = edge.getSource().getPartitionId();
-				int connectionIndex = edge.getSource().getIntermediateResult().getConnectionIndex();
-
-				PartitionInfo.PartitionLocation producerLocation;
-				RemoteAddress producerAddress = null;
-
-				if(consumerSlot.getInstance().getInstanceConnectionInfo().equals(
-						getAssignedResourceLocation())) {
-					producerLocation = PartitionInfo.PartitionLocation.LOCAL;
-				} else {
-					producerLocation = PartitionInfo.PartitionLocation.REMOTE;
-					producerAddress = new RemoteAddress(getAssignedResourceLocation(),
-							connectionIndex);
-				}
+			// ----------------------------------------------------------------
+			// Consumer is running => send update message now
+			// ----------------------------------------------------------------
+			else {
+				if (consumerState == RUNNING) {
+					final SimpleSlot consumerSlot = consumer.getAssignedResource();
 
-				PartitionInfo partitionInfo = new PartitionInfo(partitionID, attemptId,
-						producerLocation, producerAddress);
+					if (consumerSlot == null) {
+						// The consumer has been reset concurrently
+						continue;
+					}
 
-				TaskManagerMessages.UpdateTask updateTaskMessage =
-						new TaskManagerMessages.UpdateTaskSinglePartitionInfo(consumerExecutionId,
-								edge.getSource().getIntermediateResult().getId(), partitionInfo);
+					final Instance consumerInstance = consumerSlot.getInstance();
 
-				sendUpdateTaskRpcCall(consumerSlot, updateTaskMessage);
-			}
-			else if (consumerState == SCHEDULED || consumerState == DEPLOYING) {
-				consumerVertex.cachePartitionInfo(PartialPartitionInfo.fromEdge(edge));
+					final ResultPartitionID partitionId = new ResultPartitionID(
+							partition.getPartitionId(), attemptId);
 
-				// double check to resolve race conditions
-				if(consumerVertex.getExecutionState() == RUNNING){
-					consumerVertex.sendPartitionInfos();
+					final Instance partitionInstance = partition.getProducer()
+							.getCurrentAssignedResource().getInstance();
+
+					final ResultPartitionLocation partitionLocation;
+
+					if (consumerInstance.equals(partitionInstance)) {
+						// Consuming task is deployed to the same instance as the partition => local
+						partitionLocation = ResultPartitionLocation.createLocal();
+					}
+					else {
+						// Different instances => remote
+						final ConnectionID connectionId = new ConnectionID(
+								partitionInstance.getInstanceConnectionInfo(),
+								partition.getIntermediateResult().getConnectionIndex());
+
+						partitionLocation = ResultPartitionLocation.createRemote(connectionId);
+					}
+
+					final InputChannelDeploymentDescriptor descriptor = new InputChannelDeploymentDescriptor(
+							partitionId, partitionLocation);
+
+					final UpdateTask updateTaskMessage = new UpdateTaskSinglePartitionInfo(
+							consumer.getAttemptId(), partition.getIntermediateResult().getId(), descriptor);
+
+					sendUpdateTaskRpcCall(consumerSlot, updateTaskMessage);
+				}
+				// ----------------------------------------------------------------
+				// Consumer is scheduled or deploying => cache input channel
+				// deployment descriptors and send update message later
+				// ----------------------------------------------------------------
+				else if (consumerState == SCHEDULED || consumerState == DEPLOYING) {
+					final Execution partitionExecution = partition.getProducer()
+							.getCurrentExecutionAttempt();
+
+					consumerVertex.cachePartitionInfo(PartialInputChannelDeploymentDescriptor
+							.fromEdge(partition, partitionExecution));
+
+					// double check to resolve race conditions
+					if (consumerVertex.getExecutionState() == RUNNING) {
+						consumerVertex.sendPartitionInfos();
+					}
 				}
 			}
 		}
@@ -543,6 +584,23 @@ public class Execution implements Serializable {
 
 				if (transitionState(current, FINISHED)) {
 					try {
+						if (getVertex().finishAllBlockingPartitions()) {
+
+							IntermediateResult[] allResults = getVertex().getJobVertex()
+									.getProducedDataSets();
+
+							LOG.debug("Finished all produced partitions ({}). Scheduling all receivers " +
+									"of the following datasets: {}.", this, Arrays
+									.toString(allResults));
+
+							// Schedule next batch
+							for (IntermediateResult result : allResults) {
+								for (IntermediateResultPartition p : result.getPartitions()) {
+									scheduleOrUpdateConsumers(p.getConsumers());
+								}
+							}
+						}
+
 						assignedResource.releaseSlot();
 						vertex.getExecutionGraph().deregisterExecution(this);
 					}
@@ -612,28 +670,28 @@ public class Execution implements Serializable {
 		}
 	}
 
-	void cachePartitionInfo(PartialPartitionInfo partitionInfo) {
-		partialPartitionInfos.add(partitionInfo);
+	void cachePartitionInfo(PartialInputChannelDeploymentDescriptor partitionInfo) {
+		partialInputChannelDeploymentDescriptors.add(partitionInfo);
 	}
 
 	void sendPartitionInfos() {
 		// check if the ExecutionVertex has already been archived and thus cleared the
 		// partial partition infos queue
-		if(partialPartitionInfos != null && !partialPartitionInfos.isEmpty()) {
+		if(partialInputChannelDeploymentDescriptors != null && !partialInputChannelDeploymentDescriptors.isEmpty()) {
 
-			PartialPartitionInfo partialPartitionInfo;
+			PartialInputChannelDeploymentDescriptor partialInputChannelDeploymentDescriptor;
 
 			List<IntermediateDataSetID> resultIDs = new ArrayList<IntermediateDataSetID>();
-			List<PartitionInfo> partitionInfos = new ArrayList<PartitionInfo>();
+			List<InputChannelDeploymentDescriptor> inputChannelDeploymentDescriptors = new ArrayList<InputChannelDeploymentDescriptor>();
 
-			while ((partialPartitionInfo = partialPartitionInfos.poll()) != null) {
-				resultIDs.add(partialPartitionInfo.getIntermediateDataSetID());
-				partitionInfos.add(partialPartitionInfo.createPartitionInfo(this));
+			while ((partialInputChannelDeploymentDescriptor = partialInputChannelDeploymentDescriptors.poll()) != null) {
+				resultIDs.add(partialInputChannelDeploymentDescriptor.getResultId());
+				inputChannelDeploymentDescriptors.add(partialInputChannelDeploymentDescriptor.createInputChannelDeploymentDescriptor(this));
 			}
 
-			TaskManagerMessages.UpdateTask updateTaskMessage =
-					TaskManagerMessages.createUpdateTaskMultiplePartitionInfos(attemptId, resultIDs,
-							partitionInfos);
+			UpdateTask updateTaskMessage =
+					createUpdateTaskMultiplePartitionInfos(attemptId, resultIDs,
+							inputChannelDeploymentDescriptors);
 
 			sendUpdateTaskRpcCall(assignedResource, updateTaskMessage);
 		}
@@ -752,7 +810,7 @@ public class Execution implements Serializable {
 		if (slot != null) {
 
 			Future<Object> cancelResult = AkkaUtils.retry(slot.getInstance().getTaskManager(), new
-							TaskManagerMessages.CancelTask(attemptId), NUM_CANCEL_CALL_TRIES,
+							CancelTask(attemptId), NUM_CANCEL_CALL_TRIES,
 					AkkaUtils.globalExecutionContext(), timeout);
 
 			cancelResult.onComplete(new OnComplete<Object>() {
@@ -782,7 +840,7 @@ public class Execution implements Serializable {
 			if (instance.isAlive()) {
 				try {
 					// TODO For some tests this could be a problem when querying too early if all resources were released
-					instance.getTaskManager().tell(new TaskManagerMessages.FailIntermediateResultPartitions(attemptId), ActorRef.noSender());
+					instance.getTaskManager().tell(new FailIntermediateResultPartitions(attemptId), ActorRef.noSender());
 				} catch (Throwable t) {
 					fail(new Exception("Intermediate result partition could not be failed.", t));
 				}
@@ -791,7 +849,8 @@ public class Execution implements Serializable {
 	}
 
 	private void sendUpdateTaskRpcCall(final SimpleSlot consumerSlot,
-									final TaskManagerMessages.UpdateTask updateTaskMsg) {
+									final UpdateTask updateTaskMsg) {
+
 		if (consumerSlot != null) {
 			final Instance instance = consumerSlot.getInstance();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionEdge.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionEdge.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionEdge.java
index 0d81fde..3fb3825 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionEdge.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionEdge.java
@@ -43,4 +43,9 @@ public class ExecutionEdge {
 	public int getInputNum() {
 		return inputNum;
 	}
+
+	@Override
+	public String toString() {
+		return "ExecutionEdge [" + source + " <=> " + target + "]";
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
index 81f83e6..01495f1 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
@@ -25,6 +25,7 @@ import org.apache.flink.runtime.JobException;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.blob.BlobKey;
 import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.JobID;
@@ -175,17 +176,14 @@ public class ExecutionGraph implements Serializable {
 	 * Once this value has reached the number of vertices, the job is done. */
 	private int nextVertexToFinish;
 
-
 	private ActorContext parentContext;
 
 	private  ActorRef stateMonitorActor;
-	
+
 	private boolean monitoringEnabled;
-	
-	private long monitoringInterval = 10000;
 
+	private long monitoringInterval = 10000;
 
-	
 	public ExecutionGraph(JobID jobId, String jobName, Configuration jobConfig, FiniteDuration timeout) {
 		this(jobId, jobName, jobConfig, timeout, new ArrayList<BlobKey>());
 	}
@@ -446,6 +444,7 @@ public class ExecutionGraph implements Serializable {
 					for (ExecutionJobVertex ejv : getVerticesTopologically()) {
 						ejv.scheduleAll(scheduler, allowQueuedScheduling);
 					}
+
 					break;
 
 				case BACKTRACKING:
@@ -623,24 +622,24 @@ public class ExecutionGraph implements Serializable {
 	
 	public void loadOperatorStates(Map<Tuple3<JobVertexID, Integer, Long> , StateHandle> states) {
 		synchronized (this.progressLock) {
-			for (Map.Entry<Tuple3<JobVertexID, Integer, Long>, StateHandle> state : states.entrySet()) {
+			for (Map.Entry<Tuple3<JobVertexID, Integer, Long>, StateHandle> state : states.entrySet())
 				tasks.get(state.getKey()._1()).getTaskVertices()[state.getKey()._2()].setOperatorState(state.getValue());
-			}
 		}
 	}
 
-	public void scheduleOrUpdateConsumers(ExecutionAttemptID executionId, int partitionIndex) {
-		Execution execution = currentExecutions.get(executionId);
+	public void scheduleOrUpdateConsumers(ResultPartitionID partitionId) {
+
+		final Execution execution = currentExecutions.get(partitionId.getProducerId());
 
 		if (execution == null) {
 			fail(new IllegalStateException("Cannot find execution for execution ID " +
-					executionId));
+					partitionId.getPartitionId()));
 		}
-		else if(execution.getVertex() == null){
-			fail(new IllegalStateException("Execution with execution ID " + executionId +
-				" has no vertex assigned."));
+		else if (execution.getVertex() == null){
+			fail(new IllegalStateException("Execution with execution ID " +
+					partitionId.getPartitionId() + " has no vertex assigned."));
 		} else {
-			execution.getVertex().scheduleOrUpdateConsumers(partitionIndex);
+			execution.getVertex().scheduleOrUpdateConsumers(partitionId);
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
index 0444e5d..6fdc628 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
@@ -18,13 +18,6 @@
 
 package org.apache.flink.runtime.executiongraph;
 
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
 import org.apache.flink.api.common.io.StrictlyLocalAssignment;
 import org.apache.flink.core.io.InputSplit;
 import org.apache.flink.core.io.InputSplitAssigner;
@@ -39,13 +32,19 @@ import org.apache.flink.runtime.jobgraph.JobEdge;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup;
-import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
+import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
 import org.slf4j.Logger;
-
 import scala.concurrent.duration.FiniteDuration;
 
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 
 public class ExecutionJobVertex implements Serializable {
 	
@@ -82,7 +81,6 @@ public class ExecutionJobVertex implements Serializable {
 	
 	private InputSplitAssigner splitAssigner;
 	
-	
 	public ExecutionJobVertex(ExecutionGraph graph, AbstractJobVertex jobVertex,
 							int defaultParallelism, FiniteDuration timeout) throws JobException {
 		this(graph, jobVertex, defaultParallelism, timeout, System.currentTimeMillis());
@@ -118,11 +116,14 @@ public class ExecutionJobVertex implements Serializable {
 		
 		// create the intermediate results
 		this.producedDataSets = new IntermediateResult[jobVertex.getNumberOfProducedIntermediateDataSets()];
+
 		for (int i = 0; i < jobVertex.getProducedDataSets().size(); i++) {
-			IntermediateDataSet set = jobVertex.getProducedDataSets().get(i);
-			this.producedDataSets[i] = new IntermediateResult(set.getId(), this, numTaskVertices);
+			final IntermediateDataSet result = jobVertex.getProducedDataSets().get(i);
+
+			this.producedDataSets[i] = new IntermediateResult(
+					result.getId(), this, numTaskVertices, result.getResultType());
 		}
-		
+
 		// create all task vertices
 		for (int i = 0; i < numTaskVertices; i++) {
 			ExecutionVertex vertex = new ExecutionVertex(this, i, this.producedDataSets, timeout, createTimestamp);
@@ -374,6 +375,11 @@ public class ExecutionJobVertex implements Serializable {
 			catch (Throwable t) {
 				throw new RuntimeException("Re-creating the input split assigner failed: " + t.getMessage(), t);
 			}
+
+			// Reset intermediate results
+			for (IntermediateResult result : producedDataSets) {
+				result.resetForNewExecution();
+			}
 		}
 	}
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
index 794ca21..e5d9db8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
@@ -20,17 +20,19 @@ package org.apache.flink.runtime.executiongraph;
 
 import org.apache.flink.runtime.JobException;
 import org.apache.flink.runtime.blob.BlobKey;
-import org.apache.flink.runtime.deployment.PartialPartitionInfo;
-import org.apache.flink.runtime.deployment.PartitionConsumerDeploymentDescriptor;
-import org.apache.flink.runtime.deployment.PartitionDeploymentDescriptor;
-import org.apache.flink.runtime.deployment.PartitionInfo;
+import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor;
+import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor;
+import org.apache.flink.runtime.deployment.PartialInputChannelDeploymentDescriptor;
+import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor;
 import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
 import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.instance.Instance;
 import org.apache.flink.runtime.instance.InstanceConnectionInfo;
 import org.apache.flink.runtime.instance.SimpleSlot;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
+import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
 import org.apache.flink.runtime.jobgraph.JobEdge;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
@@ -46,11 +48,12 @@ import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashSet;
+import java.util.LinkedHashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.CopyOnWriteArrayList;
 
-import static com.google.common.base.Preconditions.checkElementIndex;
 import static org.apache.flink.runtime.execution.ExecutionState.CANCELED;
 import static org.apache.flink.runtime.execution.ExecutionState.FAILED;
 import static org.apache.flink.runtime.execution.ExecutionState.FINISHED;
@@ -63,35 +66,35 @@ public class ExecutionVertex implements Serializable {
 
 	private static final long serialVersionUID = 42L;
 
+	@SuppressWarnings("unused")
 	private static final Logger LOG = ExecutionGraph.LOG;
-	
+
 	private static final int MAX_DISTINCT_LOCATIONS_TO_CONSIDER = 8;
-	
+
 	// --------------------------------------------------------------------------------------------
-	
+
 	private final ExecutionJobVertex jobVertex;
-	
-	private IntermediateResultPartition[] resultPartitions;
-	
+
+	private Map<IntermediateResultPartitionID, IntermediateResultPartition> resultPartitions;
+
 	private ExecutionEdge[][] inputEdges;
 
 	private final int subTaskIndex;
-	
+
 	private final List<Execution> priorExecutions;
 
 	private final FiniteDuration timeout;
-	
+
 	private volatile CoLocationConstraint locationConstraint;
-	
+
 	private volatile Execution currentExecution;	// this field must never be null
-	
-	
+
 	private volatile List<Instance> locationConstraintInstances;
-	
+
 	private volatile boolean scheduleLocalOnly;
-	
+
 	private StateHandle operatorState;
-	
+
 	// --------------------------------------------------------------------------------------------
 
 	public ExecutionVertex(ExecutionJobVertex jobVertex, int subTaskIndex,
@@ -105,11 +108,13 @@ public class ExecutionVertex implements Serializable {
 		this.jobVertex = jobVertex;
 		this.subTaskIndex = subTaskIndex;
 
-		this.resultPartitions = new IntermediateResultPartition[producedDataSets.length];
-		for (int i = 0; i < producedDataSets.length; i++) {
-			IntermediateResultPartition irp = new IntermediateResultPartition(producedDataSets[i], this, subTaskIndex);
-			this.resultPartitions[i] = irp;
-			producedDataSets[i].setPartition(subTaskIndex, irp);
+		this.resultPartitions = new LinkedHashMap<IntermediateResultPartitionID, IntermediateResultPartition>(producedDataSets.length, 1);
+
+		for (IntermediateResult result : producedDataSets) {
+			IntermediateResultPartition irp = new IntermediateResultPartition(result, this, subTaskIndex);
+			result.setPartition(subTaskIndex, irp);
+
+			resultPartitions.put(irp.getPartitionId(), irp);
 		}
 
 		this.inputEdges = new ExecutionEdge[jobVertex.getJobVertex().getInputs().size()][];
@@ -129,71 +134,71 @@ public class ExecutionVertex implements Serializable {
 
 		this.timeout = timeout;
 	}
-	
-	
+
+
 	// --------------------------------------------------------------------------------------------
 	//  Properties
 	// --------------------------------------------------------------------------------------------
-	
+
 	public JobID getJobId() {
 		return this.jobVertex.getJobId();
 	}
-	
+
 	public ExecutionJobVertex getJobVertex() {
 		return jobVertex;
 	}
-	
+
 	public JobVertexID getJobvertexId() {
 		return this.jobVertex.getJobVertexId();
 	}
-	
+
 	public String getTaskName() {
 		return this.jobVertex.getJobVertex().getName();
 	}
-	
+
 	public int getTotalNumberOfParallelSubtasks() {
 		return this.jobVertex.getParallelism();
 	}
-	
+
 	public int getParallelSubtaskIndex() {
 		return this.subTaskIndex;
 	}
-	
+
 	public int getNumberOfInputs() {
 		return this.inputEdges.length;
 	}
-	
+
 	public ExecutionEdge[] getInputEdges(int input) {
 		if (input < 0 || input >= this.inputEdges.length) {
 			throw new IllegalArgumentException(String.format("Input %d is out of range [0..%d)", input, this.inputEdges.length));
 		}
 		return inputEdges[input];
 	}
-	
+
 	public CoLocationConstraint getLocationConstraint() {
 		return locationConstraint;
 	}
-	
+
 	public Execution getCurrentExecutionAttempt() {
 		return currentExecution;
 	}
-	
+
 	public ExecutionState getExecutionState() {
 		return currentExecution.getState();
 	}
-	
+
 	public long getStateTimestamp(ExecutionState state) {
 		return currentExecution.getStateTimestamp(state);
 	}
-	
+
 	public Throwable getFailureCause() {
 		return currentExecution.getFailureCause();
 	}
-	
+
 	public SimpleSlot getCurrentAssignedResource() {
 		return currentExecution.getAssignedResource();
 	}
-	
+
 	public InstanceConnectionInfo getCurrentAssignedResourceLocation() {
 		return currentExecution.getAssignedResourceLocation();
 	}
@@ -213,14 +218,14 @@ public class ExecutionVertex implements Serializable {
 	// --------------------------------------------------------------------------------------------
 	//  Graph building
 	// --------------------------------------------------------------------------------------------
-	
+
 	public void connectSource(int inputNumber, IntermediateResult source, JobEdge edge, int consumerNumber) {
-		
+
 		final DistributionPattern pattern = edge.getDistributionPattern();
 		final IntermediateResultPartition[] sourcePartitions = source.getPartitions();
-		
+
 		ExecutionEdge[] edges;
-		
+
 		switch (pattern) {
 			case POINTWISE:
 				edges = connectPointwise(sourcePartitions, inputNumber);
@@ -229,14 +234,14 @@ public class ExecutionVertex implements Serializable {
 			case ALL_TO_ALL:
 				edges = connectAllToAll(sourcePartitions, inputNumber);
 				break;
-				
+
 			default:
 				throw new RuntimeException("Unrecognized distribution pattern.");
-		
+
 		}
-		
+
 		this.inputEdges[inputNumber] = edges;
-		
+
 		// add the consumers to the source
 		// for now (until the receiver initiated handshake is in place), we need to register the 
 		// edges as the execution graph
@@ -244,22 +249,22 @@ public class ExecutionVertex implements Serializable {
 			ee.getSource().addConsumer(ee, consumerNumber);
 		}
 	}
-	
+
 	private ExecutionEdge[] connectAllToAll(IntermediateResultPartition[] sourcePartitions, int inputNumber) {
 		ExecutionEdge[] edges = new ExecutionEdge[sourcePartitions.length];
-		
+
 		for (int i = 0; i < sourcePartitions.length; i++) {
 			IntermediateResultPartition irp = sourcePartitions[i];
 			edges[i] = new ExecutionEdge(irp, this, inputNumber);
 		}
-		
+
 		return edges;
 	}
-	
+
 	private ExecutionEdge[] connectPointwise(IntermediateResultPartition[] sourcePartitions, int inputNumber) {
 		final int numSources = sourcePartitions.length;
 		final int parallelism = getTotalNumberOfParallelSubtasks();
-	
+
 		// simple case same number of sources as targets
 		if (numSources == parallelism) {
 			return new ExecutionEdge[] { new ExecutionEdge(sourcePartitions[subTaskIndex], this, inputNumber) };
@@ -300,7 +305,7 @@ public class ExecutionVertex implements Serializable {
 
 				int start = (int) (subTaskIndex * factor);
 				int end = (subTaskIndex == getTotalNumberOfParallelSubtasks() - 1) ?
-						sourcePartitions.length : 
+						sourcePartitions.length :
 						(int) ((subTaskIndex + 1) * factor);
 
 				ExecutionEdge[] edges = new ExecutionEdge[end - start];
@@ -312,29 +317,29 @@ public class ExecutionVertex implements Serializable {
 			}
 		}
 	}
-	
+
 	public void setLocationConstraintHosts(List<Instance> instances) {
 		this.locationConstraintInstances = instances;
 	}
-	
+
 	public void setScheduleLocalOnly(boolean scheduleLocalOnly) {
 		if (scheduleLocalOnly && inputEdges != null && inputEdges.length > 0) {
 			throw new IllegalArgumentException("Strictly local scheduling is only supported for sources.");
 		}
-		
+
 		this.scheduleLocalOnly = scheduleLocalOnly;
 	}
 
 	public boolean isScheduleLocalOnly() {
 		return scheduleLocalOnly;
 	}
-	
+
 	/**
 	 * Gets the location preferences of this task, determined by the locations of the predecessors from which
 	 * it receives input data.
 	 * If there are more than MAX_DISTINCT_LOCATIONS_TO_CONSIDER different locations of source data, this
 	 * method returns {@code null} to indicate no location preference.
-	 * 
+	 *
 	 * @return The preferred locations for this vertex execution, or null, if there is no preference.
 	 */
 	public Iterable<Instance> getPreferredLocations() {
@@ -343,13 +348,12 @@ public class ExecutionVertex implements Serializable {
 		if (constraintInstances != null && !constraintInstances.isEmpty()) {
 			return constraintInstances;
 		}
-		
+
 		// otherwise, base the preferred locations on the input connections
 		if (inputEdges == null) {
 			return Collections.emptySet();
 		}
 		else {
-
 			Set<Instance> locations = new HashSet<Instance>();
 			Set<Instance> inputLocations = new HashSet<Instance>();
 
@@ -398,11 +402,11 @@ public class ExecutionVertex implements Serializable {
 			Execution execution = currentExecution;
 			ExecutionState state = execution.getState();
 
-			if (state == FINISHED || state == CANCELED || state ==FAILED) {
+			if (state == FINISHED || state == CANCELED || state == FAILED) {
 				priorExecutions.add(execution);
 				currentExecution = new Execution(this, execution.getAttemptNumber()+1,
 						System.currentTimeMillis(), timeout);
-				
+
 				CoLocationGroup grp = jobVertex.getCoLocationGroup();
 				if (grp != null) {
 					this.locationConstraint = grp.getLocationConstraint(subTaskIndex);
@@ -436,17 +440,33 @@ public class ExecutionVertex implements Serializable {
 	}
 
 	/**
-	 * Schedules or updates the {@link IntermediateResultPartition} consumer
-	 * tasks of the intermediate result partition with the given index.
+	 * Schedules or updates the consumer tasks of the result partition with the given ID.
 	 */
-	void scheduleOrUpdateConsumers(int partitionIndex) {
-		checkElementIndex(partitionIndex, resultPartitions.length);
+	void scheduleOrUpdateConsumers(ResultPartitionID partitionId) {
+
+		final Execution execution = currentExecution;
 
-		IntermediateResultPartition partition = resultPartitions[partitionIndex];
+		// Abort this request if there was a concurrent reset
+		if (!partitionId.getProducerId().equals(execution.getAttemptId())) {
+			return;
+		}
+
+		final IntermediateResultPartition partition = resultPartitions.get(partitionId.getPartitionId());
+
+		if (partition == null) {
+			throw new IllegalStateException("Unknown partition " + partitionId + ".");
+		}
 
-		currentExecution.scheduleOrUpdateConsumers(partition.getConsumers());
+		if (partition.getIntermediateResult().getResultType().isPipelined()) {
+			// Schedule or update receivers of this partition
+			execution.scheduleOrUpdateConsumers(partition.getConsumers());
+		}
+		else {
+			throw new IllegalArgumentException("ScheduleOrUpdateConsumers msg is only valid for" +
+					"pipelined partitions.");
+		}
 	}
-	
+
 	/**
 	 * This method cleans fields that are irrelevant for the archived execution attempt.
 	 */
@@ -458,22 +478,22 @@ public class ExecutionVertex implements Serializable {
 		if (!(state == FINISHED || state == CANCELED || state == FAILED)) {
 			throw new IllegalStateException("Cannot archive ExecutionVertex that is not in a finished state.");
 		}
-		
+
 		// prepare the current execution for archiving
 		execution.prepareForArchiving();
-		
+
 		// prepare previous executions for archiving
 		for (Execution exec : priorExecutions) {
 			exec.prepareForArchiving();
 		}
-		
+
 		// clear the unnecessary fields in this class
 		this.resultPartitions = null;
 		this.inputEdges = null;
 		this.locationConstraintInstances = null;
 	}
 
-	public void cachePartitionInfo(PartialPartitionInfo partitionInfo){
+	public void cachePartitionInfo(PartialInputChannelDeploymentDescriptor partitionInfo){
 		getCurrentExecutionAttempt().cachePartitionInfo(partitionInfo);
 	}
 
@@ -481,6 +501,25 @@ public class ExecutionVertex implements Serializable {
 		currentExecution.sendPartitionInfos();
 	}
 
+	/**
+	 * Returns whether to schedule the next batch of receiving tasks.
+	 */
+	boolean finishAllBlockingPartitions() {
+		for (IntermediateResultPartition partition : resultPartitions.values()) {
+			// Nothing to do for pipelined results
+			if (partition.getResultType().isPipelined()) {
+				return false;
+			}
+			// It's a blocking partition, mark it as finished and return whether all blocking
+			// partitions have been produced.
+			else if (partition.markFinished()) {
+				return true;
+			}
+		}
+
+		return false;
+	}
+
 	// --------------------------------------------------------------------------------------------
 	//   Notifications from the Execution Attempt
 	// --------------------------------------------------------------------------------------------
@@ -507,20 +546,27 @@ public class ExecutionVertex implements Serializable {
 	void notifyStateTransition(ExecutionAttemptID executionId, ExecutionState newState, Throwable error) {
 		getExecutionGraph().notifyExecutionChange(getJobvertexId(), subTaskIndex, executionId, newState, error);
 	}
-	
-	TaskDeploymentDescriptor createDeploymentDescriptor(ExecutionAttemptID executionId, SimpleSlot slot) {
+
+	/**
+	 * Creates a task deployment descriptor to deploy a subtask to the given target slot.
+	 */
+	TaskDeploymentDescriptor createDeploymentDescriptor(
+			ExecutionAttemptID executionId,
+			SimpleSlot targetSlot) {
+
 		// Produced intermediate results
-		List<PartitionDeploymentDescriptor> producedPartitions = new ArrayList<PartitionDeploymentDescriptor>(resultPartitions.length);
+		List<ResultPartitionDeploymentDescriptor> producedPartitions = new ArrayList<ResultPartitionDeploymentDescriptor>(resultPartitions.size());
 
-		for (IntermediateResultPartition partition : resultPartitions) {
-			producedPartitions.add(PartitionDeploymentDescriptor.fromIntermediateResultPartition(partition));
+		for (IntermediateResultPartition partition : resultPartitions.values()) {
+			producedPartitions.add(ResultPartitionDeploymentDescriptor.from(partition));
 		}
 
 		// Consumed intermediate results
-		List<PartitionConsumerDeploymentDescriptor> consumedPartitions = new ArrayList<PartitionConsumerDeploymentDescriptor>();
+		List<InputGateDeploymentDescriptor> consumedPartitions = new ArrayList<InputGateDeploymentDescriptor>();
 
 		for (ExecutionEdge[] edges : inputEdges) {
-			PartitionInfo[] partitions = PartitionInfo.fromEdges(edges, slot);
+			InputChannelDeploymentDescriptor[] partitions = InputChannelDeploymentDescriptor
+					.fromEdges(edges, targetSlot);
 
 			// If the produced partition has multiple consumers registered, we
 			// need to request the one matching our sub task index.
@@ -531,7 +577,7 @@ public class ExecutionVertex implements Serializable {
 
 			IntermediateDataSetID resultId = edges[0].getSource().getIntermediateResult().getId();
 
-			consumedPartitions.add(new PartitionConsumerDeploymentDescriptor(resultId, partitions, queueToRequest));
+			consumedPartitions.add(new InputGateDeploymentDescriptor(resultId, queueToRequest, partitions));
 		}
 
 		List<BlobKey> jarFiles = getExecutionGraph().getRequiredJarFiles();
@@ -539,7 +585,7 @@ public class ExecutionVertex implements Serializable {
 		return new TaskDeploymentDescriptor(getJobId(), getJobvertexId(), executionId, getTaskName(),
 				subTaskIndex, getTotalNumberOfParallelSubtasks(), getExecutionGraph().getJobConfiguration(),
 				jobVertex.getJobVertex().getConfiguration(), jobVertex.getJobVertex().getInvokableClassName(),
-				producedPartitions, consumedPartitions, jarFiles, slot.getSlotNumber(), operatorState);
+				producedPartitions, consumedPartitions, jarFiles, targetSlot.getSlotNumber(), operatorState);
 	}
 
 	// --------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResult.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResult.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResult.java
index 64ad2d2..658a06b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResult.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResult.java
@@ -18,8 +18,13 @@
 
 package org.apache.flink.runtime.executiongraph;
 
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
-import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionType;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
 
 public class IntermediateResult {
 
@@ -31,20 +36,30 @@ public class IntermediateResult {
 
 	private final int numParallelProducers;
 
+	private final AtomicInteger numberOfRunningProducers;
+
 	private int partitionsAssigned;
 
 	private int numConsumers;
 
 	private final int connectionIndex;
 
-	private final IntermediateResultPartitionType resultType;
+	private final ResultPartitionType resultType;
+
+	public IntermediateResult(
+			IntermediateDataSetID id,
+			ExecutionJobVertex producer,
+			int numParallelProducers,
+			ResultPartitionType resultType) {
 
-	public IntermediateResult(IntermediateDataSetID id, ExecutionJobVertex producer, int numParallelProducers) {
-		this.id = id;
-		this.producer = producer;
+		this.id = checkNotNull(id);
+		this.producer = checkNotNull(producer);
 		this.partitions = new IntermediateResultPartition[numParallelProducers];
+		checkArgument(numParallelProducers >= 1);
 		this.numParallelProducers = numParallelProducers;
 
+		this.numberOfRunningProducers = new AtomicInteger(numParallelProducers);
+
 		// we do not set the intermediate result partitions here, because we let them be initialized by
 		// the execution vertex that produces them
 
@@ -52,8 +67,7 @@ public class IntermediateResult {
 		this.connectionIndex = (int) (Math.random() * Integer.MAX_VALUE);
 
 		// The runtime type for this produced result
-		// TODO The JobGraph generator has to decide which type of result this is
-		this.resultType = IntermediateResultPartitionType.PIPELINED;
+		this.resultType = checkNotNull(resultType);
 	}
 
 	public void setPartition(int partitionNumber, IntermediateResultPartition partition) {
@@ -85,7 +99,7 @@ public class IntermediateResult {
 		return partitionsAssigned;
 	}
 
-	public IntermediateResultPartitionType getResultType() {
+	public ResultPartitionType getResultType() {
 		return resultType;
 	}
 
@@ -104,4 +118,26 @@ public class IntermediateResult {
 	public int getConnectionIndex() {
 		return connectionIndex;
 	}
+
+	void resetForNewExecution() {
+		this.numberOfRunningProducers.set(numParallelProducers);
+	}
+
+	int decrementNumberOfRunningProducersAndGetRemaining() {
+		return numberOfRunningProducers.decrementAndGet();
+	}
+
+	boolean isConsumable() {
+		if (resultType.isPipelined()) {
+			return true;
+		}
+		else {
+			return numberOfRunningProducers.get() == 0;
+		}
+	}
+
+	@Override
+	public String toString() {
+		return "IntermediateResult " + id.toString();
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java
index 7d06dca..124ceb2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.runtime.executiongraph;
 
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
 import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
 
 import java.util.ArrayList;
@@ -59,10 +60,18 @@ public class IntermediateResultPartition {
 		return partitionId;
 	}
 
+	ResultPartitionType getResultType() {
+		return totalResult.getResultType();
+	}
+
 	public List<List<ExecutionEdge>> getConsumers() {
 		return consumers;
 	}
 
+	public boolean isConsumable() {
+		return totalResult.isConsumable();
+	}
+
 	int addConsumerGroup() {
 		int pos = consumers.size();
 
@@ -78,4 +87,24 @@ public class IntermediateResultPartition {
 	void addConsumer(ExecutionEdge edge, int consumerNumber) {
 		consumers.get(consumerNumber).add(edge);
 	}
+
+	boolean markFinished() {
+		// Sanity check that this is only called on blocking partitions.
+		if (!getResultType().isBlocking()) {
+			throw new IllegalStateException("Tried to mark a non-blocking result partition as finished");
+		}
+
+		final int refCnt = totalResult.decrementNumberOfRunningProducersAndGetRemaining();
+
+		if (refCnt == 0) {
+			return true;
+		}
+		else if (refCnt  < 0) {
+			throw new IllegalStateException("Decremented number of unfinished producers below 0. "
+					+ "This is most likely a bug in the execution state/intermediate result "
+					+ "partition management.");
+		}
+
+		return false;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/ChannelReaderInputViewIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/ChannelReaderInputViewIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/ChannelReaderInputViewIterator.java
index 6007db9..87a3baa 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/ChannelReaderInputViewIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/ChannelReaderInputViewIterator.java
@@ -61,7 +61,7 @@ public class ChannelReaderInputViewIterator<E> implements MutableObjectIterator<
 			segments, freeMemTarget, accessors, numBlocks);
 	}
 		
-	public ChannelReaderInputViewIterator(BlockChannelReader reader, LinkedBlockingQueue<MemorySegment> returnQueue,
+	public ChannelReaderInputViewIterator(BlockChannelReader<MemorySegment> reader, LinkedBlockingQueue<MemorySegment> returnQueue,
 			List<MemorySegment> segments, List<MemorySegment> freeMemTarget, TypeSerializer<E> accessors, int numBlocks)
 	throws IOException
 	{

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelInputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelInputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelInputView.java
index 9fb8072..736c245 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelInputView.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelInputView.java
@@ -39,7 +39,7 @@ import org.apache.flink.runtime.util.MathUtils;
  */
 public class FileChannelInputView extends AbstractPagedInputView {
 	
-	private final BlockChannelReader reader;
+	private final BlockChannelReader<MemorySegment> reader;
 	
 	private final MemoryManager memManager;
 	
@@ -53,7 +53,7 @@ public class FileChannelInputView extends AbstractPagedInputView {
 	
 	// --------------------------------------------------------------------------------------------
 	
-	public FileChannelInputView(BlockChannelReader reader, MemoryManager memManager, List<MemorySegment> memory, int sizeOfLastBlock) throws IOException {
+	public FileChannelInputView(BlockChannelReader<MemorySegment> reader, MemoryManager memManager, List<MemorySegment> memory, int sizeOfLastBlock) throws IOException {
 		super(0);
 		
 		checkNotNull(reader);
@@ -129,7 +129,7 @@ public class FileChannelInputView extends AbstractPagedInputView {
 		
 		// get the next segment
 		numBlocksRemaining--;
-		return reader.getNextReturnedSegment();
+		return reader.getNextReturnedBlock();
 	}
 	
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelOutputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelOutputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelOutputView.java
index e04759c..b6c500f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelOutputView.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelOutputView.java
@@ -35,7 +35,7 @@ import org.apache.flink.runtime.memorymanager.MemoryManager;
  */
 public class FileChannelOutputView extends AbstractPagedOutputView {
 	
-	private final BlockChannelWriter writer;		// the writer to the channel
+	private final BlockChannelWriter<MemorySegment> writer;		// the writer to the channel
 	
 	private final MemoryManager memManager;
 	
@@ -47,7 +47,7 @@ public class FileChannelOutputView extends AbstractPagedOutputView {
 	
 	// --------------------------------------------------------------------------------------------
 	
-	public FileChannelOutputView(BlockChannelWriter writer, MemoryManager memManager, List<MemorySegment> memory, int segmentSize) throws IOException {
+	public FileChannelOutputView(BlockChannelWriter<MemorySegment> writer, MemoryManager memManager, List<MemorySegment> memory, int segmentSize) throws IOException {
 		super(segmentSize, 0);
 		
 		checkNotNull(writer);
@@ -137,7 +137,7 @@ public class FileChannelOutputView extends AbstractPagedOutputView {
 		if (current != null) {
 			writeSegment(current, posInSegment);
 		}
-		return writer.getNextReturnedSegment();
+		return writer.getNextReturnedBlock();
 	}
 	
 	private void writeSegment(MemorySegment segment, int writePosition) throws IOException {

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputView.java
index 6098fdb..7d8d485 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputView.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputView.java
@@ -40,7 +40,7 @@ import org.apache.flink.runtime.util.MathUtils;
  */
 public class SeekableFileChannelInputView extends AbstractPagedInputView {
 	
-	private BlockChannelReader reader;
+	private BlockChannelReader<MemorySegment> reader;
 	
 	private final IOManager ioManager;
 	
@@ -127,7 +127,7 @@ public class SeekableFileChannelInputView extends AbstractPagedInputView {
 		}
 		
 		numBlocksRemaining--;
-		seekInput(reader.getNextReturnedSegment(), positionInBlock, numBlocksRemaining == 0 ? sizeOfLastBlock : segmentSize);
+		seekInput(reader.getNextReturnedBlock(), positionInBlock, numBlocksRemaining == 0 ? sizeOfLastBlock : segmentSize);
 	}
 	
 	public void close() throws IOException {
@@ -169,7 +169,7 @@ public class SeekableFileChannelInputView extends AbstractPagedInputView {
 		
 		// get the next segment
 		numBlocksRemaining--;
-		return reader.getNextReturnedSegment();
+		return reader.getNextReturnedBlock();
 	}
 	
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SpillingBuffer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SpillingBuffer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SpillingBuffer.java
index 655a574..5f9c2cf 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SpillingBuffer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SpillingBuffer.java
@@ -42,7 +42,7 @@ public class SpillingBuffer extends AbstractPagedOutputView {
 	
 	private final MemorySegmentSource memorySource;
 	
-	private BlockChannelWriter writer;
+	private BlockChannelWriter<MemorySegment> writer;
 	
 	private RandomAccessInputView inMemInView;
 	
@@ -86,7 +86,7 @@ public class SpillingBuffer extends AbstractPagedOutputView {
 					this.writer.writeBlock(this.fullSegments.get(i));
 				}
 				this.fullSegments.clear();
-				final MemorySegment seg = this.writer.getNextReturnedSegment();
+				final MemorySegment seg = this.writer.getNextReturnedBlock();
 				this.numMemorySegmentsInWriter--;
 				return seg;
 			}
@@ -94,7 +94,7 @@ public class SpillingBuffer extends AbstractPagedOutputView {
 			// spilling
 			this.writer.writeBlock(current);
 			this.blockCount++;
-			return this.writer.getNextReturnedSegment();
+			return this.writer.getNextReturnedBlock();
 		}
 	}
 	
@@ -116,7 +116,7 @@ public class SpillingBuffer extends AbstractPagedOutputView {
 				this.blockCount++;
 				this.writer.close();
 				for (int i = this.numMemorySegmentsInWriter; i > 0; i--) {
-					this.fullSegments.add(this.writer.getNextReturnedSegment());
+					this.fullSegments.add(this.writer.getNextReturnedBlock());
 				}
 				this.numMemorySegmentsInWriter = 0;
 			}
@@ -135,7 +135,7 @@ public class SpillingBuffer extends AbstractPagedOutputView {
 				this.externalInView.close();
 			}
 			
-			final BlockChannelReader reader = this.ioManager.createBlockChannelReader(this.writer.getChannelID());
+			final BlockChannelReader<MemorySegment> reader = this.ioManager.createBlockChannelReader(this.writer.getChannelID());
 			this.externalInView = new HeaderlessChannelReaderInputView(reader, this.fullSegments, this.blockCount, this.numBytesInLastSegment, false);
 			return this.externalInView;
 		}
@@ -161,7 +161,7 @@ public class SpillingBuffer extends AbstractPagedOutputView {
 			// closing before the first flip, collect the memory in the writer
 			this.writer.close();
 			for (int i = this.numMemorySegmentsInWriter; i > 0; i--) {
-				segments.add(this.writer.getNextReturnedSegment());
+				segments.add(this.writer.getNextReturnedBlock());
 			}
 			this.writer.closeAndDelete();
 			this.writer = null;

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AbstractFileIOChannel.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AbstractFileIOChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AbstractFileIOChannel.java
index 3991167..e79439f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AbstractFileIOChannel.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AbstractFileIOChannel.java
@@ -80,7 +80,7 @@ public abstract class AbstractFileIOChannel implements FileIOChannel {
 	
 	@Override
 	public abstract void close() throws IOException;
-	
+
 	@Override
 	public void deleteChannel() {
 		if (!isClosed() || this.fileChannel.isOpen()) {
@@ -104,4 +104,9 @@ public abstract class AbstractFileIOChannel implements FileIOChannel {
 			deleteChannel();
 		}
 	}
+
+	@Override
+	public FileChannel getNioFileChannel() {
+		return fileChannel;
+	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBlockReader.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBlockReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBlockReader.java
index acfa71f..7a80b7c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBlockReader.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBlockReader.java
@@ -18,12 +18,12 @@
 
 package org.apache.flink.runtime.io.disk.iomanager;
 
+import org.apache.flink.core.memory.MemorySegment;
+
 import java.io.IOException;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.flink.core.memory.MemorySegment;
-
 /**
  * A reader that reads data in blocks from a file channel. The reader reads the blocks into a 
  * {@link org.apache.flink.core.memory.MemorySegment} in an asynchronous fashion. That is, a read
@@ -40,7 +40,7 @@ import org.apache.flink.core.memory.MemorySegment;
  * or even whether the file was written in blocks of the same size, or in blocks at all. Ensuring that the
  * writing and reading is consistent with each other (same blocks sizes) is up to the programmer.  
  */
-public class AsynchronousBlockReader extends AsynchronousFileIOChannel<MemorySegment, ReadRequest> implements BlockChannelReader {
+public class AsynchronousBlockReader extends AsynchronousFileIOChannel<MemorySegment, ReadRequest> implements BlockChannelReader<MemorySegment> {
 	
 	private final LinkedBlockingQueue<MemorySegment> returnSegments;
 	
@@ -57,7 +57,7 @@ public class AsynchronousBlockReader extends AsynchronousFileIOChannel<MemorySeg
 			LinkedBlockingQueue<MemorySegment> returnSegments)
 	throws IOException
 	{
-		super(channelID, requestQueue, new QueuingCallback(returnSegments), false);
+		super(channelID, requestQueue, new QueuingCallback<MemorySegment>(returnSegments), false);
 		this.returnSegments = returnSegments;
 	}	
 
@@ -74,7 +74,12 @@ public class AsynchronousBlockReader extends AsynchronousFileIOChannel<MemorySeg
 	public void readBlock(MemorySegment segment) throws IOException {
 		addRequest(new SegmentReadRequest(this, segment));
 	}
-	
+
+	@Override
+	public void seekToPosition(long position) throws IOException {
+		requestQueue.add(new SeekRequest(this, position));
+	}
+
 	/**
 	 * Gets the next memory segment that has been filled with data by the reader. This method blocks until
 	 * such a segment is available, or until an error occurs in the reader, or the reader is closed.
@@ -87,7 +92,7 @@ public class AsynchronousBlockReader extends AsynchronousFileIOChannel<MemorySeg
 	 * @throws IOException Thrown, if an I/O error occurs in the reader while waiting for the request to return.
 	 */
 	@Override
-	public MemorySegment getNextReturnedSegment() throws IOException {
+	public MemorySegment getNextReturnedBlock() throws IOException {
 		try {
 			while (true) {
 				final MemorySegment next = this.returnSegments.poll(1000, TimeUnit.MILLISECONDS);
@@ -115,9 +120,4 @@ public class AsynchronousBlockReader extends AsynchronousFileIOChannel<MemorySeg
 	public LinkedBlockingQueue<MemorySegment> getReturnQueue() {
 		return this.returnSegments;
 	}
-	
-	@Override
-	public void seekToPosition(long position) throws IOException {
-		this.requestQueue.add(new SeekRequest(this, position));
-	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBlockWriter.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBlockWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBlockWriter.java
index 7e1681f..18d16a0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBlockWriter.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBlockWriter.java
@@ -24,7 +24,7 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.flink.core.memory.MemorySegment;
 
-public class AsynchronousBlockWriter extends AsynchronousBlockWriterWithCallback implements BlockChannelWriter {
+public class AsynchronousBlockWriter extends AsynchronousBlockWriterWithCallback implements BlockChannelWriter<MemorySegment> {
 	
 	private final LinkedBlockingQueue<MemorySegment> returnSegments;
 	
@@ -41,7 +41,7 @@ public class AsynchronousBlockWriter extends AsynchronousBlockWriterWithCallback
 			LinkedBlockingQueue<MemorySegment> returnSegments)
 	throws IOException
 	{
-		super(channelID, requestQueue, new QueuingCallback(returnSegments));
+		super(channelID, requestQueue, new QueuingCallback<MemorySegment>(returnSegments));
 		this.returnSegments = returnSegments;
 	}
 	
@@ -58,7 +58,7 @@ public class AsynchronousBlockWriter extends AsynchronousBlockWriterWithCallback
 	 * @throws IOException Thrown, if an I/O error occurs in the writer while waiting for the request to return.
 	 */
 	@Override
-	public MemorySegment getNextReturnedSegment() throws IOException {
+	public MemorySegment getNextReturnedBlock() throws IOException {
 		try {
 			while (true) {
 				final MemorySegment next = returnSegments.poll(1000, TimeUnit.MILLISECONDS);

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBlockWriterWithCallback.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBlockWriterWithCallback.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBlockWriterWithCallback.java
index c9fbdd2..e5dab47 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBlockWriterWithCallback.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBlockWriterWithCallback.java
@@ -26,7 +26,7 @@ import org.apache.flink.core.memory.MemorySegment;
  * An asynchronous implementation of the {@link BlockChannelWriterWithCallback} that queues I/O requests
  * and calls a callback once they have been handled.
  */
-public class AsynchronousBlockWriterWithCallback extends AsynchronousFileIOChannel<MemorySegment, WriteRequest> implements BlockChannelWriterWithCallback {
+public class AsynchronousBlockWriterWithCallback extends AsynchronousFileIOChannel<MemorySegment, WriteRequest> implements BlockChannelWriterWithCallback<MemorySegment> {
 	
 	/**
 	 * Creates a new asynchronous block writer for the given channel.

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileReader.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileReader.java
new file mode 100644
index 0000000..ba5f0ec
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileReader.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.disk.iomanager;
+
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class AsynchronousBufferFileReader extends AsynchronousFileIOChannel<Buffer, ReadRequest> implements BufferFileReader {
+
+	private final AtomicBoolean hasReachedEndOfFile = new AtomicBoolean();
+
+	protected AsynchronousBufferFileReader(ID channelID, RequestQueue<ReadRequest> requestQueue, RequestDoneCallback<Buffer> callback) throws IOException {
+		super(channelID, requestQueue, callback, false);
+	}
+
+	@Override
+	public void readInto(Buffer buffer) throws IOException {
+		addRequest(new BufferReadRequest(this, buffer, hasReachedEndOfFile));
+	}
+
+	@Override
+	public void seekToPosition(long position) throws IOException {
+		requestQueue.add(new SeekRequest(this, position));
+	}
+
+	@Override
+	public boolean hasReachedEndOfFile() {
+		return hasReachedEndOfFile.get();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileSegmentReader.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileSegmentReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileSegmentReader.java
new file mode 100644
index 0000000..c2a277a
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileSegmentReader.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.disk.iomanager;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class AsynchronousBufferFileSegmentReader extends AsynchronousFileIOChannel<FileSegment, ReadRequest> implements BufferFileSegmentReader {
+
+	private final AtomicBoolean hasReachedEndOfFile = new AtomicBoolean();
+
+	protected AsynchronousBufferFileSegmentReader(ID channelID, RequestQueue<ReadRequest> requestQueue, RequestDoneCallback<FileSegment> callback) throws IOException {
+		super(channelID, requestQueue, callback, false);
+	}
+
+	@Override
+	public void read() throws IOException {
+		addRequest(new FileSegmentReadRequest(this, hasReachedEndOfFile));
+	}
+
+	@Override
+	public void seekTo(long position) throws IOException {
+		requestQueue.add(new SeekRequest(this, position));
+	}
+
+	@Override
+	public boolean hasReachedEndOfFile() {
+		return hasReachedEndOfFile.get();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9d7acf36/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriter.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriter.java
new file mode 100644
index 0000000..14bb8f7
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriter.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.disk.iomanager;
+
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.util.event.NotificationListener;
+
+import java.io.IOException;
+
+public class AsynchronousBufferFileWriter extends AsynchronousFileIOChannel<Buffer, WriteRequest> implements BufferFileWriter {
+
+	private static final RecyclingCallback CALLBACK = new RecyclingCallback();
+
+	protected AsynchronousBufferFileWriter(ID channelID, RequestQueue<WriteRequest> requestQueue) throws IOException {
+		super(channelID, requestQueue, CALLBACK, true);
+	}
+
+	@Override
+	public void writeBlock(Buffer buffer) throws IOException {
+		addRequest(new BufferWriteRequest(this, buffer));
+	}
+
+	@Override
+	public int getNumberOfOutstandingRequests() {
+		return requestsNotReturned.get();
+	}
+
+	@Override
+	public boolean registerAllRequestsProcessedListener(NotificationListener listener) throws IOException {
+		return super.registerAllRequestsProcessedListener(listener);
+	}
+
+	/**
+	 * Recycles the buffer after the I/O request.
+	 */
+	private static class RecyclingCallback implements RequestDoneCallback<Buffer> {
+
+		@Override
+		public void requestSuccessful(Buffer buffer) {
+			buffer.recycle();
+		}
+
+		@Override
+		public void requestFailed(Buffer buffer, IOException e) {
+			buffer.recycle();
+		}
+	}
+}