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,