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 2016/12/02 08:42:35 UTC

[1/6] flink git commit: [FLINK-5169] [network] Fix spillable subpartition buffer count

Repository: flink
Updated Branches:
  refs/heads/master dbe707324 -> 2fcef5ecf


[FLINK-5169] [network] Fix spillable subpartition buffer count


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

Branch: refs/heads/master
Commit: 2fcef5ecf473f82a3894b136e48b53b81b465356
Parents: c0cdc5c
Author: Ufuk Celebi <uc...@apache.org>
Authored: Thu Dec 1 18:38:30 2016 +0100
Committer: Ufuk Celebi <uc...@apache.org>
Committed: Thu Dec 1 21:42:49 2016 +0100

----------------------------------------------------------------------
 .../netty/SequenceNumberingViewReader.java      |  10 ++
 .../partition/SpillableSubpartition.java        |   5 +
 .../partition/SpillableSubpartitionView.java    |  22 +++-
 .../partition/SpilledSubpartitionView.java      |  13 +-
 .../partition/SpillableSubpartitionTest.java    | 130 +++++++++++++++++++
 5 files changed, 178 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/2fcef5ec/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/SequenceNumberingViewReader.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/SequenceNumberingViewReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/SequenceNumberingViewReader.java
index ef611eb..5036bb7 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/SequenceNumberingViewReader.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/SequenceNumberingViewReader.java
@@ -127,4 +127,14 @@ class SequenceNumberingViewReader implements BufferAvailabilityListener {
 			requestQueue.notifyReaderNonEmpty(this);
 		}
 	}
+
+	@Override
+	public String toString() {
+		return "SequenceNumberingViewReader{" +
+			"requestLock=" + requestLock +
+			", receiverId=" + receiverId +
+			", numBuffersAvailable=" + numBuffersAvailable.get() +
+			", sequenceNumber=" + sequenceNumber +
+			'}';
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/2fcef5ec/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
index 439e08d..ad04e97 100644
--- 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
@@ -91,6 +91,11 @@ class SpillableSubpartition extends ResultSubpartition {
 				return false;
 			}
 
+			// The number of buffers are needed later when creating
+			// the read views. If you ever remove this line here,
+			// make sure to still count the number of buffers.
+			updateStatistics(buffer);
+
 			if (spillWriter == null) {
 				buffers.add(buffer);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/2fcef5ec/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
index 8119ecc..533f95b 100644
--- 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
@@ -21,6 +21,8 @@ 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.network.buffer.Buffer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.util.ArrayDeque;
@@ -30,6 +32,8 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
 
 class SpillableSubpartitionView implements ResultSubpartitionView {
 
+	private static final Logger LOG = LoggerFactory.getLogger(SpillableSubpartitionView.class);
+
 	/** The subpartition this view belongs to. */
 	private final SpillableSubpartition parent;
 
@@ -51,6 +55,9 @@ class SpillableSubpartitionView implements ResultSubpartitionView {
 
 	private final AtomicBoolean isReleased = new AtomicBoolean(false);
 
+	/** Remember the number of buffers this view was created with. */
+	private final long numBuffers;
+
 	/**
 	 * The next buffer to hand out. Everytime this is set to a non-null value,
 	 * a listener notification happens.
@@ -73,6 +80,7 @@ class SpillableSubpartitionView implements ResultSubpartitionView {
 		this.listener = checkNotNull(listener);
 
 		synchronized (buffers) {
+			numBuffers = buffers.size();
 			nextBuffer = buffers.poll();
 		}
 
@@ -94,9 +102,12 @@ class SpillableSubpartitionView implements ResultSubpartitionView {
 				// Create the spill writer and write all buffers to disk
 				BufferFileWriter spillWriter = ioManager.createBufferFileWriter(ioManager.createChannel());
 
+				long spilledBytes = 0;
+
 				int numBuffers = buffers.size();
 				for (int i = 0; i < numBuffers; i++) {
 					Buffer buffer = buffers.remove();
+					spilledBytes += buffer.getSize();
 					try {
 						spillWriter.writeBlock(buffer);
 					} finally {
@@ -111,6 +122,11 @@ class SpillableSubpartitionView implements ResultSubpartitionView {
 					numBuffers,
 					listener);
 
+				LOG.debug("Spilling {} bytes for sub partition {} of {}.",
+					spilledBytes,
+					parent.index,
+					parent.parent.getPartitionId());
+
 				return numBuffers;
 			}
 		}
@@ -188,8 +204,12 @@ class SpillableSubpartitionView implements ResultSubpartitionView {
 
 	@Override
 	public String toString() {
-		return String.format("SpillableSubpartitionView(index: %d) of ResultPartition %s",
+		boolean hasSpilled = spilledView != null;
+
+		return String.format("SpillableSubpartitionView(index: %d, buffers: %d, spilled? {}) of ResultPartition %s",
 			parent.index,
+			numBuffers,
+			hasSpilled,
 			parent.parent.getPartitionId());
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/2fcef5ec/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionView.java
index b087a4e..7488132 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionView.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionView.java
@@ -26,6 +26,8 @@ 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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.util.ArrayDeque;
@@ -46,6 +48,8 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  */
 class SpilledSubpartitionView implements ResultSubpartitionView, NotificationListener {
 
+	private static final Logger LOG = LoggerFactory.getLogger(SpilledSubpartitionView.class);
+
 	/** The subpartition this view belongs to. */
 	private final ResultSubpartition parent;
 
@@ -91,6 +95,9 @@ class SpilledSubpartitionView implements ResultSubpartitionView, NotificationLis
 		if (!spillWriter.registerAllRequestsProcessedListener(this)) {
 			isSpillInProgress = false;
 			availabilityListener.notifyBuffersAvailable(numberOfSpilledBuffers);
+			LOG.debug("No spilling in progress. Notified about {} available buffers.", numberOfSpilledBuffers);
+		} else {
+			LOG.debug("Spilling in progress. Waiting with notification about {} available buffers.", numberOfSpilledBuffers);
 		}
 	}
 
@@ -103,6 +110,7 @@ class SpilledSubpartitionView implements ResultSubpartitionView, NotificationLis
 	public void onNotification() {
 		isSpillInProgress = false;
 		availabilityListener.notifyBuffersAvailable(numberOfSpilledBuffers);
+		LOG.debug("Finished spilling. Notified about {} available buffers.", numberOfSpilledBuffers);
 	}
 
 	@Override
@@ -158,7 +166,10 @@ class SpilledSubpartitionView implements ResultSubpartitionView, NotificationLis
 
 	@Override
 	public String toString() {
-		return String.format("SpilledSubpartitionView[sync](index: %d) of ResultPartition %s", parent.index, parent.parent.getPartitionId());
+		return String.format("SpilledSubpartitionView(index: %d, buffers: {}) of ResultPartition %s",
+			parent.index,
+			numberOfSpilledBuffers,
+			parent.parent.getPartitionId());
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/2fcef5ec/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
index b7a54d7..b53ef68 100644
--- 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
@@ -18,11 +18,15 @@
 
 package org.apache.flink.runtime.io.network.partition;
 
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.io.disk.iomanager.AsynchronousBufferFileWriter;
 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.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.Buffer;
+import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
 import org.apache.flink.runtime.io.network.util.TestInfiniteBufferProvider;
 import org.junit.AfterClass;
 import org.junit.Test;
@@ -36,12 +40,16 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.doAnswer;
 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;
 
 public class SpillableSubpartitionTest extends SubpartitionTestBase {
@@ -153,4 +161,126 @@ public class SpillableSubpartitionTest extends SubpartitionTestBase {
 
 		assertNull(readView.getNextBuffer());
 	}
+
+	/**
+	 * Tests that a spilled partition is correctly read back in via a spilled
+	 * read view.
+	 */
+	@Test
+	public void testConsumeSpilledPartition() throws Exception {
+		ResultPartition parent = mock(ResultPartition.class);
+		SpillableSubpartition partition = new SpillableSubpartition(
+			0,
+			parent,
+			ioManager);
+
+		Buffer buffer = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(4096), FreeingBufferRecycler.INSTANCE);
+		buffer.retain();
+		buffer.retain();
+
+		partition.add(buffer);
+		partition.add(buffer);
+		partition.add(buffer);
+
+		assertEquals(3, partition.releaseMemory());
+
+		partition.finish();
+
+		BufferAvailabilityListener listener = mock(BufferAvailabilityListener.class);
+		SpilledSubpartitionView reader = (SpilledSubpartitionView) partition.createReadView(new TestInfiniteBufferProvider(), listener);
+
+		verify(listener, times(1)).notifyBuffersAvailable(eq(4L));
+
+		Buffer read = reader.getNextBuffer();
+		assertNotNull(read);
+		read.recycle();
+
+		read = reader.getNextBuffer();
+		assertNotNull(read);
+		read.recycle();
+
+		read = reader.getNextBuffer();
+		assertNotNull(read);
+		read.recycle();
+
+		// End of partition
+		read = reader.getNextBuffer();
+		assertNotNull(read);
+		assertEquals(EndOfPartitionEvent.class, EventSerializer.fromBuffer(read, ClassLoader.getSystemClassLoader()).getClass());
+		read.recycle();
+	}
+
+	/**
+	 * Tests that a spilled partition is correctly read back in via a spilled
+	 * read view.
+	 */
+	@Test
+	public void testConsumeSpillablePartitionSpilledDuringConsume() throws Exception {
+		ResultPartition parent = mock(ResultPartition.class);
+		SpillableSubpartition partition = new SpillableSubpartition(
+			0,
+			parent,
+			ioManager);
+
+		Buffer buffer = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(4096), FreeingBufferRecycler.INSTANCE);
+		buffer.retain();
+		buffer.retain();
+
+		partition.add(buffer);
+		partition.add(buffer);
+		partition.add(buffer);
+		partition.finish();
+
+		AwaitableBufferAvailablityListener listener = new AwaitableBufferAvailablityListener();
+		SpillableSubpartitionView reader = (SpillableSubpartitionView) partition.createReadView(new TestInfiniteBufferProvider(), listener);
+
+		// Initial notification
+		assertEquals(1, listener.getNumNotifiedBuffers());
+
+		Buffer read = reader.getNextBuffer();
+		assertNotNull(read);
+		read.recycle();
+		assertEquals(2, listener.getNumNotifiedBuffers());
+
+		// Spill now
+		assertEquals(2, partition.releaseMemory());
+
+		listener.awaitNotifications(4, 30_000);
+		assertEquals(4, listener.getNumNotifiedBuffers());
+
+		read = reader.getNextBuffer();
+		assertNotNull(read);
+		read.recycle();
+
+		read = reader.getNextBuffer();
+		assertNotNull(read);
+		read.recycle();
+
+		// End of partition
+		read = reader.getNextBuffer();
+		assertNotNull(read);
+		assertEquals(EndOfPartitionEvent.class, EventSerializer.fromBuffer(read, ClassLoader.getSystemClassLoader()).getClass());
+		read.recycle();
+	}
+
+	private static class AwaitableBufferAvailablityListener implements BufferAvailabilityListener {
+
+		private long numNotifiedBuffers;
+
+		@Override
+		public void notifyBuffersAvailable(long numBuffers) {
+			numNotifiedBuffers += numBuffers;
+		}
+
+		long getNumNotifiedBuffers() {
+			return numNotifiedBuffers;
+		}
+
+		void awaitNotifications(long awaitedNumNotifiedBuffers, long timeoutMillis) throws InterruptedException {
+			long deadline = System.currentTimeMillis() + timeoutMillis;
+			while (numNotifiedBuffers < awaitedNumNotifiedBuffers && System.currentTimeMillis() < deadline) {
+				Thread.sleep(1);
+			}
+		}
+	}
 }


[5/6] flink git commit: [FLINK-5169] [network] Make consumption of InputChannels fair

Posted by uc...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/f728129b/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
deleted file mode 100644
index c86697f..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewSyncIO.java
+++ /dev/null
@@ -1,196 +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 org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.core.memory.MemorySegmentFactory;
-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 org.apache.flink.util.Preconditions.checkArgument;
-import static org.apache.flink.util.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();
-
-	/** Spilled file size */
-	private final long fileSize;
-
-	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);
-		}
-
-		this.fileSize = fileReader.getSize();
-	}
-
-	@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 parent.isReleased() || isReleased.get();
-	}
-
-	@Override
-	public Throwable getFailureCause() {
-		return parent.getFailureCause();
-	}
-
-	@Override
-	public String toString() {
-		return String.format("SpilledSubpartitionView[sync](index: %d, file size: %d bytes) of ResultPartition %s",
-				parent.index,
-				fileSize,
-				parent.parent.getPartitionId());
-	}
-
-	/**
-	 * 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(MemorySegmentFactory.allocateUnpooledSegment(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/f728129b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/BufferOrEvent.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/BufferOrEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/BufferOrEvent.java
index 885e738..3e93ae6 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/BufferOrEvent.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/BufferOrEvent.java
@@ -34,18 +34,35 @@ public class BufferOrEvent {
 
 	private final AbstractEvent event;
 
+	/**
+	 * Indicate availability of further instances for the union input gate.
+	 * This is not needed outside of the input gate unioning logic and cannot
+	 * be set outside of the consumer package.
+	 */
+	private final boolean moreAvailable;
+
 	private int channelIndex;
 
-	public BufferOrEvent(Buffer buffer, int channelIndex) {
+	BufferOrEvent(Buffer buffer, int channelIndex, boolean moreAvailable) {
 		this.buffer = checkNotNull(buffer);
 		this.event = null;
 		this.channelIndex = channelIndex;
+		this.moreAvailable = moreAvailable;
 	}
 
-	public BufferOrEvent(AbstractEvent event, int channelIndex) {
+	BufferOrEvent(AbstractEvent event, int channelIndex, boolean moreAvailable) {
 		this.buffer = null;
 		this.event = checkNotNull(event);
 		this.channelIndex = channelIndex;
+		this.moreAvailable = moreAvailable;
+	}
+
+	public BufferOrEvent(Buffer buffer, int channelIndex) {
+		this(buffer, channelIndex, true);
+	}
+
+	public BufferOrEvent(AbstractEvent event, int channelIndex) {
+		this(event, channelIndex, true);
 	}
 
 	public boolean isBuffer() {
@@ -73,6 +90,10 @@ public class BufferOrEvent {
 		this.channelIndex = channelIndex;
 	}
 
+	boolean moreAvailable() {
+		return moreAvailable;
+	}
+
 	@Override
 	public String toString() {
 		return String.format("BufferOrEvent [%s, channelIndex = %d]",

http://git-wip-us.apache.org/repos/asf/flink/blob/f728129b/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 35094e2..f46abfd 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
@@ -101,10 +101,19 @@ public abstract class InputChannel {
 	}
 
 	/**
-	 * Notifies the owning {@link SingleInputGate} about an available {@link Buffer} instance.
+	 * Notifies the owning {@link SingleInputGate} that this channel became non-empty.
+	 * 
+	 * <p>This is guaranteed to be called only when a Buffer was added to a previously
+	 * empty input channel. The notion of empty is atomically consistent with the flag
+	 * {@link BufferAndAvailability#moreAvailable()} when polling the next buffer
+	 * from this channel.
+	 * 
+	 * <p><b>Note:</b> When the input channel observes an exception, this
+	 * method is called regardless of whether the channel was empty before. That ensures
+	 * that the parent InputGate will always be notified about the exception.
 	 */
-	protected void notifyAvailableBuffer() {
-		inputGate.onAvailableBuffer(this);
+	protected void notifyChannelNonEmpty() {
+		inputGate.notifyChannelNonEmpty(this);
 	}
 
 	// ------------------------------------------------------------------------
@@ -123,7 +132,7 @@ public abstract class InputChannel {
 	/**
 	 * Returns the next buffer from the consumed subpartition.
 	 */
-	abstract Buffer getNextBuffer() throws IOException, InterruptedException;
+	abstract BufferAndAvailability getNextBuffer() throws IOException, InterruptedException;
 
 	// ------------------------------------------------------------------------
 	// Task events
@@ -182,7 +191,7 @@ public abstract class InputChannel {
 	protected void setError(Throwable cause) {
 		if (this.cause.compareAndSet(null, checkNotNull(cause))) {
 			// Notify the input gate.
-			notifyAvailableBuffer();
+			notifyChannelNonEmpty();
 		}
 	}
 
@@ -225,4 +234,28 @@ public abstract class InputChannel {
 		// Reached maximum backoff
 		return false;
 	}
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * A combination of a {@link Buffer} and a flag indicating availability of further buffers.
+	 */
+	public static final class BufferAndAvailability {
+
+		private final Buffer buffer;
+		private final boolean moreAvailable;
+
+		public BufferAndAvailability(Buffer buffer, boolean moreAvailable) {
+			this.buffer = checkNotNull(buffer);
+			this.moreAvailable = moreAvailable;
+		}
+
+		public Buffer buffer() {
+			return buffer;
+		}
+
+		public boolean moreAvailable() {
+			return moreAvailable;
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f728129b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGate.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGate.java
index 1cd5fc5..1f2182e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGate.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGate.java
@@ -19,7 +19,6 @@
 package org.apache.flink.runtime.io.network.partition.consumer;
 
 import org.apache.flink.runtime.event.TaskEvent;
-import org.apache.flink.runtime.util.event.EventListener;
 
 import java.io.IOException;
 
@@ -77,7 +76,7 @@ public interface InputGate {
 
 	void sendTaskEvent(TaskEvent event) throws IOException;
 
-	void registerListener(EventListener<InputGate> listener);
+	void registerListener(InputGateListener listener);
 
 	int getPageSize();
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f728129b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGateListener.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGateListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGateListener.java
new file mode 100644
index 0000000..00fa782
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGateListener.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition.consumer;
+
+/**
+ * Listener interface implemented by consumers of {@link InputGate} instances
+ * that want to be notified of availability of buffer or event instances.
+ */
+public interface InputGateListener {
+
+	/**
+	 * Notification callback if the input gate moves from zero to non-zero
+	 * available input channels with data.
+	 *
+	 * @param inputGate Input Gate that became available.
+	 */
+	void notifyInputGateNonEmpty(InputGate inputGate);
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f728129b/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 55ff539..d5308a8 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
@@ -18,24 +18,23 @@
 
 package org.apache.flink.runtime.io.network.partition.consumer;
 
-import org.apache.flink.runtime.metrics.groups.TaskIOMetricGroup;
 import org.apache.flink.runtime.event.TaskEvent;
 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.BufferAvailabilityListener;
 import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException;
 import org.apache.flink.runtime.io.network.partition.ProducerFailedException;
 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.apache.flink.runtime.metrics.groups.TaskIOMetricGroup;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.util.Timer;
 import java.util.TimerTask;
+import java.util.concurrent.atomic.AtomicLong;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 import static org.apache.flink.util.Preconditions.checkState;
@@ -43,11 +42,13 @@ import static org.apache.flink.util.Preconditions.checkState;
 /**
  * An input channel, which requests a local subpartition.
  */
-public class LocalInputChannel extends InputChannel implements NotificationListener {
+public class LocalInputChannel extends InputChannel implements BufferAvailabilityListener {
 
 	private static final Logger LOG = LoggerFactory.getLogger(LocalInputChannel.class);
 
-	private final Object requestLock = new Object();
+	// ------------------------------------------------------------------------
+
+	private final Object requestReleaseLock = new Object();
 
 	/** The local partition manager. */
 	private final ResultPartitionManager partitionManager;
@@ -55,39 +56,41 @@ public class LocalInputChannel extends InputChannel implements NotificationListe
 	/** Task event dispatcher for backwards events. */
 	private final TaskEventDispatcher taskEventDispatcher;
 
+	/** Number of available buffers used to keep track of non-empty gate notifications. */
+	private final AtomicLong numBuffersAvailable;
+
 	/** The consumed subpartition */
 	private volatile ResultSubpartitionView subpartitionView;
 
 	private volatile boolean isReleased;
 
-	private volatile Buffer lookAhead;
-
 	LocalInputChannel(
-			SingleInputGate inputGate,
-			int channelIndex,
-			ResultPartitionID partitionId,
-			ResultPartitionManager partitionManager,
-			TaskEventDispatcher taskEventDispatcher,
-			TaskIOMetricGroup metrics) {
+		SingleInputGate inputGate,
+		int channelIndex,
+		ResultPartitionID partitionId,
+		ResultPartitionManager partitionManager,
+		TaskEventDispatcher taskEventDispatcher,
+		TaskIOMetricGroup metrics) {
 
 		this(inputGate, channelIndex, partitionId, partitionManager, taskEventDispatcher,
-				0, 0, metrics);
+			0, 0, metrics);
 	}
 
 	LocalInputChannel(
-			SingleInputGate inputGate,
-			int channelIndex,
-			ResultPartitionID partitionId,
-			ResultPartitionManager partitionManager,
-			TaskEventDispatcher taskEventDispatcher,
-			int initialBackoff,
-			int maxBackoff,
-			TaskIOMetricGroup metrics) {
+		SingleInputGate inputGate,
+		int channelIndex,
+		ResultPartitionID partitionId,
+		ResultPartitionManager partitionManager,
+		TaskEventDispatcher taskEventDispatcher,
+		int initialBackoff,
+		int maxBackoff,
+		TaskIOMetricGroup metrics) {
 
 		super(inputGate, channelIndex, partitionId, initialBackoff, maxBackoff, metrics.getNumBytesInLocalCounter());
 
 		this.partitionManager = checkNotNull(partitionManager);
 		this.taskEventDispatcher = checkNotNull(taskEventDispatcher);
+		this.numBuffersAvailable = new AtomicLong();
 	}
 
 	// ------------------------------------------------------------------------
@@ -97,30 +100,36 @@ public class LocalInputChannel extends InputChannel implements NotificationListe
 	@Override
 	void requestSubpartition(int subpartitionIndex) throws IOException, InterruptedException {
 		// The lock is required to request only once in the presence of retriggered requests.
-		synchronized (requestLock) {
+		synchronized (requestReleaseLock) {
+			checkState(!isReleased, "released");
+
 			if (subpartitionView == null) {
 				LOG.debug("{}: Requesting LOCAL subpartition {} of partition {}.",
-						this, subpartitionIndex, partitionId);
+					this, subpartitionIndex, partitionId);
 
 				try {
-					subpartitionView = partitionManager.createSubpartitionView(
-							partitionId, subpartitionIndex, inputGate.getBufferProvider());
-				}
-				catch (PartitionNotFoundException notFound) {
+					ResultSubpartitionView subpartitionView = partitionManager.createSubpartitionView(
+						partitionId, subpartitionIndex, inputGate.getBufferProvider(), this);
+
+					if (subpartitionView == null) {
+						throw new IOException("Error requesting subpartition.");
+					}
+
+					// make the subpartition view visible
+					this.subpartitionView = subpartitionView;
+
+					// check if the channel was released in the meantime
+					if (isReleased) {
+						subpartitionView.releaseAllResources();
+						this.subpartitionView = null;
+					}
+				} catch (PartitionNotFoundException notFound) {
 					if (increaseBackoff()) {
 						inputGate.retriggerPartitionRequest(partitionId.getPartitionId());
-						return;
-					}
-					else {
+					} else {
 						throw notFound;
 					}
 				}
-
-				if (subpartitionView == null) {
-					throw new IOException("Error requesting subpartition.");
-				}
-
-				getNextLookAhead();
 			}
 		}
 	}
@@ -128,17 +137,16 @@ public class LocalInputChannel extends InputChannel implements NotificationListe
 	/**
 	 * Retriggers a subpartition request.
 	 */
-	void retriggerSubpartitionRequest(Timer timer, final int subpartitionIndex) throws IOException, InterruptedException {
-		synchronized (requestLock) {
-			checkState(subpartitionView == null, "Already requested partition.");
+	void retriggerSubpartitionRequest(Timer timer, final int subpartitionIndex) {
+		synchronized (requestReleaseLock) {
+			checkState(subpartitionView == null, "already requested partition");
 
 			timer.schedule(new TimerTask() {
 				@Override
 				public void run() {
 					try {
 						requestSubpartition(subpartitionIndex);
-					}
-					catch (Throwable t) {
+					} catch (Throwable t) {
 						setError(t);
 					}
 				}
@@ -147,29 +155,49 @@ public class LocalInputChannel extends InputChannel implements NotificationListe
 	}
 
 	@Override
-	Buffer getNextBuffer() throws IOException, InterruptedException {
+	BufferAndAvailability getNextBuffer() throws IOException, InterruptedException {
 		checkError();
-		checkState(subpartitionView != null, "Queried for a buffer before requesting the subpartition.");
 
-		// After subscribe notification
-		if (lookAhead == null) {
-			lookAhead = subpartitionView.getNextBuffer();
+		ResultSubpartitionView subpartitionView = this.subpartitionView;
+		if (subpartitionView == null) {
+			// this can happen if the request for the partition was triggered asynchronously
+			// by the time trigger
+			// would be good to avoid that, by guaranteeing that the requestPartition() and
+			// getNextBuffer() always come from the same thread
+			// we could do that by letting the timer insert a special "requesting channel" into the input gate's queue
+			subpartitionView = checkAndWaitForSubpartitionView();
 		}
 
-		Buffer next = lookAhead;
-		lookAhead = null;
+		Buffer next = subpartitionView.getNextBuffer();
+		long remaining = numBuffersAvailable.decrementAndGet();
 
-		if (!next.isBuffer() && EventSerializer
-				.fromBuffer(next, getClass().getClassLoader())
-				.getClass() == EndOfPartitionEvent.class) {
-
-				return next;
+		if (remaining >= 0) {
+			numBytesIn.inc(next.getSize());
+			return new BufferAndAvailability(next, remaining > 0);
+		} else if (subpartitionView.isReleased()) {
+			throw new ProducerFailedException(subpartitionView.getFailureCause());
+		} else {
+			throw new IllegalStateException("No buffer available and producer partition not released.");
 		}
+	}
 
-		getNextLookAhead();
+	@Override
+	public void notifyBuffersAvailable(long numBuffers) {
+		// if this request made the channel non-empty, notify the input gate
+		if (numBuffers > 0 && numBuffersAvailable.getAndAdd(numBuffers) == 0) {
+			notifyChannelNonEmpty();
+		}
+	}
 
-		numBytesIn.inc(next.getSize());
-		return next;
+	private ResultSubpartitionView checkAndWaitForSubpartitionView() {
+		// synchronizing on the request lock means this blocks until the asynchronous request
+		// for the partition view has been completed
+		// by then the subpartition view is visible or the channel is released
+		synchronized (requestReleaseLock) {
+			checkState(!isReleased, "released");
+			checkState(subpartitionView != null, "Queried for a buffer before requesting the subpartition.");
+			return subpartitionView;
+		}
 	}
 
 	// ------------------------------------------------------------------------
@@ -208,18 +236,15 @@ public class LocalInputChannel extends InputChannel implements NotificationListe
 	 */
 	@Override
 	void releaseAllResources() throws IOException {
-		if (!isReleased) {
-			if (lookAhead != null) {
-				lookAhead.recycle();
-				lookAhead = null;
-			}
+		synchronized (requestReleaseLock) {
+			if (!isReleased) {
+				isReleased = true;
 
-			if (subpartitionView != null) {
-				subpartitionView.releaseAllResources();
-				subpartitionView = null;
+				if (subpartitionView != null) {
+					subpartitionView.releaseAllResources();
+					subpartitionView = null;
+				}
 			}
-
-			isReleased = true;
 		}
 	}
 
@@ -227,55 +252,4 @@ public class LocalInputChannel extends InputChannel implements NotificationListe
 	public String toString() {
 		return "LocalInputChannel [" + partitionId + "]";
 	}
-
-	// ------------------------------------------------------------------------
-	// Queue iterator listener (called by producing or disk I/O thread)
-	// ------------------------------------------------------------------------
-
-	@Override
-	public void onNotification() {
-		if (isReleased) {
-			return;
-		}
-
-		try {
-			getNextLookAhead();
-		}
-		catch (Exception e) {
-			throw new RuntimeException(e);
-		}
-	}
-
-	// ------------------------------------------------------------------------
-
-	private void getNextLookAhead() throws IOException, InterruptedException {
-
-		final ResultSubpartitionView view = subpartitionView;
-
-		if (view == null) {
-			return;
-		}
-
-		while (true) {
-			lookAhead = view.getNextBuffer();
-
-			if (lookAhead != null) {
-				notifyAvailableBuffer();
-				break;
-			}
-
-			if (view.registerListener(this)) {
-				return;
-			}
-			else if (view.isReleased()) {
-				Throwable cause = view.getFailureCause();
-
-				if (cause != null) {
-					setError(new ProducerFailedException(cause));
-				}
-
-				return;
-			}
-		}
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f728129b/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 13a71a9..ed3122e 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
@@ -18,7 +18,6 @@
 
 package org.apache.flink.runtime.io.network.partition.consumer;
 
-import org.apache.flink.runtime.metrics.groups.TaskIOMetricGroup;
 import org.apache.flink.runtime.event.TaskEvent;
 import org.apache.flink.runtime.io.network.ConnectionID;
 import org.apache.flink.runtime.io.network.ConnectionManager;
@@ -27,8 +26,7 @@ import org.apache.flink.runtime.io.network.buffer.BufferProvider;
 import org.apache.flink.runtime.io.network.netty.PartitionRequestClient;
 import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.flink.runtime.metrics.groups.TaskIOMetricGroup;
 
 import java.io.IOException;
 import java.util.ArrayDeque;
@@ -43,8 +41,6 @@ import static org.apache.flink.util.Preconditions.checkState;
  */
 public class RemoteInputChannel extends InputChannel {
 
-	private static final Logger LOG = LoggerFactory.getLogger(RemoteInputChannel.class);
-
 	/** ID to distinguish this channel from other channels sharing the same TCP connection. */
 	private final InputChannelID id = new InputChannelID();
 
@@ -58,7 +54,7 @@ public class RemoteInputChannel extends InputChannel {
 	 * The received buffers. Received buffers are enqueued by the network I/O thread and the queue
 	 * is consumed by the receiving task thread.
 	 */
-	private final Queue<Buffer> receivedBuffers = new ArrayDeque<Buffer>();
+	private final Queue<Buffer> receivedBuffers = new ArrayDeque<>();
 
 	/**
 	 * Flag indicating whether this channel has been released. Either called by the receiving task
@@ -76,28 +72,27 @@ public class RemoteInputChannel extends InputChannel {
 	private int expectedSequenceNumber = 0;
 
 	public RemoteInputChannel(
-			SingleInputGate inputGate,
-			int channelIndex,
-			ResultPartitionID partitionId,
-			ConnectionID connectionId,
-			ConnectionManager connectionManager,
-			TaskIOMetricGroup metrics) {
-
-		this(inputGate, channelIndex, partitionId, connectionId, connectionManager,
-				0, 0, metrics);
+		SingleInputGate inputGate,
+		int channelIndex,
+		ResultPartitionID partitionId,
+		ConnectionID connectionId,
+		ConnectionManager connectionManager,
+		TaskIOMetricGroup metrics) {
+
+		this(inputGate, channelIndex, partitionId, connectionId, connectionManager, 0, 0, metrics);
 	}
 
 	public RemoteInputChannel(
-			SingleInputGate inputGate,
-			int channelIndex,
-			ResultPartitionID partitionId,
-			ConnectionID connectionId,
-			ConnectionManager connectionManager,
-			int initialBackoff,
-			int maxBackoff,
-			TaskIOMetricGroup metrics) {
+		SingleInputGate inputGate,
+		int channelIndex,
+		ResultPartitionID partitionId,
+		ConnectionID connectionId,
+		ConnectionManager connectionManager,
+		int initialBackOff,
+		int maxBackoff,
+		TaskIOMetricGroup metrics) {
 
-		super(inputGate, channelIndex, partitionId, initialBackoff, maxBackoff, metrics.getNumBytesInRemoteCounter());
+		super(inputGate, channelIndex, partitionId, initialBackOff, maxBackoff, metrics.getNumBytesInRemoteCounter());
 
 		this.connectionId = checkNotNull(connectionId);
 		this.connectionManager = checkNotNull(connectionManager);
@@ -115,7 +110,7 @@ public class RemoteInputChannel extends InputChannel {
 		if (partitionRequestClient == null) {
 			// Create a client and request the partition
 			partitionRequestClient = connectionManager
-					.createPartitionRequestClient(connectionId);
+				.createPartitionRequestClient(connectionId);
 
 			partitionRequestClient.requestSubpartition(partitionId, subpartitionIndex, this, 0);
 		}
@@ -129,31 +124,29 @@ public class RemoteInputChannel extends InputChannel {
 
 		if (increaseBackoff()) {
 			partitionRequestClient.requestSubpartition(
-					partitionId, subpartitionIndex, this, getCurrentBackoff());
-		}
-		else {
+				partitionId, subpartitionIndex, this, getCurrentBackoff());
+		} else {
 			failPartitionRequest();
 		}
 	}
 
 	@Override
-	Buffer getNextBuffer() throws IOException {
+	BufferAndAvailability getNextBuffer() throws IOException {
 		checkState(!isReleased.get(), "Queried for a buffer after channel has been closed.");
 		checkState(partitionRequestClient != null, "Queried for a buffer before requesting a queue.");
 
 		checkError();
 
-		synchronized (receivedBuffers) {
-			Buffer buffer = receivedBuffers.poll();
-
-			// Sanity check that channel is only queried after a notification
-			if (buffer == null) {
-				throw new IOException("Queried input channel for data although non is available.");
-			}
+		final Buffer next;
+		final int remaining;
 
-			numBytesIn.inc(buffer.getSize());
-			return buffer;
+		synchronized (receivedBuffers) {
+			next = receivedBuffers.poll();
+			remaining = receivedBuffers.size();
 		}
+
+		numBytesIn.inc(next.getSize());
+		return new BufferAndAvailability(next, remaining > 0);
 	}
 
 	// ------------------------------------------------------------------------
@@ -201,14 +194,13 @@ public class RemoteInputChannel extends InputChannel {
 			// buffers received concurrently with closing are properly recycled.
 			if (partitionRequestClient != null) {
 				partitionRequestClient.close(this);
-			}
-			else {
+			} else {
 				connectionManager.closeOpenChannelConnections(connectionId);
 			}
 		}
 	}
 
-	public void failPartitionRequest() {
+	private void failPartitionRequest() {
 		setError(new PartitionNotFoundException(partitionId));
 	}
 
@@ -246,20 +238,22 @@ public class RemoteInputChannel extends InputChannel {
 			synchronized (receivedBuffers) {
 				if (!isReleased.get()) {
 					if (expectedSequenceNumber == sequenceNumber) {
+						int available = receivedBuffers.size();
+
 						receivedBuffers.add(buffer);
 						expectedSequenceNumber++;
 
-						notifyAvailableBuffer();
+						if (available == 0) {
+							notifyChannelNonEmpty();
+						}
 
 						success = true;
-					}
-					else {
+					} else {
 						onError(new BufferReorderingException(expectedSequenceNumber, sequenceNumber));
 					}
 				}
 			}
-		}
-		finally {
+		} finally {
 			if (!success) {
 				buffer.recycle();
 			}
@@ -271,8 +265,7 @@ public class RemoteInputChannel extends InputChannel {
 			if (!isReleased.get()) {
 				if (expectedSequenceNumber == sequenceNumber) {
 					expectedSequenceNumber++;
-				}
-				else {
+				} else {
 					onError(new BufferReorderingException(expectedSequenceNumber, sequenceNumber));
 				}
 			}
@@ -287,7 +280,7 @@ public class RemoteInputChannel extends InputChannel {
 		setError(cause);
 	}
 
-	public static class BufferReorderingException extends IOException {
+	private static class BufferReorderingException extends IOException {
 
 		private static final long serialVersionUID = -888282210356266816L;
 
@@ -295,7 +288,7 @@ public class RemoteInputChannel extends InputChannel {
 
 		private final int actualSequenceNumber;
 
-		public BufferReorderingException(int expectedSequenceNumber, int actualSequenceNumber) {
+		BufferReorderingException(int expectedSequenceNumber, int actualSequenceNumber) {
 			this.expectedSequenceNumber = expectedSequenceNumber;
 			this.actualSequenceNumber = actualSequenceNumber;
 		}
@@ -303,7 +296,7 @@ public class RemoteInputChannel extends InputChannel {
 		@Override
 		public String getMessage() {
 			return String.format("Buffer re-ordering: expected buffer with sequence number %d, but received %d.",
-					expectedSequenceNumber, actualSequenceNumber);
+				expectedSequenceNumber, actualSequenceNumber);
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f728129b/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 d7ed33c..bcbb2c4 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
@@ -21,8 +21,6 @@ package org.apache.flink.runtime.io.network.partition.consumer;
 import com.google.common.collect.Maps;
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.JobID;
-import org.apache.flink.runtime.metrics.groups.TaskIOMetricGroup;
-import org.apache.flink.runtime.taskmanager.TaskActions;
 import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor;
 import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor;
 import org.apache.flink.runtime.deployment.ResultPartitionLocation;
@@ -36,22 +34,22 @@ 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.io.network.partition.consumer.InputChannel.BufferAndAvailability;
 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;
+import org.apache.flink.runtime.metrics.groups.TaskIOMetricGroup;
+import org.apache.flink.runtime.taskmanager.TaskActions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.BitSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Timer;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
 
 import static org.apache.flink.util.Preconditions.checkArgument;
 import static org.apache.flink.util.Preconditions.checkNotNull;
@@ -136,7 +134,7 @@ public class SingleInputGate implements InputGate {
 	private final Map<IntermediateResultPartitionID, InputChannel> inputChannels;
 
 	/** Channels, which notified this input gate about available data. */
-	private final BlockingQueue<InputChannel> inputChannelsWithData = new LinkedBlockingQueue<InputChannel>();
+	private final ArrayDeque<InputChannel> inputChannelsWithData = new ArrayDeque<>();
 
 	private final BitSet channelsWithEndOfPartitionEvents;
 
@@ -158,9 +156,9 @@ public class SingleInputGate implements InputGate {
 	private volatile boolean isReleased;
 
 	/** Registered listener to forward buffer notifications to. */
-	private volatile EventListener<InputGate> registeredListener;
+	private volatile InputGateListener inputGateListener;
 
-	private final List<TaskEvent> pendingEvents = new ArrayList<TaskEvent>();
+	private final List<TaskEvent> pendingEvents = new ArrayList<>();
 
 	private int numberOfUninitializedChannels;
 
@@ -168,14 +166,14 @@ public class SingleInputGate implements InputGate {
 	private Timer retriggerLocalRequestTimer;
 
 	public SingleInputGate(
-			String owningTaskName,
-			JobID jobId,
-			ExecutionAttemptID executionId,
-			IntermediateDataSetID consumedResultId,
-			int consumedSubpartitionIndex,
-			int numberOfInputChannels,
-			TaskActions taskActions,
-			TaskIOMetricGroup metrics) {
+		String owningTaskName,
+		JobID jobId,
+		ExecutionAttemptID executionId,
+		IntermediateDataSetID consumedResultId,
+		int consumedSubpartitionIndex,
+		int numberOfInputChannels,
+		TaskActions taskActions,
+		TaskIOMetricGroup metrics) {
 
 		this.owningTaskName = checkNotNull(owningTaskName);
 		this.jobId = checkNotNull(jobId);
@@ -263,7 +261,7 @@ public class SingleInputGate implements InputGate {
 		this.bufferPool = checkNotNull(bufferPool);
 	}
 
-	public void setInputChannel(IntermediateResultPartitionID partitionId, InputChannel inputChannel) {
+	void setInputChannel(IntermediateResultPartitionID partitionId, InputChannel inputChannel) {
 		synchronized (requestLock) {
 			if (inputChannels.put(checkNotNull(partitionId), checkNotNull(inputChannel)) == null
 					&& inputChannel.getClass() == UnknownInputChannel.class) {
@@ -355,6 +353,7 @@ public class SingleInputGate implements InputGate {
 	}
 
 	public void releaseAllResources() throws IOException {
+		boolean released = false;
 		synchronized (requestLock) {
 			if (!isReleased) {
 				try {
@@ -381,9 +380,16 @@ public class SingleInputGate implements InputGate {
 				}
 				finally {
 					isReleased = true;
+					released = true;
 				}
 			}
 		}
+
+		if (released) {
+			synchronized (inputChannelsWithData) {
+				inputChannelsWithData.notifyAll();
+			}
+		}
 	}
 
 	@Override
@@ -429,32 +435,50 @@ public class SingleInputGate implements InputGate {
 
 	@Override
 	public BufferOrEvent getNextBufferOrEvent() throws IOException, InterruptedException {
-
 		if (hasReceivedAllEndOfPartitionEvents) {
 			return null;
 		}
 
+		if (isReleased) {
+			throw new IllegalStateException("Released");
+		}
+
 		requestPartitions();
 
-		InputChannel currentChannel = null;
-		while (currentChannel == null) {
-			if (isReleased) {
-				throw new IllegalStateException("Released");
+		InputChannel currentChannel;
+		boolean moreAvailable;
+
+		synchronized (inputChannelsWithData) {
+			while (inputChannelsWithData.size() == 0) {
+				if (isReleased) {
+					throw new IllegalStateException("Released");
+				}
+
+				inputChannelsWithData.wait();
 			}
 
-			currentChannel = inputChannelsWithData.poll(2, TimeUnit.SECONDS);
+			currentChannel = inputChannelsWithData.remove();
+			moreAvailable = inputChannelsWithData.size() > 0;
 		}
 
-		final Buffer buffer = currentChannel.getNextBuffer();
+		final BufferAndAvailability result = currentChannel.getNextBuffer();
 
 		// Sanity check that notifications only happen when data is available
-		if (buffer == null) {
+		if (result == null) {
 			throw new IllegalStateException("Bug in input gate/channel logic: input gate got " +
 					"notified by channel about available data, but none was available.");
 		}
 
+		// this channel was now removed from the non-empty channels queue
+		// we re-add it in case it has more data, because in that case no "non-empty" notification
+		// will come for that channel
+		if (result.moreAvailable()) {
+			queueChannel(currentChannel);
+		}
+
+		final Buffer buffer = result.buffer();
 		if (buffer.isBuffer()) {
-			return new BufferOrEvent(buffer, currentChannel.getChannelIndex());
+			return new BufferOrEvent(buffer, currentChannel.getChannelIndex(), moreAvailable);
 		}
 		else {
 			final AbstractEvent event = EventSerializer.fromBuffer(buffer, getClass().getClassLoader());
@@ -471,7 +495,7 @@ public class SingleInputGate implements InputGate {
 				currentChannel.releaseAllResources();
 			}
 
-			return new BufferOrEvent(event, currentChannel.getChannelIndex());
+			return new BufferOrEvent(event, currentChannel.getChannelIndex(), moreAvailable);
 		}
 	}
 
@@ -493,29 +517,45 @@ public class SingleInputGate implements InputGate {
 	// ------------------------------------------------------------------------
 
 	@Override
-	public void registerListener(EventListener<InputGate> listener) {
-		if (registeredListener == null) {
-			registeredListener = listener;
-		}
-		else {
+	public void registerListener(InputGateListener inputGateListener) {
+		if (this.inputGateListener == null) {
+			this.inputGateListener = inputGateListener;
+		} else {
 			throw new IllegalStateException("Multiple listeners");
 		}
 	}
 
-	public void onAvailableBuffer(InputChannel channel) {
-		inputChannelsWithData.add(channel);
-		EventListener<InputGate> listener = registeredListener;
-		if (listener != null) {
-			listener.onEvent(this);
-		}
+	void notifyChannelNonEmpty(InputChannel channel) {
+		queueChannel(checkNotNull(channel));
 	}
 
 	void triggerPartitionStateCheck(ResultPartitionID partitionId) {
 		taskActions.triggerPartitionStateCheck(
-				jobId,
-				executionId,
-				consumedResultId,
-				partitionId);
+			jobId,
+			executionId,
+			consumedResultId,
+			partitionId);
+	}
+
+	private void queueChannel(InputChannel channel) {
+		int availableChannels;
+
+		synchronized (inputChannelsWithData) {
+			availableChannels = inputChannelsWithData.size();
+
+			inputChannelsWithData.add(channel);
+
+			if (availableChannels == 0) {
+				inputChannelsWithData.notify();
+			}
+		}
+
+		if (availableChannels == 0) {
+			InputGateListener listener = inputGateListener;
+			if (listener != null) {
+				listener.notifyInputGateNonEmpty(this);
+			}
+		}
 	}
 
 	// ------------------------------------------------------------------------
@@ -531,13 +571,13 @@ public class SingleInputGate implements InputGate {
 	 * Creates an input gate and all of its input channels.
 	 */
 	public static SingleInputGate create(
-			String owningTaskName,
-			JobID jobId,
-			ExecutionAttemptID executionId,
-			InputGateDeploymentDescriptor igdd,
-			NetworkEnvironment networkEnvironment,
-			TaskActions taskActions,
-			TaskIOMetricGroup metrics) {
+		String owningTaskName,
+		JobID jobId,
+		ExecutionAttemptID executionId,
+		InputGateDeploymentDescriptor igdd,
+		NetworkEnvironment networkEnvironment,
+		TaskActions taskActions,
+		TaskIOMetricGroup metrics) {
 
 		final IntermediateDataSetID consumedResultId = checkNotNull(igdd.getConsumedResultId());
 
@@ -547,8 +587,8 @@ public class SingleInputGate implements InputGate {
 		final InputChannelDeploymentDescriptor[] icdd = checkNotNull(igdd.getInputChannelDeploymentDescriptors());
 
 		final SingleInputGate inputGate = new SingleInputGate(
-				owningTaskName, jobId, executionId, consumedResultId, consumedSubpartitionIndex,
-				icdd.length, taskActions, metrics);
+			owningTaskName, jobId, executionId, consumedResultId, consumedSubpartitionIndex,
+			icdd.length, taskActions, metrics);
 
 		// Create the input channels. There is one input channel for each consumed partition.
 		final InputChannel[] inputChannels = new InputChannel[icdd.length];

http://git-wip-us.apache.org/repos/asf/flink/blob/f728129b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
index b1b8911..e8ccbb4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
@@ -22,15 +22,11 @@ import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import org.apache.flink.runtime.event.TaskEvent;
 import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
-import org.apache.flink.runtime.util.event.EventListener;
 
 import java.io.IOException;
-import java.util.List;
+import java.util.ArrayDeque;
 import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.LinkedBlockingQueue;
 
 import static org.apache.flink.util.Preconditions.checkArgument;
 import static org.apache.flink.util.Preconditions.checkNotNull;
@@ -63,19 +59,22 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  *
  * It is possible to recursively union union input gates.
  */
-public class UnionInputGate implements InputGate {
+public class UnionInputGate implements InputGate, InputGateListener {
 
 	/** The input gates to union. */
 	private final InputGate[] inputGates;
 
 	private final Set<InputGate> inputGatesWithRemainingData;
 
-	/** Data availability listener across all unioned input gates. */
-	private final InputGateListener inputGateListener;
+	/** Gates, which notified this input gate about available data. */
+	private final ArrayDeque<InputGate> inputGatesWithData = new ArrayDeque<>();
 
 	/** The total number of input channels across all unioned input gates. */
 	private final int totalNumberOfInputChannels;
 
+	/** Registered listener to forward input gate notifications to. */
+	private volatile InputGateListener inputGateListener;
+
 	/**
 	 * A mapping from input gate to (logical) channel index offset. Valid channel indexes go from 0
 	 * (inclusive) to the total number of input channels (exclusive).
@@ -100,11 +99,12 @@ public class UnionInputGate implements InputGate {
 			inputGatesWithRemainingData.add(inputGate);
 
 			currentNumberOfInputChannels += inputGate.getNumberOfInputChannels();
+
+			// Register the union gate as a listener for all input gates
+			inputGate.registerListener(this);
 		}
 
 		this.totalNumberOfInputChannels = currentNumberOfInputChannels;
-
-		this.inputGateListener = new InputGateListener(inputGates, this);
 	}
 
 	/**
@@ -139,7 +139,6 @@ public class UnionInputGate implements InputGate {
 
 	@Override
 	public BufferOrEvent getNextBufferOrEvent() throws IOException, InterruptedException {
-
 		if (inputGatesWithRemainingData.isEmpty()) {
 			return null;
 		}
@@ -147,17 +146,31 @@ public class UnionInputGate implements InputGate {
 		// Make sure to request the partitions, if they have not been requested before.
 		requestPartitions();
 
-		final InputGate inputGate = inputGateListener.getNextInputGateToReadFrom();
+		final InputGate inputGate;
+		synchronized (inputGatesWithData) {
+			while (inputGatesWithData.size() == 0) {
+				inputGatesWithData.wait();
+			}
+
+			inputGate = inputGatesWithData.remove();
+		}
 
 		final BufferOrEvent bufferOrEvent = inputGate.getNextBufferOrEvent();
 
+		if (bufferOrEvent.moreAvailable()) {
+			// this buffer or event was now removed from the non-empty gates queue
+			// we re-add it in case it has more data, because in that case no "non-empty" notification
+			// will come for that gate
+			queueInputGate(inputGate);
+		}
+
 		if (bufferOrEvent.isEvent()
-				&& bufferOrEvent.getEvent().getClass() == EndOfPartitionEvent.class
-				&& inputGate.isFinished()) {
+			&& bufferOrEvent.getEvent().getClass() == EndOfPartitionEvent.class
+			&& inputGate.isFinished()) {
 
 			if (!inputGatesWithRemainingData.remove(inputGate)) {
 				throw new IllegalStateException("Couldn't find input gate in set of remaining " +
-						"input gates.");
+					"input gates.");
 			}
 		}
 
@@ -177,9 +190,12 @@ public class UnionInputGate implements InputGate {
 	}
 
 	@Override
-	public void registerListener(EventListener<InputGate> listener) {
-		// This method is called from the consuming task thread.
-		inputGateListener.registerListener(listener);
+	public void registerListener(InputGateListener listener) {
+		if (this.inputGateListener == null) {
+			this.inputGateListener = listener;
+		} else {
+			throw new IllegalStateException("Multiple listeners");
+		}
 	}
 
 	@Override
@@ -195,45 +211,29 @@ public class UnionInputGate implements InputGate {
 		return pageSize;
 	}
 
-	/**
-	 * Data availability listener at all unioned input gates.
-	 *
-	 * <p> The listener registers itself at each input gate and is notified for *each incoming
-	 * buffer* at one of the unioned input gates.
-	 */
-	private static class InputGateListener implements EventListener<InputGate> {
-
-		private final UnionInputGate unionInputGate;
-
-		private final BlockingQueue<InputGate> inputGatesWithData = new LinkedBlockingQueue<InputGate>();
+	@Override
+	public void notifyInputGateNonEmpty(InputGate inputGate) {
+		queueInputGate(checkNotNull(inputGate));
+	}
 
-		private final List<EventListener<InputGate>> registeredListeners = new CopyOnWriteArrayList<EventListener<InputGate>>();
+	private void queueInputGate(InputGate inputGate) {
+		int availableInputGates;
 
-		public InputGateListener(InputGate[] inputGates, UnionInputGate unionInputGate) {
-			for (InputGate inputGate : inputGates) {
-				inputGate.registerListener(this);
-			}
+		synchronized (inputGatesWithData) {
+			availableInputGates = inputGatesWithData.size();
 
-			this.unionInputGate = unionInputGate;
-		}
-
-		@Override
-		public void onEvent(InputGate inputGate) {
-			// This method is called from the input channel thread, which can be either the same
-			// thread as the consuming task thread or a different one.
 			inputGatesWithData.add(inputGate);
 
-			for (int i = 0; i < registeredListeners.size(); i++) {
-				registeredListeners.get(i).onEvent(unionInputGate);
+			if (availableInputGates == 0) {
+				inputGatesWithData.notify();
 			}
 		}
 
-		InputGate getNextInputGateToReadFrom() throws InterruptedException {
-			return inputGatesWithData.take();
-		}
-
-		public void registerListener(EventListener<InputGate> listener) {
-			registeredListeners.add(checkNotNull(listener));
+		if (availableInputGates == 0) {
+			InputGateListener listener = inputGateListener;
+			if (listener != null) {
+				listener.notifyInputGateNonEmpty(this);
+			}
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f728129b/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 08b5044..d887ab6 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
@@ -23,8 +23,6 @@ import org.apache.flink.runtime.event.TaskEvent;
 import org.apache.flink.runtime.io.network.ConnectionID;
 import org.apache.flink.runtime.io.network.ConnectionManager;
 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.ResultPartitionID;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionManager;
 
@@ -36,7 +34,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  * An input channel place holder to be replaced by either a {@link RemoteInputChannel}
  * or {@link LocalInputChannel} at runtime.
  */
-public class UnknownInputChannel extends InputChannel {
+class UnknownInputChannel extends InputChannel {
 
 	private final ResultPartitionManager partitionManager;
 
@@ -78,9 +76,9 @@ public class UnknownInputChannel extends InputChannel {
 	}
 
 	@Override
-	public Buffer getNextBuffer() throws IOException {
+	public BufferAndAvailability getNextBuffer() throws IOException {
 		// Nothing to do here
-		return null;
+		throw new UnsupportedOperationException("Cannot retrieve a buffer from an UnknownInputChannel");
 	}
 
 	@Override
@@ -93,8 +91,7 @@ public class UnknownInputChannel extends InputChannel {
 	 * <p>
 	 * <strong>Important</strong>: It is important that the method correctly
 	 * always <code>false</code> for unknown input channels in order to not
-	 * finish the consumption of an intermediate result partition early in
-	 * {@link BufferReader}.
+	 * finish the consumption of an intermediate result partition early.
 	 */
 	@Override
 	public boolean isReleased() {

http://git-wip-us.apache.org/repos/asf/flink/blob/f728129b/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 bd8c196..14ef1bf 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
@@ -347,7 +347,6 @@ public class Task implements Runnable, TaskActions {
 				networkEnvironment.getResultPartitionManager(),
 				resultPartitionConsumableNotifier,
 				ioManager,
-				networkEnvironment.getDefaultIOMode(),
 				desc.sendScheduleOrUpdateConsumersMessage());
 
 			writers[counter] = new ResultPartitionWriter(producedPartitions[counter]);


[6/6] flink git commit: [FLINK-5169] [network] Make consumption of InputChannels fair

Posted by uc...@apache.org.
[FLINK-5169] [network] Make consumption of InputChannels fair


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

Branch: refs/heads/master
Commit: f728129bdb8c3176fba03c3e74c65ed254146061
Parents: dbe7073
Author: Stephan Ewen <se...@apache.org>
Authored: Mon Nov 28 09:59:29 2016 +0100
Committer: Ufuk Celebi <uc...@apache.org>
Committed: Thu Dec 1 21:42:49 2016 +0100

----------------------------------------------------------------------
 .../io/network/api/reader/BufferReader.java     |  50 ---
 .../io/network/netty/PartitionRequestQueue.java | 255 +++++--------
 .../netty/PartitionRequestServerHandler.java    |  40 +-
 .../netty/SequenceNumberingViewReader.java      | 130 +++++++
 .../partition/BufferAvailabilityListener.java   |  33 ++
 .../partition/PipelinedSubpartition.java        | 148 ++++----
 .../partition/PipelinedSubpartitionView.java    |  18 +-
 .../io/network/partition/ResultPartition.java   |   9 +-
 .../partition/ResultPartitionManager.java       |   5 +-
 .../partition/ResultPartitionProvider.java      |   3 +-
 .../network/partition/ResultSubpartition.java   |   6 +-
 .../partition/ResultSubpartitionView.java       |   9 +-
 .../partition/SpillableSubpartition.java        | 179 ++++-----
 .../partition/SpillableSubpartitionView.java    | 210 ++++++-----
 .../partition/SpilledSubpartitionView.java      | 223 +++++++++++
 .../SpilledSubpartitionViewAsyncIO.java         | 377 -------------------
 .../SpilledSubpartitionViewSyncIO.java          | 196 ----------
 .../partition/consumer/BufferOrEvent.java       |  25 +-
 .../partition/consumer/InputChannel.java        |  43 ++-
 .../network/partition/consumer/InputGate.java   |   3 +-
 .../partition/consumer/InputGateListener.java   |  35 ++
 .../partition/consumer/LocalInputChannel.java   | 212 +++++------
 .../partition/consumer/RemoteInputChannel.java  |  95 +++--
 .../partition/consumer/SingleInputGate.java     | 144 ++++---
 .../partition/consumer/UnionInputGate.java      |  98 ++---
 .../partition/consumer/UnknownInputChannel.java |  11 +-
 .../apache/flink/runtime/taskmanager/Task.java  |   1 -
 27 files changed, 1175 insertions(+), 1383 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/f728129b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/BufferReader.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/BufferReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/BufferReader.java
deleted file mode 100644
index ca59609..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/BufferReader.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.io.network.api.reader;
-
-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 java.io.IOException;
-
-/**
- * A buffer-oriented reader.
- */
-public final class BufferReader extends AbstractReader {
-
-	public BufferReader(InputGate gate) {
-		super(gate);
-	}
-
-	public Buffer getNextBuffer() throws IOException, InterruptedException {
-		while (true) {
-			final BufferOrEvent bufferOrEvent = inputGate.getNextBufferOrEvent();
-
-			if (bufferOrEvent.isBuffer()) {
-				return bufferOrEvent.getBuffer();
-			}
-			else {
-				if (handleEvent(bufferOrEvent.getEvent())) {
-					return null;
-				}
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/f728129b/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 094c9c7..dc80675 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
@@ -27,10 +27,10 @@ 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.netty.NettyMessage.ErrorResponse;
 import org.apache.flink.runtime.io.network.partition.ProducerFailedException;
-import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView;
+import org.apache.flink.runtime.io.network.partition.consumer.InputChannel.BufferAndAvailability;
 import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
-import org.apache.flink.runtime.util.event.NotificationListener;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -39,11 +39,10 @@ import java.util.ArrayDeque;
 import java.util.Queue;
 import java.util.Set;
 
-import static org.apache.flink.util.Preconditions.checkNotNull;
 import static org.apache.flink.runtime.io.network.netty.NettyMessage.BufferResponse;
 
 /**
- * A queue of partition queues, which listens for channel writability changed
+ * A nonEmptyReader of partition queues, which listens for channel writability changed
  * events before writing and flushing {@link Buffer} instances.
  */
 class PartitionRequestQueue extends ChannelInboundHandlerAdapter {
@@ -52,12 +51,10 @@ class PartitionRequestQueue extends ChannelInboundHandlerAdapter {
 
 	private final ChannelFutureListener writeListener = new WriteAndFlushNextMessageIfPossibleListener();
 
-	private final Queue<SequenceNumberingSubpartitionView> queue = new ArrayDeque<SequenceNumberingSubpartitionView>();
+	private final Queue<SequenceNumberingViewReader> nonEmptyReader = new ArrayDeque<>();
 
 	private final Set<InputChannelID> released = Sets.newHashSet();
 
-	private SequenceNumberingSubpartitionView currentPartitionQueue;
-
 	private boolean fatalError;
 
 	private ChannelHandlerContext ctx;
@@ -71,8 +68,22 @@ class PartitionRequestQueue extends ChannelInboundHandlerAdapter {
 		super.channelRegistered(ctx);
 	}
 
-	public void enqueue(ResultSubpartitionView partitionQueue, InputChannelID receiverId) throws Exception {
-		ctx.pipeline().fireUserEventTriggered(new SequenceNumberingSubpartitionView(partitionQueue, receiverId));
+	void notifyReaderNonEmpty(final SequenceNumberingViewReader reader) {
+		// The notification might come from the same thread. For the initial writes this
+		// might happen before the reader has set its reference to the view, because
+		// creating the queue and the initial notification happen in the same method call.
+		// This can be resolved by separating the creation of the view and allowing
+		// notifications.
+
+		// TODO This could potentially have a bad performance impact as in the
+		// worst case (network consumes faster than the producer) each buffer
+		// will trigger a separate event loop task being scheduled.
+		ctx.executor().execute(new Runnable() {
+			@Override
+			public void run() {
+				ctx.pipeline().fireUserEventTriggered(reader);
+			}
+		});
 	}
 
 	public void cancel(InputChannelID receiverId) {
@@ -87,45 +98,37 @@ class PartitionRequestQueue extends ChannelInboundHandlerAdapter {
 
 	@Override
 	public void userEventTriggered(ChannelHandlerContext ctx, Object msg) throws Exception {
-		if (msg.getClass() == SequenceNumberingSubpartitionView.class) {
-			boolean triggerWrite = queue.isEmpty();
-
-			queue.add((SequenceNumberingSubpartitionView) msg);
-
+		// The user event triggered event loop callback is used for thread-safe
+		// hand over of reader queues and cancelled producers.
+
+		if (msg.getClass() == SequenceNumberingViewReader.class) {
+			// Queue a non-empty reader for consumption. If the queue
+			// is empty, we try trigger the actual write. Otherwise this
+			// will be handled by the writeAndFlushIfPossible calls.
+			boolean triggerWrite = nonEmptyReader.isEmpty();
+			nonEmptyReader.add((SequenceNumberingViewReader) msg);
 			if (triggerWrite) {
 				writeAndFlushNextMessageIfPossible(ctx.channel());
 			}
-		}
-		else if (msg.getClass() == InputChannelID.class) {
+		} else if (msg.getClass() == InputChannelID.class) {
+			// Release partition view that get a cancel request.
 			InputChannelID toCancel = (InputChannelID) msg;
-
 			if (released.contains(toCancel)) {
 				return;
 			}
 
 			// Cancel the request for the input channel
-			if (currentPartitionQueue != null && currentPartitionQueue.getReceiverId().equals(toCancel)) {
-				currentPartitionQueue.releaseAllResources();
-				markAsReleased(currentPartitionQueue.receiverId);
-				currentPartitionQueue = null;
-			}
-			else {
-				int size = queue.size();
-
-				for (int i = 0; i < size; i++) {
-					SequenceNumberingSubpartitionView curr = queue.poll();
-
-					if (curr.getReceiverId().equals(toCancel)) {
-						curr.releaseAllResources();
-						markAsReleased(curr.receiverId);
-					}
-					else {
-						queue.add(curr);
-					}
+			int size = nonEmptyReader.size();
+			for (int i = 0; i < size; i++) {
+				SequenceNumberingViewReader reader = nonEmptyReader.poll();
+				if (reader.getReceiverId().equals(toCancel)) {
+					reader.releaseAllResources();
+					markAsReleased(reader.getReceiverId());
+				} else {
+					nonEmptyReader.add(reader);
 				}
 			}
-		}
-		else {
+		} else {
 			ctx.fireUserEventTriggered(msg);
 		}
 	}
@@ -140,64 +143,84 @@ class PartitionRequestQueue extends ChannelInboundHandlerAdapter {
 			return;
 		}
 
-		Buffer buffer = null;
+		// The logic here is very similar to the combined input gate and local
+		// input channel logic. You can think of this class acting as the input
+		// gate and the consumed views as the local input channels.
 
+		BufferAndAvailability next = null;
 		try {
 			if (channel.isWritable()) {
 				while (true) {
-					if (currentPartitionQueue == null && (currentPartitionQueue = queue.poll()) == null) {
+					SequenceNumberingViewReader reader = nonEmptyReader.poll();
+
+					// No queue with available data. We allow this here, because
+					// of the write callbacks that are executed after each write.
+					if (reader == null) {
 						return;
 					}
 
-					buffer = currentPartitionQueue.getNextBuffer();
+					next = reader.getNextBuffer();
 
-					if (buffer == null) {
-						if (currentPartitionQueue.registerListener(null)) {
-							currentPartitionQueue = null;
-						}
-						else if (currentPartitionQueue.isReleased()) {
-							markAsReleased(currentPartitionQueue.getReceiverId());
-
-							Throwable cause = currentPartitionQueue.getFailureCause();
+					if (next == null) {
+						if (reader.isReleased()) {
+							markAsReleased(reader.getReceiverId());
+							Throwable cause = reader.getFailureCause();
 
 							if (cause != null) {
-								ctx.writeAndFlush(new NettyMessage.ErrorResponse(
-										new ProducerFailedException(cause),
-										currentPartitionQueue.receiverId));
-							}
+								ErrorResponse msg = new ErrorResponse(
+									new ProducerFailedException(cause),
+									reader.getReceiverId());
 
-							currentPartitionQueue = null;
+								ctx.writeAndFlush(msg);
+							}
+						} else {
+							IllegalStateException err = new IllegalStateException(
+								"Bug in Netty consumer logic: reader queue got notified by partition " +
+									"about available data, but none was available.");
+							handleException(ctx.channel(), err);
+							return;
+						}
+					} else {
+						// this channel was now removed from the non-empty reader queue
+						// we re-add it in case it has more data, because in that case no
+						// "non-empty" notification will come for that reader from the queue.
+						if (next.moreAvailable()) {
+							nonEmptyReader.add(reader);
 						}
-					}
-					else {
-						BufferResponse resp = new BufferResponse(buffer, currentPartitionQueue.getSequenceNumber(), currentPartitionQueue.getReceiverId());
 
-						if (!buffer.isBuffer() &&
-								EventSerializer.fromBuffer(buffer, getClass().getClassLoader()).getClass() == EndOfPartitionEvent.class) {
+						BufferResponse msg = new BufferResponse(
+							next.buffer(),
+							reader.getSequenceNumber(),
+							reader.getReceiverId());
 
-							currentPartitionQueue.notifySubpartitionConsumed();
-							currentPartitionQueue.releaseAllResources();
-							markAsReleased(currentPartitionQueue.getReceiverId());
+						if (isEndOfPartitionEvent(next.buffer())) {
+							reader.notifySubpartitionConsumed();
+							reader.releaseAllResources();
 
-							currentPartitionQueue = null;
+							markAsReleased(reader.getReceiverId());
 						}
 
-						channel.writeAndFlush(resp).addListener(writeListener);
+						// Write and flush and wait until this is done before
+						// trying to continue with the next buffer.
+						channel.writeAndFlush(msg).addListener(writeListener);
 
 						return;
 					}
 				}
 			}
-		}
-		catch (Throwable t) {
-			if (buffer != null) {
-				buffer.recycle();
+		} catch (Throwable t) {
+			if (next != null) {
+				next.buffer().recycle();
 			}
 
 			throw new IOException(t.getMessage(), t);
 		}
 	}
 
+	private boolean isEndOfPartitionEvent(Buffer buffer) throws IOException {
+		return !buffer.isBuffer() && EventSerializer.fromBuffer(buffer, getClass().getClassLoader()).getClass() == EndOfPartitionEvent.class;
+	}
+
 	@Override
 	public void channelInactive(ChannelHandlerContext ctx) throws Exception {
 		releaseAllResources();
@@ -215,22 +238,15 @@ class PartitionRequestQueue extends ChannelInboundHandlerAdapter {
 		releaseAllResources();
 
 		if (channel.isActive()) {
-			channel.writeAndFlush(new NettyMessage.ErrorResponse(cause)).addListener(ChannelFutureListener.CLOSE);
+			channel.writeAndFlush(new ErrorResponse(cause)).addListener(ChannelFutureListener.CLOSE);
 		}
 	}
 
 	private void releaseAllResources() throws IOException {
-		if (currentPartitionQueue != null) {
-			currentPartitionQueue.releaseAllResources();
-			markAsReleased(currentPartitionQueue.getReceiverId());
-
-			currentPartitionQueue = null;
-		}
-
-		while ((currentPartitionQueue = queue.poll()) != null) {
-			currentPartitionQueue.releaseAllResources();
-
-			markAsReleased(currentPartitionQueue.getReceiverId());
+		SequenceNumberingViewReader reader;
+		while ((reader = nonEmptyReader.poll()) != null) {
+			reader.releaseAllResources();
+			markAsReleased(reader.getReceiverId());
 		}
 	}
 
@@ -241,7 +257,7 @@ class PartitionRequestQueue extends ChannelInboundHandlerAdapter {
 		released.add(receiverId);
 	}
 
-	// This listener is called after an element of the current queue has been
+	// This listener is called after an element of the current nonEmptyReader has been
 	// flushed. If successful, the listener triggers further processing of the
 	// queues.
 	private class WriteAndFlushNextMessageIfPossibleListener implements ChannelFutureListener {
@@ -251,87 +267,14 @@ class PartitionRequestQueue extends ChannelInboundHandlerAdapter {
 			try {
 				if (future.isSuccess()) {
 					writeAndFlushNextMessageIfPossible(future.channel());
-				}
-				else if (future.cause() != null) {
+				} else if (future.cause() != null) {
 					handleException(future.channel(), future.cause());
-				}
-				else {
+				} else {
 					handleException(future.channel(), new IllegalStateException("Sending cancelled by user."));
 				}
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				handleException(future.channel(), t);
 			}
 		}
 	}
-
-	/**
-	 * Simple wrapper for the partition queue iterator, which increments a
-	 * sequence number for each returned buffer and remembers the receiver ID.
-	 */
-	private class SequenceNumberingSubpartitionView implements ResultSubpartitionView, NotificationListener {
-
-		private final ResultSubpartitionView queueIterator;
-
-		private final InputChannelID receiverId;
-
-		private int sequenceNumber = -1;
-
-		private SequenceNumberingSubpartitionView(ResultSubpartitionView queueIterator, InputChannelID receiverId) {
-			this.queueIterator = checkNotNull(queueIterator);
-			this.receiverId = checkNotNull(receiverId);
-		}
-
-		private InputChannelID getReceiverId() {
-			return receiverId;
-		}
-
-		private int getSequenceNumber() {
-			return sequenceNumber;
-		}
-
-		@Override
-		public Buffer getNextBuffer() throws IOException, InterruptedException {
-			Buffer buffer = queueIterator.getNextBuffer();
-
-			if (buffer != null) {
-				sequenceNumber++;
-			}
-
-			return buffer;
-		}
-
-		@Override
-		public void notifySubpartitionConsumed() throws IOException {
-			queueIterator.notifySubpartitionConsumed();
-		}
-
-		@Override
-		public boolean isReleased() {
-			return queueIterator.isReleased();
-		}
-
-		@Override
-		public Throwable getFailureCause() {
-			return queueIterator.getFailureCause();
-		}
-
-		@Override
-		public boolean registerListener(NotificationListener ignored) throws IOException {
-			return queueIterator.registerListener(this);
-		}
-
-		@Override
-		public void releaseAllResources() throws IOException {
-			queueIterator.releaseAllResources();
-		}
-
-		/**
-		 * Enqueue this iterator again after a notification.
-		 */
-		@Override
-		public void onNotification() {
-			ctx.pipeline().fireUserEventTriggered(this);
-		}
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f728129b/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 e278d07..12b52ec 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
@@ -27,7 +27,6 @@ import org.apache.flink.runtime.io.network.netty.NettyMessage.CancelPartitionReq
 import org.apache.flink.runtime.io.network.netty.NettyMessage.CloseRequest;
 import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException;
 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.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -53,10 +52,10 @@ class PartitionRequestServerHandler extends SimpleChannelInboundHandler<NettyMes
 	private BufferPool bufferPool;
 
 	PartitionRequestServerHandler(
-			ResultPartitionProvider partitionProvider,
-			TaskEventDispatcher taskEventDispatcher,
-			PartitionRequestQueue outboundQueue,
-			NetworkBufferPool networkBufferPool) {
+		ResultPartitionProvider partitionProvider,
+		TaskEventDispatcher taskEventDispatcher,
+		PartitionRequestQueue outboundQueue,
+		NetworkBufferPool networkBufferPool) {
 
 		this.partitionProvider = partitionProvider;
 		this.taskEventDispatcher = taskEventDispatcher;
@@ -94,15 +93,16 @@ class PartitionRequestServerHandler extends SimpleChannelInboundHandler<NettyMes
 				LOG.debug("Read channel on {}: {}.", ctx.channel().localAddress(), request);
 
 				try {
-					ResultSubpartitionView subpartition =
-							partitionProvider.createSubpartitionView(
-									request.partitionId,
-									request.queueIndex,
-									bufferPool);
-
-					outboundQueue.enqueue(subpartition, request.receiverId);
-				}
-				catch (PartitionNotFoundException notFound) {
+					SequenceNumberingViewReader reader = new SequenceNumberingViewReader(
+						request.receiverId,
+						outboundQueue);
+
+					reader.requestSubpartitionView(
+						partitionProvider,
+						request.partitionId,
+						request.queueIndex,
+						bufferPool);
+				} catch (PartitionNotFoundException notFound) {
 					respondWithError(ctx, notFound, request.receiverId);
 				}
 			}
@@ -115,20 +115,16 @@ class PartitionRequestServerHandler extends SimpleChannelInboundHandler<NettyMes
 				if (!taskEventDispatcher.publish(request.partitionId, request.event)) {
 					respondWithError(ctx, new IllegalArgumentException("Task event receiver not found."), request.receiverId);
 				}
-			}
-			else if (msgClazz == CancelPartitionRequest.class) {
+			} else if (msgClazz == CancelPartitionRequest.class) {
 				CancelPartitionRequest request = (CancelPartitionRequest) msg;
 
 				outboundQueue.cancel(request.receiverId);
-			}
-			else if (msgClazz == CloseRequest.class) {
+			} else if (msgClazz == CloseRequest.class) {
 				outboundQueue.close();
-			}
-			else {
+			} else {
 				LOG.warn("Received unexpected client request: {}", msg);
 			}
-		}
-		catch (Throwable t) {
+		} catch (Throwable t) {
 			respondWithError(ctx, t);
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/f728129b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/SequenceNumberingViewReader.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/SequenceNumberingViewReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/SequenceNumberingViewReader.java
new file mode 100644
index 0000000..ef611eb
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/SequenceNumberingViewReader.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.netty;
+
+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.BufferAvailabilityListener;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+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.InputChannel.BufferAndAvailability;
+import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
+import org.apache.flink.runtime.io.network.partition.consumer.LocalInputChannel;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * Simple wrapper for the partition readerQueue iterator, which increments a
+ * sequence number for each returned buffer and remembers the receiver ID.
+ *
+ * <p>It also keeps track of available buffers and notifies the outbound
+ * handler about non-emptiness, similar to the {@link LocalInputChannel}.
+ */
+class SequenceNumberingViewReader implements BufferAvailabilityListener {
+
+	private final Object requestLock = new Object();
+
+	private final InputChannelID receiverId;
+
+	private final AtomicLong numBuffersAvailable = new AtomicLong();
+
+	private final PartitionRequestQueue requestQueue;
+
+	private volatile ResultSubpartitionView subpartitionView;
+
+	private int sequenceNumber = -1;
+
+	SequenceNumberingViewReader(InputChannelID receiverId, PartitionRequestQueue requestQueue) {
+		this.receiverId = receiverId;
+		this.requestQueue = requestQueue;
+	}
+
+	void requestSubpartitionView(
+		ResultPartitionProvider partitionProvider,
+		ResultPartitionID resultPartitionId,
+		int subPartitionIndex,
+		BufferProvider bufferProvider) throws IOException {
+
+		synchronized (requestLock) {
+			if (subpartitionView == null) {
+				// This this call can trigger a notification we have to
+				// schedule a separate task at the event loop that will
+				// start consuming this. Otherwise the reference to the
+				// view cannot be available in getNextBuffer().
+				this.subpartitionView = partitionProvider.createSubpartitionView(
+					resultPartitionId,
+					subPartitionIndex,
+					bufferProvider,
+					this);
+			} else {
+				throw new IllegalStateException("Subpartition already requested");
+			}
+		}
+	}
+
+	InputChannelID getReceiverId() {
+		return receiverId;
+	}
+
+	int getSequenceNumber() {
+		return sequenceNumber;
+	}
+
+	public BufferAndAvailability getNextBuffer() throws IOException, InterruptedException {
+		Buffer next = subpartitionView.getNextBuffer();
+		if (next != null) {
+			long remaining = numBuffersAvailable.decrementAndGet();
+			sequenceNumber++;
+
+			if (remaining >= 0) {
+				return new BufferAndAvailability(next, remaining > 0);
+			} else {
+				throw new IllegalStateException("no buffer available");
+			}
+		} else {
+			return null;
+		}
+	}
+
+	public void notifySubpartitionConsumed() throws IOException {
+		subpartitionView.notifySubpartitionConsumed();
+	}
+
+	public boolean isReleased() {
+		return subpartitionView.isReleased();
+	}
+
+	public Throwable getFailureCause() {
+		return subpartitionView.getFailureCause();
+	}
+
+	public void releaseAllResources() throws IOException {
+		subpartitionView.releaseAllResources();
+	}
+
+	@Override
+	public void notifyBuffersAvailable(long numBuffers) {
+		// if this request made the channel non-empty, notify the input gate
+		if (numBuffers > 0 && numBuffersAvailable.getAndAdd(numBuffers) == 0) {
+			requestQueue.notifyReaderNonEmpty(this);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f728129b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferAvailabilityListener.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferAvailabilityListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferAvailabilityListener.java
new file mode 100644
index 0000000..114ef7c
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferAvailabilityListener.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;
+
+/**
+ * Listener interface implemented by consumers of {@link ResultSubpartitionView}
+ * that want to be notified of availability of further buffers.
+ */
+public interface BufferAvailabilityListener {
+
+	/**
+	 * Called whenever a new number of buffers becomes available.
+	 *
+	 * @param numBuffers The number of buffers that became available.
+	 */
+	void notifyBuffersAvailable(long numBuffers);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f728129b/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
index 3981a26..e9400f0 100644
--- 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
@@ -22,7 +22,7 @@ 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;
 
@@ -30,6 +30,7 @@ import java.io.IOException;
 import java.util.ArrayDeque;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
 
 /**
  * A pipelined in-memory only subpartition, which can be consumed once.
@@ -38,51 +39,47 @@ class PipelinedSubpartition extends ResultSubpartition {
 
 	private static final Logger LOG = LoggerFactory.getLogger(PipelinedSubpartition.class);
 
+	// ------------------------------------------------------------------------
+
+	/** All buffers of this subpartition. Access to the buffers is synchronized on this object. */
+	private final ArrayDeque<Buffer> buffers = new ArrayDeque<>();
+
+	/** The read view to consume this subpartition. */
+	private PipelinedSubpartitionView readView;
+
 	/** Flag indicating whether the subpartition has been finished. */
 	private boolean isFinished;
 
 	/** Flag indicating whether the subpartition has been released. */
 	private volatile 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) {
+	public boolean add(Buffer buffer) throws IOException {
 		checkNotNull(buffer);
 
-		final NotificationListener listener;
+		// view reference accessible outside the lock, but assigned inside the locked scope
+		final PipelinedSubpartitionView reader;
 
 		synchronized (buffers) {
-			if (isReleased || isFinished) {
+			if (isFinished || isReleased) {
 				return false;
 			}
 
 			// Add the buffer and update the stats
 			buffers.add(buffer);
+			reader = readView;
 			updateStatistics(buffer);
-
-			// Get the listener...
-			listener = registeredListener;
-			registeredListener = null;
 		}
 
 		// Notify the listener outside of the synchronized block
-		if (listener != null) {
-			listener.onNotification();
+		if (reader != null) {
+			reader.notifyBuffersAvailable(1);
 		}
 
 		return true;
@@ -90,36 +87,34 @@ class PipelinedSubpartition extends ResultSubpartition {
 
 	@Override
 	public void finish() throws IOException {
-		final NotificationListener listener;
+		final Buffer buffer = EventSerializer.toBuffer(EndOfPartitionEvent.INSTANCE);
+
+		// view reference accessible outside the lock, but assigned inside the locked scope
+		final PipelinedSubpartitionView reader;
 
 		synchronized (buffers) {
-			if (isReleased || isFinished) {
+			if (isFinished || isReleased) {
 				return;
 			}
 
-			final Buffer buffer = EventSerializer.toBuffer(EndOfPartitionEvent.INSTANCE);
-
 			buffers.add(buffer);
+			reader = readView;
 			updateStatistics(buffer);
 
 			isFinished = true;
-
-			LOG.debug("Finished {}.", this);
-
-			// Get the listener...
-			listener = registeredListener;
-			registeredListener = null;
 		}
 
+		LOG.debug("Finished {}.", this);
+
 		// Notify the listener outside of the synchronized block
-		if (listener != null) {
-			listener.onNotification();
+		if (reader != null) {
+			reader.notifyBuffersAvailable(1);
 		}
 	}
 
 	@Override
 	public void release() {
-		final NotificationListener listener;
+		// view reference accessible outside the lock, but assigned inside the locked scope
 		final PipelinedSubpartitionView view;
 
 		synchronized (buffers) {
@@ -130,40 +125,35 @@ class PipelinedSubpartition extends ResultSubpartition {
 			// Release all available buffers
 			Buffer buffer;
 			while ((buffer = buffers.poll()) != null) {
-				if (!buffer.isRecycled()) {
-					buffer.recycle();
-				}
+				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);
 		}
 
+		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();
+	Buffer pollBuffer() {
+		synchronized (buffers) {
+			return buffers.pollFirst();
 		}
 	}
 
 	@Override
 	public int releaseMemory() {
-		// The pipelined subpartition does not react to memory release requests. The buffers will be
-		// recycled by the consuming task.
+		// The pipelined subpartition does not react to memory release requests.
+		// The buffers will be recycled by the consuming task.
 		return 0;
 	}
 
@@ -173,53 +163,43 @@ class PipelinedSubpartition extends ResultSubpartition {
 	}
 
 	@Override
-	public PipelinedSubpartitionView createReadView(BufferProvider bufferProvider) {
-		synchronized (buffers) {
-			if (readView != null) {
-				throw new IllegalStateException("Subpartition " + index + " of "
-						+ parent.getPartitionId() + " is being or already has been " +
-						"consumed, but pipelined subpartitions can only be consumed once.");
-			}
+	public PipelinedSubpartitionView createReadView(BufferProvider bufferProvider, BufferAvailabilityListener availabilityListener) throws IOException {
+		final int queueSize;
 
-			readView = new PipelinedSubpartitionView(this);
+		synchronized (buffers) {
+			checkState(!isReleased);
+			checkState(readView == null,
+					"Subpartition %s of is being (or already has been) consumed, " +
+					"but pipelined subpartitions can only be consumed once.", index, parent.getPartitionId());
 
-			LOG.debug("Created read view for subpartition {} of partition {}.", index, parent.getPartitionId());
+			LOG.debug("Creating read view for subpartition {} of partition {}.", index, parent.getPartitionId());
 
-			return readView;
+			queueSize = buffers.size();
+			readView = new PipelinedSubpartitionView(this, availabilityListener);
 		}
+
+		readView.notifyBuffersAvailable(queueSize);
+
+		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);
-		}
-	}
+		final long numBuffers;
+		final long numBytes;
+		final boolean finished;
+		final boolean hasReadView;
 
-	/**
-	 * 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.");
+			numBuffers = getTotalNumberOfBuffers();
+			numBytes = getTotalNumberOfBytes();
+			finished = isFinished;
+			hasReadView = readView != null;
 		}
+
+		return String.format(
+				"PipelinedSubpartition [number of buffers: %d (%d bytes), finished? %s, read view? %s]",
+				numBuffers, numBytes, finished, hasReadView);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/f728129b/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
index f8d81a4..52c78ea 100644
--- 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
@@ -19,8 +19,8 @@
 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;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
@@ -33,23 +33,25 @@ class PipelinedSubpartitionView implements ResultSubpartitionView {
 	/** The subpartition this view belongs to. */
 	private final PipelinedSubpartition parent;
 
+	private final BufferAvailabilityListener availabilityListener;
+
 	/** Flag indicating whether this view has been released. */
-	private AtomicBoolean isReleased = new AtomicBoolean();
+	private final AtomicBoolean isReleased;
 
-	PipelinedSubpartitionView(PipelinedSubpartition parent) {
+	PipelinedSubpartitionView(PipelinedSubpartition parent, BufferAvailabilityListener listener) {
 		this.parent = checkNotNull(parent);
+		this.availabilityListener = checkNotNull(listener);
+		this.isReleased = new AtomicBoolean();
 	}
 
 	@Override
 	public Buffer getNextBuffer() {
-		synchronized (parent.buffers) {
-			return parent.buffers.poll();
-		}
+		return parent.pollBuffer();
 	}
 
 	@Override
-	public boolean registerListener(NotificationListener listener) {
-		return !isReleased.get() && parent.registerListener(listener);
+	public void notifyBuffersAvailable(long numBuffers) throws IOException {
+		availabilityListener.notifyBuffersAvailable(numBuffers);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/f728129b/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
index 834318c..474c25c 100644
--- 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
@@ -21,7 +21,6 @@ package org.apache.flink.runtime.io.network.partition;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.executiongraph.IntermediateResultPartition;
 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.buffer.Buffer;
 import org.apache.flink.runtime.io.network.buffer.BufferPool;
 import org.apache.flink.runtime.io.network.buffer.BufferPoolOwner;
@@ -135,7 +134,6 @@ public class ResultPartition implements BufferPoolOwner {
 		ResultPartitionManager partitionManager,
 		ResultPartitionConsumableNotifier partitionConsumableNotifier,
 		IOManager ioManager,
-		IOMode defaultIoMode,
 		boolean sendScheduleOrUpdateConsumersMessage) {
 
 		this.owningTaskName = checkNotNull(owningTaskName);
@@ -152,8 +150,7 @@ public class ResultPartition implements BufferPoolOwner {
 		switch (partitionType) {
 			case BLOCKING:
 				for (int i = 0; i < subpartitions.length; i++) {
-					subpartitions[i] = new SpillableSubpartition(
-							i, this, ioManager, defaultIoMode);
+					subpartitions[i] = new SpillableSubpartition(i, this, ioManager);
 				}
 
 				break;
@@ -340,7 +337,7 @@ public class ResultPartition implements BufferPoolOwner {
 	/**
 	 * Returns the requested subpartition.
 	 */
-	public ResultSubpartitionView createSubpartitionView(int index, BufferProvider bufferProvider) throws IOException {
+	public ResultSubpartitionView createSubpartitionView(int index, BufferProvider bufferProvider, BufferAvailabilityListener availabilityListener) throws IOException {
 		int refCnt = pendingReferences.get();
 
 		checkState(refCnt != -1, "Partition released.");
@@ -348,7 +345,7 @@ public class ResultPartition implements BufferPoolOwner {
 
 		checkElementIndex(index, subpartitions.length, "Subpartition not found.");
 
-		ResultSubpartitionView readView = subpartitions[index].createReadView(bufferProvider);
+		ResultSubpartitionView readView = subpartitions[index].createReadView(bufferProvider, availabilityListener);
 
 		LOG.debug("Created {}", readView);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f728129b/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
index 9da3e14..8ad3e34 100644
--- 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
@@ -66,7 +66,8 @@ public class ResultPartitionManager implements ResultPartitionProvider {
 	public ResultSubpartitionView createSubpartitionView(
 			ResultPartitionID partitionId,
 			int subpartitionIndex,
-			BufferProvider bufferProvider) throws IOException {
+			BufferProvider bufferProvider,
+			BufferAvailabilityListener availabilityListener) throws IOException {
 
 		synchronized (registeredPartitions) {
 			final ResultPartition partition = registeredPartitions.get(partitionId.getProducerId(),
@@ -78,7 +79,7 @@ public class ResultPartitionManager implements ResultPartitionProvider {
 
 			LOG.debug("Requesting subpartition {} of {}.", subpartitionIndex, partition);
 
-			return partition.createSubpartitionView(subpartitionIndex, bufferProvider);
+			return partition.createSubpartitionView(subpartitionIndex, bufferProvider, availabilityListener);
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f728129b/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
index 23dd1d3..3fbfd49 100644
--- 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
@@ -30,6 +30,7 @@ public interface ResultPartitionProvider {
 	ResultSubpartitionView createSubpartitionView(
 			ResultPartitionID partitionId,
 			int index,
-			BufferProvider bufferProvider) throws IOException;
+			BufferProvider bufferProvider,
+			BufferAvailabilityListener availabilityListener) throws IOException;
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f728129b/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
index 31c8f73..dd0e152 100644
--- 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
@@ -37,7 +37,7 @@ public abstract class ResultSubpartition {
 	// - Statistics ----------------------------------------------------------
 
 	/** The total number of buffers (both data and event buffers) */
-	private int totalNumberOfBuffers;
+	private long totalNumberOfBuffers;
 
 	/** The total number of bytes (both data and event buffers) */
 	private long totalNumberOfBytes;
@@ -52,7 +52,7 @@ public abstract class ResultSubpartition {
 		totalNumberOfBytes += buffer.getSize();
 	}
 
-	protected int getTotalNumberOfBuffers() {
+	protected long getTotalNumberOfBuffers() {
 		return totalNumberOfBuffers;
 	}
 
@@ -77,7 +77,7 @@ public abstract class ResultSubpartition {
 
 	abstract public void release() throws IOException;
 
-	abstract public ResultSubpartitionView createReadView(BufferProvider bufferProvider) throws IOException;
+	abstract public ResultSubpartitionView createReadView(BufferProvider bufferProvider, BufferAvailabilityListener availabilityListener) throws IOException;
 
 	abstract int releaseMemory() throws IOException;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f728129b/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
index cfc5455..98be90f 100644
--- 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
@@ -19,7 +19,6 @@
 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;
 
@@ -41,13 +40,7 @@ public interface ResultSubpartitionView {
 	 */
 	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 notifyBuffersAvailable(long buffers) throws IOException;
 
 	void releaseAllResources() throws IOException;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f728129b/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
index 3f19559..439e08d 100644
--- 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
@@ -18,42 +18,54 @@
 
 package org.apache.flink.runtime.io.network.partition;
 
+import org.apache.flink.configuration.ConfigConstants;
 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.BufferPool;
 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 java.util.ArrayDeque;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * A blocking in-memory subpartition, which is able to spill to disk.
+ * A spillable sub partition starts out in-memory and spills to disk if asked
+ * to do so.
  *
- * <p> Buffers are kept in-memory as long as possible. If not possible anymore, all buffers are
- * spilled to disk.
+ * <p>Buffers for the partition come from a {@link BufferPool}. The buffer pool
+ * is also responsible to trigger the release of the buffers if it needs them
+ * back. At this point, the spillable sub partition will write all in-memory
+ * buffers to disk. All added buffers after that point directly go to disk.
+ *
+ * <p>This partition type is used for {@link ResultPartitionType#BLOCKING}
+ * results that are fully produced before they can be consumed. At the point
+ * when they are consumed, the buffers are (i) all in-memory, (ii) currently
+ * being spilled to disk, or (iii) completely spilled to disk. Depending on
+ * this state, different reader variants are returned (see
+ * {@link SpillableSubpartitionView} and {@link SpilledSubpartitionView}).
+ *
+ * <p>Since the network buffer pool size is usually quite small (default is
+ * {@link ConfigConstants#DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS}), most
+ * spillable partitions will be spilled for real-world data sets.
  */
 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>();
+	/** Buffers are kept in this queue as long as we weren't ask to release any. */
+	private final ArrayDeque<Buffer> buffers = new ArrayDeque<>();
 
-	/** The I/O manager to create the spill writer from. */
-	final IOManager ioManager;
-
-	/** The default I/O mode to use. */
-	final IOMode ioMode;
+	/** The I/O manager used for spilling buffers to disk. */
+	private final IOManager ioManager;
 
 	/** The writer used for spilling. As long as this is null, we are in-memory. */
-	BufferFileWriter spillWriter;
+	private BufferFileWriter spillWriter;
 
 	/** Flag indicating whether the subpartition has been finished. */
 	private boolean isFinished;
@@ -64,11 +76,10 @@ class SpillableSubpartition extends ResultSubpartition {
 	/** The read view to consume this subpartition. */
 	private ResultSubpartitionView readView;
 
-	SpillableSubpartition(int index, ResultPartition parent, IOManager ioManager, IOMode ioMode) {
+	SpillableSubpartition(int index, ResultPartition parent, IOManager ioManager) {
 		super(index, parent);
 
 		this.ioManager = checkNotNull(ioManager);
-		this.ioMode = checkNotNull(ioMode);
 	}
 
 	@Override
@@ -80,7 +91,6 @@ class SpillableSubpartition extends ResultSubpartition {
 				return false;
 			}
 
-			// In-memory
 			if (spillWriter == null) {
 				buffers.add(buffer);
 
@@ -88,7 +98,7 @@ class SpillableSubpartition extends ResultSubpartition {
 			}
 		}
 
-		// Else: Spilling
+		// Didn't return early => go to disk
 		spillWriter.writeBlock(buffer);
 
 		return true;
@@ -102,7 +112,7 @@ class SpillableSubpartition extends ResultSubpartition {
 			}
 		}
 
-		// If we are spilling/have spilled, wait for the writer to finish.
+		// If we are spilling/have spilled, wait for the writer to finish
 		if (spillWriter != null) {
 			spillWriter.close();
 		}
@@ -117,51 +127,93 @@ class SpillableSubpartition extends ResultSubpartition {
 				return;
 			}
 
-			// Recycle all in-memory buffers
-			for (Buffer buffer : buffers) {
-				buffer.recycle();
-			}
-
-			buffers.clear();
-			buffers.trimToSize();
+			view = readView;
 
-			// If we are spilling/have spilled, wait for the writer to finish and delete the file.
-			if (spillWriter != null) {
-				spillWriter.closeAndDelete();
+			// No consumer yet, we are responsible to clean everything up. If
+			// one is available, the view is responsible is to clean up (see
+			// below).
+			if (view == null) {
+				for (Buffer buffer : buffers) {
+					buffer.recycle();
+				}
+				buffers.clear();
+
+				// TODO This can block until all buffers are written out to
+				// disk if a spill is in-progress before deleting the file.
+				// It is possibly called from the Netty event loop threads,
+				// which can bring down the network.
+				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();
+			view.releaseAllResources();
+		}
+	}
+
+	@Override
+	public ResultSubpartitionView createReadView(BufferProvider bufferProvider, BufferAvailabilityListener availabilityListener) 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.");
+			}
+
+			if (spillWriter != null) {
+				readView = new SpilledSubpartitionView(
+					this,
+					bufferProvider.getMemorySegmentSize(),
+					spillWriter,
+					getTotalNumberOfBuffers(),
+					availabilityListener);
+			} else {
+				readView = new SpillableSubpartitionView(
+					this,
+					buffers,
+					ioManager,
+					bufferProvider.getMemorySegmentSize(),
+					availabilityListener);
+			}
+
+			return readView;
 		}
 	}
 
 	@Override
 	public int releaseMemory() throws IOException {
 		synchronized (buffers) {
-			if (spillWriter == null) {
-				// Create the spill writer
+			ResultSubpartitionView view = readView;
+
+			if (view != null && view.getClass() == SpillableSubpartitionView.class) {
+				// If there is a spilalble view, it's the responsibility of the
+				// view to release memory.
+				SpillableSubpartitionView spillableView = (SpillableSubpartitionView) view;
+				return spillableView.releaseMemory();
+			} else if (spillWriter == null) {
+				// No view and in-memory => spill to disk
 				spillWriter = ioManager.createBufferFileWriter(ioManager.createChannel());
 
-				final int numberOfBuffers = buffers.size();
-
+				int numberOfBuffers = buffers.size();
 				long spilledBytes = 0;
 
 				// Spill all buffers
 				for (int i = 0; i < numberOfBuffers; i++) {
-					Buffer buffer = buffers.remove(0);
+					Buffer buffer = buffers.remove();
 					spilledBytes += buffer.getSize();
 					spillWriter.writeBlock(buffer);
 				}
 
-				LOG.debug("Spilled {} bytes for sub partition {} of {}.", spilledBytes, index, parent.getPartitionId());
+				LOG.debug("Spilling {} bytes for sub partition {} of {}.", spilledBytes, index, parent.getPartitionId());
 
 				return numberOfBuffers;
 			}
@@ -177,47 +229,8 @@ class SpillableSubpartition extends ResultSubpartition {
 	}
 
 	@Override
-	public ResultSubpartitionView createReadView(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.getMemorySegmentSize(),
-							spillWriter.getChannelID(),
-							0);
-				}
-				else {
-					readView = new SpilledSubpartitionViewAsyncIO(
-							this,
-							bufferProvider,
-							ioManager,
-							spillWriter.getChannelID(),
-							0);
-				}
-			}
-			else {
-				readView = new SpillableSubpartitionView(
-						this, bufferProvider, buffers.size(), ioMode);
-			}
-
-			return readView;
-		}
+	public int getNumberOfQueuedBuffers() {
+		return buffers.size();
 	}
 
 	@Override
@@ -228,8 +241,4 @@ class SpillableSubpartition extends ResultSubpartition {
 				spillWriter != null);
 	}
 
-	@Override
-	public int getNumberOfQueuedBuffers() {
-			return buffers.size();
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f728129b/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
index 29c2002..8119ecc 100644
--- 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
@@ -18,15 +18,14 @@
 
 package org.apache.flink.runtime.io.network.partition;
 
-import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode;
+import org.apache.flink.runtime.io.disk.iomanager.BufferFileWriter;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 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.ArrayDeque;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import static org.apache.flink.util.Preconditions.checkArgument;
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 class SpillableSubpartitionView implements ResultSubpartitionView {
@@ -34,146 +33,163 @@ 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;
+	/** All buffers of this subpartition. Access to the buffers is synchronized on this object. */
+	private final ArrayDeque<Buffer> buffers;
 
-	/** The number of buffers in-memory at the subpartition. */
-	private final int numberOfBuffers;
+	/** IO manager if we need to spill (for spilled case). */
+	private final IOManager ioManager;
 
-	/** The default I/O mode to use. */
-	private final IOMode ioMode;
+	/** Size of memory segments (for spilled case). */
+	private final int memorySegmentSize;
 
-	private ResultSubpartitionView spilledView;
-
-	private int currentQueuePosition;
-
-	private long currentBytesRead;
+	/**
+	 * The buffer availability listener. As long as in-memory, notifications
+	 * happen on a buffer per buffer basis as spilling may happen after a
+	 * notification has been sent out.
+	 */
+	private final BufferAvailabilityListener listener;
 
 	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.isReleased()) {
-				return null;
-			}
+	/**
+	 * The next buffer to hand out. Everytime this is set to a non-null value,
+	 * a listener notification happens.
+	 */
+	private Buffer nextBuffer;
 
-			if (parent.spillWriter == null) {
-				if (currentQueuePosition < numberOfBuffers) {
-					Buffer buffer = parent.buffers.get(currentQueuePosition);
+	private volatile SpilledSubpartitionView spilledView;
 
-					buffer.retain();
+	SpillableSubpartitionView(
+		SpillableSubpartition parent,
+		ArrayDeque<Buffer> buffers,
+		IOManager ioManager,
+		int memorySegmentSize,
+		BufferAvailabilityListener listener) {
 
-					// TODO Fix hard coding of 8 bytes for the header
-					currentBytesRead += buffer.getSize() + 8;
-					currentQueuePosition++;
-
-					return buffer;
-				}
+		this.parent = checkNotNull(parent);
+		this.buffers = checkNotNull(buffers);
+		this.ioManager = checkNotNull(ioManager);
+		this.memorySegmentSize = memorySegmentSize;
+		this.listener = checkNotNull(listener);
 
-				return null;
-			}
+		synchronized (buffers) {
+			nextBuffer = buffers.poll();
 		}
 
-		// 2) Spilled
-		if (spilledView != null) {
-			return spilledView.getNextBuffer();
+		if (nextBuffer != null) {
+			listener.notifyBuffersAvailable(1);
 		}
+	}
 
-		// 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;
-		}
+	int releaseMemory() throws IOException {
+		synchronized (buffers) {
+			if (spilledView != null || nextBuffer == null) {
+				// Already spilled or nothing in-memory
+				return 0;
+			} else {
+				// We don't touch next buffer, because a notification has
+				// already been sent for it. Only when it is consumed, will
+				// it be recycled.
+
+				// Create the spill writer and write all buffers to disk
+				BufferFileWriter spillWriter = ioManager.createBufferFileWriter(ioManager.createChannel());
+
+				int numBuffers = buffers.size();
+				for (int i = 0; i < numBuffers; i++) {
+					Buffer buffer = buffers.remove();
+					try {
+						spillWriter.writeBlock(buffer);
+					} finally {
+						buffer.recycle();
+					}
+				}
 
-		if (ioMode.isSynchronous()) {
-			spilledView = new SpilledSubpartitionViewSyncIO(
-					parent,
-					bufferProvider.getMemorySegmentSize(),
-					parent.spillWriter.getChannelID(),
-					currentBytesRead);
-		}
-		else {
-			spilledView = new SpilledSubpartitionViewAsyncIO(
+				spilledView = new SpilledSubpartitionView(
 					parent,
-					bufferProvider,
-					parent.ioManager,
-					parent.spillWriter.getChannelID(),
-					currentBytesRead);
-		}
+					memorySegmentSize,
+					spillWriter,
+					numBuffers,
+					listener);
 
-		return spilledView.getNextBuffer();
+				return numBuffers;
+			}
+		}
 	}
 
 	@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;
+	public Buffer getNextBuffer() throws IOException, InterruptedException {
+		synchronized (buffers) {
+			if (isReleased.get()) {
+				return null;
+			} else if (nextBuffer != null) {
+				Buffer current = nextBuffer;
+				nextBuffer = buffers.poll();
+
+				if (nextBuffer != null) {
+					listener.notifyBuffersAvailable(1);
 				}
-			}
 
-			// Spilling
-			if (parent.spillWriter.getNumberOfOutstandingRequests() > 0) {
-				return parent.spillWriter.registerAllRequestsProcessedListener(listener);
+				return current;
 			}
+		} // else: spilled
 
-			return false;
+		SpilledSubpartitionView spilled = spilledView;
+		if (spilled != null) {
+			return spilled.getNextBuffer();
+		} else {
+			throw new IllegalStateException("No in-memory buffers available, but also nothing spilled.");
 		}
-
-		return spilledView.registerListener(listener);
 	}
 
 	@Override
-	public void notifySubpartitionConsumed() throws IOException {
-		parent.onConsumedSubpartition();
+	public void notifyBuffersAvailable(long buffers) throws IOException {
+		// We do the availability listener notification one by one
 	}
 
 	@Override
 	public void releaseAllResources() throws IOException {
 		if (isReleased.compareAndSet(false, true)) {
-			if (spilledView != null) {
-				spilledView.releaseAllResources();
+			SpilledSubpartitionView spilled = spilledView;
+			if (spilled != null) {
+				spilled.releaseAllResources();
 			}
 		}
 	}
 
 	@Override
+	public void notifySubpartitionConsumed() throws IOException {
+		SpilledSubpartitionView spilled = spilledView;
+		if (spilled != null) {
+			spilled.notifySubpartitionConsumed();
+		} else {
+			parent.onConsumedSubpartition();
+		}
+	}
+
+	@Override
 	public boolean isReleased() {
-		return parent.isReleased() || isReleased.get();
+		SpilledSubpartitionView spilled = spilledView;
+		if (spilled != null) {
+			return spilled.isReleased();
+		} else {
+			return parent.isReleased() || isReleased.get();
+		}
 	}
 
 	@Override
 	public Throwable getFailureCause() {
-		return parent.getFailureCause();
+		SpilledSubpartitionView spilled = spilledView;
+		if (spilled != null) {
+			return spilled.getFailureCause();
+		} else {
+			return parent.getFailureCause();
+		}
 	}
 
 	@Override
 	public String toString() {
 		return String.format("SpillableSubpartitionView(index: %d) of ResultPartition %s",
-				parent.index,
-				parent.parent.getPartitionId());
+			parent.index,
+			parent.parent.getPartitionId());
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f728129b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionView.java
new file mode 100644
index 0000000..b087a4e
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionView.java
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.core.memory.MemorySegmentFactory;
+import org.apache.flink.runtime.io.disk.iomanager.BufferFileReader;
+import org.apache.flink.runtime.io.disk.iomanager.BufferFileWriter;
+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 org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Reader for a spilled sub partition.
+ *
+ * <p>The partition availability listener is notified about available buffers
+ * only when the spilling is done. Spilling is done async and if it is still
+ * in progress, we wait with the notification until the spilling is done.
+ *
+ * <p>Reads of the spilled file are done in synchronously.
+ */
+class SpilledSubpartitionView implements ResultSubpartitionView, NotificationListener {
+
+	/** The subpartition this view belongs to. */
+	private final ResultSubpartition parent;
+
+	/** Writer for spills. */
+	private final BufferFileWriter spillWriter;
+
+	/** 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;
+
+	/** Buffer availability listener. */
+	private final BufferAvailabilityListener availabilityListener;
+
+	/** The total number of spilled buffers. */
+	private final long numberOfSpilledBuffers;
+
+	/** Flag indicating whether all resources have been released. */
+	private AtomicBoolean isReleased = new AtomicBoolean();
+
+	/** Flag indicating whether a spill is still in progress. */
+	private volatile boolean isSpillInProgress = true;
+
+	SpilledSubpartitionView(
+		ResultSubpartition parent,
+		int memorySegmentSize,
+		BufferFileWriter spillWriter,
+		long numberOfSpilledBuffers,
+		BufferAvailabilityListener availabilityListener) throws IOException {
+
+		this.parent = checkNotNull(parent);
+		this.bufferPool = new SpillReadBufferPool(2, memorySegmentSize);
+		this.spillWriter = checkNotNull(spillWriter);
+		this.fileReader = new SynchronousBufferFileReader(spillWriter.getChannelID(), false);
+		checkArgument(numberOfSpilledBuffers >= 0);
+		this.numberOfSpilledBuffers = numberOfSpilledBuffers;
+		this.availabilityListener = checkNotNull(availabilityListener);
+
+		// Check whether async spilling is still in progress. If not, this returns
+		// false and we can notify our availability listener about all available buffers.
+		// Otherwise, we notify only when the spill writer callback happens.
+		if (!spillWriter.registerAllRequestsProcessedListener(this)) {
+			isSpillInProgress = false;
+			availabilityListener.notifyBuffersAvailable(numberOfSpilledBuffers);
+		}
+	}
+
+	/**
+	 * This is the call back method for the spill writer. If a spill is still
+	 * in progress when this view is created we wait until this method is called
+	 * before we notify the availability listener.
+	 */
+	@Override
+	public void onNotification() {
+		isSpillInProgress = false;
+		availabilityListener.notifyBuffersAvailable(numberOfSpilledBuffers);
+	}
+
+	@Override
+	public Buffer getNextBuffer() throws IOException, InterruptedException {
+		if (fileReader.hasReachedEndOfFile() || isSpillInProgress) {
+			return null;
+		}
+
+		// TODO This is fragile as we implicitly expect that multiple calls to
+		// this method don't happen before recycling buffers returned earlier.
+		Buffer buffer = bufferPool.requestBufferBlocking();
+		fileReader.readInto(buffer);
+
+		return buffer;
+	}
+
+	@Override
+	public void notifyBuffersAvailable(long buffers) throws IOException {
+		// We do the availability listener notification either directly on
+		// construction of this view (when everything has been spilled) or
+		// as soon as spilling is done and we are notified about it in the
+		// #onNotification callback.
+	}
+
+	@Override
+	public void notifySubpartitionConsumed() throws IOException {
+		parent.onConsumedSubpartition();
+	}
+
+	@Override
+	public void releaseAllResources() throws IOException {
+		if (isReleased.compareAndSet(false, true)) {
+			// TODO This can block until all buffers are written out to
+			// disk if a spill is in-progress before deleting the file.
+			// It is possibly called from the Netty event loop threads,
+			// which can bring down the network.
+			spillWriter.closeAndDelete();
+
+			fileReader.close();
+			bufferPool.destroy();
+		}
+	}
+
+	@Override
+	public boolean isReleased() {
+		return parent.isReleased() || isReleased.get();
+	}
+
+	@Override
+	public Throwable getFailureCause() {
+		return parent.getFailureCause();
+	}
+
+	@Override
+	public String toString() {
+		return String.format("SpilledSubpartitionView[sync](index: %d) of ResultPartition %s", parent.index, parent.parent.getPartitionId());
+	}
+
+	/**
+	 * 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.
+	 */
+	private static class SpillReadBufferPool implements BufferRecycler {
+
+		private final Queue<Buffer> buffers;
+
+		private boolean isDestroyed;
+
+		SpillReadBufferPool(int numberOfBuffers, int memorySegmentSize) {
+			this.buffers = new ArrayDeque<>(numberOfBuffers);
+
+			synchronized (buffers) {
+				for (int i = 0; i < numberOfBuffers; i++) {
+					buffers.add(new Buffer(MemorySegmentFactory.allocateUnpooledSegment(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/f728129b/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
deleted file mode 100644
index ca25536..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewAsyncIO.java
+++ /dev/null
@@ -1,377 +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 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 java.util.concurrent.atomic.AtomicReference;
-
-import static org.apache.flink.util.Preconditions.checkArgument;
-import static org.apache.flink.util.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 final AtomicReference<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;
-
-	/** Spilled file size */
-	private final long fileSize;
-
-	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.registeredListener = new AtomicReference<>();
-		
-		this.asyncFileReader = ioManager.createBufferFileReader(channelId, new IOThreadCallback(this));
-
-		if (initialSeekPosition > 0) {
-			asyncFileReader.seekToPosition(initialSeekPosition);
-		}
-
-		this.readBatchSize = readBatchSize;
-
-		this.fileSize = asyncFileReader.getSize();
-
-		// 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.compareAndSet(null, listener)) {
-				return true;
-			} else {
-				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 parent.isReleased() || isReleased;
-	}
-
-	@Override
-	public Throwable getFailureCause() {
-		return parent.getFailureCause();
-	}
-
-	/**
-	 * 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);
-
-			// after this, the listener should be null
-			listener = registeredListener.getAndSet(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 = registeredListener.getAndSet(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);
-		}
-	}
-
-	@Override
-	public String toString() {
-		return String.format("SpilledSubpartitionView[async](index: %d, file size: %d bytes) of ResultPartition %s",
-				parent.index,
-				fileSize,
-				parent.parent.getPartitionId());
-	}
-
-	/**
-	 * 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);
-		}
-	}
-}


[4/6] flink git commit: [FLINK-5169] [network] Add tests for channel consumption

Posted by uc...@apache.org.
[FLINK-5169] [network] Add tests for channel consumption

This closes #2882.


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

Branch: refs/heads/master
Commit: c0cdc5c4ec08e35a8ea319d1bbf2b24e03e24fd3
Parents: d3ac0ad
Author: Stephan Ewen <se...@apache.org>
Authored: Sun Nov 27 18:15:40 2016 +0100
Committer: Ufuk Celebi <uc...@apache.org>
Committed: Thu Dec 1 21:42:49 2016 +0100

----------------------------------------------------------------------
 .../partition/PipelinedSubpartition.java        |   8 +
 .../partition/consumer/LocalInputChannel.java   |   4 +-
 .../partition/consumer/SingleInputGate.java     |   4 +-
 .../partition/consumer/UnionInputGate.java      |   2 +-
 .../partition/InputChannelTestUtils.java        |  89 +++++
 .../partition/InputGateConcurrentTest.java      | 323 +++++++++++++++
 .../partition/InputGateFairnessTest.java        | 395 +++++++++++++++++++
 7 files changed, 820 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/c0cdc5c4/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
index e9400f0..9e2f5ba 100644
--- 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
@@ -183,6 +183,14 @@ class PipelinedSubpartition extends ResultSubpartition {
 		return readView;
 	}
 
+	// ------------------------------------------------------------------------
+
+	int getCurrentNumberOfBuffers() {
+		return buffers.size();
+	}
+
+	// ------------------------------------------------------------------------
+
 	@Override
 	public String toString() {
 		final long numBuffers;

http://git-wip-us.apache.org/repos/asf/flink/blob/c0cdc5c4/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 d5308a8..1936da2 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
@@ -64,7 +64,7 @@ public class LocalInputChannel extends InputChannel implements BufferAvailabilit
 
 	private volatile boolean isReleased;
 
-	LocalInputChannel(
+	public LocalInputChannel(
 		SingleInputGate inputGate,
 		int channelIndex,
 		ResultPartitionID partitionId,
@@ -76,7 +76,7 @@ public class LocalInputChannel extends InputChannel implements BufferAvailabilit
 			0, 0, metrics);
 	}
 
-	LocalInputChannel(
+	public LocalInputChannel(
 		SingleInputGate inputGate,
 		int channelIndex,
 		ResultPartitionID partitionId,

http://git-wip-us.apache.org/repos/asf/flink/blob/c0cdc5c4/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 bcbb2c4..b4d8d2c 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
@@ -261,7 +261,7 @@ public class SingleInputGate implements InputGate {
 		this.bufferPool = checkNotNull(bufferPool);
 	}
 
-	void setInputChannel(IntermediateResultPartitionID partitionId, InputChannel inputChannel) {
+	public void setInputChannel(IntermediateResultPartitionID partitionId, InputChannel inputChannel) {
 		synchronized (requestLock) {
 			if (inputChannels.put(checkNotNull(partitionId), checkNotNull(inputChannel)) == null
 					&& inputChannel.getClass() == UnknownInputChannel.class) {
@@ -546,7 +546,7 @@ public class SingleInputGate implements InputGate {
 			inputChannelsWithData.add(channel);
 
 			if (availableChannels == 0) {
-				inputChannelsWithData.notify();
+				inputChannelsWithData.notifyAll();
 			}
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/c0cdc5c4/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
index e8ccbb4..55c78af 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
@@ -225,7 +225,7 @@ public class UnionInputGate implements InputGate, InputGateListener {
 			inputGatesWithData.add(inputGate);
 
 			if (availableInputGates == 0) {
-				inputGatesWithData.notify();
+				inputGatesWithData.notifyAll();
 			}
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/c0cdc5c4/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputChannelTestUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputChannelTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputChannelTestUtils.java
new file mode 100644
index 0000000..e292576
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputChannelTestUtils.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.io.network.partition;
+
+import org.apache.flink.runtime.io.network.ConnectionID;
+import org.apache.flink.runtime.io.network.ConnectionManager;
+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.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Some utility methods used for testing InputChannels and InputGates.
+ */
+class InputChannelTestUtils {
+
+	/**
+	 * Creates a simple Buffer that is not recycled (never will be) of the given size.
+	 */
+	public static Buffer createMockBuffer(int size) {
+		final Buffer mockBuffer = mock(Buffer.class);
+		when(mockBuffer.isBuffer()).thenReturn(true);
+		when(mockBuffer.getSize()).thenReturn(size);
+		when(mockBuffer.isRecycled()).thenReturn(false);
+
+		return mockBuffer;
+	}
+
+	/**
+	 * Creates a result partition manager that ignores all IDs, and simply returns the given
+	 * subpartitions in sequence.
+	 */
+	public static ResultPartitionManager createResultPartitionManager(final ResultSubpartition[] sources) throws Exception {
+
+		final Answer<ResultSubpartitionView> viewCreator = new Answer<ResultSubpartitionView>() {
+
+			private int num = 0;
+
+			@Override
+			public ResultSubpartitionView answer(InvocationOnMock invocation) throws Throwable {
+				BufferAvailabilityListener channel = (BufferAvailabilityListener) invocation.getArguments()[3];
+				return sources[num++].createReadView(null, channel);
+			}
+		};
+
+		ResultPartitionManager manager = mock(ResultPartitionManager.class);
+		when(manager.createSubpartitionView(
+				any(ResultPartitionID.class), anyInt(), any(BufferProvider.class), any(BufferAvailabilityListener.class)))
+				.thenAnswer(viewCreator);
+
+		return manager;
+	}
+	
+	public static ConnectionManager createDummyConnectionManager() throws Exception {
+		final PartitionRequestClient mockClient = mock(PartitionRequestClient.class);
+
+		final ConnectionManager connManager = mock(ConnectionManager.class);
+		when(connManager.createPartitionRequestClient(any(ConnectionID.class))).thenReturn(mockClient);
+
+		return connManager;
+	}
+
+	// ------------------------------------------------------------------------
+
+	/** This class is not meant to be instantiated */
+	private InputChannelTestUtils() {}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/c0cdc5c4/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateConcurrentTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateConcurrentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateConcurrentTest.java
new file mode 100644
index 0000000..6570679
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateConcurrentTest.java
@@ -0,0 +1,323 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.api.common.JobID;
+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.TaskEventDispatcher;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+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.io.network.partition.consumer.SingleInputGate;
+import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
+import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
+import org.apache.flink.runtime.operators.testutils.UnregisteredTaskMetricsGroup;
+import org.apache.flink.runtime.taskmanager.TaskActions;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+
+import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createDummyConnectionManager;
+import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createResultPartitionManager;
+import static org.junit.Assert.assertNotNull;
+import static org.mockito.Mockito.mock;
+
+public class InputGateConcurrentTest {
+
+	@Test
+	public void testConsumptionWithLocalChannels() throws Exception {
+		final int numChannels = 11;
+		final int buffersPerChannel = 1000;
+
+		final ResultPartition resultPartition = mock(ResultPartition.class);
+
+		final PipelinedSubpartition[] partitions = new PipelinedSubpartition[numChannels];
+		final Source[] sources = new Source[numChannels];
+
+		final ResultPartitionManager resultPartitionManager = createResultPartitionManager(partitions);
+
+		final SingleInputGate gate = new SingleInputGate(
+				"Test Task Name",
+				new JobID(),
+				new ExecutionAttemptID(),
+				new IntermediateDataSetID(),
+				0, numChannels,
+				mock(TaskActions.class),
+				new UnregisteredTaskMetricsGroup.DummyTaskIOMetricGroup());
+
+		for (int i = 0; i < numChannels; i++) {
+			LocalInputChannel channel = new LocalInputChannel(gate, i, new ResultPartitionID(),
+					resultPartitionManager, mock(TaskEventDispatcher.class), new UnregisteredTaskMetricsGroup.DummyTaskIOMetricGroup());
+			gate.setInputChannel(new IntermediateResultPartitionID(), channel);
+
+			partitions[i] = new PipelinedSubpartition(0, resultPartition);
+			sources[i] = new PipelinedSubpartitionSource(partitions[i]);
+		}
+
+		ProducerThread producer = new ProducerThread(sources, numChannels * buffersPerChannel, 4, 10);
+		ConsumerThread consumer = new ConsumerThread(gate, numChannels * buffersPerChannel);
+		producer.start();
+		consumer.start();
+
+		// the 'sync()' call checks for exceptions and failed assertions
+		producer.sync();
+		consumer.sync();
+	}
+
+	@Test
+	public void testConsumptionWithRemoteChannels() throws Exception {
+		final int numChannels = 11;
+		final int buffersPerChannel = 1000;
+
+		final ConnectionManager connManager = createDummyConnectionManager();
+		final Source[] sources = new Source[numChannels];
+
+		final SingleInputGate gate = new SingleInputGate(
+				"Test Task Name",
+				new JobID(),
+				new ExecutionAttemptID(),
+				new IntermediateDataSetID(),
+				0,
+				numChannels,
+				mock(TaskActions.class),
+				new UnregisteredTaskMetricsGroup.DummyTaskIOMetricGroup());
+
+		for (int i = 0; i < numChannels; i++) {
+			RemoteInputChannel channel = new RemoteInputChannel(
+					gate, i, new ResultPartitionID(), mock(ConnectionID.class),
+					connManager, 0, 0, new UnregisteredTaskMetricsGroup.DummyTaskIOMetricGroup());
+			gate.setInputChannel(new IntermediateResultPartitionID(), channel);
+
+			sources[i] = new RemoteChannelSource(channel);
+		}
+
+		ProducerThread producer = new ProducerThread(sources, numChannels * buffersPerChannel, 4, 10);
+		ConsumerThread consumer = new ConsumerThread(gate, numChannels * buffersPerChannel);
+		producer.start();
+		consumer.start();
+
+		// the 'sync()' call checks for exceptions and failed assertions
+		producer.sync();
+		consumer.sync();
+	}
+
+	@Test
+	public void testConsumptionWithMixedChannels() throws Exception {
+		final int numChannels = 61;
+		final int numLocalChannels = 20;
+		final int buffersPerChannel = 1000;
+
+		// fill the local/remote decision
+		List<Boolean> localOrRemote = new ArrayList<>(numChannels);
+		for (int i = 0; i < numChannels; i++) {
+			localOrRemote.add(i < numLocalChannels);
+		}
+		Collections.shuffle(localOrRemote);
+
+		final ConnectionManager connManager = createDummyConnectionManager();
+		final ResultPartition resultPartition = mock(ResultPartition.class);
+
+		final PipelinedSubpartition[] localPartitions = new PipelinedSubpartition[numLocalChannels];
+		final ResultPartitionManager resultPartitionManager = createResultPartitionManager(localPartitions);
+
+		final Source[] sources = new Source[numChannels];
+
+		final SingleInputGate gate = new SingleInputGate(
+				"Test Task Name",
+				new JobID(),
+				new ExecutionAttemptID(),
+				new IntermediateDataSetID(),
+				0,
+				numChannels,
+				mock(TaskActions.class),
+				new UnregisteredTaskMetricsGroup.DummyTaskIOMetricGroup());
+
+		for (int i = 0, local = 0; i < numChannels; i++) {
+			if (localOrRemote.get(i)) {
+				// local channel
+				PipelinedSubpartition psp = new PipelinedSubpartition(0, resultPartition);
+				localPartitions[local++] = psp;
+				sources[i] = new PipelinedSubpartitionSource(psp);
+
+				LocalInputChannel channel = new LocalInputChannel(gate, i, new ResultPartitionID(),
+						resultPartitionManager, mock(TaskEventDispatcher.class), new UnregisteredTaskMetricsGroup.DummyTaskIOMetricGroup());
+				gate.setInputChannel(new IntermediateResultPartitionID(), channel);
+			}
+			else {
+				//remote channel
+				RemoteInputChannel channel = new RemoteInputChannel(
+						gate, i, new ResultPartitionID(), mock(ConnectionID.class),
+						connManager, 0, 0, new UnregisteredTaskMetricsGroup.DummyTaskIOMetricGroup());
+				gate.setInputChannel(new IntermediateResultPartitionID(), channel);
+
+				sources[i] = new RemoteChannelSource(channel);
+			}
+		}
+
+		ProducerThread producer = new ProducerThread(sources, numChannels * buffersPerChannel, 4, 10);
+		ConsumerThread consumer = new ConsumerThread(gate, numChannels * buffersPerChannel);
+		producer.start();
+		consumer.start();
+
+		// the 'sync()' call checks for exceptions and failed assertions
+		producer.sync();
+		consumer.sync();
+	}
+
+	// ------------------------------------------------------------------------
+	//  testing threads
+	// ------------------------------------------------------------------------
+
+	private static abstract class Source {
+	
+		abstract void addBuffer(Buffer buffer) throws Exception;
+	}
+
+	private static class PipelinedSubpartitionSource extends Source {
+
+		final PipelinedSubpartition partition;
+
+		PipelinedSubpartitionSource(PipelinedSubpartition partition) {
+			this.partition = partition;
+		}
+
+		@Override
+		void addBuffer(Buffer buffer) throws Exception {
+			partition.add(buffer);
+		}
+	}
+
+	private static class RemoteChannelSource extends Source {
+
+		final RemoteInputChannel channel;
+		private int seq = 0;
+
+		RemoteChannelSource(RemoteInputChannel channel) {
+			this.channel = channel;
+		}
+
+		@Override
+		void addBuffer(Buffer buffer) throws Exception {
+			channel.onBuffer(buffer, seq++);
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  testing threads
+	// ------------------------------------------------------------------------
+
+	private static abstract class CheckedThread extends Thread {
+
+		private volatile Throwable error;
+
+		public abstract void go() throws Exception;
+
+		@Override
+		public void run() {
+			try {
+				go();
+			}
+			catch (Throwable t) {
+				error = t;
+			}
+		}
+
+		public void sync() throws Exception {
+			join();
+
+			// propagate the error
+			if (error != null) {
+				if (error instanceof Error) {
+					throw (Error) error;
+				}
+				else if (error instanceof Exception) {
+					throw (Exception) error;
+				}
+				else {
+					throw new Exception(error.getMessage(), error);
+				}
+			}
+		}
+	}
+
+	private static class ProducerThread extends CheckedThread {
+
+		private final Random rnd = new Random();
+		private final Source[] sources;
+		private final int numTotal;
+		private final int maxChunk;
+		private final int yieldAfter;
+
+		ProducerThread(Source[] sources, int numTotal, int maxChunk, int yieldAfter) {
+			this.sources = sources;
+			this.numTotal = numTotal;
+			this.maxChunk = maxChunk;
+			this.yieldAfter = yieldAfter;
+		}
+
+		@Override
+		public void go() throws Exception {
+			final Buffer buffer = InputChannelTestUtils.createMockBuffer(100);
+			int nextYield = numTotal - yieldAfter;
+
+			for (int i = numTotal; i > 0;) {
+				final int nextChannel = rnd.nextInt(sources.length);
+				final int chunk = Math.min(i, rnd.nextInt(maxChunk) + 1);
+
+				final Source next = sources[nextChannel];
+
+				for (int k = chunk; k > 0; --k) {
+					next.addBuffer(buffer);
+				}
+
+				i -= chunk;
+
+				if (i <= nextYield) {
+					nextYield -= yieldAfter;
+					//noinspection CallToThreadYield
+					Thread.yield();
+				}
+
+			}
+		}
+	}
+
+	private static class ConsumerThread extends CheckedThread {
+
+		private final SingleInputGate gate;
+		private final int numBuffers;
+
+		ConsumerThread(SingleInputGate gate, int numBuffers) {
+			this.gate = gate;
+			this.numBuffers = numBuffers;
+		}
+
+		@Override
+		public void go() throws Exception {
+			for (int i = numBuffers; i > 0; --i) {
+				assertNotNull(gate.getNextBufferOrEvent());
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/c0cdc5c4/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java
new file mode 100644
index 0000000..b35612a
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java
@@ -0,0 +1,395 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.api.common.JobID;
+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.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.consumer.BufferOrEvent;
+import org.apache.flink.runtime.io.network.partition.consumer.InputChannel;
+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.io.network.partition.consumer.SingleInputGate;
+import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
+import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
+import org.apache.flink.runtime.metrics.groups.TaskIOMetricGroup;
+import org.apache.flink.runtime.operators.testutils.UnregisteredTaskMetricsGroup;
+import org.apache.flink.runtime.taskmanager.TaskActions;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+
+import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createDummyConnectionManager;
+import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createMockBuffer;
+import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createResultPartitionManager;
+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 InputGateFairnessTest {
+
+	@Test
+	public void testFairConsumptionLocalChannelsPreFilled() throws Exception {
+		final int numChannels = 37;
+		final int buffersPerChannel = 27;
+
+		final ResultPartition resultPartition = mock(ResultPartition.class);
+		final Buffer mockBuffer = createMockBuffer(42);
+
+		// ----- create some source channels and fill them with buffers -----
+
+		final PipelinedSubpartition[] sources = new PipelinedSubpartition[numChannels];
+
+		for (int i = 0; i < numChannels; i++) {
+			PipelinedSubpartition partition = new PipelinedSubpartition(0, resultPartition);
+
+			for (int p = 0; p < buffersPerChannel; p++) {
+				partition.add(mockBuffer);
+			}
+
+			partition.finish();
+			sources[i] = partition;
+		}
+
+		// ----- create reading side -----
+
+		ResultPartitionManager resultPartitionManager = createResultPartitionManager(sources);
+
+		SingleInputGate gate = new FairnessVerifyingInputGate(
+				"Test Task Name",
+				new JobID(),
+				new ExecutionAttemptID(),
+				new IntermediateDataSetID(),
+				0, numChannels,
+				mock(TaskActions.class),
+				new UnregisteredTaskMetricsGroup.DummyTaskIOMetricGroup());
+
+		for (int i = 0; i < numChannels; i++) {
+			LocalInputChannel channel = new LocalInputChannel(gate, i, new ResultPartitionID(),
+					resultPartitionManager, mock(TaskEventDispatcher.class), new UnregisteredTaskMetricsGroup.DummyTaskIOMetricGroup());
+			gate.setInputChannel(new IntermediateResultPartitionID(), channel);
+		}
+
+		// read all the buffers and the EOF event
+		for (int i = numChannels * (buffersPerChannel + 1); i > 0; --i) {
+			assertNotNull(gate.getNextBufferOrEvent());
+
+			int min = Integer.MAX_VALUE;
+			int max = 0;
+
+			for (PipelinedSubpartition source : sources) {
+				int size = source.getCurrentNumberOfBuffers();
+				min = Math.min(min, size);
+				max = Math.max(max, size);
+			}
+
+			assertTrue(max == min || max == min+1);
+		}
+
+		assertNull(gate.getNextBufferOrEvent());
+	}
+
+	@Test
+	public void testFairConsumptionLocalChannels() throws Exception {
+		final int numChannels = 37;
+		final int buffersPerChannel = 27;
+
+		final ResultPartition resultPartition = mock(ResultPartition.class);
+		final Buffer mockBuffer = createMockBuffer(42);
+
+		// ----- create some source channels and fill them with one buffer each -----
+
+		final PipelinedSubpartition[] sources = new PipelinedSubpartition[numChannels];
+
+		for (int i = 0; i < numChannels; i++) {
+			sources[i] = new PipelinedSubpartition(0, resultPartition);
+		}
+
+		// ----- create reading side -----
+
+		ResultPartitionManager resultPartitionManager = createResultPartitionManager(sources);
+
+		SingleInputGate gate = new FairnessVerifyingInputGate(
+				"Test Task Name",
+				new JobID(),
+				new ExecutionAttemptID(),
+				new IntermediateDataSetID(),
+				0, numChannels,
+				mock(TaskActions.class),
+				new UnregisteredTaskMetricsGroup.DummyTaskIOMetricGroup());
+
+		for (int i = 0; i < numChannels; i++) {
+			LocalInputChannel channel = new LocalInputChannel(gate, i, new ResultPartitionID(),
+					resultPartitionManager, mock(TaskEventDispatcher.class), new UnregisteredTaskMetricsGroup.DummyTaskIOMetricGroup());
+			gate.setInputChannel(new IntermediateResultPartitionID(), channel);
+		}
+
+		// seed one initial buffer
+		sources[12].add(mockBuffer);
+
+		// read all the buffers and the EOF event
+		for (int i = 0; i < numChannels * buffersPerChannel; i++) {
+			assertNotNull(gate.getNextBufferOrEvent());
+
+			int min = Integer.MAX_VALUE;
+			int max = 0;
+
+			for (PipelinedSubpartition source : sources) {
+				int size = source.getCurrentNumberOfBuffers();
+				min = Math.min(min, size);
+				max = Math.max(max, size);
+			}
+
+			assertTrue(max == min || max == min+1);
+
+			if (i % (2 * numChannels) == 0) {
+				// add three buffers to each channel, in random order
+				fillRandom(sources, 3, mockBuffer);
+			}
+		}
+
+		// there is still more in the queues
+	}
+
+	@Test
+	public void testFairConsumptionRemoteChannelsPreFilled() throws Exception {
+		final int numChannels = 37;
+		final int buffersPerChannel = 27;
+
+		final Buffer mockBuffer = createMockBuffer(42);
+
+		// ----- create some source channels and fill them with buffers -----
+
+		SingleInputGate gate = new FairnessVerifyingInputGate(
+				"Test Task Name",
+				new JobID(),
+				new ExecutionAttemptID(),
+				new IntermediateDataSetID(),
+				0, numChannels,
+				mock(TaskActions.class),
+				new UnregisteredTaskMetricsGroup.DummyTaskIOMetricGroup());
+
+		final ConnectionManager connManager = createDummyConnectionManager();
+
+		final RemoteInputChannel[] channels = new RemoteInputChannel[numChannels];
+
+		for (int i = 0; i < numChannels; i++) {
+			RemoteInputChannel channel = new RemoteInputChannel(
+					gate, i, new ResultPartitionID(), mock(ConnectionID.class), 
+					connManager, 0, 0, new UnregisteredTaskMetricsGroup.DummyTaskIOMetricGroup());
+
+			channels[i] = channel;
+			
+			for (int p = 0; p < buffersPerChannel; p++) {
+				channel.onBuffer(mockBuffer, p);
+			}
+			channel.onBuffer(EventSerializer.toBuffer(EndOfPartitionEvent.INSTANCE), buffersPerChannel);
+
+			gate.setInputChannel(new IntermediateResultPartitionID(), channel);
+		}
+
+		// read all the buffers and the EOF event
+		for (int i = numChannels * (buffersPerChannel + 1); i > 0; --i) {
+			assertNotNull(gate.getNextBufferOrEvent());
+
+			int min = Integer.MAX_VALUE;
+			int max = 0;
+
+			for (RemoteInputChannel channel : channels) {
+				int size = channel.getNumberOfQueuedBuffers();
+				min = Math.min(min, size);
+				max = Math.max(max, size);
+			}
+
+			assertTrue(max == min || max == min+1);
+		}
+
+		assertNull(gate.getNextBufferOrEvent());
+	}
+
+	@Test
+	public void testFairConsumptionRemoteChannels() throws Exception {
+		final int numChannels = 37;
+		final int buffersPerChannel = 27;
+
+		final Buffer mockBuffer = createMockBuffer(42);
+
+		// ----- create some source channels and fill them with buffers -----
+
+		SingleInputGate gate = new FairnessVerifyingInputGate(
+				"Test Task Name",
+				new JobID(),
+				new ExecutionAttemptID(),
+				new IntermediateDataSetID(),
+				0, numChannels,
+				mock(TaskActions.class),
+				new UnregisteredTaskMetricsGroup.DummyTaskIOMetricGroup());
+
+		final ConnectionManager connManager = createDummyConnectionManager();
+
+		final RemoteInputChannel[] channels = new RemoteInputChannel[numChannels];
+		final int[] channelSequenceNums = new int[numChannels];
+
+		for (int i = 0; i < numChannels; i++) {
+			RemoteInputChannel channel = new RemoteInputChannel(
+					gate, i, new ResultPartitionID(), mock(ConnectionID.class),
+					connManager, 0, 0, new UnregisteredTaskMetricsGroup.DummyTaskIOMetricGroup());
+
+			channels[i] = channel;
+			gate.setInputChannel(new IntermediateResultPartitionID(), channel);
+		}
+
+		channels[11].onBuffer(mockBuffer, 0);
+		channelSequenceNums[11]++;
+
+		// read all the buffers and the EOF event
+		for (int i = 0; i < numChannels * buffersPerChannel; i++) {
+			assertNotNull(gate.getNextBufferOrEvent());
+
+			int min = Integer.MAX_VALUE;
+			int max = 0;
+
+			for (RemoteInputChannel channel : channels) {
+				int size = channel.getNumberOfQueuedBuffers();
+				min = Math.min(min, size);
+				max = Math.max(max, size);
+			}
+
+			assertTrue(max == min || max == min+1);
+
+			if (i % (2 * numChannels) == 0) {
+				// add three buffers to each channel, in random order
+				fillRandom(channels, channelSequenceNums, 3, mockBuffer);
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+	private void fillRandom(PipelinedSubpartition[] partitions, int numPerPartition, Buffer buffer) throws Exception {
+		ArrayList<Integer> poss = new ArrayList<>(partitions.length * numPerPartition);
+
+		for (int i = 0; i < partitions.length; i++) {
+			for (int k = 0; k < numPerPartition; k++) {
+				poss.add(i);
+			}
+		}
+
+		Collections.shuffle(poss);
+
+		for (Integer i : poss) {
+			partitions[i].add(buffer);
+		}
+	}
+
+	private void fillRandom(
+			RemoteInputChannel[] partitions,
+			int[] sequenceNumbers,
+			int numPerPartition,
+			Buffer buffer) throws Exception {
+
+		ArrayList<Integer> poss = new ArrayList<>(partitions.length * numPerPartition);
+
+		for (int i = 0; i < partitions.length; i++) {
+			for (int k = 0; k < numPerPartition; k++) {
+				poss.add(i);
+			}
+		}
+
+		Collections.shuffle(poss);
+
+		for (int i : poss) {
+			partitions[i].onBuffer(buffer, sequenceNumbers[i]++);
+		}
+	}
+	
+	// ------------------------------------------------------------------------
+
+	private static class FairnessVerifyingInputGate extends SingleInputGate {
+
+		private final ArrayDeque<InputChannel> channelsWithData;
+
+		private final HashSet<InputChannel> uniquenessChecker;
+
+		@SuppressWarnings("unchecked")
+		public FairnessVerifyingInputGate(
+				String owningTaskName,
+				JobID jobId,
+				ExecutionAttemptID executionId,
+				IntermediateDataSetID consumedResultId,
+				int consumedSubpartitionIndex,
+				int numberOfInputChannels,
+				TaskActions taskActions,
+				TaskIOMetricGroup metrics) {
+
+			super(owningTaskName, jobId, executionId, consumedResultId, consumedSubpartitionIndex,
+					numberOfInputChannels, taskActions, metrics);
+
+			try {
+				Field f = SingleInputGate.class.getDeclaredField("inputChannelsWithData");
+				f.setAccessible(true);
+				channelsWithData = (ArrayDeque<InputChannel>) f.get(this);
+			}
+			catch (Exception e) {
+				throw new RuntimeException(e);
+			}
+
+			this.uniquenessChecker = new HashSet<>();
+		}
+
+
+		@Override
+		public BufferOrEvent getNextBufferOrEvent() throws IOException, InterruptedException {
+			synchronized (channelsWithData) {
+				assertTrue("too many input channels", channelsWithData.size() <= getNumberOfInputChannels());
+				ensureUnique(channelsWithData);
+			}
+
+			return super.getNextBufferOrEvent();
+		}
+
+		private void ensureUnique(Collection<InputChannel> channels) {
+			HashSet<InputChannel> uniquenessChecker = this.uniquenessChecker;
+
+			for (InputChannel channel : channels) {
+				if (!uniquenessChecker.add(channel)) {
+					fail("Duplicate channel in input gate: " + channel);
+				}
+			}
+
+			assertTrue("found duplicate input channels", uniquenessChecker.size() == channels.size());
+			uniquenessChecker.clear();
+		}
+	}
+}


[3/6] flink git commit: [FLINK-5169] [network] Adjust tests to new consumer logic

Posted by uc...@apache.org.
[FLINK-5169] [network] Adjust tests to new consumer logic


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

Branch: refs/heads/master
Commit: d3ac0adfd7ed8878f0c80e0c454c580969e40cfc
Parents: f728129
Author: Ufuk Celebi <uc...@apache.org>
Authored: Mon Nov 28 09:59:58 2016 +0100
Committer: Ufuk Celebi <uc...@apache.org>
Committed: Thu Dec 1 21:42:49 2016 +0100

----------------------------------------------------------------------
 .../runtime/io/disk/SpillingBufferTest.java     |  40 ++--
 .../iomanager/BufferFileWriterReaderTest.java   |   1 -
 .../io/network/api/reader/BufferReaderTest.java | 115 -----------
 .../netty/CancelPartitionRequestTest.java       |  37 ++--
 .../netty/PartitionRequestQueueTest.java        |  23 ++-
 .../netty/ServerTransportErrorHandlingTest.java |  54 +++---
 .../PartialConsumePipelinedResultTest.java      |  18 +-
 .../partition/PipelinedSubpartitionTest.java    | 118 +++---------
 .../network/partition/ResultPartitionTest.java  |   1 -
 .../partition/SpillableSubpartitionTest.java    |  20 +-
 .../SpilledSubpartitionViewAsyncIOTest.java     |  65 -------
 .../SpilledSubpartitionViewSyncIOTest.java      | 103 ----------
 .../partition/SpilledSubpartitionViewTest.java  | 192 ++++++++++++-------
 .../network/partition/SubpartitionTestBase.java |  10 +-
 .../partition/consumer/InputChannelTest.java    |  13 +-
 .../IteratorWrappingTestSingleInputGate.java    |  23 +--
 .../consumer/LocalInputChannelTest.java         |  84 ++++----
 .../consumer/RemoteInputChannelTest.java        |   2 +-
 .../partition/consumer/SingleInputGateTest.java |  78 +++-----
 .../partition/consumer/TestInputChannel.java    |  32 +---
 .../partition/consumer/TestSingleInputGate.java | 101 ++--------
 .../partition/consumer/UnionInputGateTest.java  |  43 +++--
 .../network/util/TestSubpartitionConsumer.java  |  69 ++++---
 .../runtime/operators/DataSinkTaskTest.java     |   6 +-
 .../operators/chaining/ChainTaskTest.java       |  12 +-
 .../operators/testutils/MockEnvironment.java    |  12 +-
 .../operators/testutils/TaskTestBase.java       |   2 +-
 .../TaskCancelAsyncProducerConsumerITCase.java  |  34 +---
 .../consumer/StreamTestSingleInputGate.java     |  46 +++--
 .../io/BarrierBufferMassiveRandomTest.java      |  17 +-
 .../streaming/runtime/io/MockInputGate.java     |  28 +--
 .../tasks/OneInputStreamTaskTestHarness.java    |  27 +--
 .../runtime/tasks/StreamMockEnvironment.java    |  21 +-
 .../StreamTaskCancellationBarrierTest.java      |   2 -
 .../runtime/tasks/StreamTaskTestHarness.java    |  14 +-
 .../runtime/tasks/TwoInputStreamTaskTest.java   |   6 +-
 .../tasks/TwoInputStreamTaskTestHarness.java    |   1 -
 37 files changed, 554 insertions(+), 916 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java
index 538c416..01a9723 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java
@@ -109,7 +109,7 @@ public class SpillingBufferTest {
 		DataInputView inView = outView.flip();
 		generator.reset();
 		
-		// read and re-generate all records and compare them
+		// notifyNonEmpty and re-generate all records and compare them
 		final Tuple2<Integer, String> readRec = new Tuple2<>();
 		for (int i = 0; i < NUM_PAIRS_INMEM; i++) {
 			generator.next(rec);
@@ -121,14 +121,14 @@ public class SpillingBufferTest {
 			int k2 = readRec.f0;
 			String v2 = readRec.f1;
 			
-			Assert.assertTrue("The re-generated and the read record do not match.", k1 == k2 && v1.equals(v2));
+			Assert.assertTrue("The re-generated and the notifyNonEmpty record do not match.", k1 == k2 && v1.equals(v2));
 		}
 		
-		// re-read the data
+		// re-notifyNonEmpty the data
 		inView = outView.flip();
 		generator.reset();
 		
-		// read and re-generate all records and compare them
+		// notifyNonEmpty and re-generate all records and compare them
 		for (int i = 0; i < NUM_PAIRS_INMEM; i++) {
 			generator.next(rec);
 			serializer.deserialize(readRec, inView);
@@ -139,7 +139,7 @@ public class SpillingBufferTest {
 			int k2 = readRec.f0;
 			String v2 = readRec.f1;
 			
-			Assert.assertTrue("The re-generated and the read record do not match.", k1 == k2 && v1.equals(v2));
+			Assert.assertTrue("The re-generated and the notifyNonEmpty record do not match.", k1 == k2 && v1.equals(v2));
 		}
 		
 		this.memoryManager.release(outView.close());
@@ -169,7 +169,7 @@ public class SpillingBufferTest {
 		DataInputView inView = outView.flip();
 		generator.reset();
 		
-		// read and re-generate all records and compare them
+		// notifyNonEmpty and re-generate all records and compare them
 		final Tuple2<Integer, String> readRec = new Tuple2<>();
 		try {
 			for (int i = 0; i < NUM_PAIRS_INMEM + 1; i++) {
@@ -182,7 +182,7 @@ public class SpillingBufferTest {
 				int k2 = readRec.f0;
 				String v2 = readRec.f1;
 				
-				Assert.assertTrue("The re-generated and the read record do not match.", k1 == k2 && v1.equals(v2));
+				Assert.assertTrue("The re-generated and the notifyNonEmpty record do not match.", k1 == k2 && v1.equals(v2));
 			}
 			Assert.fail("Read too much, expected EOFException.");
 		}
@@ -190,11 +190,11 @@ public class SpillingBufferTest {
 			// expected
 		}
 		
-		// re-read the data
+		// re-notifyNonEmpty the data
 		inView = outView.flip();
 		generator.reset();
 		
-		// read and re-generate all records and compare them
+		// notifyNonEmpty and re-generate all records and compare them
 		for (int i = 0; i < NUM_PAIRS_INMEM; i++) {
 			generator.next(rec);
 			serializer.deserialize(readRec, inView);
@@ -205,7 +205,7 @@ public class SpillingBufferTest {
 			int k2 = readRec.f0;
 			String v2 = readRec.f1;
 			
-			Assert.assertTrue("The re-generated and the read record do not match.", k1 == k2 && v1.equals(v2));
+			Assert.assertTrue("The re-generated and the notifyNonEmpty record do not match.", k1 == k2 && v1.equals(v2));
 		}
 		
 		this.memoryManager.release(outView.close());
@@ -237,7 +237,7 @@ public class SpillingBufferTest {
 		DataInputView inView = outView.flip();
 		generator.reset();
 		
-		// read and re-generate all records and compare them
+		// notifyNonEmpty and re-generate all records and compare them
 		final Tuple2<Integer, String> readRec = new Tuple2<>();
 		for (int i = 0; i < NUM_PAIRS_EXTERNAL; i++) {
 			generator.next(rec);
@@ -249,14 +249,14 @@ public class SpillingBufferTest {
 			int k2 = readRec.f0;
 			String v2 = readRec.f1;
 			
-			Assert.assertTrue("The re-generated and the read record do not match.", k1 == k2 && v1.equals(v2));
+			Assert.assertTrue("The re-generated and the notifyNonEmpty record do not match.", k1 == k2 && v1.equals(v2));
 		}
 		
-		// re-read the data
+		// re-notifyNonEmpty the data
 		inView = outView.flip();
 		generator.reset();
 		
-		// read and re-generate all records and compare them
+		// notifyNonEmpty and re-generate all records and compare them
 		for (int i = 0; i < NUM_PAIRS_EXTERNAL; i++) {
 			generator.next(rec);
 			serializer.deserialize(readRec, inView);
@@ -267,7 +267,7 @@ public class SpillingBufferTest {
 			int k2 = readRec.f0;
 			String v2 = readRec.f1;
 			
-			Assert.assertTrue("The re-generated and the read record do not match.", k1 == k2 && v1.equals(v2));
+			Assert.assertTrue("The re-generated and the notifyNonEmpty record do not match.", k1 == k2 && v1.equals(v2));
 		}
 		
 		this.memoryManager.release(outView.close());
@@ -297,7 +297,7 @@ public class SpillingBufferTest {
 		DataInputView inView = outView.flip();
 		generator.reset();
 		
-		// read and re-generate all records and compare them
+		// notifyNonEmpty and re-generate all records and compare them
 		final Tuple2<Integer, String> readRec = new Tuple2<>();
 		try {
 			for (int i = 0; i < NUM_PAIRS_EXTERNAL + 1; i++) {
@@ -310,7 +310,7 @@ public class SpillingBufferTest {
 				int k2 = readRec.f0;
 				String v2 = readRec.f1;
 				
-				Assert.assertTrue("The re-generated and the read record do not match.", k1 == k2 && v1.equals(v2));
+				Assert.assertTrue("The re-generated and the notifyNonEmpty record do not match.", k1 == k2 && v1.equals(v2));
 			}
 			Assert.fail("Read too much, expected EOFException.");
 		}
@@ -318,11 +318,11 @@ public class SpillingBufferTest {
 			// expected
 		}
 		
-		// re-read the data
+		// re-notifyNonEmpty the data
 		inView = outView.flip();
 		generator.reset();
 		
-		// read and re-generate all records and compare them
+		// notifyNonEmpty and re-generate all records and compare them
 		for (int i = 0; i < NUM_PAIRS_EXTERNAL; i++) {
 			generator.next(rec);
 			serializer.deserialize(readRec, inView);
@@ -333,7 +333,7 @@ public class SpillingBufferTest {
 			int k2 = readRec.f0;
 			String v2 = readRec.f1;
 			
-			Assert.assertTrue("The re-generated and the read record do not match.", k1 == k2 && v1.equals(v2));
+			Assert.assertTrue("The re-generated and the notifyNonEmpty record do not match.", k1 == k2 && v1.equals(v2));
 		}
 		
 		this.memoryManager.release(outView.close());

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/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
index 375be45..2da0f7e 100644
--- 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
@@ -23,7 +23,6 @@ import org.apache.flink.core.memory.MemorySegmentFactory;
 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;

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/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
deleted file mode 100644
index 099b6fb..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/BufferReaderTest.java
+++ /dev/null
@@ -1,115 +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.runtime.event.TaskEvent;
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.partition.consumer.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;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.powermock.core.classloader.annotations.PowerMockIgnore;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.powermock.modules.junit4.PowerMockRunner;
-
-import java.io.IOException;
-
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-
-@RunWith(PowerMockRunner.class)
-@PrepareForTest(Task.class)
-@PowerMockIgnore({"javax.management.*", "com.sun.jndi.*"})
-@SuppressWarnings("unchecked")
-public class BufferReaderTest {
-
-	@Test
-	public void testGetNextBufferOrEvent() throws IOException, InterruptedException {
-
-		final TestSingleInputGate inputGate = new TestSingleInputGate(1)
-				.readBuffer().readBuffer().readEvent()
-				.readBuffer().readBuffer().readEvent()
-				.readBuffer().readEndOfPartitionEvent();
-
-		final BufferReader reader = new BufferReader(inputGate.getInputGate());
-
-		// Task event listener to be notified...
-		final EventListener<TaskEvent> listener = mock(EventListener.class);
-		reader.registerTaskEventListener(listener, TestTaskEvent.class);
-
-		int numReadBuffers = 0;
-		while ((reader.getNextBuffer()) != null) {
-			numReadBuffers++;
-		}
-
-		assertEquals(5, numReadBuffers);
-		verify(listener, times(2)).onEvent(any(TaskEvent.class));
-	}
-
-	@Test
-	public void testIterativeGetNextBufferOrEvent() throws IOException, InterruptedException {
-
-		final TestSingleInputGate inputGate = new TestSingleInputGate(1)
-				.readBuffer().readBuffer().readEvent()
-				.readBuffer().readBuffer().readEvent()
-				.readBuffer().readEndOfSuperstepEvent()
-				.readBuffer().readBuffer().readEvent()
-				.readBuffer().readBuffer().readEvent()
-				.readBuffer().readEndOfPartitionEvent();
-
-		final BufferReader reader = new BufferReader(inputGate.getInputGate());
-
-		// Set reader iterative
-		reader.setIterativeReader();
-
-		// Task event listener to be notified...
-		final EventListener<TaskEvent> listener = mock(EventListener.class);
-		// Task event listener to be notified...
-		reader.registerTaskEventListener(listener, TestTaskEvent.class);
-
-		int numReadBuffers = 0;
-		int numEndOfSuperstepEvents = 0;
-
-		while (true) {
-			Buffer buffer = reader.getNextBuffer();
-
-			if (buffer != null) {
-				numReadBuffers++;
-			}
-			else if (reader.hasReachedEndOfSuperstep()) {
-				reader.startNextSuperstep();
-
-				numEndOfSuperstepEvents++;
-			}
-			else if (reader.isFinished()) {
-				break;
-			}
-		}
-
-		assertEquals(10, numReadBuffers);
-		assertEquals(1, numEndOfSuperstepEvents);
-
-		verify(listener, times(4)).onEvent(any(TaskEvent.class));
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CancelPartitionRequestTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CancelPartitionRequestTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CancelPartitionRequestTest.java
index 1ff1e99..a2f866a 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CancelPartitionRequestTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CancelPartitionRequestTest.java
@@ -24,14 +24,16 @@ 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.NetworkBufferPool;
 import org.apache.flink.runtime.io.network.netty.NettyTestUtil.NettyServerAndClient;
+import org.apache.flink.runtime.io.network.partition.BufferAvailabilityListener;
 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.partition.consumer.InputChannelID;
 import org.apache.flink.runtime.io.network.util.TestPooledBufferProvider;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
-import org.apache.flink.runtime.util.event.NotificationListener;
 import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 import java.io.IOException;
 import java.util.concurrent.CountDownLatch;
@@ -73,11 +75,18 @@ public class CancelPartitionRequestTest {
 
 			CountDownLatch sync = new CountDownLatch(1);
 
-			ResultSubpartitionView view = spy(new InfiniteSubpartitionView(outboundBuffers, sync));
+			final ResultSubpartitionView view = spy(new InfiniteSubpartitionView(outboundBuffers, sync));
 
 			// Return infinite subpartition
-			when(partitions.createSubpartitionView(eq(pid), eq(0), any(BufferProvider.class)))
-					.thenReturn(view);
+			when(partitions.createSubpartitionView(eq(pid), eq(0), any(BufferProvider.class), any(BufferAvailabilityListener.class)))
+				.thenAnswer(new Answer<ResultSubpartitionView>() {
+					@Override
+					public ResultSubpartitionView answer(InvocationOnMock invocationOnMock) throws Throwable {
+						BufferAvailabilityListener listener = (BufferAvailabilityListener) invocationOnMock.getArguments()[3];
+						listener.notifyBuffersAvailable(Long.MAX_VALUE);
+						return view;
+					}
+				});
 
 			PartitionRequestProtocol protocol = new PartitionRequestProtocol(
 					partitions, mock(TaskEventDispatcher.class), mock(NetworkBufferPool.class));
@@ -109,19 +118,26 @@ public class CancelPartitionRequestTest {
 		NettyServerAndClient serverAndClient = null;
 
 		try {
-			TestPooledBufferProvider outboundBuffers = new TestPooledBufferProvider(16);
+			final TestPooledBufferProvider outboundBuffers = new TestPooledBufferProvider(16);
 
 			ResultPartitionManager partitions = mock(ResultPartitionManager.class);
 
 			ResultPartitionID pid = new ResultPartitionID();
 
-			CountDownLatch sync = new CountDownLatch(1);
+			final CountDownLatch sync = new CountDownLatch(1);
 
-			ResultSubpartitionView view = spy(new InfiniteSubpartitionView(outboundBuffers, sync));
+			final ResultSubpartitionView view = spy(new InfiniteSubpartitionView(outboundBuffers, sync));
 
 			// Return infinite subpartition
-			when(partitions.createSubpartitionView(eq(pid), eq(0), any(BufferProvider.class)))
-					.thenReturn(view);
+			when(partitions.createSubpartitionView(eq(pid), eq(0), any(BufferProvider.class), any(BufferAvailabilityListener.class)))
+					.thenAnswer(new Answer<ResultSubpartitionView>() {
+						@Override
+						public ResultSubpartitionView answer(InvocationOnMock invocationOnMock) throws Throwable {
+							BufferAvailabilityListener listener = (BufferAvailabilityListener) invocationOnMock.getArguments()[3];
+							listener.notifyBuffersAvailable(Long.MAX_VALUE);
+							return view;
+						}
+					});
 
 			PartitionRequestProtocol protocol = new PartitionRequestProtocol(
 					partitions, mock(TaskEventDispatcher.class), mock(NetworkBufferPool.class));
@@ -174,8 +190,7 @@ public class CancelPartitionRequestTest {
 		}
 
 		@Override
-		public boolean registerListener(final NotificationListener listener) throws IOException {
-			return false;
+		public void notifyBuffersAvailable(long buffers) throws IOException {
 		}
 
 		@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueueTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueueTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueueTest.java
index 3f281bd..7224e96 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueueTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueueTest.java
@@ -20,12 +20,18 @@ package org.apache.flink.runtime.io.network.netty;
 
 import io.netty.channel.embedded.EmbeddedChannel;
 import org.apache.flink.runtime.execution.CancelTaskException;
+import org.apache.flink.runtime.io.network.buffer.BufferProvider;
+import org.apache.flink.runtime.io.network.partition.BufferAvailabilityListener;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+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.junit.Test;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -35,14 +41,27 @@ public class PartitionRequestQueueTest {
 	public void testProducerFailedException() throws Exception {
 		PartitionRequestQueue queue = new PartitionRequestQueue();
 
-		EmbeddedChannel ch = new EmbeddedChannel(queue);
+		ResultPartitionProvider partitionProvider = mock(ResultPartitionProvider.class);
+		ResultPartitionID rpid = new ResultPartitionID();
+		BufferProvider bufferProvider = mock(BufferProvider.class);
 
 		ResultSubpartitionView view = mock(ResultSubpartitionView.class);
 		when(view.isReleased()).thenReturn(true);
 		when(view.getFailureCause()).thenReturn(new RuntimeException("Expected test exception"));
 
+		when(partitionProvider.createSubpartitionView(
+			eq(rpid),
+			eq(0),
+			eq(bufferProvider),
+			any(BufferAvailabilityListener.class))).thenReturn(view);
+
+		EmbeddedChannel ch = new EmbeddedChannel(queue);
+
+		SequenceNumberingViewReader seqView = new SequenceNumberingViewReader(new InputChannelID(), queue);
+		seqView.requestSubpartitionView(partitionProvider, rpid, 0, bufferProvider);
+
 		// Enqueue the erroneous view
-		queue.enqueue(view, new InputChannelID());
+		queue.notifyReaderNonEmpty(seqView);
 		ch.runPendingTasks();
 
 		// Read the enqueued msg

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ServerTransportErrorHandlingTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ServerTransportErrorHandlingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ServerTransportErrorHandlingTest.java
index 1515f83..1c3557e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ServerTransportErrorHandlingTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ServerTransportErrorHandlingTest.java
@@ -25,20 +25,20 @@ import io.netty.channel.ChannelInboundHandlerAdapter;
 import org.apache.flink.runtime.io.network.TaskEventDispatcher;
 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.netty.CancelPartitionRequestTest.InfiniteSubpartitionView;
-import org.apache.flink.runtime.io.network.netty.NettyTestUtil.NettyServerAndClient;
+import org.apache.flink.runtime.io.network.partition.BufferAvailabilityListener;
 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.partition.consumer.InputChannelID;
 import org.apache.flink.runtime.io.network.util.TestPooledBufferProvider;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 
-import static org.apache.flink.runtime.io.network.netty.NettyMessage.NettyMessageEncoder;
-import static org.apache.flink.runtime.io.network.netty.NettyMessage.PartitionRequest;
 import static org.apache.flink.runtime.io.network.netty.NettyTestUtil.connect;
 import static org.apache.flink.runtime.io.network.netty.NettyTestUtil.createConfig;
 import static org.apache.flink.runtime.io.network.netty.NettyTestUtil.initServerAndClient;
@@ -63,36 +63,43 @@ public class ServerTransportErrorHandlingTest {
 		final ResultPartitionManager partitionManager = mock(ResultPartitionManager.class);
 
 		when(partitionManager
-				.createSubpartitionView(any(ResultPartitionID.class), anyInt(), any(BufferProvider.class)))
-				.thenReturn(new InfiniteSubpartitionView(outboundBuffers, sync));
+			.createSubpartitionView(any(ResultPartitionID.class), anyInt(), any(BufferProvider.class), any(BufferAvailabilityListener.class)))
+			.thenAnswer(new Answer<ResultSubpartitionView>() {
+				@Override
+				public ResultSubpartitionView answer(InvocationOnMock invocationOnMock) throws Throwable {
+					BufferAvailabilityListener listener = (BufferAvailabilityListener) invocationOnMock.getArguments()[3];
+					listener.notifyBuffersAvailable(Long.MAX_VALUE);
+					return new CancelPartitionRequestTest.InfiniteSubpartitionView(outboundBuffers, sync);
+				}
+			});
 
 		NettyProtocol protocol = new NettyProtocol() {
 			@Override
 			public ChannelHandler[] getServerChannelHandlers() {
 				return new PartitionRequestProtocol(
-						partitionManager,
-						mock(TaskEventDispatcher.class),
-						mock(NetworkBufferPool.class)).getServerChannelHandlers();
+					partitionManager,
+					mock(TaskEventDispatcher.class),
+					mock(NetworkBufferPool.class)).getServerChannelHandlers();
 			}
 
 			@Override
 			public ChannelHandler[] getClientChannelHandlers() {
-				return new ChannelHandler[] {
-						new NettyMessageEncoder(),
-						// Close on read
-						new ChannelInboundHandlerAdapter() {
-							@Override
-							public void channelRead(ChannelHandlerContext ctx, Object msg)
-									throws Exception {
-
-								ctx.channel().close();
-							}
+				return new ChannelHandler[]{
+					new NettyMessage.NettyMessageEncoder(),
+					// Close on read
+					new ChannelInboundHandlerAdapter() {
+						@Override
+						public void channelRead(ChannelHandlerContext ctx, Object msg)
+							throws Exception {
+
+							ctx.channel().close();
 						}
+					}
 				};
 			}
 		};
 
-		NettyServerAndClient serverAndClient = null;
+		NettyTestUtil.NettyServerAndClient serverAndClient = null;
 
 		try {
 			serverAndClient = initServerAndClient(protocol, createConfig());
@@ -100,15 +107,14 @@ public class ServerTransportErrorHandlingTest {
 			Channel ch = connect(serverAndClient);
 
 			// Write something to trigger close by server
-			ch.writeAndFlush(new PartitionRequest(new ResultPartitionID(), 0, new InputChannelID()));
+			ch.writeAndFlush(new NettyMessage.PartitionRequest(new ResultPartitionID(), 0, new InputChannelID()));
 
 			// Wait for the notification
 			if (!sync.await(TestingUtils.TESTING_DURATION().toMillis(), TimeUnit.MILLISECONDS)) {
 				fail("Timed out after waiting for " + TestingUtils.TESTING_DURATION().toMillis() +
-						" ms to be notified about released partition.");
+					" ms to be notified about released partition.");
 			}
-		}
-		finally {
+		} finally {
 			shutdown(serverAndClient);
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java
index af8aa69..1ec4ad3 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java
@@ -21,12 +21,12 @@ package org.apache.flink.runtime.io.network.partition;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.io.network.api.reader.BufferReader;
 import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
 import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
 import org.apache.flink.runtime.testingUtils.TestingCluster;
@@ -87,12 +87,12 @@ public class PartialConsumePipelinedResultTest {
 		// The partition needs to be pipelined, otherwise the original issue does not occur, because
 		// the sender and receiver are not online at the same time.
 		receiver.connectNewDataSetAsInput(
-				sender, DistributionPattern.POINTWISE, ResultPartitionType.PIPELINED);
+			sender, DistributionPattern.POINTWISE, ResultPartitionType.PIPELINED);
 
 		final JobGraph jobGraph = new JobGraph("Partial Consume of Pipelined Result", sender, receiver);
 
 		final SlotSharingGroup slotSharingGroup = new SlotSharingGroup(
-				sender.getID(), receiver.getID());
+			sender.getID(), receiver.getID());
 
 		sender.setSlotSharingGroup(slotSharingGroup);
 		receiver.setSlotSharingGroup(slotSharingGroup);
@@ -127,11 +127,11 @@ public class PartialConsumePipelinedResultTest {
 
 		@Override
 		public void invoke() throws Exception {
-			final BufferReader reader = new BufferReader(getEnvironment().getInputGate(0));
-
-			final Buffer buffer = reader.getNextBuffer();
-
-			buffer.recycle();
+			InputGate gate = getEnvironment().getInputGate(0);
+			Buffer buffer = gate.getNextBufferOrEvent().getBuffer();
+			if (buffer != null) {
+				buffer.recycle();
+			}
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/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
index 8750a1a..a56177e 100644
--- 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
@@ -24,7 +24,6 @@ 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.TestConsumerCallback;
-import org.apache.flink.runtime.io.network.util.TestNotificationListener;
 import org.apache.flink.runtime.io.network.util.TestPooledBufferProvider;
 import org.apache.flink.runtime.io.network.util.TestProducerSource;
 import org.apache.flink.runtime.io.network.util.TestSubpartitionConsumer;
@@ -38,12 +37,13 @@ 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.Matchers.eq;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
 
 public class PipelinedSubpartitionTest extends SubpartitionTestBase {
 
@@ -63,80 +63,25 @@ public class PipelinedSubpartitionTest extends SubpartitionTestBase {
 	}
 
 	@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.createReadView(null));
+		assertNotNull(subpartition.createReadView(null, new BufferAvailabilityListener() {
+			@Override
+			public void notifyBuffersAvailable(long numBuffers) {
+			}
+		}));
 
 		try {
-			subpartition.createReadView(null);
+			subpartition.createReadView(null, new BufferAvailabilityListener() {
+				@Override
+				public void notifyBuffersAvailable(long numBuffers) {
+				}
+			});
 
-			fail("Did not throw expected exception after duplicate read view request.");
-		}
-		catch (IllegalStateException expected) {
+			fail("Did not throw expected exception after duplicate notifyNonEmpty view request.");
+		} catch (IllegalStateException expected) {
 		}
 	}
 
@@ -144,23 +89,19 @@ public class PipelinedSubpartitionTest extends SubpartitionTestBase {
 	public void testBasicPipelinedProduceConsumeLogic() throws Exception {
 		final PipelinedSubpartition subpartition = createSubpartition();
 
-		TestNotificationListener listener = new TestNotificationListener();
+		BufferAvailabilityListener listener = mock(BufferAvailabilityListener.class);
 
-		ResultSubpartitionView view = subpartition.createReadView(null);
+		ResultSubpartitionView view = subpartition.createReadView(null, listener);
 
 		// Empty => should return null
 		assertNull(view.getNextBuffer());
-
-		// Register listener for notifications
-		assertTrue(view.registerListener(listener));
-
-		assertEquals(0, listener.getNumberOfNotifications());
+		verify(listener, times(1)).notifyBuffersAvailable(eq(0L));
 
 		// Add data to the queue...
 		subpartition.add(createBuffer());
 
 		// ...should have resulted in a notification
-		assertEquals(1, listener.getNumberOfNotifications());
+		verify(listener, times(1)).notifyBuffersAvailable(eq(1L));
 
 		// ...and one available result
 		assertNotNull(view.getNextBuffer());
@@ -168,10 +109,7 @@ public class PipelinedSubpartitionTest extends SubpartitionTestBase {
 
 		// 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());
+		verify(listener, times(2)).notifyBuffersAvailable(eq(1L));
 	}
 
 	@Test
@@ -208,7 +146,6 @@ public class PipelinedSubpartitionTest extends SubpartitionTestBase {
 
 			@Override
 			public BufferOrEvent getNextBufferOrEvent() throws Exception {
-
 				if (numberOfBuffers == producerNumberOfBuffersToProduce) {
 					return null;
 				}
@@ -261,16 +198,17 @@ public class PipelinedSubpartitionTest extends SubpartitionTestBase {
 
 		final PipelinedSubpartition subpartition = createSubpartition();
 
-		final PipelinedSubpartitionView view = subpartition.createReadView(null);
+		TestSubpartitionConsumer consumer = new TestSubpartitionConsumer(isSlowConsumer, consumerCallback);
+		final PipelinedSubpartitionView view = subpartition.createReadView(null, consumer);
+		consumer.setSubpartitionView(view);
 
-		Future<Boolean> producer = executorService.submit(
-				new TestSubpartitionProducer(subpartition, isSlowProducer, producerSource));
+		Future<Boolean> producerResult = executorService.submit(
+			new TestSubpartitionProducer(subpartition, isSlowProducer, producerSource));
 
-		Future<Boolean> consumer = executorService.submit(
-				new TestSubpartitionConsumer(view, isSlowConsumer, consumerCallback));
+		Future<Boolean> consumerResult = executorService.submit(consumer);
 
 		// Wait for producer and consumer to finish
-		producer.get();
-		consumer.get();
+		producerResult.get();
+		consumerResult.get();
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java
index f6fddfa..4eb4fd1 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java
@@ -86,7 +86,6 @@ public class ResultPartitionTest {
 			mock(ResultPartitionManager.class),
 			notifier,
 			mock(IOManager.class),
-			IOManager.IOMode.SYNC,
 			sendScheduleOrUpdateConsumersMessage);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/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
index d7e56c8..b7a54d7 100644
--- 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
@@ -22,6 +22,7 @@ import org.apache.flink.runtime.io.disk.iomanager.AsynchronousBufferFileWriter;
 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.IOManagerAsync;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
 import org.apache.flink.runtime.io.network.util.TestInfiniteBufferProvider;
 import org.junit.AfterClass;
 import org.junit.Test;
@@ -34,7 +35,7 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 
-import static org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode.SYNC;
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.doAnswer;
@@ -59,7 +60,7 @@ public class SpillableSubpartitionTest extends SubpartitionTestBase {
 
 	@Override
 	ResultSubpartition createSubpartition() {
-		return new SpillableSubpartition(0, mock(ResultPartition.class), ioManager, SYNC);
+		return new SpillableSubpartition(0, mock(ResultPartition.class), ioManager);
 	}
 
 	/**
@@ -87,14 +88,14 @@ public class SpillableSubpartitionTest extends SubpartitionTestBase {
 		// Mock I/O manager returning the blocking spill writer
 		IOManager ioManager = mock(IOManager.class);
 		when(ioManager.createBufferFileWriter(any(FileIOChannel.ID.class)))
-				.thenReturn(spillWriter);
+			.thenReturn(spillWriter);
 
 		// The partition
 		final SpillableSubpartition partition = new SpillableSubpartition(
-				0, mock(ResultPartition.class), ioManager, SYNC);
+			0, mock(ResultPartition.class), ioManager);
 
 		// Spill the partition initially (creates the spill writer)
-		partition.releaseMemory();
+		assertEquals(0, partition.releaseMemory());
 
 		ExecutorService executor = Executors.newSingleThreadExecutor();
 
@@ -130,13 +131,18 @@ public class SpillableSubpartitionTest extends SubpartitionTestBase {
 	public void testReleasePartitionAndGetNext() throws Exception {
 		// Create partition and add some buffers
 		SpillableSubpartition partition = new SpillableSubpartition(
-				0, mock(ResultPartition.class), ioManager, SYNC);
+			0, mock(ResultPartition.class), ioManager);
 
 		partition.finish();
 
 		// Create the read view
 		ResultSubpartitionView readView = spy(partition
-				.createReadView(new TestInfiniteBufferProvider()));
+			.createReadView(new TestInfiniteBufferProvider(), new BufferAvailabilityListener() {
+				@Override
+				public void notifyBuffersAvailable(long numBuffers) {
+
+				}
+			}));
 
 		// The released state check (of the parent) needs to be independent
 		// of the released state of the view.

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/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
deleted file mode 100644
index 981c8ee..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewAsyncIOTest.java
+++ /dev/null
@@ -1,65 +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 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/d3ac0adf/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
deleted file mode 100644
index f8baae4..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewSyncIOTest.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.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/d3ac0adf/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 5722cac..8f8da93 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
@@ -18,26 +18,21 @@
 
 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.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 org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -47,55 +42,103 @@ 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 IOManager IO_MANAGER = new IOManagerAsync();
 
 	private static final TestInfiniteBufferProvider writerBufferPool =
-			new TestInfiniteBufferProvider();
-
-	private IOMode ioMode;
-
-	public SpilledSubpartitionViewTest(IOMode ioMode) {
-		this.ioMode = ioMode;
-	}
+		new TestInfiniteBufferProvider();
 
 	@AfterClass
 	public static void shutdown() {
-		ioManager.shutdown();
-		executor.shutdown();
+		IO_MANAGER.shutdown();
 	}
 
-	@Parameterized.Parameters
-	public static Collection<Object[]> ioMode() {
-		return Arrays.asList(new Object[][]{
-				{IOMode.SYNC},
-				{IOMode.ASYNC}});
+	@Test
+	public void testWriteConsume() throws Exception {
+		// Config
+		final int numberOfBuffersToWrite = 512;
+
+		// Setup
+		final BufferFileWriter writer = createWriterAndWriteBuffers(IO_MANAGER, writerBufferPool, numberOfBuffersToWrite);
+
+		writer.close();
+
+		TestPooledBufferProvider viewBufferPool = new TestPooledBufferProvider(1);
+
+		TestSubpartitionConsumer consumer = new TestSubpartitionConsumer(
+			false, new TestConsumerCallback.RecyclingCallback());
+
+		SpilledSubpartitionView view = new SpilledSubpartitionView(
+			mock(ResultSubpartition.class),
+			viewBufferPool.getMemorySegmentSize(),
+			writer,
+			numberOfBuffersToWrite + 1, // +1 for end-of-partition
+			consumer);
+
+		consumer.setSubpartitionView(view);
+
+		// Consume subpartition
+		consumer.call();
 	}
 
 	@Test
-	public void testReadMultipleFilesWithSingleBufferPool() throws Exception {
+	public void testConsumeWithFewBuffers() throws Exception {
+		// Config
+		final int numberOfBuffersToWrite = 512;
+
 		// Setup
-		BufferFileWriter[] writers = new BufferFileWriter[]{
-				createWriterAndWriteBuffers(ioManager, writerBufferPool, 512),
-				createWriterAndWriteBuffers(ioManager, writerBufferPool, 512)
-		};
+		final BufferFileWriter writer = createWriterAndWriteBuffers(IO_MANAGER, writerBufferPool, numberOfBuffersToWrite);
+
+		writer.close();
+
+		TestSubpartitionConsumer consumer = new TestSubpartitionConsumer(
+			false, new TestConsumerCallback.RecyclingCallback());
+
+		SpilledSubpartitionView view = new SpilledSubpartitionView(
+			mock(ResultSubpartition.class),
+			32 * 1024,
+			writer,
+			numberOfBuffersToWrite + 1,
+			consumer);
+
+		consumer.setSubpartitionView(view);
+
+		// 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.
+		consumer.call();
+	}
 
-		final ResultSubpartitionView[] readers = new ResultSubpartitionView[writers.length];
+	@Test
+	public void testReadMultipleFilesWithSingleBufferPool() throws Exception {
+		ExecutorService executor = null;
+		BufferFileWriter[] writers = null;
+		ResultSubpartitionView[] readers = null;
 
-		// 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);
+		try {
+			executor = Executors.newCachedThreadPool();
 
-		final ResultSubpartition parent = mock(ResultSubpartition.class);
+			// Setup
+			writers = new BufferFileWriter[]{
+				createWriterAndWriteBuffers(IO_MANAGER, writerBufferPool, 512),
+				createWriterAndWriteBuffers(IO_MANAGER, writerBufferPool, 512)
+			};
+
+			readers = new ResultSubpartitionView[writers.length];
+			TestSubpartitionConsumer[] consumers = new TestSubpartitionConsumer[writers.length];
+
+			BufferProvider inputBuffers = new TestPooledBufferProvider(2);
+
+			ResultSubpartition parent = mock(ResultSubpartition.class);
 
-		try {
 			// Wait for writers to finish
 			for (BufferFileWriter writer : writers) {
 				writer.close();
@@ -103,56 +146,56 @@ public class SpilledSubpartitionViewTest {
 
 			// 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);
-				}
+				consumers[i] = new TestSubpartitionConsumer(
+					false, new TestConsumerCallback.RecyclingCallback());
+
+				readers[i] = new SpilledSubpartitionView(
+					parent,
+					inputBuffers.getMemorySegmentSize(),
+					writers[i],
+					512 + 1, // +1 for end of partition event
+					consumers[i]);
+
+				consumers[i].setSubpartitionView(readers[i]);
 			}
 
-			final List<Future<Boolean>> results = new ArrayList<>();
+			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())));
+			for (TestSubpartitionConsumer consumer : consumers) {
+				results.add(executor.submit(consumer));
 			}
 
 			// Wait for the results
 			for (Future<Boolean> res : results) {
 				try {
 					res.get(2, TimeUnit.MINUTES);
-				}
-				catch (TimeoutException e) {
+				} 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.");
+						"indicates that there is a bug/deadlock in the tested subpartition " +
+						"view.");
 				}
 			}
-		}
-		finally {
-			for (BufferFileWriter writer : writers) {
-				if (writer != null) {
-					writer.deleteChannel();
+		} finally {
+			if (writers != null) {
+				for (BufferFileWriter writer : writers) {
+					if (writer != null) {
+						writer.deleteChannel();
+					}
 				}
 			}
 
-			for (ResultSubpartitionView reader : readers) {
-				if (reader != null) {
-					reader.releaseAllResources();
+			if (readers != null) {
+				for (ResultSubpartitionView reader : readers) {
+					if (reader != null) {
+						reader.releaseAllResources();
+					}
 				}
 			}
+
+			if (executor != null) {
+				executor.shutdown();
+			}
 		}
 	}
 
@@ -163,9 +206,9 @@ public class SpilledSubpartitionViewTest {
 	 * <p> Call {@link BufferFileWriter#close()} to ensure that all buffers have been written.
 	 */
 	static BufferFileWriter createWriterAndWriteBuffers(
-			IOManager ioManager,
-			BufferProvider bufferProvider,
-			int numberOfBuffers) throws IOException {
+		IOManager ioManager,
+		BufferProvider bufferProvider,
+		int numberOfBuffers) throws IOException {
 
 		final BufferFileWriter writer = ioManager.createBufferFileWriter(ioManager.createChannel());
 
@@ -177,4 +220,5 @@ public class SpilledSubpartitionViewTest {
 
 		return writer;
 	}
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/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
index 26a8f29..14942bc 100644
--- 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
@@ -24,7 +24,6 @@ import org.apache.flink.runtime.io.network.util.TestInfiniteBufferProvider;
 import org.apache.flink.util.TestLogger;
 import org.junit.Test;
 
-import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
@@ -50,8 +49,7 @@ public abstract class SubpartitionTestBase extends TestLogger {
 			subpartition.finish();
 
 			assertFalse(subpartition.add(mock(Buffer.class)));
-		}
-		finally {
+		} finally {
 			if (subpartition != null) {
 				subpartition.release();
 			}
@@ -66,8 +64,7 @@ public abstract class SubpartitionTestBase extends TestLogger {
 			subpartition.release();
 
 			assertFalse(subpartition.add(mock(Buffer.class)));
-		}
-		finally {
+		} finally {
 			if (subpartition != null) {
 				subpartition.release();
 			}
@@ -97,7 +94,8 @@ public abstract class SubpartitionTestBase extends TestLogger {
 		TestInfiniteBufferProvider buffers = new TestInfiniteBufferProvider();
 
 		// Create the view
-		ResultSubpartitionView view = partition.createReadView(buffers);
+		BufferAvailabilityListener listener = mock(BufferAvailabilityListener.class);
+		ResultSubpartitionView view = partition.createReadView(buffers, listener);
 
 		// The added buffer and end-of-partition event
 		assertNotNull(view.getNextBuffer());

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelTest.java
index 8884b29..cd75a7b 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelTest.java
@@ -20,7 +20,6 @@ package org.apache.flink.runtime.io.network.partition.consumer;
 
 import org.apache.flink.metrics.SimpleCounter;
 import org.apache.flink.runtime.event.TaskEvent;
-import org.apache.flink.runtime.io.network.buffer.Buffer;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.junit.Test;
 
@@ -114,11 +113,11 @@ public class InputChannelTest {
 	private static class MockInputChannel extends InputChannel {
 
 		private MockInputChannel(
-				SingleInputGate inputGate,
-				int channelIndex,
-				ResultPartitionID partitionId,
-				int initialBackoff,
-				int maxBackoff) {
+			SingleInputGate inputGate,
+			int channelIndex,
+			ResultPartitionID partitionId,
+			int initialBackoff,
+			int maxBackoff) {
 
 			super(inputGate, channelIndex, partitionId, initialBackoff, maxBackoff, new SimpleCounter());
 		}
@@ -128,7 +127,7 @@ public class InputChannelTest {
 		}
 
 		@Override
-		Buffer getNextBuffer() throws IOException, InterruptedException {
+		BufferAndAvailability getNextBuffer() throws IOException, InterruptedException {
 			return null;
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/IteratorWrappingTestSingleInputGate.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/IteratorWrappingTestSingleInputGate.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/IteratorWrappingTestSingleInputGate.java
index cfbe99e..fa44393 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/IteratorWrappingTestSingleInputGate.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/IteratorWrappingTestSingleInputGate.java
@@ -64,24 +64,25 @@ public class IteratorWrappingTestSingleInputGate<T extends IOReadableWritable> e
 
 		// 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>() {
+		final Answer<InputChannel.BufferAndAvailability> answer = new Answer<InputChannel.BufferAndAvailability>() {
+
+			private boolean hasData = inputIterator.next(reuse) != null;
+
 			@Override
-			public Buffer answer(InvocationOnMock invocationOnMock) throws Throwable {
-				if (inputIterator.next(reuse) != null) {
+			public InputChannel.BufferAndAvailability answer(InvocationOnMock invocationOnMock) throws Throwable {
+				if (hasData) {
 					final Buffer buffer = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(bufferSize), mock(BufferRecycler.class));
 					serializer.setNextBuffer(buffer);
 					serializer.addRecord(reuse);
 
-					inputGate.onAvailableBuffer(inputChannel.getInputChannel());
+					hasData = inputIterator.next(reuse) != null;
 
 					// Call getCurrentBuffer to ensure size is set
-					return serializer.getCurrentBuffer();
-				}
-				else {
-
+					return new InputChannel.BufferAndAvailability(serializer.getCurrentBuffer(), true);
+				} else {
 					when(inputChannel.getInputChannel().isReleased()).thenReturn(true);
 
-					return EventSerializer.toBuffer(EndOfPartitionEvent.INSTANCE);
+					return new InputChannel.BufferAndAvailability(EventSerializer.toBuffer(EndOfPartitionEvent.INSTANCE), false);
 				}
 			}
 		};
@@ -93,8 +94,8 @@ public class IteratorWrappingTestSingleInputGate<T extends IOReadableWritable> e
 		return this;
 	}
 
-	public IteratorWrappingTestSingleInputGate<T> read() {
-		inputGate.onAvailableBuffer(inputChannel.getInputChannel());
+	public IteratorWrappingTestSingleInputGate<T> notifyNonEmpty() {
+		inputGate.notifyChannelNonEmpty(inputChannel.getInputChannel());
 
 		return this;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java
index 4ca1d1f..0b72f95 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java
@@ -28,6 +28,7 @@ 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.buffer.NetworkBufferPool;
+import org.apache.flink.runtime.io.network.partition.BufferAvailabilityListener;
 import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException;
 import org.apache.flink.runtime.io.network.partition.ResultPartition;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier;
@@ -57,7 +58,6 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 
-import static org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode.ASYNC;
 import static org.apache.flink.util.Preconditions.checkArgument;
 import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
@@ -93,11 +93,11 @@ public class LocalInputChannelTest {
 		final ExecutorService executor = Executors.newFixedThreadPool(2 * parallelism);
 
 		final NetworkBufferPool networkBuffers = new NetworkBufferPool(
-				(parallelism * producerBufferPoolSize) + (parallelism * parallelism),
-				TestBufferFactory.BUFFER_SIZE, MemoryType.HEAP);
+			(parallelism * producerBufferPoolSize) + (parallelism * parallelism),
+			TestBufferFactory.BUFFER_SIZE, MemoryType.HEAP);
 
 		final ResultPartitionConsumableNotifier partitionConsumableNotifier =
-				mock(ResultPartitionConsumableNotifier.class);
+			mock(ResultPartitionConsumableNotifier.class);
 
 		final TaskActions taskActions = mock(TaskActions.class);
 
@@ -124,21 +124,20 @@ public class LocalInputChannelTest {
 				partitionManager,
 				partitionConsumableNotifier,
 				ioManager,
-				ASYNC,
 				true);
 
 			// Create a buffer pool for this partition
 			partition.registerBufferPool(
-					networkBuffers.createBufferPool(producerBufferPoolSize, true));
+				networkBuffers.createBufferPool(producerBufferPoolSize, true));
 
 			// Create the producer
 			partitionProducers[i] = new TestPartitionProducer(
-					partition,
-					false,
-					new TestPartitionProducerBufferSource(
-							parallelism,
-							partition.getBufferProvider(),
-							numberOfBuffersPerChannel)
+				partition,
+				false,
+				new TestPartitionProducerBufferSource(
+					parallelism,
+					partition.getBufferProvider(),
+					numberOfBuffersPerChannel)
 			);
 
 			// Register with the partition manager in order to allow the local input channels to
@@ -150,7 +149,7 @@ public class LocalInputChannelTest {
 		try {
 			// Submit producer tasks
 			List<Future<?>> results = Lists.newArrayListWithCapacity(
-					parallelism + 1);
+				parallelism + 1);
 
 			for (int i = 0; i < parallelism; i++) {
 				results.add(executor.submit(partitionProducers[i]));
@@ -159,14 +158,14 @@ public class LocalInputChannelTest {
 			// Submit consumer
 			for (int i = 0; i < parallelism; i++) {
 				results.add(executor.submit(
-						new TestLocalInputChannelConsumer(
-								i,
-								parallelism,
-								numberOfBuffersPerChannel,
-								networkBuffers.createBufferPool(parallelism, true),
-								partitionManager,
-								new TaskEventDispatcher(),
-								partitionIds)));
+					new TestLocalInputChannelConsumer(
+						i,
+						parallelism,
+						numberOfBuffersPerChannel,
+						networkBuffers.createBufferPool(parallelism, true),
+						partitionManager,
+						new TaskEventDispatcher(),
+						partitionIds)));
 			}
 
 			// Wait for all to finish
@@ -183,7 +182,7 @@ public class LocalInputChannelTest {
 	@Test
 	public void testPartitionRequestExponentialBackoff() throws Exception {
 		// Config
-		Tuple2<Integer, Integer> backoff = new Tuple2<Integer, Integer>(500, 3000);
+		Tuple2<Integer, Integer> backoff = new Tuple2<>(500, 3000);
 
 		// Start with initial backoff, then keep doubling, and cap at max.
 		int[] expectedDelays = {backoff._1(), 1000, 2000, backoff._2()};
@@ -199,7 +198,7 @@ public class LocalInputChannelTest {
 		LocalInputChannel ch = createLocalInputChannel(inputGate, partitionManager, backoff);
 
 		when(partitionManager
-				.createSubpartitionView(eq(ch.partitionId), eq(0), eq(bufferProvider)))
+				.createSubpartitionView(eq(ch.partitionId), eq(0), eq(bufferProvider), any(BufferAvailabilityListener.class)))
 				.thenThrow(new PartitionNotFoundException(ch.partitionId));
 
 		Timer timer = mock(Timer.class);
@@ -215,7 +214,7 @@ public class LocalInputChannelTest {
 		// Initial request
 		ch.requestSubpartition(0);
 		verify(partitionManager)
-				.createSubpartitionView(eq(ch.partitionId), eq(0), eq(bufferProvider));
+				.createSubpartitionView(eq(ch.partitionId), eq(0), eq(bufferProvider), any(BufferAvailabilityListener.class));
 
 		// Request subpartition and verify that the actual requests are delayed.
 		for (long expected : expectedDelays) {
@@ -236,14 +235,13 @@ public class LocalInputChannelTest {
 
 	@Test(expected = CancelTaskException.class)
 	public void testProducerFailedException() throws Exception {
-
 		ResultSubpartitionView view = mock(ResultSubpartitionView.class);
 		when(view.isReleased()).thenReturn(true);
 		when(view.getFailureCause()).thenReturn(new Exception("Expected test exception"));
 
 		ResultPartitionManager partitionManager = mock(ResultPartitionManager.class);
 		when(partitionManager
-				.createSubpartitionView(any(ResultPartitionID.class), anyInt(), any(BufferProvider.class)))
+				.createSubpartitionView(any(ResultPartitionID.class), anyInt(), any(BufferProvider.class), any(BufferAvailabilityListener.class)))
 				.thenReturn(view);
 
 		SingleInputGate inputGate = mock(SingleInputGate.class);
@@ -251,7 +249,7 @@ public class LocalInputChannelTest {
 		when(inputGate.getBufferProvider()).thenReturn(bufferProvider);
 
 		LocalInputChannel ch = createLocalInputChannel(
-				inputGate, partitionManager, new Tuple2<Integer, Integer>(0, 0));
+				inputGate, partitionManager, new Tuple2<>(0, 0));
 
 		ch.requestSubpartition(0);
 
@@ -268,14 +266,14 @@ public class LocalInputChannelTest {
 			throws IOException, InterruptedException {
 
 		return new LocalInputChannel(
-			inputGate,
-			0,
-			new ResultPartitionID(),
-			partitionManager,
-			mock(TaskEventDispatcher.class),
-			initialAndMaxRequestBackoff._1(),
-			initialAndMaxRequestBackoff._2(),
-			new UnregisteredTaskMetricsGroup.DummyTaskIOMetricGroup());
+				inputGate,
+				0,
+				new ResultPartitionID(),
+				partitionManager,
+				mock(TaskEventDispatcher.class),
+				initialAndMaxRequestBackoff._1(),
+				initialAndMaxRequestBackoff._2(),
+				new UnregisteredTaskMetricsGroup.DummyTaskIOMetricGroup());
 	}
 
 	/**
@@ -344,14 +342,14 @@ public class LocalInputChannelTest {
 			checkArgument(numberOfExpectedBuffersPerChannel >= 1);
 
 			this.inputGate = new SingleInputGate(
-				"Test Name",
-				new JobID(),
-				new ExecutionAttemptID(),
-				new IntermediateDataSetID(),
-				subpartitionIndex,
-				numberOfInputChannels,
-				mock(TaskActions.class),
-				new UnregisteredTaskMetricsGroup.DummyTaskIOMetricGroup());
+					"Test Name",
+					new JobID(),
+					new ExecutionAttemptID(),
+					new IntermediateDataSetID(),
+					subpartitionIndex,
+					numberOfInputChannels,
+					mock(TaskActions.class),
+					new UnregisteredTaskMetricsGroup.DummyTaskIOMetricGroup());
 
 			// Set buffer pool
 			inputGate.setBufferPool(bufferPool);

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java
index 2c2f966..1d30a9a 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java
@@ -70,7 +70,7 @@ public class RemoteInputChannelTest {
 
 		// Need to notify the input gate for the out-of-order buffer as well. Otherwise the
 		// receiving task will not notice the error.
-		verify(inputGate, times(2)).onAvailableBuffer(eq(inputChannel));
+		verify(inputGate, times(2)).notifyChannelNonEmpty(eq(inputChannel));
 	}
 
 	@Test

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/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 0b7b10d..7cae362 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
@@ -34,6 +34,7 @@ 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.BufferRecycler;
+import org.apache.flink.runtime.io.network.partition.BufferAvailabilityListener;
 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;
@@ -72,18 +73,18 @@ public class SingleInputGateTest {
 	public void testBasicGetNextLogic() throws Exception {
 		// Setup
 		final SingleInputGate inputGate = new SingleInputGate(
-				"Test Task Name", new JobID(), new ExecutionAttemptID(), new IntermediateDataSetID(), 0, 2, mock(TaskActions.class), new UnregisteredTaskMetricsGroup.DummyTaskIOMetricGroup());
+			"Test Task Name", new JobID(), new ExecutionAttemptID(), new IntermediateDataSetID(), 0, 2, mock(TaskActions.class), new UnregisteredTaskMetricsGroup.DummyTaskIOMetricGroup());
 
 		final TestInputChannel[] inputChannels = new TestInputChannel[]{
-				new TestInputChannel(inputGate, 0),
-				new TestInputChannel(inputGate, 1)
+			new TestInputChannel(inputGate, 0),
+			new TestInputChannel(inputGate, 1)
 		};
 
 		inputGate.setInputChannel(
-				new IntermediateResultPartitionID(), inputChannels[0].getInputChannel());
+			new IntermediateResultPartitionID(), inputChannels[0].getInputChannel());
 
 		inputGate.setInputChannel(
-				new IntermediateResultPartitionID(), inputChannels[1].getInputChannel());
+			new IntermediateResultPartitionID(), inputChannels[1].getInputChannel());
 
 		// Test
 		inputChannels[0].readBuffer();
@@ -92,9 +93,12 @@ public class SingleInputGateTest {
 		inputChannels[1].readEndOfPartitionEvent();
 		inputChannels[0].readEndOfPartitionEvent();
 
-		verifyBufferOrEvent(inputGate, true, 0);
+		inputGate.notifyChannelNonEmpty(inputChannels[0].getInputChannel());
+		inputGate.notifyChannelNonEmpty(inputChannels[1].getInputChannel());
+
 		verifyBufferOrEvent(inputGate, true, 0);
 		verifyBufferOrEvent(inputGate, true, 1);
+		verifyBufferOrEvent(inputGate, true, 0);
 		verifyBufferOrEvent(inputGate, false, 1);
 		verifyBufferOrEvent(inputGate, false, 0);
 
@@ -111,10 +115,14 @@ public class SingleInputGateTest {
 
 		final ResultSubpartitionView iterator = mock(ResultSubpartitionView.class);
 		when(iterator.getNextBuffer()).thenReturn(
-				new Buffer(MemorySegmentFactory.allocateUnpooledSegment(1024), mock(BufferRecycler.class)));
+			new Buffer(MemorySegmentFactory.allocateUnpooledSegment(1024), mock(BufferRecycler.class)));
 
 		final ResultPartitionManager partitionManager = mock(ResultPartitionManager.class);
-		when(partitionManager.createSubpartitionView(any(ResultPartitionID.class), anyInt(), any(BufferProvider.class))).thenReturn(iterator);
+		when(partitionManager.createSubpartitionView(
+			any(ResultPartitionID.class),
+			anyInt(),
+			any(BufferProvider.class),
+			any(BufferAvailabilityListener.class))).thenReturn(iterator);
 
 		// Setup reader with one local and one unknown input channel
 		final IntermediateDataSetID resultId = new IntermediateDataSetID();
@@ -143,7 +151,7 @@ public class SingleInputGateTest {
 		inputGate.requestPartitions();
 
 		// Only the local channel can request
-		verify(partitionManager, times(1)).createSubpartitionView(any(ResultPartitionID.class), anyInt(), any(BufferProvider.class));
+		verify(partitionManager, times(1)).createSubpartitionView(any(ResultPartitionID.class), anyInt(), any(BufferProvider.class), any(BufferAvailabilityListener.class));
 
 		// Send event backwards and initialize unknown channel afterwards
 		final TaskEvent event = new TestTaskEvent();
@@ -155,7 +163,7 @@ public class SingleInputGateTest {
 		// After the update, the pending event should be send to local channel
 		inputGate.updateInputChannel(new InputChannelDeploymentDescriptor(new ResultPartitionID(unknownPartitionId.getPartitionId(), unknownPartitionId.getProducerId()), ResultPartitionLocation.createLocal()));
 
-		verify(partitionManager, times(2)).createSubpartitionView(any(ResultPartitionID.class), anyInt(), any(BufferProvider.class));
+		verify(partitionManager, times(2)).createSubpartitionView(any(ResultPartitionID.class), anyInt(), any(BufferProvider.class), any(BufferAvailabilityListener.class));
 		verify(taskEventDispatcher, times(2)).publish(any(ResultPartitionID.class), any(TaskEvent.class));
 	}
 
@@ -174,8 +182,7 @@ public class SingleInputGateTest {
 			new IntermediateDataSetID(),
 			0,
 			1,
-			mock(TaskActions.class),
-			new UnregisteredTaskMetricsGroup.DummyTaskIOMetricGroup());
+			mock(TaskActions.class), new UnregisteredTaskMetricsGroup.DummyTaskIOMetricGroup());
 
 		ResultPartitionManager partitionManager = mock(ResultPartitionManager.class);
 
@@ -186,19 +193,17 @@ public class SingleInputGateTest {
 			partitionManager,
 			new TaskEventDispatcher(),
 			new LocalConnectionManager(),
-			0,
-			0,
-			new UnregisteredTaskMetricsGroup.DummyTaskIOMetricGroup());
+			0, 0, new UnregisteredTaskMetricsGroup.DummyTaskIOMetricGroup());
 
 		inputGate.setInputChannel(unknown.partitionId.getPartitionId(), unknown);
 
 		// Update to a local channel and verify that no request is triggered
 		inputGate.updateInputChannel(new InputChannelDeploymentDescriptor(
-				unknown.partitionId,
-				ResultPartitionLocation.createLocal()));
+			unknown.partitionId,
+			ResultPartitionLocation.createLocal()));
 
 		verify(partitionManager, never()).createSubpartitionView(
-				any(ResultPartitionID.class), anyInt(), any(BufferProvider.class));
+			any(ResultPartitionID.class), anyInt(), any(BufferProvider.class), any(BufferAvailabilityListener.class));
 	}
 
 	/**
@@ -227,8 +232,7 @@ public class SingleInputGateTest {
 			new ResultPartitionManager(),
 			new TaskEventDispatcher(),
 			new LocalConnectionManager(),
-			0,
-			0,
+			0, 0,
 			new UnregisteredTaskMetricsGroup.DummyTaskIOMetricGroup());
 
 		inputGate.setInputChannel(unknown.partitionId.getPartitionId(), unknown);
@@ -249,16 +253,15 @@ public class SingleInputGateTest {
 		// Wait for blocking queue poll call and release input gate
 		boolean success = false;
 		for (int i = 0; i < 50; i++) {
-			if (asyncConsumer != null && asyncConsumer.isAlive()) {
-				StackTraceElement[] stackTrace = asyncConsumer.getStackTrace();
-				success = isInBlockingQueuePoll(stackTrace);
+			if (asyncConsumer.isAlive()) {
+				success = asyncConsumer.getState() == Thread.State.WAITING;
 			}
 
 			if (success) {
 				break;
 			} else {
 				// Retry
-				Thread.sleep(500);
+				Thread.sleep(100);
 			}
 		}
 
@@ -355,33 +358,12 @@ public class SingleInputGateTest {
 		}
 	}
 
-	/**
-	 * Returns whether the stack trace represents a Thread in a blocking queue
-	 * poll call.
-	 *
-	 * @param stackTrace Stack trace of the Thread to check
-	 *
-	 * @return Flag indicating whether the Thread is in a blocking queue poll
-	 * call.
-	 */
-	private boolean isInBlockingQueuePoll(StackTraceElement[] stackTrace) {
-		for (StackTraceElement elem : stackTrace) {
-			if (elem.getMethodName().equals("poll") &&
-					elem.getClassName().equals("java.util.concurrent.LinkedBlockingQueue")) {
-
-				return true;
-			}
-		}
-
-		return false;
-	}
-
 	// ---------------------------------------------------------------------------------------------
 
 	static void verifyBufferOrEvent(
-			InputGate inputGate,
-			boolean isBuffer,
-			int channelIndex) throws IOException, InterruptedException {
+		InputGate inputGate,
+		boolean isBuffer,
+		int channelIndex) throws IOException, InterruptedException {
 
 		final BufferOrEvent boe = inputGate.getNextBufferOrEvent();
 		assertEquals(isBuffer, boe.isBuffer());

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java
index 7ea67b3..a6597a2 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java
@@ -19,10 +19,8 @@
 package org.apache.flink.runtime.io.network.partition.consumer;
 
 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.util.TestTaskEvent;
 import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
 import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
@@ -46,7 +44,7 @@ public class TestInputChannel {
 	private final SingleInputGate inputGate;
 
 	// Abusing Mockito here... ;)
-	protected OngoingStubbing<Buffer> stubbing;
+	protected OngoingStubbing<InputChannel.BufferAndAvailability> stubbing;
 
 	public TestInputChannel(SingleInputGate inputGate, int channelIndex) {
 		checkArgument(channelIndex >= 0);
@@ -57,13 +55,10 @@ public class TestInputChannel {
 
 	public TestInputChannel read(Buffer buffer) throws IOException, InterruptedException {
 		if (stubbing == null) {
-			stubbing = when(mock.getNextBuffer()).thenReturn(buffer);
+			stubbing = when(mock.getNextBuffer()).thenReturn(new InputChannel.BufferAndAvailability(buffer, true));
+		} else {
+			stubbing = stubbing.thenReturn(new InputChannel.BufferAndAvailability(buffer, true));
 		}
-		else {
-			stubbing = stubbing.thenReturn(buffer);
-		}
-
-		inputGate.onAvailableBuffer(mock);
 
 		return this;
 	}
@@ -75,34 +70,23 @@ public class TestInputChannel {
 		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>() {
+		final Answer<InputChannel.BufferAndAvailability> answer = new Answer<InputChannel.BufferAndAvailability>() {
 			@Override
-			public Buffer answer(InvocationOnMock invocationOnMock) throws Throwable {
+			public InputChannel.BufferAndAvailability answer(InvocationOnMock invocationOnMock) throws Throwable {
 				// Return true after finishing
 				when(mock.isReleased()).thenReturn(true);
 
-				return EventSerializer.toBuffer(EndOfPartitionEvent.INSTANCE);
+				return new InputChannel.BufferAndAvailability(EventSerializer.toBuffer(EndOfPartitionEvent.INSTANCE), false);
 			}
 		};
 
 		if (stubbing == null) {
 			stubbing = when(mock.getNextBuffer()).thenAnswer(answer);
-		}
-		else {
+		} else {
 			stubbing = stubbing.thenAnswer(answer);
 		}
 
-		inputGate.onAvailableBuffer(mock);
-
 		return this;
 	}
 


[2/6] flink git commit: [FLINK-5169] [network] Adjust tests to new consumer logic

Posted by uc...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestSingleInputGate.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestSingleInputGate.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestSingleInputGate.java
index 3972917..126a96e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestSingleInputGate.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestSingleInputGate.java
@@ -20,24 +20,17 @@ package org.apache.flink.runtime.io.network.partition.consumer;
 
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.taskmanager.TaskActions;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
 import org.apache.flink.runtime.operators.testutils.UnregisteredTaskMetricsGroup;
-import org.apache.flink.runtime.util.event.EventListener;
+import org.apache.flink.runtime.taskmanager.TaskActions;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
-import java.io.IOException;
 import java.lang.reflect.Field;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.BlockingQueue;
+import java.util.ArrayDeque;
 
 import static org.apache.flink.util.Preconditions.checkArgument;
-import static org.apache.flink.util.Preconditions.checkElementIndex;
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.mock;
@@ -73,28 +66,29 @@ public class TestSingleInputGate {
 
 		// Notify about late registrations (added for DataSinkTaskTest#testUnionDataSinkTask).
 		// After merging registerInputOutput and invoke, we have to make sure that the test
-		// notifcations happen at the expected time. In real programs, this is guaranteed by
+		// notifications happen at the expected time. In real programs, this is guaranteed by
 		// the instantiation and request partition life cycle.
 		try {
 			Field f = realGate.getClass().getDeclaredField("inputChannelsWithData");
 			f.setAccessible(true);
-			final BlockingQueue<InputChannel> notifications = (BlockingQueue<InputChannel>) f.get(realGate);
+			final ArrayDeque<InputChannel> notifications = (ArrayDeque<InputChannel>) f.get(realGate);
 
 			doAnswer(new Answer<Void>() {
 				@Override
 				public Void answer(InvocationOnMock invocation) throws Throwable {
 					invocation.callRealMethod();
 
-					if (!notifications.isEmpty()) {
-						EventListener<InputGate> listener = (EventListener<InputGate>) invocation.getArguments()[0];
-						listener.onEvent(inputGate);
+					synchronized (notifications) {
+						if (!notifications.isEmpty()) {
+							InputGateListener listener = (InputGateListener) invocation.getArguments()[0];
+							listener.notifyInputGateNonEmpty(inputGate);
+						}
 					}
 
 					return null;
 				}
-			}).when(inputGate).registerListener(any(EventListener.class));
-		}
-		catch (Exception e) {
+			}).when(inputGate).registerListener(any(InputGateListener.class));
+		} catch (Exception e) {
 			throw new RuntimeException(e);
 		}
 
@@ -108,81 +102,8 @@ public class TestSingleInputGate {
 		}
 	}
 
-	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/d3ac0adf/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 cba3199..84ec202 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
@@ -21,7 +21,6 @@ package org.apache.flink.runtime.io.network.partition.consumer;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
-
 import org.apache.flink.runtime.operators.testutils.UnregisteredTaskMetricsGroup;
 import org.apache.flink.runtime.taskmanager.TaskActions;
 import org.junit.Test;
@@ -73,22 +72,32 @@ public class UnionInputGateTest {
 		inputChannels[1][1].readEndOfPartitionEvent(); // 0 => 3
 		inputChannels[1][0].readEndOfPartitionEvent(); // 0 => 3
 
-		SingleInputGateTest.verifyBufferOrEvent(union, true, 0);
-		SingleInputGateTest.verifyBufferOrEvent(union, false, 0);
-		SingleInputGateTest.verifyBufferOrEvent(union, true, 5);
-		SingleInputGateTest.verifyBufferOrEvent(union, false, 5);
-		SingleInputGateTest.verifyBufferOrEvent(union, true, 3);
-		SingleInputGateTest.verifyBufferOrEvent(union, true, 4);
-		SingleInputGateTest.verifyBufferOrEvent(union, true, 1);
-		SingleInputGateTest.verifyBufferOrEvent(union, true, 6);
-		SingleInputGateTest.verifyBufferOrEvent(union, false, 1);
-		SingleInputGateTest.verifyBufferOrEvent(union, false, 6);
-		SingleInputGateTest.verifyBufferOrEvent(union, true, 2);
-		SingleInputGateTest.verifyBufferOrEvent(union, false, 2);
-		SingleInputGateTest.verifyBufferOrEvent(union, true, 7);
-		SingleInputGateTest.verifyBufferOrEvent(union, false, 7);
-		SingleInputGateTest.verifyBufferOrEvent(union, false, 4);
-		SingleInputGateTest.verifyBufferOrEvent(union, false, 3);
+		ig1.notifyChannelNonEmpty(inputChannels[0][0].getInputChannel());
+		ig1.notifyChannelNonEmpty(inputChannels[0][1].getInputChannel());
+		ig1.notifyChannelNonEmpty(inputChannels[0][2].getInputChannel());
+
+		ig2.notifyChannelNonEmpty(inputChannels[1][0].getInputChannel());
+		ig2.notifyChannelNonEmpty(inputChannels[1][1].getInputChannel());
+		ig2.notifyChannelNonEmpty(inputChannels[1][2].getInputChannel());
+		ig2.notifyChannelNonEmpty(inputChannels[1][3].getInputChannel());
+		ig2.notifyChannelNonEmpty(inputChannels[1][4].getInputChannel());
+
+		SingleInputGateTest.verifyBufferOrEvent(union, true, 0); // gate 1, channel 0
+		SingleInputGateTest.verifyBufferOrEvent(union, true, 3); // gate 2, channel 0
+		SingleInputGateTest.verifyBufferOrEvent(union, true, 1); // gate 1, channel 1
+		SingleInputGateTest.verifyBufferOrEvent(union, true, 4); // gate 2, channel 1
+		SingleInputGateTest.verifyBufferOrEvent(union, true, 2); // gate 1, channel 2
+		SingleInputGateTest.verifyBufferOrEvent(union, true, 5); // gate 2, channel 1
+		SingleInputGateTest.verifyBufferOrEvent(union, false, 0); // gate 1, channel 0
+		SingleInputGateTest.verifyBufferOrEvent(union, true, 6); // gate 2, channel 1
+		SingleInputGateTest.verifyBufferOrEvent(union, false, 1); // gate 1, channel 1
+		SingleInputGateTest.verifyBufferOrEvent(union, true, 7); // gate 2, channel 1
+		SingleInputGateTest.verifyBufferOrEvent(union, false, 2); // gate 1, channel 2
+		SingleInputGateTest.verifyBufferOrEvent(union, false, 3); // gate 2, channel 0
+		SingleInputGateTest.verifyBufferOrEvent(union, false, 4); // gate 2, channel 1
+		SingleInputGateTest.verifyBufferOrEvent(union, false, 5); // gate 2, channel 2
+		SingleInputGateTest.verifyBufferOrEvent(union, false, 6); // gate 2, channel 3
+		SingleInputGateTest.verifyBufferOrEvent(union, false, 7); // gate 2, channel 4
 
 		// Return null when the input gate has received all end-of-partition events
 		assertTrue(union.isFinished());

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/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
index 1b51805..676a304 100644
--- 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
@@ -22,26 +22,32 @@ import org.apache.flink.runtime.event.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.BufferAvailabilityListener;
 import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView;
 
 import java.util.Random;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicLong;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * A test subpartition view consumer.
+ * A test subpartition viewQueue consumer.
  *
  * <p> The behaviour of the consumer is customizable by specifying a callback.
  *
  * @see TestConsumerCallback
  */
-public class TestSubpartitionConsumer implements Callable<Boolean> {
+public class TestSubpartitionConsumer implements Callable<Boolean>, BufferAvailabilityListener {
 
 	private static final int MAX_SLEEP_TIME_MS = 20;
 
-	/** The subpartition view to consume. */
-	private final ResultSubpartitionView subpartitionView;
+	/** The subpartition viewQueue to consume. */
+	private volatile ResultSubpartitionView subpartitionView;
+
+	private BlockingQueue<ResultSubpartitionView> viewQueue = new ArrayBlockingQueue<>(1);
 
 	/**
 	 * Flag indicating whether the consumer is slow. If true, the consumer will sleep a random
@@ -49,33 +55,43 @@ public class TestSubpartitionConsumer implements Callable<Boolean> {
 	 */
 	private final boolean isSlowConsumer;
 
-	/** The callback to handle a read buffer. */
+	/** The callback to handle a notifyNonEmpty buffer. */
 	private final TestConsumerCallback callback;
 
 	/** Random source for sleeps. */
 	private final Random random;
 
+	private final AtomicLong numBuffersAvailable = new AtomicLong();
+
 	public TestSubpartitionConsumer(
-			ResultSubpartitionView subpartitionView,
-			boolean isSlowConsumer,
-			TestConsumerCallback callback) {
+		boolean isSlowConsumer,
+		TestConsumerCallback callback) {
 
-		this.subpartitionView = checkNotNull(subpartitionView);
 		this.isSlowConsumer = isSlowConsumer;
 		this.random = isSlowConsumer ? new Random() : null;
 		this.callback = checkNotNull(callback);
 	}
 
+	public void setSubpartitionView(ResultSubpartitionView subpartitionView) {
+		this.subpartitionView = checkNotNull(subpartitionView);
+	}
+
 	@Override
 	public Boolean call() throws Exception {
-		final TestNotificationListener listener = new TestNotificationListener();
-
 		try {
 			while (true) {
 				if (Thread.interrupted()) {
 					throw new InterruptedException();
 				}
 
+				if (numBuffersAvailable.get() == 0) {
+					synchronized (numBuffersAvailable) {
+						while (numBuffersAvailable.get() == 0) {
+							numBuffersAvailable.wait();
+						}
+					}
+				}
+
 				final Buffer buffer = subpartitionView.getNextBuffer();
 
 				if (isSlowConsumer) {
@@ -83,12 +99,13 @@ public class TestSubpartitionConsumer implements Callable<Boolean> {
 				}
 
 				if (buffer != null) {
+					numBuffersAvailable.decrementAndGet();
+
 					if (buffer.isBuffer()) {
 						callback.onBuffer(buffer);
-					}
-					else {
+					} else {
 						final AbstractEvent event = EventSerializer.fromBuffer(buffer,
-								getClass().getClassLoader());
+							getClass().getClassLoader());
 
 						callback.onEvent(event);
 
@@ -100,22 +117,22 @@ public class TestSubpartitionConsumer implements Callable<Boolean> {
 							return true;
 						}
 					}
-				}
-				else {
-					int current = listener.getNumberOfNotifications();
-
-					if (subpartitionView.registerListener(listener)) {
-						listener.waitForNotification(current);
-					}
-					else if (subpartitionView.isReleased()) {
-						return true;
-					}
+				} else if (subpartitionView.isReleased()) {
+					return true;
 				}
 			}
-		}
-		finally {
+		} finally {
 			subpartitionView.releaseAllResources();
 		}
 	}
 
+	@Override
+	public void notifyBuffersAvailable(long numBuffers) {
+		if (numBuffers > 0 && numBuffersAvailable.getAndAdd(numBuffers) == 0) {
+			synchronized (numBuffersAvailable) {
+				numBuffersAvailable.notifyAll();
+			}
+			;
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/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 226dc91..f9aea89 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
@@ -129,7 +129,6 @@ public class DataSinkTaskTest extends TaskTestBase {
 
 	@Test
 	public void testUnionDataSinkTask() {
-
 		int keyCnt = 10;
 		int valCnt = 20;
 
@@ -147,9 +146,10 @@ public class DataSinkTaskTest extends TaskTestBase {
 
 		try {
 			// For the union reader to work, we need to start notifications *after* the union reader
-			// has been initialized.
+			// has been initialized. This is accomplished via a mockito hack in TestSingleInputGate,
+			// which checks forwards existing notifications on registerListener calls.
 			for (IteratorWrappingTestSingleInputGate<?> reader : readers) {
-				reader.read();
+				reader.notifyNonEmpty();
 			}
 
 			testTask.invoke();

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/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 0c9fd79..fb8ed68 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,9 @@ import org.apache.flink.api.common.functions.GroupReduceFunction;
 import org.apache.flink.api.common.operators.util.UserCodeClassWrapper;
 import org.apache.flink.runtime.testutils.recordutils.RecordComparatorFactory;
 import org.apache.flink.runtime.testutils.recordutils.RecordSerializerFactory;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
 import org.apache.flink.runtime.operators.DriverStrategy;
 import org.apache.flink.runtime.operators.BatchTask;
 import org.apache.flink.runtime.operators.FlatMapDriver;
@@ -37,14 +39,20 @@ import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
 import org.apache.flink.runtime.operators.testutils.TaskTestBase;
 import org.apache.flink.runtime.operators.testutils.UniformRecordGenerator;
 import org.apache.flink.runtime.operators.util.TaskConfig;
+import org.apache.flink.runtime.taskmanager.Task;
 import org.apache.flink.types.IntValue;
 import org.apache.flink.types.Record;
 import org.apache.flink.util.Collector;
-
 import org.junit.Assert;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.core.classloader.annotations.PowerMockIgnore;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
 
-
+@RunWith(PowerMockRunner.class)
+@PrepareForTest({Task.class, ResultPartitionWriter.class})
+@PowerMockIgnore({"javax.management.*", "com.sun.jndi.*"})
 public class ChainTaskTest extends TaskTestBase {
 	
 	private static final int MEMORY_MANAGER_SIZE = 1024 * 1024 * 3;

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/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 646c038..8f4bc77 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
@@ -67,9 +67,9 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 public class MockEnvironment implements Environment {
-	
+
 	private final TaskInfo taskInfo;
-	
+
 	private final ExecutionConfig executionConfig;
 
 	private final MemoryManager memManager;
@@ -184,7 +184,7 @@ public class MockEnvironment implements Environment {
 						}
 
 						if (result == RecordDeserializer.DeserializationResult.LAST_RECORD_FROM_BUFFER
-								|| result == RecordDeserializer.DeserializationResult.PARTIAL_RECORD) {
+							|| result == RecordDeserializer.DeserializationResult.PARTIAL_RECORD) {
 							break;
 						}
 					}
@@ -234,9 +234,9 @@ public class MockEnvironment implements Environment {
 	@Override
 	public TaskManagerRuntimeInfo getTaskManagerInfo() {
 		return new TaskManagerRuntimeInfo(
-				"localhost",
-				new UnmodifiableConfiguration(new Configuration()),
-				System.getProperty("java.io.tmpdir"));
+			"localhost",
+			new UnmodifiableConfiguration(new Configuration()),
+			System.getProperty("java.io.tmpdir"));
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/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 eaf44db..53d75b3 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
@@ -67,7 +67,7 @@ public abstract class TaskTestBase extends TestLogger {
 		conf.setInputSerializer(RecordSerializerFactory.get(), groupId);
 
 		if (read) {
-			reader.read();
+			reader.notifyNonEmpty();
 		}
 
 		return reader;

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelAsyncProducerConsumerITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelAsyncProducerConsumerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelAsyncProducerConsumerITCase.java
index a093233..876e908 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelAsyncProducerConsumerITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelAsyncProducerConsumerITCase.java
@@ -18,7 +18,6 @@
 
 package org.apache.flink.runtime.taskmanager;
 
-import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.instance.ActorGateway;
@@ -43,6 +42,7 @@ import scala.concurrent.Future;
 import scala.concurrent.duration.Deadline;
 import scala.concurrent.duration.FiniteDuration;
 
+import java.util.Arrays;
 import java.util.concurrent.TimeUnit;
 
 import static org.junit.Assert.assertEquals;
@@ -129,18 +129,17 @@ public class TaskCancelAsyncProducerConsumerITCase extends TestLogger {
 			}
 
 			// Verify that async producer is in blocking request
-			assertTrue("Producer thread is not blocked.", producerBlocked);
+			assertTrue("Producer thread is not blocked: " + Arrays.toString(ASYNC_CONSUMER_THREAD.getStackTrace()), producerBlocked);
 
-			boolean consumerBlocked = false;
+			boolean consumerWaiting = false;
 			for (int i = 0; i < 50; i++) {
 				Thread thread = ASYNC_CONSUMER_THREAD;
 
 				if (thread != null && thread.isAlive()) {
-					StackTraceElement[] stackTrace = thread.getStackTrace();
-					consumerBlocked = isInBlockingQueuePoll(stackTrace);
+					consumerWaiting = thread.getState() == Thread.State.WAITING;
 				}
 
-				if (consumerBlocked) {
+				if (consumerWaiting) {
 					break;
 				} else {
 					// Retry
@@ -149,7 +148,7 @@ public class TaskCancelAsyncProducerConsumerITCase extends TestLogger {
 			}
 
 			// Verify that async consumer is in blocking request
-			assertTrue("Consumer thread is not blocked.", consumerBlocked);
+			assertTrue("Consumer thread is not blocked.", consumerWaiting);
 
 			msg = new CancelJob(jobGraph.getJobID());
 			Future<?> cancelFuture = jobManager.ask(msg, deadline.timeLeft());
@@ -186,27 +185,6 @@ public class TaskCancelAsyncProducerConsumerITCase extends TestLogger {
 	}
 
 	/**
-	 * Returns whether the stack trace represents a Thread in a blocking queue
-	 * poll call.
-	 *
-	 * @param stackTrace Stack trace of the Thread to check
-	 *
-	 * @return Flag indicating whether the Thread is in a blocking queue poll
-	 * call.
-	 */
-	private boolean isInBlockingQueuePoll(StackTraceElement[] stackTrace) {
-		for (StackTraceElement elem : stackTrace) {
-			if (elem.getMethodName().equals("poll") &&
-					elem.getClassName().equals("java.util.concurrent.LinkedBlockingQueue")) {
-
-				return true;
-			}
-		}
-
-		return false;
-	}
-
-	/**
 	 * Invokable emitting records in a separate Thread (not the main Task
 	 * thread).
 	 */

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/flink-streaming-java/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java b/flink-streaming-java/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
index 322a0f0..6dcb56b 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
@@ -28,10 +28,11 @@ 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.partition.consumer.InputChannel.BufferAndAvailability;
 import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
 import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer;
 import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
@@ -62,9 +63,9 @@ public class StreamTestSingleInputGate<T> extends TestSingleInputGate {
 
 	@SuppressWarnings("unchecked")
 	public StreamTestSingleInputGate(
-			int numInputChannels,
-			int bufferSize,
-			TypeSerializer<T> serializer) throws IOException, InterruptedException {
+		int numInputChannels,
+		int bufferSize,
+		TypeSerializer<T> serializer) throws IOException, InterruptedException {
 		super(numInputChannels, false);
 
 		this.bufferSize = bufferSize;
@@ -86,39 +87,36 @@ public class StreamTestSingleInputGate<T> extends TestSingleInputGate {
 			final int channelIndex = i;
 			final RecordSerializer<SerializationDelegate<Object>> recordSerializer = new SpanningRecordSerializer<SerializationDelegate<Object>>();
 			final SerializationDelegate<Object> delegate = (SerializationDelegate<Object>) (SerializationDelegate<?>)
-					new SerializationDelegate<StreamElement>(new StreamElementSerializer<T>(serializer));
+				new SerializationDelegate<StreamElement>(new StreamElementSerializer<T>(serializer));
 
 			inputQueues[channelIndex] = new ConcurrentLinkedQueue<InputValue<Object>>();
 			inputChannels[channelIndex] = new TestInputChannel(inputGate, i);
 
 
-			final Answer<Buffer> answer = new Answer<Buffer>() {
+			final Answer<BufferAndAvailability> answer = new Answer<BufferAndAvailability>() {
 				@Override
-				public Buffer answer(InvocationOnMock invocationOnMock) throws Throwable {
+				public BufferAndAvailability answer(InvocationOnMock invocationOnMock) throws Throwable {
 					InputValue<Object> input = inputQueues[channelIndex].poll();
 					if (input != null && input.isStreamEnd()) {
 						when(inputChannels[channelIndex].getInputChannel().isReleased()).thenReturn(
-								true);
-						return EventSerializer.toBuffer(EndOfPartitionEvent.INSTANCE);
-					}
-					else if (input != null && input.isStreamRecord()) {
+							true);
+						return new BufferAndAvailability(EventSerializer.toBuffer(EndOfPartitionEvent.INSTANCE), false);
+					} else if (input != null && input.isStreamRecord()) {
 						Object inputElement = input.getStreamRecord();
 						final Buffer buffer = new Buffer(
-								MemorySegmentFactory.allocateUnpooledSegment(bufferSize),
-								mock(BufferRecycler.class));
-						
+							MemorySegmentFactory.allocateUnpooledSegment(bufferSize),
+							mock(BufferRecycler.class));
+
 						recordSerializer.setNextBuffer(buffer);
 						delegate.setInstance(inputElement);
 						recordSerializer.addRecord(delegate);
 
 						// Call getCurrentBuffer to ensure size is set
-						return recordSerializer.getCurrentBuffer();
-					}
-					else if (input != null && input.isEvent()) {
+						return new BufferAndAvailability(recordSerializer.getCurrentBuffer(), false);
+					} else if (input != null && input.isEvent()) {
 						AbstractEvent event = input.getEvent();
-						return EventSerializer.toBuffer(event);
-					}
-					else {
+						return new BufferAndAvailability(EventSerializer.toBuffer(event), false);
+					} else {
 						synchronized (inputQueues[channelIndex]) {
 							inputQueues[channelIndex].wait();
 							return answer(invocationOnMock);
@@ -130,7 +128,7 @@ public class StreamTestSingleInputGate<T> extends TestSingleInputGate {
 			when(inputChannels[channelIndex].getInputChannel().getNextBuffer()).thenAnswer(answer);
 
 			inputGate.setInputChannel(new IntermediateResultPartitionID(),
-					inputChannels[channelIndex].getInputChannel());
+				inputChannels[channelIndex].getInputChannel());
 		}
 	}
 
@@ -139,7 +137,7 @@ public class StreamTestSingleInputGate<T> extends TestSingleInputGate {
 			inputQueues[channel].add(InputValue.element(element));
 			inputQueues[channel].notifyAll();
 		}
-		inputGate.onAvailableBuffer(inputChannels[channel].getInputChannel());
+		inputGate.notifyChannelNonEmpty(inputChannels[channel].getInputChannel());
 	}
 
 	public void sendEvent(AbstractEvent event, int channel) {
@@ -147,7 +145,7 @@ public class StreamTestSingleInputGate<T> extends TestSingleInputGate {
 			inputQueues[channel].add(InputValue.event(event));
 			inputQueues[channel].notifyAll();
 		}
-		inputGate.onAvailableBuffer(inputChannels[channel].getInputChannel());
+		inputGate.notifyChannelNonEmpty(inputChannels[channel].getInputChannel());
 	}
 
 	public void endInput() {
@@ -156,7 +154,7 @@ public class StreamTestSingleInputGate<T> extends TestSingleInputGate {
 				inputQueues[i].add(InputValue.streamEnd());
 				inputQueues[i].notifyAll();
 			}
-			inputGate.onAvailableBuffer(inputChannels[i].getInputChannel());
+			inputGate.notifyChannelNonEmpty(inputChannels[i].getInputChannel());
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java
index a8a989b..0cf866a 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java
@@ -17,25 +17,24 @@
 
 package org.apache.flink.streaming.runtime.io;
 
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.util.Random;
-
 import org.apache.flink.core.memory.MemoryType;
 import org.apache.flink.runtime.event.TaskEvent;
 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.CheckpointBarrier;
 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.NetworkBufferPool;
 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.util.event.EventListener;
-import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
-
+import org.apache.flink.runtime.io.network.partition.consumer.InputGateListener;
 import org.junit.Test;
 
+import java.io.IOException;
+import java.util.Random;
+
+import static org.junit.Assert.fail;
+
 /**
  * The test generates two random streams (input channels) which independently
  * and randomly generate checkpoint barriers. The two streams are very
@@ -165,7 +164,7 @@ public class BarrierBufferMassiveRandomTest {
 		public void sendTaskEvent(TaskEvent event) {}
 
 		@Override
-		public void registerListener(EventListener<InputGate> listener) {}
+		public void registerListener(InputGateListener listener) {}
 
 		@Override
 		public int getPageSize() {

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/MockInputGate.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/MockInputGate.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/MockInputGate.java
index cb8a058..3e2a75a 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/MockInputGate.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/MockInputGate.java
@@ -22,7 +22,7 @@ import org.apache.flink.runtime.event.TaskEvent;
 import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
 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.util.event.EventListener;
+import org.apache.flink.runtime.io.network.partition.consumer.InputGateListener;
 
 import java.util.ArrayDeque;
 import java.util.List;
@@ -31,16 +31,15 @@ import java.util.Queue;
 public class MockInputGate implements InputGate {
 
 	private final int pageSize;
-	
+
 	private final int numChannels;
-	
+
 	private final Queue<BufferOrEvent> boes;
 
 	private final boolean[] closed;
-	
+
 	private int closedChannels;
 
-	
 	public MockInputGate(int pageSize, int numChannels, List<BufferOrEvent> boes) {
 		this.pageSize = pageSize;
 		this.numChannels = numChannels;
@@ -52,7 +51,7 @@ public class MockInputGate implements InputGate {
 	public int getPageSize() {
 		return pageSize;
 	}
-	
+
 	@Override
 	public int getNumberOfInputChannels() {
 		return numChannels;
@@ -69,11 +68,11 @@ public class MockInputGate implements InputGate {
 		if (next == null) {
 			return null;
 		}
-		
+
 		int channelIdx = next.getChannelIndex();
 		if (closed[channelIdx]) {
 			throw new RuntimeException("Inconsistent: Channel " + channelIdx
-					+ " has data even though it is already closed.");
+				+ " has data even though it is already closed.");
 		}
 		if (next.isEvent() && next.getEvent() instanceof EndOfPartitionEvent) {
 			closed[channelIdx] = true;
@@ -83,12 +82,15 @@ public class MockInputGate implements InputGate {
 	}
 
 	@Override
-	public void requestPartitions() {}
+	public void requestPartitions() {
+	}
 
 	@Override
-	public void sendTaskEvent(TaskEvent event) {}
+	public void sendTaskEvent(TaskEvent event) {
+	}
 
 	@Override
-	public void registerListener(EventListener<InputGate> listener) {}
-	
-}
\ No newline at end of file
+	public void registerListener(InputGateListener listener) {
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java
index 3126d71..d86c809 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java
@@ -45,7 +45,7 @@ import java.io.IOException;
  *
  * <p>
  * When Elements or Events are offered to the Task they are put into a queue. The input gates
- * of the Task read from this queue. Use {@link #waitForInputProcessing()} to wait until all
+ * of the Task notifyNonEmpty from this queue. Use {@link #waitForInputProcessing()} to wait until all
  * queues are empty. This must be used after entering some elements before checking the
  * desired output.
  */
@@ -58,11 +58,13 @@ public class OneInputStreamTaskTestHarness<IN, OUT> extends StreamTaskTestHarnes
 	 * Creates a test harness with the specified number of input gates and specified number
 	 * of channels per input gate.
 	 */
-	public OneInputStreamTaskTestHarness(OneInputStreamTask<IN, OUT> task,
-			int numInputGates,
-			int numInputChannelsPerGate,
-			TypeInformation<IN> inputType,
-			TypeInformation<OUT> outputType) {
+	public OneInputStreamTaskTestHarness(
+		OneInputStreamTask<IN, OUT> task,
+		int numInputGates,
+		int numInputChannelsPerGate,
+		TypeInformation<IN> inputType,
+		TypeInformation<OUT> outputType) {
+		
 		super(task, outputType);
 
 		this.inputType = inputType;
@@ -75,9 +77,10 @@ public class OneInputStreamTaskTestHarness<IN, OUT> extends StreamTaskTestHarnes
 	/**
 	 * Creates a test harness with one input gate that has one input channel.
 	 */
-	public OneInputStreamTaskTestHarness(OneInputStreamTask<IN, OUT> task,
-			TypeInformation<IN> inputType,
-			TypeInformation<OUT> outputType) {
+	public OneInputStreamTaskTestHarness(
+		OneInputStreamTask<IN, OUT> task,
+		TypeInformation<IN> inputType,
+		TypeInformation<OUT> outputType) {
 		this(task, 1, 1, inputType, outputType);
 	}
 
@@ -87,9 +90,9 @@ public class OneInputStreamTaskTestHarness<IN, OUT> extends StreamTaskTestHarnes
 
 		for (int i = 0; i < numInputGates; i++) {
 			inputGates[i] = new StreamTestSingleInputGate<IN>(
-					numInputChannelsPerGate,
-					bufferSize,
-					inputSerializer);
+				numInputChannelsPerGate,
+				bufferSize,
+				inputSerializer);
 			this.mockEnv.addInputGate(inputGates[i].getInputGate());
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java
index 52daf6f..17a0857 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java
@@ -102,13 +102,13 @@ public class StreamMockEnvironment implements Environment {
 	private volatile boolean wasFailedExternally = false;
 
 	public StreamMockEnvironment(Configuration jobConfig, Configuration taskConfig, ExecutionConfig executionConfig,
-									long memorySize, MockInputSplitProvider inputSplitProvider, int bufferSize) {
+								 long memorySize, MockInputSplitProvider inputSplitProvider, int bufferSize) {
 		this.taskInfo = new TaskInfo(
-				"", /* task name */
-				1, /* num key groups / max parallelism */
-				0, /* index of this subtask */
-				1, /* num subtasks */
-				0 /* attempt number */);
+			"", /* task name */
+			1, /* num key groups / max parallelism */
+			0, /* index of this subtask */
+			1, /* num subtasks */
+			0 /* attempt number */);
 		this.jobConfiguration = jobConfig;
 		this.taskConfiguration = taskConfig;
 		this.inputs = new LinkedList<InputGate>();
@@ -146,8 +146,8 @@ public class StreamMockEnvironment implements Environment {
 				@Override
 				public Buffer answer(InvocationOnMock invocationOnMock) throws Throwable {
 					return new Buffer(
-							MemorySegmentFactory.allocateUnpooledSegment(bufferSize),
-							mock(BufferRecycler.class));
+						MemorySegmentFactory.allocateUnpooledSegment(bufferSize),
+						mock(BufferRecycler.class));
 				}
 			});
 
@@ -175,7 +175,7 @@ public class StreamMockEnvironment implements Environment {
 						}
 
 						if (result == RecordDeserializer.DeserializationResult.LAST_RECORD_FROM_BUFFER
-								|| result == RecordDeserializer.DeserializationResult.PARTIAL_RECORD) {
+							|| result == RecordDeserializer.DeserializationResult.PARTIAL_RECORD) {
 							break;
 						}
 					}
@@ -318,7 +318,7 @@ public class StreamMockEnvironment implements Environment {
 
 	@Override
 	public void acknowledgeCheckpoint(
-			CheckpointMetaData checkpointMetaData, SubtaskState subtaskState) {
+		CheckpointMetaData checkpointMetaData, SubtaskState subtaskState) {
 	}
 
 	@Override
@@ -343,4 +343,3 @@ public class StreamMockEnvironment implements Environment {
 		return new UnregisteredTaskMetricsGroup();
 	}
 }
-

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskCancellationBarrierTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskCancellationBarrierTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskCancellationBarrierTest.java
index 95828f8..ebe5285 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskCancellationBarrierTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskCancellationBarrierTest.java
@@ -28,13 +28,11 @@ import org.apache.flink.streaming.api.graph.StreamConfig;
 import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
 import org.apache.flink.streaming.api.operators.StreamMap;
 import org.apache.flink.streaming.api.operators.co.CoStreamMap;
-
 import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
-
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.any;
 import static org.mockito.Mockito.eq;

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
index b71e38d..b20b3a3 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
@@ -92,7 +92,7 @@ public class StreamTaskTestHarness<OUT> {
 	// input related methods only need to be implemented once, in generic form
 	protected int numInputGates;
 	protected int numInputChannelsPerGate;
-	
+
 	@SuppressWarnings("rawtypes")
 	protected StreamTestSingleInputGate[] inputGates;
 
@@ -329,7 +329,7 @@ public class StreamTaskTestHarness<OUT> {
 			try {
 				Thread.sleep(10);
 			} catch (InterruptedException ignored) {}
-			
+
 			if (allEmpty) {
 				break;
 			}
@@ -337,7 +337,7 @@ public class StreamTaskTestHarness<OUT> {
 
 		// then wait for the Task Thread to be in a blocked state
 		// Check whether the state is blocked, this should be the case if it cannot
-		// read more input, i.e. all currently available input has been processed.
+		// notifyNonEmpty more input, i.e. all currently available input has been processed.
 		while (true) {
 			Thread.State state = taskThread.getState();
 			if (state == Thread.State.BLOCKED || state == Thread.State.TERMINATED ||
@@ -360,13 +360,13 @@ public class StreamTaskTestHarness<OUT> {
 			inputGates[i].endInput();
 		}
 	}
-	
+
 	// ------------------------------------------------------------------------
-	
+
 	private class TaskThread extends Thread {
-		
+
 		private final AbstractInvokable task;
-		
+
 		private volatile Throwable error;
 
 		TaskThread(AbstractInvokable task) {

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java
index 1076eeb..3cd9c9a 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java
@@ -199,6 +199,8 @@ public class TwoInputStreamTaskTest {
 		testHarness.processElement(new StreamRecord<String>("Ciao-0-0", initialTime), 0, 1);
 		expectedOutput.add(new StreamRecord<String>("Ciao-0-0", initialTime));
 
+		testHarness.waitForInputProcessing();
+
 		// These elements should be forwarded, since we did not yet receive a checkpoint barrier
 		// on that input, only add to same input, otherwise we would not know the ordering
 		// of the output since the Task might read the inputs in any order
@@ -221,8 +223,8 @@ public class TwoInputStreamTaskTest {
 
 		// we should not yet see the barrier, only the two elements from non-blocked input
 		TestHarnessUtil.assertOutputEquals("Output was not correct.",
-				testHarness.getOutput(),
-				expectedOutput);
+			expectedOutput,
+			testHarness.getOutput());
 
 		testHarness.processEvent(new CheckpointBarrier(0, 0), 0, 1);
 		testHarness.processEvent(new CheckpointBarrier(0, 0), 1, 0);

http://git-wip-us.apache.org/repos/asf/flink/blob/d3ac0adf/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java
index edb1642..f5b7566 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java
@@ -124,7 +124,6 @@ public class TwoInputStreamTaskTestHarness<IN1, IN2, OUT> extends StreamTaskTest
 							bufferSize,
 							inputSerializer1);
 
-
 					StreamEdge streamEdge = new StreamEdge(sourceVertexDummy,
 							targetVertexDummy,
 							1,