You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by pn...@apache.org on 2019/07/01 14:41:09 UTC

[flink] branch master updated (4beba91 -> 6311a65)

This is an automated email from the ASF dual-hosted git repository.

pnowojski pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git.


    from 4beba91  [hotfix][docs] Update configuration page to list new netty shuffle options
     new ff8a946  [hotfix][network] Rename BufferBlocker to BufferStorage
     new 3adc3eb  [hotfix][network] Make toNotifyOnCheckpoint field final in ChekpointBarrierHandlers
     new d6093eb  [hotfix][network] Move queuedBuffered and currentBuffered fields to BufferStorage
     new ad44048  [hotfix][test] Drop unnecessary pageSize argument in BufferBarierTestBase#createBuffer
     new 62927c9  [hotfix][network] Do not abort the same checkpoint barrier twice when cancellation marker was lost
     new 814f47e  [hotfix][test] Drop mockito usage from BarrierTrackerTest
     new aa5f061  [FLINK-12777][network] Extract CheckpointBarrierAligner from BarrierBuffer
     new 5252588  [FLIKN-12777][network] Refactor BarrierTracker to use the same code structure as BarrierBuffer
     new 0d502b6  [FLINK-12777][network] Rename existing classes to make them in sync with the refactor
     new 2e5c499  [hotfix][network] Split InputProcessorUtil into smaller methods
     new 76b2993  [FLINK-12777][network] Introduce LinkedBufferStorage class
     new 530a5e4  [hotfix][operator] Fix checkpointing lock in StreamTwoInputSelectableProcessor
     new 6c7a69a  [FLINK-12777][operator] Use CheckpointedInputGate StreamTwoInputSelectableProcessor
     new 9bdcfac  [hotfix][network] Drop unneccessary reference in the comment
     new 777aefb  [hotfix][network] Minor code simplification in CachedBufferStorage
     new 6311a65  [hotfix][network] Drop one testing constructor of CheckpointedInputGate

The 16 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../runtime/io/AbstractBufferStorage.java          | 170 ++++
 .../flink/streaming/runtime/io/BarrierBuffer.java  | 617 -------------
 .../streaming/runtime/io/BarrierDiscarder.java     | 120 ---
 .../runtime/io/BufferOrEventSequence.java          |   2 +-
 .../flink/streaming/runtime/io/BufferSpiller.java  |  28 +-
 .../io/{BufferBlocker.java => BufferStorage.java}  |  51 +-
 ...BufferBlocker.java => CachedBufferStorage.java} |  40 +-
 .../runtime/io/CheckpointBarrierAligner.java       | 336 ++++++++
 .../runtime/io/CheckpointBarrierHandler.java       |  88 +-
 ...rTracker.java => CheckpointBarrierTracker.java} | 167 ++--
 .../runtime/io/CheckpointedInputGate.java          | 258 ++++++
 .../streaming/runtime/io/EmptyBufferStorage.java   |  73 ++
 .../streaming/runtime/io/InputProcessorUtil.java   | 125 ++-
 .../streaming/runtime/io/LinkedBufferStorage.java  |  92 ++
 .../streaming/runtime/io/StreamInputProcessor.java |   2 +-
 .../runtime/io/StreamTaskNetworkInput.java         |  24 +-
 .../runtime/io/StreamTwoInputProcessor.java        |   4 +-
 .../io/StreamTwoInputSelectableProcessor.java      |  22 +-
 .../tasks/TwoInputSelectableStreamTask.java        |   7 +-
 .../streaming/runtime/io/BufferSpillerTest.java    |   6 +-
 ...kerTestBase.java => BufferStorageTestBase.java} | 217 ++---
 ...ockerTest.java => CachedBufferStorageTest.java} |  20 +-
 ...heckpointBarrierAlignerAlignmentLimitTest.java} |  27 +-
 ...CheckpointBarrierAlignerMassiveRandomTest.java} |   6 +-
 ....java => CheckpointBarrierAlignerTestBase.java} | 950 +++++++++++----------
 ...Test.java => CheckpointBarrierTrackerTest.java} | 165 ++--
 .../runtime/io/CheckpointSequenceValidator.java    |  90 ++
 ...> CreditBasedCheckpointBarrierAlignerTest.java} |  11 +-
 .../runtime/io/LinkedBufferStorageTest.java        | 165 ++++
 ...a => SpillingCheckpointBarrierAlignerTest.java} |  11 +-
 .../tasks/StreamTaskCancellationBarrierTest.java   |   7 +-
 31 files changed, 2172 insertions(+), 1729 deletions(-)
 create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/AbstractBufferStorage.java
 delete mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
 delete mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierDiscarder.java
 rename flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/{BufferBlocker.java => BufferStorage.java} (53%)
 rename flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/{CachedBufferBlocker.java => CachedBufferStorage.java} (72%)
 create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierAligner.java
 rename flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/{BarrierTracker.java => CheckpointBarrierTracker.java} (60%)
 create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java
 create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/EmptyBufferStorage.java
 create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/LinkedBufferStorage.java
 rename flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/{BufferBlockerTestBase.java => BufferStorageTestBase.java} (53%)
 rename flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/{CachedBufferBlockerTest.java => CachedBufferStorageTest.java} (73%)
 rename flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/{BarrierBufferAlignmentLimitTest.java => CheckpointBarrierAlignerAlignmentLimitTest.java} (93%)
 rename flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/{BarrierBufferMassiveRandomTest.java => CheckpointBarrierAlignerMassiveRandomTest.java} (95%)
 rename flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/{BarrierBufferTestBase.java => CheckpointBarrierAlignerTestBase.java} (50%)
 rename flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/{BarrierTrackerTest.java => CheckpointBarrierTrackerTest.java} (70%)
 create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CheckpointSequenceValidator.java
 rename flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/{CreditBasedBarrierBufferTest.java => CreditBasedCheckpointBarrierAlignerTest.java} (74%)
 create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/LinkedBufferStorageTest.java
 rename flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/{SpillingBarrierBufferTest.java => SpillingCheckpointBarrierAlignerTest.java} (82%)


[flink] 16/16: [hotfix][network] Drop one testing constructor of CheckpointedInputGate

Posted by pn...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pnowojski pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 6311a651bf66c393bca86af02fbabd9a9c4129e9
Author: Piotr Nowojski <pi...@gmail.com>
AuthorDate: Wed Jun 26 17:02:15 2019 +0200

    [hotfix][network] Drop one testing constructor of CheckpointedInputGate
---
 .../flink/streaming/runtime/io/CheckpointedInputGate.java  | 14 --------------
 .../io/CheckpointBarrierAlignerMassiveRandomTest.java      |  2 +-
 2 files changed, 1 insertion(+), 15 deletions(-)

diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java
index ce80e30..8e82a00 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java
@@ -18,7 +18,6 @@
 package org.apache.flink.streaming.runtime.io;
 
 import org.apache.flink.annotation.Internal;
-import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.runtime.io.AsyncDataInput;
 import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker;
 import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
@@ -63,19 +62,6 @@ public class CheckpointedInputGate implements AsyncDataInput<BufferOrEvent> {
 	 * {@link #bufferStorage}. */
 	private boolean isFinished;
 
-	/**
-	 * Creates a new checkpoint stream aligner.
-	 *
-	 * <p>There is no limit to how much data may be buffered during an alignment.
-	 *
-	 * @param inputGate The input gate to draw the buffers and events from.
-	 * @param bufferStorage The storage to hold the buffers and events for blocked channels.
-	 */
-	@VisibleForTesting
-	CheckpointedInputGate(InputGate inputGate, BufferStorage bufferStorage) {
-		this (inputGate, bufferStorage, "Testing: No task associated", null);
-	}
-
 	public CheckpointedInputGate(
 			InputGate inputGate,
 			BufferStorage bufferStorage,
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierAlignerMassiveRandomTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierAlignerMassiveRandomTest.java
index 7da0aa3..82920aa 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierAlignerMassiveRandomTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierAlignerMassiveRandomTest.java
@@ -62,7 +62,7 @@ public class CheckpointBarrierAlignerMassiveRandomTest {
 					new BufferPool[] { pool1, pool2 },
 					new BarrierGenerator[] { new CountBarrier(100000), new RandomBarrier(100000) });
 
-			CheckpointedInputGate checkpointedInputGate = new CheckpointedInputGate(myIG, new BufferSpiller(ioMan, myIG.getPageSize()));
+			CheckpointedInputGate checkpointedInputGate = new CheckpointedInputGate(myIG, new BufferSpiller(ioMan, myIG.getPageSize()), "Testing: No task associated", null);
 
 			for (int i = 0; i < 2000000; i++) {
 				BufferOrEvent boe = checkpointedInputGate.pollNext().get();


[flink] 06/16: [hotfix][test] Drop mockito usage from BarrierTrackerTest

Posted by pn...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pnowojski pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 814f47e8b6b5ac9c8ba606ae901ded92da8174d2
Author: Piotr Nowojski <pi...@gmail.com>
AuthorDate: Fri Jun 14 18:02:28 2019 +0200

    [hotfix][test] Drop mockito usage from BarrierTrackerTest
---
 .../flink/streaming/runtime/io/BarrierTrackerTest.java     | 14 ++++----------
 1 file changed, 4 insertions(+), 10 deletions(-)

diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java
index 1be2aab..a8e7727 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java
@@ -37,11 +37,7 @@ import java.util.Arrays;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 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.times;
-import static org.mockito.Mockito.verify;
+
 
 /**
  * Tests for the behavior of the barrier tracker.
@@ -355,17 +351,15 @@ public class BarrierTrackerTest {
 			createCancellationBarrier(2L, 2),
 			createBuffer(0)
 		};
-		AbstractInvokable statefulTask = mock(AbstractInvokable.class);
-		tracker = createBarrierTracker(3, sequence, statefulTask);
+		CheckpointSequenceValidator validator =
+			new CheckpointSequenceValidator(-1, -2);
+		tracker = createBarrierTracker(3, sequence, validator);
 
 		for (BufferOrEvent boe : sequence) {
 			if (boe.isBuffer() || (boe.getEvent().getClass() != CheckpointBarrier.class && boe.getEvent().getClass() != CancelCheckpointMarker.class)) {
 				assertEquals(boe, tracker.pollNext().get());
 			}
 		}
-
-		verify(statefulTask, times(1)).abortCheckpointOnBarrier(eq(1L), any(Throwable.class));
-		verify(statefulTask, times(1)).abortCheckpointOnBarrier(eq(2L), any(Throwable.class));
 	}
 
 	// ------------------------------------------------------------------------


[flink] 14/16: [hotfix][network] Drop unneccessary reference in the comment

Posted by pn...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pnowojski pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 9bdcfacf7212715cf194854886143593896b2bfc
Author: Piotr Nowojski <pi...@gmail.com>
AuthorDate: Wed Jun 26 16:30:12 2019 +0200

    [hotfix][network] Drop unneccessary reference in the comment
    
    This reference was introducing unwanted dependency between CachedBufferStorage and a
    class that was using it.
---
 .../java/org/apache/flink/streaming/runtime/io/CachedBufferStorage.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CachedBufferStorage.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CachedBufferStorage.java
index 4927c35..318497d 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CachedBufferStorage.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CachedBufferStorage.java
@@ -118,7 +118,7 @@ public class CachedBufferStorage extends AbstractBufferStorage {
 	 */
 	public static class CachedBufferOrEventSequence implements BufferOrEventSequence {
 
-		/** The sequence of buffers and events to be consumed by {@link CheckpointedInputGate}.*/
+		/** The sequence of buffers and events to be consumed. */
 		private final ArrayDeque<BufferOrEvent> queuedBuffers;
 
 		/** The total size of the cached data. */


[flink] 02/16: [hotfix][network] Make toNotifyOnCheckpoint field final in ChekpointBarrierHandlers

Posted by pn...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pnowojski pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 3adc3eb2a6ece13f3befddd41f508d01bde067d7
Author: Piotr Nowojski <pi...@gmail.com>
AuthorDate: Thu Jun 13 15:07:01 2019 +0200

    [hotfix][network] Make toNotifyOnCheckpoint field final in ChekpointBarrierHandlers
---
 .../flink/streaming/runtime/io/BarrierBuffer.java  | 27 ++++----
 .../streaming/runtime/io/BarrierDiscarder.java     | 15 -----
 .../flink/streaming/runtime/io/BarrierTracker.java | 19 +++---
 .../runtime/io/CheckpointBarrierHandler.java       |  9 ---
 .../streaming/runtime/io/InputProcessorUtil.java   | 10 ++-
 .../io/BarrierBufferAlignmentLimitTest.java        | 18 ++++--
 .../runtime/io/BarrierBufferTestBase.java          | 73 ++++++++++------------
 .../streaming/runtime/io/BarrierTrackerTest.java   | 59 ++++++++---------
 .../runtime/io/CreditBasedBarrierBufferTest.java   |  7 ++-
 .../runtime/io/SpillingBarrierBufferTest.java      |  7 ++-
 10 files changed, 105 insertions(+), 139 deletions(-)

diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
index ad62360..b2e6ea1 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
@@ -33,6 +33,8 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.annotation.Nullable;
+
 import java.io.IOException;
 import java.util.ArrayDeque;
 import java.util.Optional;
@@ -86,8 +88,8 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 	 */
 	private BufferOrEventSequence currentBuffered;
 
-	/** Handler that receives the checkpoint notifications. */
-	private AbstractInvokable toNotifyOnCheckpoint;
+	@Nullable
+	private final AbstractInvokable toNotifyOnCheckpoint;
 
 	/** The ID of the checkpoint for which we expect barriers. */
 	private long currentCheckpointId = -1L;
@@ -127,7 +129,7 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 	 */
 	@VisibleForTesting
 	BarrierBuffer(InputGate inputGate, BufferStorage bufferStorage) {
-		this (inputGate, bufferStorage, -1, "Testing: No task associated");
+		this (inputGate, bufferStorage, -1, "Testing: No task associated", null);
 	}
 
 	/**
@@ -141,8 +143,14 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 	 * @param bufferStorage The storage to hold the buffers and events for blocked channels.
 	 * @param maxBufferedBytes The maximum bytes to be buffered before the checkpoint aborts.
 	 * @param taskName The task name for logging.
+	 * @param toNotifyOnCheckpoint optional Handler that receives the checkpoint notifications.
 	 */
-	BarrierBuffer(InputGate inputGate, BufferStorage bufferStorage, long maxBufferedBytes, String taskName) {
+	BarrierBuffer(
+			InputGate inputGate,
+			BufferStorage bufferStorage,
+			long maxBufferedBytes,
+			String taskName,
+			@Nullable AbstractInvokable toNotifyOnCheckpoint) {
 		checkArgument(maxBufferedBytes == -1 || maxBufferedBytes > 0);
 
 		this.inputGate = inputGate;
@@ -154,6 +162,7 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 		this.queuedBuffered = new ArrayDeque<BufferOrEventSequence>();
 
 		this.taskName = taskName;
+		this.toNotifyOnCheckpoint = toNotifyOnCheckpoint;
 	}
 
 	@Override
@@ -452,16 +461,6 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 	}
 
 	@Override
-	public void registerCheckpointEventHandler(AbstractInvokable toNotifyOnCheckpoint) {
-		if (this.toNotifyOnCheckpoint == null) {
-			this.toNotifyOnCheckpoint = toNotifyOnCheckpoint;
-		}
-		else {
-			throw new IllegalStateException("BarrierBuffer already has a registered checkpoint notifyee");
-		}
-	}
-
-	@Override
 	public boolean isEmpty() {
 		return currentBuffered == null;
 	}
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierDiscarder.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierDiscarder.java
index e8d9f34..c33c940 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierDiscarder.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierDiscarder.java
@@ -23,7 +23,6 @@ import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker;
 import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
 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.jobgraph.tasks.AbstractInvokable;
 
 import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
@@ -45,10 +44,6 @@ public class BarrierDiscarder implements CheckpointBarrierHandler {
 	 */
 	private final int totalNumberOfInputChannels;
 
-
-	/** The listener to be notified on complete checkpoints. */
-	private AbstractInvokable toNotifyOnCheckpoint;
-
 	// ------------------------------------------------------------------------
 
 	public BarrierDiscarder(InputGate inputGate) {
@@ -88,16 +83,6 @@ public class BarrierDiscarder implements CheckpointBarrierHandler {
 	}
 
 	@Override
-	public void registerCheckpointEventHandler(AbstractInvokable toNotifyOnCheckpoint) {
-		if (this.toNotifyOnCheckpoint == null) {
-			this.toNotifyOnCheckpoint = toNotifyOnCheckpoint;
-		}
-		else {
-			throw new IllegalStateException("BarrierDiscarder already has a registered checkpoint notifyee");
-		}
-	}
-
-	@Override
 	public void cleanup() {
 
 	}
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierTracker.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierTracker.java
index 49d2991..f7629bb 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierTracker.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierTracker.java
@@ -33,6 +33,8 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.annotation.Nullable;
+
 import java.util.ArrayDeque;
 import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
@@ -77,7 +79,7 @@ public class BarrierTracker implements CheckpointBarrierHandler {
 	private final ArrayDeque<CheckpointBarrierCount> pendingCheckpoints;
 
 	/** The listener to be notified on complete checkpoints. */
-	private AbstractInvokable toNotifyOnCheckpoint;
+	private final AbstractInvokable toNotifyOnCheckpoint;
 
 	/** The highest checkpoint ID encountered so far. */
 	private long latestPendingCheckpointID = -1;
@@ -85,9 +87,14 @@ public class BarrierTracker implements CheckpointBarrierHandler {
 	// ------------------------------------------------------------------------
 
 	public BarrierTracker(InputGate inputGate) {
+		this(inputGate, null);
+	}
+
+	public BarrierTracker(InputGate inputGate, @Nullable AbstractInvokable toNotifyOnCheckpoint) {
 		this.inputGate = inputGate;
 		this.totalNumberOfInputChannels = inputGate.getNumberOfInputChannels();
 		this.pendingCheckpoints = new ArrayDeque<>();
+		this.toNotifyOnCheckpoint = toNotifyOnCheckpoint;
 	}
 
 	@Override
@@ -127,16 +134,6 @@ public class BarrierTracker implements CheckpointBarrierHandler {
 	}
 
 	@Override
-	public void registerCheckpointEventHandler(AbstractInvokable toNotifyOnCheckpoint) {
-		if (this.toNotifyOnCheckpoint == null) {
-			this.toNotifyOnCheckpoint = toNotifyOnCheckpoint;
-		}
-		else {
-			throw new IllegalStateException("BarrierTracker already has a registered checkpoint notifyee");
-		}
-	}
-
-	@Override
 	public void cleanup() {
 		pendingCheckpoints.clear();
 	}
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierHandler.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierHandler.java
index faffd44..2ee1a97 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierHandler.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierHandler.java
@@ -21,7 +21,6 @@ package org.apache.flink.streaming.runtime.io;
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.runtime.io.AsyncDataInput;
 import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 
 import java.io.IOException;
 
@@ -32,14 +31,6 @@ import java.io.IOException;
  */
 @Internal
 public interface CheckpointBarrierHandler extends AsyncDataInput<BufferOrEvent> {
-
-	/**
-	 * Registers the task be notified once all checkpoint barriers have been received for a checkpoint.
-	 *
-	 * @param task The task to notify
-	 */
-	void registerCheckpointEventHandler(AbstractInvokable task);
-
 	/**
 	 * Cleans up all internally held resources.
 	 *
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java
index 289dd1a..ebef48f 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java
@@ -58,13 +58,15 @@ public class InputProcessorUtil {
 					inputGate,
 					new CachedBufferStorage(inputGate.getPageSize()),
 					maxAlign,
-					taskName);
+					taskName,
+					checkpointedTask);
 			} else {
 				barrierHandler = new BarrierBuffer(
 					inputGate,
 					new BufferSpiller(ioManager, inputGate.getPageSize()),
 					maxAlign,
-					taskName);
+					taskName,
+					checkpointedTask);
 			}
 		} else if (checkpointMode == CheckpointingMode.AT_LEAST_ONCE) {
 			barrierHandler = new BarrierTracker(inputGate);
@@ -72,10 +74,6 @@ public class InputProcessorUtil {
 			throw new IllegalArgumentException("Unrecognized Checkpointing Mode: " + checkpointMode);
 		}
 
-		if (checkpointedTask != null) {
-			barrierHandler.registerCheckpointEventHandler(checkpointedTask);
-		}
-
 		return barrierHandler;
 	}
 }
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferAlignmentLimitTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferAlignmentLimitTest.java
index 0a284e1..8c97938 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferAlignmentLimitTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferAlignmentLimitTest.java
@@ -115,10 +115,13 @@ public class BarrierBufferAlignmentLimitTest {
 
 		// the barrier buffer has a limit that only 1000 bytes may be spilled in alignment
 		MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence));
-		BarrierBuffer buffer = new BarrierBuffer(gate, new BufferSpiller(ioManager, gate.getPageSize()), 1000, "Testing");
-
 		AbstractInvokable toNotify = mock(AbstractInvokable.class);
-		buffer.registerCheckpointEventHandler(toNotify);
+		BarrierBuffer buffer = new BarrierBuffer(
+			gate,
+			new BufferSpiller(ioManager, gate.getPageSize()),
+			1000,
+			"Testing",
+			toNotify);
 
 		// validating the sequence of buffers
 
@@ -210,10 +213,13 @@ public class BarrierBufferAlignmentLimitTest {
 
 		// the barrier buffer has a limit that only 1000 bytes may be spilled in alignment
 		MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence));
-		BarrierBuffer buffer = new BarrierBuffer(gate, new BufferSpiller(ioManager, gate.getPageSize()), 500, "Testing");
-
 		AbstractInvokable toNotify = mock(AbstractInvokable.class);
-		buffer.registerCheckpointEventHandler(toNotify);
+		BarrierBuffer buffer = new BarrierBuffer(
+			gate,
+			new BufferSpiller(ioManager, gate.getPageSize()),
+			500,
+			"Testing",
+			toNotify);
 
 		// validating the sequence of buffers
 		long startTs;
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTestBase.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTestBase.java
index 4bc05ff..908a199 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTestBase.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTestBase.java
@@ -41,6 +41,8 @@ import org.hamcrest.Description;
 import org.junit.After;
 import org.junit.Test;
 
+import javax.annotation.Nullable;
+
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Random;
@@ -70,12 +72,23 @@ public abstract class BarrierBufferTestBase {
 
 	BarrierBuffer buffer;
 
-	protected BarrierBuffer createBarrierBuffer(int numberOfChannels, BufferOrEvent[] sequence) throws IOException {
+	protected BarrierBuffer createBarrierBuffer(
+		int numberOfChannels,
+		BufferOrEvent[] sequence,
+		@Nullable AbstractInvokable toNotify) throws IOException {
 		MockInputGate gate = new MockInputGate(PAGE_SIZE, numberOfChannels, Arrays.asList(sequence));
-		return createBarrierBuffer(gate);
+		return createBarrierBuffer(gate, toNotify);
+	}
+
+	protected BarrierBuffer createBarrierBuffer(int numberOfChannels, BufferOrEvent[] sequence) throws IOException {
+		return createBarrierBuffer(numberOfChannels, sequence, null);
 	}
 
-	abstract BarrierBuffer createBarrierBuffer(InputGate gate) throws IOException;
+	protected BarrierBuffer createBarrierBuffer(InputGate gate) throws IOException {
+		return createBarrierBuffer(gate, null);
+	}
+
+	abstract BarrierBuffer createBarrierBuffer(InputGate gate, @Nullable AbstractInvokable toNotify) throws IOException;
 
 	abstract void validateAlignmentBuffered(long actualBytesBuffered, BufferOrEvent... sequence);
 
@@ -147,10 +160,9 @@ public abstract class BarrierBufferTestBase {
 			createBarrier(4, 0), createBarrier(5, 0), createBarrier(6, 0),
 			createBuffer(0, PAGE_SIZE), createEndOfPartition(0)
 		};
-		buffer = createBarrierBuffer(1, sequence);
-
 		ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler();
-		buffer.registerCheckpointEventHandler(handler);
+		buffer = createBarrierBuffer(1, sequence, handler);
+
 		handler.setNextExpectedCheckpointId(1L);
 
 		for (BufferOrEvent boe : sequence) {
@@ -198,10 +210,9 @@ public abstract class BarrierBufferTestBase {
 			createBuffer(0, PAGE_SIZE),
 			createEndOfPartition(0), createEndOfPartition(1), createEndOfPartition(2)
 		};
-		buffer = createBarrierBuffer(3, sequence);
-
 		ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler();
-		buffer.registerCheckpointEventHandler(handler);
+		buffer = createBarrierBuffer(3, sequence, handler);
+
 		handler.setNextExpectedCheckpointId(1L);
 
 		// pre checkpoint 1
@@ -292,10 +303,9 @@ public abstract class BarrierBufferTestBase {
 			createBarrier(2, 2),
 			createBuffer(2, PAGE_SIZE), createEndOfPartition(2), createBuffer(0, PAGE_SIZE), createEndOfPartition(0)
 		};
-		buffer = createBarrierBuffer(3, sequence);
-
 		ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler();
-		buffer.registerCheckpointEventHandler(handler);
+		buffer = createBarrierBuffer(3, sequence, handler);
+
 		handler.setNextExpectedCheckpointId(1L);
 
 		// pre-checkpoint 1
@@ -368,10 +378,9 @@ public abstract class BarrierBufferTestBase {
 			createBuffer(2, PAGE_SIZE), createEndOfPartition(2),
 			createBuffer(0, PAGE_SIZE), createEndOfPartition(0)
 		};
-		buffer = createBarrierBuffer(3, sequence);
-
 		ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler();
-		buffer.registerCheckpointEventHandler(handler);
+		buffer = createBarrierBuffer(3, sequence, handler);
+
 		handler.setNextExpectedCheckpointId(1L);
 
 		// around checkpoint 1
@@ -461,10 +470,8 @@ public abstract class BarrierBufferTestBase {
 			createBuffer(2, PAGE_SIZE), createEndOfPartition(2),
 			createBuffer(0, PAGE_SIZE), createEndOfPartition(0)
 		};
-		buffer = createBarrierBuffer(3, sequence);
-
 		AbstractInvokable toNotify = mock(AbstractInvokable.class);
-		buffer.registerCheckpointEventHandler(toNotify);
+		buffer = createBarrierBuffer(3, sequence, toNotify);
 
 		long startTs;
 
@@ -548,10 +555,9 @@ public abstract class BarrierBufferTestBase {
 			createBuffer(2, PAGE_SIZE), createEndOfPartition(2),
 			createBuffer(0, PAGE_SIZE), createEndOfPartition(0)
 		};
-		buffer = createBarrierBuffer(3, sequence);
-
 		ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler();
-		buffer.registerCheckpointEventHandler(handler);
+		buffer = createBarrierBuffer(3, sequence, handler);
+
 		handler.setNextExpectedCheckpointId(1L);
 
 		// checkpoint 1
@@ -694,10 +700,9 @@ public abstract class BarrierBufferTestBase {
 			createBarrier(2, 2),
 			createBuffer(2, PAGE_SIZE), createEndOfPartition(2), createBuffer(0, PAGE_SIZE), createEndOfPartition(0)
 		};
-		buffer = createBarrierBuffer(3, sequence);
-
 		ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler();
-		buffer.registerCheckpointEventHandler(handler);
+		buffer = createBarrierBuffer(3, sequence, handler);
+
 		handler.setNextExpectedCheckpointId(1L);
 
 		// pre-checkpoint 1
@@ -841,10 +846,8 @@ public abstract class BarrierBufferTestBase {
 			createCancellationBarrier(6, 0),
 			createBuffer(0, PAGE_SIZE)
 		};
-		buffer = createBarrierBuffer(1, sequence);
-
 		AbstractInvokable toNotify = mock(AbstractInvokable.class);
-		buffer.registerCheckpointEventHandler(toNotify);
+		buffer = createBarrierBuffer(1, sequence, toNotify);
 
 		check(sequence[0], buffer.pollNext().get(), PAGE_SIZE);
 		check(sequence[2], buffer.pollNext().get(), PAGE_SIZE);
@@ -902,10 +905,8 @@ public abstract class BarrierBufferTestBase {
 
 			/* 37 */ createBuffer(0, PAGE_SIZE)
 		};
-		buffer = createBarrierBuffer(3, sequence);
-
 		AbstractInvokable toNotify = mock(AbstractInvokable.class);
-		buffer.registerCheckpointEventHandler(toNotify);
+		buffer = createBarrierBuffer(3, sequence, toNotify);
 
 		long startTs;
 
@@ -990,10 +991,8 @@ public abstract class BarrierBufferTestBase {
 				// some more buffers
 			/* 16 */ createBuffer(0, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createBuffer(2, PAGE_SIZE)
 		};
-		buffer = createBarrierBuffer(3, sequence);
-
 		AbstractInvokable toNotify = mock(AbstractInvokable.class);
-		buffer.registerCheckpointEventHandler(toNotify);
+		buffer = createBarrierBuffer(3, sequence, toNotify);
 
 		long startTs;
 
@@ -1074,10 +1073,8 @@ public abstract class BarrierBufferTestBase {
 				// some more buffers
 			/* 18 */ createBuffer(0, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createBuffer(2, PAGE_SIZE)
 		};
-		buffer = createBarrierBuffer(3, sequence);
-
 		AbstractInvokable toNotify = mock(AbstractInvokable.class);
-		buffer.registerCheckpointEventHandler(toNotify);
+		buffer = createBarrierBuffer(3, sequence, toNotify);
 
 		long startTs;
 
@@ -1151,10 +1148,8 @@ public abstract class BarrierBufferTestBase {
 				// some more buffers
 			/* 16 */ createBuffer(0, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createBuffer(2, PAGE_SIZE)
 		};
-		buffer = createBarrierBuffer(3, sequence);
-
 		AbstractInvokable toNotify = mock(AbstractInvokable.class);
-		buffer.registerCheckpointEventHandler(toNotify);
+		buffer = createBarrierBuffer(3, sequence, toNotify);
 
 		long startTs;
 
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java
index 398a95a..cb58837 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java
@@ -33,6 +33,8 @@ import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
 import org.junit.After;
 import org.junit.Test;
 
+import javax.annotation.Nullable;
+
 import java.util.Arrays;
 
 import static org.junit.Assert.assertEquals;
@@ -95,11 +97,9 @@ public class BarrierTrackerTest {
 				createBarrier(4, 0), createBarrier(5, 0), createBarrier(6, 0),
 				createBuffer(0)
 		};
-		tracker = createBarrierTracker(1, sequence);
-
 		CheckpointSequenceValidator validator =
-				new CheckpointSequenceValidator(1, 2, 3, 4, 5, 6);
-		tracker.registerCheckpointEventHandler(validator);
+			new CheckpointSequenceValidator(1, 2, 3, 4, 5, 6);
+		tracker = createBarrierTracker(1, sequence, validator);
 
 		for (BufferOrEvent boe : sequence) {
 			if (boe.isBuffer() || boe.getEvent().getClass() != CheckpointBarrier.class) {
@@ -119,11 +119,9 @@ public class BarrierTrackerTest {
 				createBarrier(7, 0), createBuffer(0), createBarrier(10, 0),
 				createBuffer(0)
 		};
-		tracker = createBarrierTracker(1, sequence);
-
 		CheckpointSequenceValidator validator =
-				new CheckpointSequenceValidator(1, 3, 4, 6, 7, 10);
-		tracker.registerCheckpointEventHandler(validator);
+			new CheckpointSequenceValidator(1, 3, 4, 6, 7, 10);
+		tracker = createBarrierTracker(1, sequence, validator);
 
 		for (BufferOrEvent boe : sequence) {
 			if (boe.isBuffer() || boe.getEvent().getClass() != CheckpointBarrier.class) {
@@ -152,11 +150,9 @@ public class BarrierTrackerTest {
 
 				createBuffer(0)
 		};
-		tracker = createBarrierTracker(3, sequence);
-
 		CheckpointSequenceValidator validator =
-				new CheckpointSequenceValidator(1, 2, 3, 4);
-		tracker.registerCheckpointEventHandler(validator);
+			new CheckpointSequenceValidator(1, 2, 3, 4);
+		tracker = createBarrierTracker(3, sequence, validator);
 
 		for (BufferOrEvent boe : sequence) {
 			if (boe.isBuffer() || boe.getEvent().getClass() != CheckpointBarrier.class) {
@@ -189,11 +185,9 @@ public class BarrierTrackerTest {
 
 				createBuffer(0)
 		};
-		tracker = createBarrierTracker(3, sequence);
-
 		CheckpointSequenceValidator validator =
-				new CheckpointSequenceValidator(1, 2, 4);
-		tracker.registerCheckpointEventHandler(validator);
+			new CheckpointSequenceValidator(1, 2, 4);
+		tracker = createBarrierTracker(3, sequence, validator);
 
 		for (BufferOrEvent boe : sequence) {
 			if (boe.isBuffer() || boe.getEvent().getClass() != CheckpointBarrier.class) {
@@ -265,11 +259,9 @@ public class BarrierTrackerTest {
 				// complete checkpoint 10
 				createBarrier(10, 0), createBarrier(10, 1),
 		};
-		tracker = createBarrierTracker(3, sequence);
-
 		CheckpointSequenceValidator validator =
-				new CheckpointSequenceValidator(2, 3, 4, 5, 7, 8, 9, 10);
-		tracker.registerCheckpointEventHandler(validator);
+			new CheckpointSequenceValidator(2, 3, 4, 5, 7, 8, 9, 10);
+		tracker = createBarrierTracker(3, sequence, validator);
 
 		for (BufferOrEvent boe : sequence) {
 			if (boe.isBuffer() || boe.getEvent().getClass() != CheckpointBarrier.class) {
@@ -291,12 +283,10 @@ public class BarrierTrackerTest {
 				createCancellationBarrier(6, 0),
 				createBuffer(0)
 		};
-		tracker = createBarrierTracker(1, sequence);
-
 		// negative values mean an expected cancellation call!
 		CheckpointSequenceValidator validator =
-				new CheckpointSequenceValidator(1, 2, -4, 5, -6);
-		tracker.registerCheckpointEventHandler(validator);
+			new CheckpointSequenceValidator(1, 2, -4, 5, -6);
+		tracker = createBarrierTracker(1, sequence, validator);
 
 		for (BufferOrEvent boe : sequence) {
 			if (boe.isBuffer()) {
@@ -342,12 +332,10 @@ public class BarrierTrackerTest {
 
 				createBuffer(0)
 		};
-		tracker = createBarrierTracker(3, sequence);
-
 		// negative values mean an expected cancellation call!
 		CheckpointSequenceValidator validator =
-				new CheckpointSequenceValidator(1, -2, 3, -4, 5, -6);
-		tracker.registerCheckpointEventHandler(validator);
+			new CheckpointSequenceValidator(1, -2, 3, -4, 5, -6);
+		tracker = createBarrierTracker(3, sequence, validator);
 
 		for (BufferOrEvent boe : sequence) {
 			if (boe.isBuffer()) {
@@ -371,11 +359,8 @@ public class BarrierTrackerTest {
 			createCancellationBarrier(2L, 2),
 			createBuffer(0)
 		};
-		tracker = createBarrierTracker(3, sequence);
-
 		AbstractInvokable statefulTask = mock(AbstractInvokable.class);
-
-		tracker.registerCheckpointEventHandler(statefulTask);
+		tracker = createBarrierTracker(3, sequence, statefulTask);
 
 		for (BufferOrEvent boe : sequence) {
 			if (boe.isBuffer() || (boe.getEvent().getClass() != CheckpointBarrier.class && boe.getEvent().getClass() != CancelCheckpointMarker.class)) {
@@ -390,10 +375,16 @@ public class BarrierTrackerTest {
 	// ------------------------------------------------------------------------
 	//  Utils
 	// ------------------------------------------------------------------------
-
 	private static BarrierTracker createBarrierTracker(int numberOfChannels, BufferOrEvent[] sequence) {
+		return createBarrierTracker(numberOfChannels, sequence, null);
+	}
+
+	private static BarrierTracker createBarrierTracker(
+			int numberOfChannels,
+			BufferOrEvent[] sequence,
+			@Nullable AbstractInvokable toNotifyOnCheckpoint) {
 		MockInputGate gate = new MockInputGate(PAGE_SIZE, numberOfChannels, Arrays.asList(sequence));
-		return new BarrierTracker(gate);
+		return new BarrierTracker(gate, toNotifyOnCheckpoint);
 	}
 
 	private static BufferOrEvent createBarrier(long id, int channel) {
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CreditBasedBarrierBufferTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CreditBasedBarrierBufferTest.java
index da88ffb..bbfe8b6 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CreditBasedBarrierBufferTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CreditBasedBarrierBufferTest.java
@@ -20,8 +20,9 @@ package org.apache.flink.streaming.runtime.io;
 
 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.jobgraph.tasks.AbstractInvokable;
 
-import java.io.IOException;
+import javax.annotation.Nullable;
 
 import static org.junit.Assert.assertEquals;
 
@@ -31,8 +32,8 @@ import static org.junit.Assert.assertEquals;
 public class CreditBasedBarrierBufferTest extends BarrierBufferTestBase {
 
 	@Override
-	public BarrierBuffer createBarrierBuffer(InputGate gate) throws IOException {
-		return new BarrierBuffer(gate, new CachedBufferStorage(PAGE_SIZE));
+	BarrierBuffer createBarrierBuffer(InputGate gate, @Nullable AbstractInvokable toNotify) {
+		return new BarrierBuffer(gate, new CachedBufferStorage(PAGE_SIZE), -1, "Testing", toNotify);
 	}
 
 	@Override
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/SpillingBarrierBufferTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/SpillingBarrierBufferTest.java
index 546fb62..2101f40 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/SpillingBarrierBufferTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/SpillingBarrierBufferTest.java
@@ -22,10 +22,13 @@ 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.partition.consumer.BufferOrEvent;
 import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
+import javax.annotation.Nullable;
+
 import java.io.File;
 import java.io.IOException;
 
@@ -64,8 +67,8 @@ public class SpillingBarrierBufferTest extends BarrierBufferTestBase {
 	}
 
 	@Override
-	public BarrierBuffer createBarrierBuffer(InputGate gate) throws IOException{
-		return new BarrierBuffer(gate, new BufferSpiller(ioManager, PAGE_SIZE));
+	BarrierBuffer createBarrierBuffer(InputGate gate, @Nullable AbstractInvokable toNotify) throws IOException {
+		return new BarrierBuffer(gate, new BufferSpiller(ioManager, PAGE_SIZE), -1, "Testing", toNotify);
 	}
 
 	@Override


[flink] 15/16: [hotfix][network] Minor code simplification in CachedBufferStorage

Posted by pn...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pnowojski pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 777aefb84574a039f4030116519cda2347d9b679
Author: Piotr Nowojski <pi...@gmail.com>
AuthorDate: Wed Jun 26 16:51:07 2019 +0200

    [hotfix][network] Minor code simplification in CachedBufferStorage
---
 .../java/org/apache/flink/streaming/runtime/io/CachedBufferStorage.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CachedBufferStorage.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CachedBufferStorage.java
index 318497d..033ff15 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CachedBufferStorage.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CachedBufferStorage.java
@@ -88,7 +88,7 @@ public class CachedBufferStorage extends AbstractBufferStorage {
 		}
 
 		CachedBufferOrEventSequence currentSequence = new CachedBufferOrEventSequence(currentBuffers, bytesBlocked);
-		currentBuffers = new ArrayDeque<BufferOrEvent>();
+		currentBuffers = new ArrayDeque<>();
 		bytesBlocked = 0L;
 
 		return currentSequence;


[flink] 08/16: [FLIKN-12777][network] Refactor BarrierTracker to use the same code structure as BarrierBuffer

Posted by pn...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pnowojski pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 52525884826cd0e99b2309d89d74fa270d51ec90
Author: Piotr Nowojski <pi...@gmail.com>
AuthorDate: Sat Jun 15 10:18:01 2019 +0200

    [FLIKN-12777][network] Refactor BarrierTracker to use the same code structure as BarrierBuffer
---
 .../flink/streaming/runtime/io/BarrierBuffer.java  |  52 ++++---
 .../streaming/runtime/io/BarrierDiscarder.java     | 105 --------------
 .../runtime/io/BufferOrEventSequence.java          |   2 +-
 .../runtime/io/CheckpointBarrierAligner.java       |  65 ++-------
 .../runtime/io/CheckpointBarrierDiscarder.java     |  74 ++++++++++
 .../runtime/io/CheckpointBarrierHandler.java       |  85 ++++++++---
 ...rTracker.java => CheckpointBarrierTracker.java} | 158 +++++++--------------
 ...rierHandler.java => CheckpointedInputGate.java} |   7 +-
 .../streaming/runtime/io/EmptyBufferStorage.java   |  73 ++++++++++
 .../streaming/runtime/io/InputProcessorUtil.java   |  11 +-
 .../streaming/runtime/io/StreamInputProcessor.java |   2 +-
 .../runtime/io/StreamTaskNetworkInput.java         |   6 +-
 .../runtime/io/StreamTwoInputProcessor.java        |   2 +-
 .../io/StreamTwoInputSelectableProcessor.java      |   5 +-
 .../runtime/io/BarrierBufferTestBase.java          |  32 ++---
 .../streaming/runtime/io/BarrierTrackerTest.java   |  11 +-
 16 files changed, 349 insertions(+), 341 deletions(-)

diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
index 1594389..8dcc005 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
@@ -38,7 +38,7 @@ import java.util.concurrent.CompletableFuture;
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * The barrier buffer is {@link CheckpointBarrierHandler} that blocks inputs with barriers until
+ * The barrier buffer is {@link CheckpointedInputGate} that blocks inputs with barriers until
  * all inputs have received the barrier for a given checkpoint.
  *
  * <p>To avoid back-pressuring the input streams (which may cause distributed deadlocks), the
@@ -46,11 +46,11 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  * the blocks are released.
  */
 @Internal
-public class BarrierBuffer implements CheckpointBarrierHandler {
+public class BarrierBuffer implements CheckpointedInputGate {
 
 	private static final Logger LOG = LoggerFactory.getLogger(BarrierBuffer.class);
 
-	private final CheckpointBarrierAligner barrierAligner;
+	private final CheckpointBarrierHandler barrierHandler;
 
 	/** The gate that the buffer draws its input from. */
 	private final InputGate inputGate;
@@ -77,6 +77,21 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 		this (inputGate, bufferStorage, "Testing: No task associated", null);
 	}
 
+	BarrierBuffer(
+			InputGate inputGate,
+			BufferStorage bufferStorage,
+			String taskName,
+			@Nullable AbstractInvokable toNotifyOnCheckpoint) {
+		this(
+			inputGate,
+			bufferStorage,
+			new CheckpointBarrierAligner(
+				inputGate.getNumberOfInputChannels(),
+				taskName,
+				toNotifyOnCheckpoint)
+		);
+	}
+
 	/**
 	 * Creates a new checkpoint stream aligner.
 	 *
@@ -86,20 +101,15 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 	 *
 	 * @param inputGate The input gate to draw the buffers and events from.
 	 * @param bufferStorage The storage to hold the buffers and events for blocked channels.
-	 * @param taskName The task name for logging.
-	 * @param toNotifyOnCheckpoint optional Handler that receives the checkpoint notifications.
+	 * @param barrierHandler Handler that controls which channels are blocked.
 	 */
 	BarrierBuffer(
 			InputGate inputGate,
 			BufferStorage bufferStorage,
-			String taskName,
-			@Nullable AbstractInvokable toNotifyOnCheckpoint) {
+			CheckpointBarrierHandler barrierHandler) {
 		this.inputGate = inputGate;
 		this.bufferStorage = checkNotNull(bufferStorage);
-		this.barrierAligner = new CheckpointBarrierAligner(
-			inputGate.getNumberOfInputChannels(),
-			taskName,
-			toNotifyOnCheckpoint);
+		this.barrierHandler = barrierHandler;
 	}
 
 	@Override
@@ -131,11 +141,11 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 			}
 
 			BufferOrEvent bufferOrEvent = next.get();
-			if (barrierAligner.isBlocked(bufferOrEvent.getChannelIndex())) {
+			if (barrierHandler.isBlocked(bufferOrEvent.getChannelIndex())) {
 				// if the channel is blocked, we just store the BufferOrEvent
 				bufferStorage.add(bufferOrEvent);
 				if (bufferStorage.isFull()) {
-					barrierAligner.checkpointSizeLimitExceeded(bufferStorage.getMaxBufferedBytes());
+					barrierHandler.checkpointSizeLimitExceeded(bufferStorage.getMaxBufferedBytes());
 					bufferStorage.rollOver();
 				}
 			}
@@ -146,19 +156,19 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 				CheckpointBarrier checkpointBarrier = (CheckpointBarrier) bufferOrEvent.getEvent();
 				if (!endOfInputGate) {
 					// process barriers only if there is a chance of the checkpoint completing
-					if (barrierAligner.processBarrier(checkpointBarrier, bufferOrEvent.getChannelIndex(), bufferStorage.getPendingBytes())) {
+					if (barrierHandler.processBarrier(checkpointBarrier, bufferOrEvent.getChannelIndex(), bufferStorage.getPendingBytes())) {
 						bufferStorage.rollOver();
 					}
 				}
 			}
 			else if (bufferOrEvent.getEvent().getClass() == CancelCheckpointMarker.class) {
-				if (barrierAligner.processCancellationBarrier((CancelCheckpointMarker) bufferOrEvent.getEvent())) {
+				if (barrierHandler.processCancellationBarrier((CancelCheckpointMarker) bufferOrEvent.getEvent())) {
 					bufferStorage.rollOver();
 				}
 			}
 			else {
 				if (bufferOrEvent.getEvent().getClass() == EndOfPartitionEvent.class) {
-					if (barrierAligner.processEndOfPartition()) {
+					if (barrierHandler.processEndOfPartition()) {
 						bufferStorage.rollOver();
 					}
 				}
@@ -178,7 +188,7 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 		} else {
 			// end of input stream. stream continues with the buffered data
 			endOfInputGate = true;
-			barrierAligner.releaseBlocksAndResetBarriers();
+			barrierHandler.releaseBlocksAndResetBarriers();
 			bufferStorage.rollOver();
 			return pollNext();
 		}
@@ -208,13 +218,13 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 	 *
 	 * @return The ID of the pending of completed checkpoint.
 	 */
-	public long getCurrentCheckpointId() {
-		return barrierAligner.getCurrentCheckpointId();
+	public long getLatestCheckpointId() {
+		return barrierHandler.getLatestCheckpointId();
 	}
 
 	@Override
 	public long getAlignmentDurationNanos() {
-		return barrierAligner.getAlignmentDurationNanos();
+		return barrierHandler.getAlignmentDurationNanos();
 	}
 
 	@Override
@@ -228,6 +238,6 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 
 	@Override
 	public String toString() {
-		return barrierAligner.toString();
+		return barrierHandler.toString();
 	}
 }
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierDiscarder.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierDiscarder.java
deleted file mode 100644
index c33c940..0000000
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierDiscarder.java
+++ /dev/null
@@ -1,105 +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.streaming.runtime.io;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker;
-import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
-import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
-import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
-
-import java.util.Optional;
-import java.util.concurrent.CompletableFuture;
-
-/**
- * The BarrierDiscarder discards checkpoint barriers have been received from which input channels.
- */
-@Internal
-public class BarrierDiscarder implements CheckpointBarrierHandler {
-
-	// ------------------------------------------------------------------------
-
-	/** The input gate, to draw the buffers and events from. */
-	private final InputGate inputGate;
-
-	/**
-	 * The number of channels. Once that many barriers have been received for a checkpoint, the
-	 * checkpoint is considered complete.
-	 */
-	private final int totalNumberOfInputChannels;
-
-	// ------------------------------------------------------------------------
-
-	public BarrierDiscarder(InputGate inputGate) {
-		this.inputGate = inputGate;
-		this.totalNumberOfInputChannels = inputGate.getNumberOfInputChannels();
-	}
-
-	@Override
-	public CompletableFuture<?> isAvailable() {
-		return inputGate.isAvailable();
-	}
-
-	@Override
-	public boolean isFinished() {
-		return inputGate.isFinished();
-	}
-
-	@Override
-	public Optional<BufferOrEvent> pollNext() throws Exception {
-		while (true) {
-			Optional<BufferOrEvent> next = inputGate.pollNext();
-			if (!next.isPresent()) {
-				// buffer or input exhausted
-				return next;
-			}
-
-			BufferOrEvent bufferOrEvent = next.get();
-			if (bufferOrEvent.isBuffer()) {
-				return next;
-			}
-			else if (bufferOrEvent.getEvent().getClass() != CheckpointBarrier.class &&
-				bufferOrEvent.getEvent().getClass() != CancelCheckpointMarker.class) {
-				// some other event
-				return next;
-			}
-		}
-	}
-
-	@Override
-	public void cleanup() {
-
-	}
-
-	@Override
-	public boolean isEmpty() {
-		return true;
-	}
-
-	@Override
-	public long getAlignmentDurationNanos() {
-		// this one does not do alignment at all
-		return 0L;
-	}
-
-	@Override
-	public int getNumberOfInputChannels() {
-		return totalNumberOfInputChannels;
-	}
-}
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BufferOrEventSequence.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BufferOrEventSequence.java
index c5bde1b..21649bf 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BufferOrEventSequence.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BufferOrEventSequence.java
@@ -27,7 +27,7 @@ import java.io.IOException;
 
 /**
  * This class represents a sequence of buffers and events which are blocked by
- * {@link CheckpointBarrierHandler}. The sequence of buffers and events can be
+ * {@link CheckpointedInputGate}. The sequence of buffers and events can be
  * read back using the method {@link #getNext()}.
  */
 @Internal
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierAligner.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierAligner.java
index 30e05c1..482ba65 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierAligner.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierAligner.java
@@ -21,8 +21,6 @@ package org.apache.flink.streaming.runtime.io;
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.runtime.checkpoint.CheckpointException;
 import org.apache.flink.runtime.checkpoint.CheckpointFailureReason;
-import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
-import org.apache.flink.runtime.checkpoint.CheckpointMetrics;
 import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker;
 import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
@@ -40,7 +38,7 @@ import java.io.IOException;
  * release blocked channels.
  */
 @Internal
-public class CheckpointBarrierAligner {
+public class CheckpointBarrierAligner extends CheckpointBarrierHandler {
 
 	private static final Logger LOG = LoggerFactory.getLogger(CheckpointBarrierAligner.class);
 
@@ -52,9 +50,6 @@ public class CheckpointBarrierAligner {
 
 	private final String taskName;
 
-	@Nullable
-	private final AbstractInvokable toNotifyOnCheckpoint;
-
 	/** The ID of the checkpoint for which we expect barriers. */
 	private long currentCheckpointId = -1L;
 
@@ -77,13 +72,14 @@ public class CheckpointBarrierAligner {
 			int totalNumberOfInputChannels,
 			String taskName,
 			@Nullable AbstractInvokable toNotifyOnCheckpoint) {
+		super(toNotifyOnCheckpoint);
 		this.totalNumberOfInputChannels = totalNumberOfInputChannels;
 		this.taskName = taskName;
-		this.toNotifyOnCheckpoint = toNotifyOnCheckpoint;
 
 		this.blockedChannels = new boolean[totalNumberOfInputChannels];
 	}
 
+	@Override
 	public void releaseBlocksAndResetBarriers() throws IOException {
 		LOG.debug("{}: End of stream alignment, feeding buffered data back.", taskName);
 
@@ -100,19 +96,12 @@ public class CheckpointBarrierAligner {
 		}
 	}
 
-	/**
-	 * Checks whether the channel with the given index is blocked.
-	 *
-	 * @param channelIndex The channel index to check.
-	 * @return True if the channel is blocked, false if not.
-	 */
+	@Override
 	public boolean isBlocked(int channelIndex) {
 		return blockedChannels[channelIndex];
 	}
 
-	/**
-	 * @return true if some blocked data should be unblocked/rolled over.
-	 */
+	@Override
 	public boolean processBarrier(CheckpointBarrier receivedBarrier, int channelIndex, long bufferedBytes) throws Exception {
 		final long barrierId = receivedBarrier.getId();
 
@@ -121,7 +110,7 @@ public class CheckpointBarrierAligner {
 			if (barrierId > currentCheckpointId) {
 				// new checkpoint
 				currentCheckpointId = barrierId;
-				notifyCheckpoint(receivedBarrier, bufferedBytes);
+				notifyCheckpoint(receivedBarrier, bufferedBytes, latestAlignmentDurationNanos);
 			}
 			return false;
 		}
@@ -185,7 +174,7 @@ public class CheckpointBarrierAligner {
 			}
 
 			releaseBlocksAndResetBarriers();
-			notifyCheckpoint(receivedBarrier, bufferedBytes);
+			notifyCheckpoint(receivedBarrier, bufferedBytes, latestAlignmentDurationNanos);
 			return true;
 		}
 		return checkpointAborted;
@@ -222,9 +211,7 @@ public class CheckpointBarrierAligner {
 		}
 	}
 
-	/**
-	 * @return true if some blocked data should be unblocked/rolled over.
-	 */
+	@Override
 	public boolean processCancellationBarrier(CancelCheckpointMarker cancelBarrier) throws Exception {
 		final long barrierId = cancelBarrier.getCheckpointId();
 
@@ -300,9 +287,7 @@ public class CheckpointBarrierAligner {
 		return false;
 	}
 
-	/**
-	 * @return true if some blocked data should be unblocked/rolled over.
-	 */
+	@Override
 	public boolean processEndOfPartition() throws Exception {
 		numClosedChannels++;
 
@@ -317,37 +302,12 @@ public class CheckpointBarrierAligner {
 		return false;
 	}
 
-	private void notifyCheckpoint(CheckpointBarrier checkpointBarrier, long bufferedBytes) throws Exception {
-		if (toNotifyOnCheckpoint != null) {
-			CheckpointMetaData checkpointMetaData =
-				new CheckpointMetaData(checkpointBarrier.getId(), checkpointBarrier.getTimestamp());
-
-			CheckpointMetrics checkpointMetrics = new CheckpointMetrics()
-				.setBytesBufferedInAlignment(bufferedBytes)
-				.setAlignmentDurationNanos(latestAlignmentDurationNanos);
-
-			toNotifyOnCheckpoint.triggerCheckpointOnBarrier(
-				checkpointMetaData,
-				checkpointBarrier.getCheckpointOptions(),
-				checkpointMetrics);
-		}
-	}
-
-	private void notifyAbortOnCancellationBarrier(long checkpointId) throws Exception {
-		notifyAbort(checkpointId,
-			new CheckpointException(CheckpointFailureReason.CHECKPOINT_DECLINED_ON_CANCELLATION_BARRIER));
-	}
-
-	private void notifyAbort(long checkpointId, CheckpointException cause) throws Exception {
-		if (toNotifyOnCheckpoint != null) {
-			toNotifyOnCheckpoint.abortCheckpointOnBarrier(checkpointId, cause);
-		}
-	}
-
-	public long getCurrentCheckpointId() {
+	@Override
+	public long getLatestCheckpointId() {
 		return currentCheckpointId;
 	}
 
+	@Override
 	public long getAlignmentDurationNanos() {
 		if (startOfAlignmentTimestamp <= 0) {
 			return latestAlignmentDurationNanos;
@@ -365,6 +325,7 @@ public class CheckpointBarrierAligner {
 			numClosedChannels);
 	}
 
+	@Override
 	public void checkpointSizeLimitExceeded(long maxBufferedBytes) throws Exception {
 		releaseBlocksAndResetBarriers();
 		notifyAbort(currentCheckpointId,
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierDiscarder.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierDiscarder.java
new file mode 100644
index 0000000..4c6cdab
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierDiscarder.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.runtime.io;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker;
+import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
+
+import java.io.IOException;
+
+/**
+ * The {@link CheckpointBarrierDiscarder} discards checkpoint barriers have been received from which input channels.
+ */
+@Internal
+public class CheckpointBarrierDiscarder extends CheckpointBarrierHandler {
+	public CheckpointBarrierDiscarder() {
+		super(null);
+	}
+
+	@Override
+	public void releaseBlocksAndResetBarriers() throws IOException {
+	}
+
+	@Override
+	public boolean isBlocked(int channelIndex) {
+		return false;
+	}
+
+	@Override
+	public boolean processBarrier(CheckpointBarrier receivedBarrier, int channelIndex, long bufferedBytes) throws Exception {
+		return false;
+	}
+
+	@Override
+	public boolean processCancellationBarrier(CancelCheckpointMarker cancelBarrier) throws Exception {
+		return false;
+	}
+
+	@Override
+	public boolean processEndOfPartition() throws Exception {
+		return false;
+	}
+
+	@Override
+	public long getLatestCheckpointId() {
+		return 0;
+	}
+
+	@Override
+	public long getAlignmentDurationNanos() {
+		return 0;
+	}
+
+	@Override
+	public void checkpointSizeLimitExceeded(long maxBufferedBytes) throws Exception {
+
+	}
+}
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierHandler.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierHandler.java
index 2ee1a97..41e043e 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierHandler.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierHandler.java
@@ -18,43 +18,88 @@
 
 package org.apache.flink.streaming.runtime.io;
 
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.runtime.io.AsyncDataInput;
-import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
+import org.apache.flink.runtime.checkpoint.CheckpointException;
+import org.apache.flink.runtime.checkpoint.CheckpointFailureReason;
+import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
+import org.apache.flink.runtime.checkpoint.CheckpointMetrics;
+import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker;
+import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+
+import javax.annotation.Nullable;
 
 import java.io.IOException;
 
 /**
- * The CheckpointBarrierHandler reacts to checkpoint barrier arriving from the input channels.
+ * The {@link CheckpointBarrierHandler} reacts to checkpoint barrier arriving from the input channels.
  * Different implementations may either simply track barriers, or block certain inputs on
  * barriers.
  */
-@Internal
-public interface CheckpointBarrierHandler extends AsyncDataInput<BufferOrEvent> {
+public abstract class CheckpointBarrierHandler {
+
+	/** The listener to be notified on complete checkpoints. */
+	@Nullable
+	private final AbstractInvokable toNotifyOnCheckpoint;
+
+	public CheckpointBarrierHandler(@Nullable AbstractInvokable toNotifyOnCheckpoint) {
+		this.toNotifyOnCheckpoint = toNotifyOnCheckpoint;
+	}
+
+	public abstract void releaseBlocksAndResetBarriers() throws IOException;
+
 	/**
-	 * Cleans up all internally held resources.
+	 * Checks whether the channel with the given index is blocked.
 	 *
-	 * @throws IOException Thrown if the cleanup of I/O resources failed.
+	 * @param channelIndex The channel index to check.
+	 * @return True if the channel is blocked, false if not.
 	 */
-	void cleanup() throws IOException;
+	public abstract boolean isBlocked(int channelIndex);
 
 	/**
-	 * Checks if the barrier handler has buffered any data internally.
-	 * @return {@code True}, if no data is buffered internally, {@code false} otherwise.
+	 * @return true if some blocked data should be unblocked/rolled over.
 	 */
-	boolean isEmpty();
+	public abstract boolean processBarrier(CheckpointBarrier receivedBarrier, int channelIndex, long bufferedBytes) throws Exception;
 
 	/**
-	 * Gets the time that the latest alignment took, in nanoseconds.
-	 * If there is currently an alignment in progress, it will return the time spent in the
-	 * current alignment so far.
-	 *
-	 * @return The duration in nanoseconds
+	 * @return true if some blocked data should be unblocked/rolled over.
 	 */
-	long getAlignmentDurationNanos();
+	public abstract boolean processCancellationBarrier(CancelCheckpointMarker cancelBarrier) throws Exception;
 
 	/**
-	 * @return number of underlying input channels.
+	 * @return true if some blocked data should be unblocked/rolled over.
 	 */
-	int getNumberOfInputChannels();
+	public abstract boolean processEndOfPartition() throws Exception;
+
+	public abstract long getLatestCheckpointId();
+
+	public abstract long getAlignmentDurationNanos();
+
+	public abstract void checkpointSizeLimitExceeded(long maxBufferedBytes) throws Exception;
+
+	protected void notifyCheckpoint(CheckpointBarrier checkpointBarrier, long bufferedBytes, long alignmentDurationNanos) throws Exception {
+		if (toNotifyOnCheckpoint != null) {
+			CheckpointMetaData checkpointMetaData =
+				new CheckpointMetaData(checkpointBarrier.getId(), checkpointBarrier.getTimestamp());
+
+			CheckpointMetrics checkpointMetrics = new CheckpointMetrics()
+				.setBytesBufferedInAlignment(bufferedBytes)
+				.setAlignmentDurationNanos(alignmentDurationNanos);
+
+			toNotifyOnCheckpoint.triggerCheckpointOnBarrier(
+				checkpointMetaData,
+				checkpointBarrier.getCheckpointOptions(),
+				checkpointMetrics);
+		}
+	}
+
+	protected void notifyAbortOnCancellationBarrier(long checkpointId) throws Exception {
+		notifyAbort(checkpointId,
+			new CheckpointException(CheckpointFailureReason.CHECKPOINT_DECLINED_ON_CANCELLATION_BARRIER));
+	}
+
+	protected void notifyAbort(long checkpointId, CheckpointException cause) throws Exception {
+		if (toNotifyOnCheckpoint != null) {
+			toNotifyOnCheckpoint.abortCheckpointOnBarrier(checkpointId, cause);
+		}
+	}
 }
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierTracker.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierTracker.java
similarity index 61%
rename from flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierTracker.java
rename to flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierTracker.java
index f7629bb..0ec9004 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierTracker.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierTracker.java
@@ -21,13 +21,8 @@ package org.apache.flink.streaming.runtime.io;
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.runtime.checkpoint.CheckpointException;
 import org.apache.flink.runtime.checkpoint.CheckpointFailureReason;
-import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
-import org.apache.flink.runtime.checkpoint.CheckpointMetrics;
-import org.apache.flink.runtime.checkpoint.CheckpointOptions;
 import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker;
 import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
-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.jobgraph.tasks.AbstractInvokable;
 
 import org.slf4j.Logger;
@@ -36,24 +31,22 @@ import org.slf4j.LoggerFactory;
 import javax.annotation.Nullable;
 
 import java.util.ArrayDeque;
-import java.util.Optional;
-import java.util.concurrent.CompletableFuture;
 
 /**
- * The BarrierTracker keeps track of what checkpoint barriers have been received from
+ * The {@link CheckpointBarrierTracker} keeps track of what checkpoint barriers have been received from
  * which input channels. Once it has observed all checkpoint barriers for a checkpoint ID,
  * it notifies its listener of a completed checkpoint.
  *
- * <p>Unlike the {@link BarrierBuffer}, the BarrierTracker does not block the input
+ * <p>Unlike the {@link CheckpointBarrierAligner}, the BarrierTracker does not block the input
  * channels that have sent barriers, so it cannot be used to gain "exactly-once" processing
  * guarantees. It can, however, be used to gain "at least once" processing guarantees.
  *
  * <p>NOTE: This implementation strictly assumes that newer checkpoints have higher checkpoint IDs.
  */
 @Internal
-public class BarrierTracker implements CheckpointBarrierHandler {
+public class CheckpointBarrierTracker extends CheckpointBarrierHandler {
 
-	private static final Logger LOG = LoggerFactory.getLogger(BarrierTracker.class);
+	private static final Logger LOG = LoggerFactory.getLogger(CheckpointBarrierTracker.class);
 
 	/**
 	 * The tracker tracks a maximum number of checkpoints, for which some, but not all barriers
@@ -63,9 +56,6 @@ public class BarrierTracker implements CheckpointBarrierHandler {
 
 	// ------------------------------------------------------------------------
 
-	/** The input gate, to draw the buffers and events from. */
-	private final InputGate inputGate;
-
 	/**
 	 * The number of channels. Once that many barriers have been received for a checkpoint, the
 	 * checkpoint is considered complete.
@@ -78,89 +68,36 @@ public class BarrierTracker implements CheckpointBarrierHandler {
 	 */
 	private final ArrayDeque<CheckpointBarrierCount> pendingCheckpoints;
 
-	/** The listener to be notified on complete checkpoints. */
-	private final AbstractInvokable toNotifyOnCheckpoint;
-
 	/** The highest checkpoint ID encountered so far. */
 	private long latestPendingCheckpointID = -1;
 
-	// ------------------------------------------------------------------------
-
-	public BarrierTracker(InputGate inputGate) {
-		this(inputGate, null);
+	public CheckpointBarrierTracker(int totalNumberOfInputChannels) {
+		this(totalNumberOfInputChannels, null);
 	}
 
-	public BarrierTracker(InputGate inputGate, @Nullable AbstractInvokable toNotifyOnCheckpoint) {
-		this.inputGate = inputGate;
-		this.totalNumberOfInputChannels = inputGate.getNumberOfInputChannels();
+	public CheckpointBarrierTracker(int totalNumberOfInputChannels, @Nullable AbstractInvokable toNotifyOnCheckpoint) {
+		super(toNotifyOnCheckpoint);
+		this.totalNumberOfInputChannels = totalNumberOfInputChannels;
 		this.pendingCheckpoints = new ArrayDeque<>();
-		this.toNotifyOnCheckpoint = toNotifyOnCheckpoint;
 	}
 
 	@Override
-	public CompletableFuture<?> isAvailable() {
-		return inputGate.isAvailable();
+	public void releaseBlocksAndResetBarriers() {
 	}
 
 	@Override
-	public boolean isFinished() {
-		return inputGate.isFinished();
+	public boolean isBlocked(int channelIndex) {
+		return false;
 	}
 
 	@Override
-	public Optional<BufferOrEvent> pollNext() throws Exception {
-		while (true) {
-			Optional<BufferOrEvent> next = inputGate.pollNext();
-			if (!next.isPresent()) {
-				// buffer or input exhausted
-				return next;
-			}
-
-			BufferOrEvent bufferOrEvent = next.get();
-			if (bufferOrEvent.isBuffer()) {
-				return next;
-			}
-			else if (bufferOrEvent.getEvent().getClass() == CheckpointBarrier.class) {
-				processBarrier((CheckpointBarrier) bufferOrEvent.getEvent(), bufferOrEvent.getChannelIndex());
-			}
-			else if (bufferOrEvent.getEvent().getClass() == CancelCheckpointMarker.class) {
-				processCheckpointAbortBarrier((CancelCheckpointMarker) bufferOrEvent.getEvent(), bufferOrEvent.getChannelIndex());
-			}
-			else {
-				// some other event
-				return next;
-			}
-		}
-	}
-
-	@Override
-	public void cleanup() {
-		pendingCheckpoints.clear();
-	}
-
-	@Override
-	public boolean isEmpty() {
-		return pendingCheckpoints.isEmpty();
-	}
-
-	@Override
-	public long getAlignmentDurationNanos() {
-		// this one does not do alignment at all
-		return 0L;
-	}
-
-	@Override
-	public int getNumberOfInputChannels() {
-		return totalNumberOfInputChannels;
-	}
-
-	private void processBarrier(CheckpointBarrier receivedBarrier, int channelIndex) throws Exception {
+	public boolean processBarrier(CheckpointBarrier receivedBarrier, int channelIndex, long bufferedBytes) throws Exception {
 		final long barrierId = receivedBarrier.getId();
 
 		// fast path for single channel trackers
 		if (totalNumberOfInputChannels == 1) {
-			notifyCheckpoint(barrierId, receivedBarrier.getTimestamp(), receivedBarrier.getCheckpointOptions());
-			return;
+			notifyCheckpoint(receivedBarrier, 0, 0);
+			return false;
 		}
 
 		// general path for multiple input channels
@@ -169,20 +106,20 @@ public class BarrierTracker implements CheckpointBarrierHandler {
 		}
 
 		// find the checkpoint barrier in the queue of pending barriers
-		CheckpointBarrierCount cbc = null;
+		CheckpointBarrierCount barrierCount = null;
 		int pos = 0;
 
 		for (CheckpointBarrierCount next : pendingCheckpoints) {
 			if (next.checkpointId == barrierId) {
-				cbc = next;
+				barrierCount = next;
 				break;
 			}
 			pos++;
 		}
 
-		if (cbc != null) {
+		if (barrierCount != null) {
 			// add one to the count to that barrier and check for completion
-			int numBarriersNew = cbc.incrementBarrierCount();
+			int numBarriersNew = barrierCount.incrementBarrierCount();
 			if (numBarriersNew == totalNumberOfInputChannels) {
 				// checkpoint can be triggered (or is aborted and all barriers have been seen)
 				// first, remove this checkpoint and all all prior pending
@@ -192,12 +129,12 @@ public class BarrierTracker implements CheckpointBarrierHandler {
 				}
 
 				// notify the listener
-				if (!cbc.isAborted()) {
+				if (!barrierCount.isAborted()) {
 					if (LOG.isDebugEnabled()) {
 						LOG.debug("Received all barriers for checkpoint {}", barrierId);
 					}
 
-					notifyCheckpoint(receivedBarrier.getId(), receivedBarrier.getTimestamp(), receivedBarrier.getCheckpointOptions());
+					notifyCheckpoint(receivedBarrier, 0, 0);
 				}
 			}
 		}
@@ -216,19 +153,21 @@ public class BarrierTracker implements CheckpointBarrierHandler {
 				}
 			}
 		}
+		return false;
 	}
 
-	private void processCheckpointAbortBarrier(CancelCheckpointMarker barrier, int channelIndex) throws Exception {
-		final long checkpointId = barrier.getCheckpointId();
+	@Override
+	public boolean processCancellationBarrier(CancelCheckpointMarker cancelBarrier) throws Exception {
+		final long checkpointId = cancelBarrier.getCheckpointId();
 
 		if (LOG.isDebugEnabled()) {
-			LOG.debug("Received cancellation barrier for checkpoint {} from channel {}", checkpointId, channelIndex);
+			LOG.debug("Received cancellation barrier for checkpoint {}", checkpointId);
 		}
 
 		// fast path for single channel trackers
 		if (totalNumberOfInputChannels == 1) {
-			notifyAbort(checkpointId);
-			return;
+			notifyAbortOnCancellationBarrier(checkpointId);
+			return false;
 		}
 
 		// -- general path for multiple input channels --
@@ -241,7 +180,7 @@ public class BarrierTracker implements CheckpointBarrierHandler {
 
 			if (cbc.markAborted()) {
 				// abort the subsumed checkpoints if not already done
-				notifyAbort(cbc.checkpointId());
+				notifyAbortOnCancellationBarrier(cbc.checkpointId());
 			}
 		}
 
@@ -249,7 +188,7 @@ public class BarrierTracker implements CheckpointBarrierHandler {
 			// make sure the checkpoint is remembered as aborted
 			if (cbc.markAborted()) {
 				// this was the first time the checkpoint was aborted - notify
-				notifyAbort(checkpointId);
+				notifyAbortOnCancellationBarrier(checkpointId);
 			}
 
 			// we still count the barriers to be able to remove the entry once all barriers have been seen
@@ -259,7 +198,7 @@ public class BarrierTracker implements CheckpointBarrierHandler {
 			}
 		}
 		else if (checkpointId > latestPendingCheckpointID) {
-			notifyAbort(checkpointId);
+			notifyAbortOnCancellationBarrier(checkpointId);
 
 			latestPendingCheckpointID = checkpointId;
 
@@ -272,28 +211,33 @@ public class BarrierTracker implements CheckpointBarrierHandler {
 		} else {
 			// trailing cancellation barrier which was already cancelled
 		}
+		return false;
 	}
 
-	private void notifyCheckpoint(long checkpointId, long timestamp, CheckpointOptions checkpointOptions) throws Exception {
-		if (toNotifyOnCheckpoint != null) {
-			CheckpointMetaData checkpointMetaData = new CheckpointMetaData(checkpointId, timestamp);
-			CheckpointMetrics checkpointMetrics = new CheckpointMetrics()
-				.setBytesBufferedInAlignment(0L)
-				.setAlignmentDurationNanos(0L);
-
-			toNotifyOnCheckpoint.triggerCheckpointOnBarrier(checkpointMetaData, checkpointOptions, checkpointMetrics);
+	@Override
+	public boolean processEndOfPartition() throws Exception {
+		while (!pendingCheckpoints.isEmpty()) {
+			CheckpointBarrierCount barrierCount = pendingCheckpoints.removeFirst();
+			if (barrierCount.markAborted()) {
+				notifyAbort(barrierCount.checkpointId(),
+					new CheckpointException(CheckpointFailureReason.CHECKPOINT_DECLINED_INPUT_END_OF_STREAM));
+			}
 		}
+		return false;
 	}
 
-	private void notifyAbort(long checkpointId) throws Exception {
-		if (toNotifyOnCheckpoint != null) {
-			toNotifyOnCheckpoint.abortCheckpointOnBarrier(
-				checkpointId,
-				new CheckpointException(CheckpointFailureReason.CHECKPOINT_DECLINED_ON_CANCELLATION_BARRIER));
-		}
+	public long getLatestCheckpointId() {
+		return pendingCheckpoints.isEmpty() ? -1 : pendingCheckpoints.peekLast().checkpointId();
 	}
 
-	// ------------------------------------------------------------------------
+	public long getAlignmentDurationNanos() {
+		return 0;
+	}
+
+	@Override
+	public void checkpointSizeLimitExceeded(long maxBufferedBytes) throws Exception {
+		throw new UnsupportedOperationException("This should never happened as this class doesn't block any data");
+	}
 
 	/**
 	 * Simple class for a checkpoint ID with a barrier counter.
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierHandler.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java
similarity index 84%
copy from flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierHandler.java
copy to flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java
index 2ee1a97..cdbbfbc 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierHandler.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java
@@ -25,12 +25,11 @@ import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
 import java.io.IOException;
 
 /**
- * The CheckpointBarrierHandler reacts to checkpoint barrier arriving from the input channels.
- * Different implementations may either simply track barriers, or block certain inputs on
- * barriers.
+ * The {@link CheckpointedInputGate} uses {@link CheckpointBarrierHandler} to handle incoming
+ * {@link org.apache.flink.runtime.io.network.api.CheckpointBarrier} from the {@link org.apache.flink.runtime.io.network.partition.consumer.InputGate}.
  */
 @Internal
-public interface CheckpointBarrierHandler extends AsyncDataInput<BufferOrEvent> {
+public interface CheckpointedInputGate extends AsyncDataInput<BufferOrEvent> {
 	/**
 	 * Cleans up all internally held resources.
 	 *
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/EmptyBufferStorage.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/EmptyBufferStorage.java
new file mode 100644
index 0000000..5535c49
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/EmptyBufferStorage.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.runtime.io;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
+
+import java.io.IOException;
+import java.util.Optional;
+
+/**
+ * Always empty implementation of {@link BufferStorage}. It doesn't allow for adding any data.
+ */
+@Internal
+public class EmptyBufferStorage implements BufferStorage {
+	@Override
+	public void add(BufferOrEvent boe) throws IOException {
+		throw new UnsupportedOperationException("Adding to EmptyBufferStorage is unsupported");
+	}
+
+	@Override
+	public boolean isFull() {
+		return false;
+	}
+
+	@Override
+	public void rollOver() throws IOException {
+	}
+
+	@Override
+	public long getPendingBytes() {
+		return 0;
+	}
+
+	@Override
+	public long getRolledBytes() {
+		return 0;
+	}
+
+	@Override
+	public boolean isEmpty() {
+		return true;
+	}
+
+	@Override
+	public Optional<BufferOrEvent> pollNext() throws IOException {
+		return Optional.empty();
+	}
+
+	@Override
+	public long getMaxBufferedBytes() {
+		return -1;
+	}
+
+	@Override
+	public void close() throws IOException {
+	}
+}
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java
index c9ec6bf..75926b9 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java
@@ -30,13 +30,13 @@ import org.apache.flink.streaming.runtime.tasks.StreamTask;
 import java.io.IOException;
 
 /**
- * Utility for creating {@link CheckpointBarrierHandler} based on checkpoint mode
+ * Utility for creating {@link CheckpointedInputGate} based on checkpoint mode
  * for {@link StreamInputProcessor} and {@link StreamTwoInputProcessor}.
  */
 @Internal
 public class InputProcessorUtil {
 
-	public static CheckpointBarrierHandler createCheckpointBarrierHandler(
+	public static CheckpointedInputGate createCheckpointBarrierHandler(
 			StreamTask<?, ?> checkpointedTask,
 			CheckpointingMode checkpointMode,
 			IOManager ioManager,
@@ -44,7 +44,7 @@ public class InputProcessorUtil {
 			Configuration taskManagerConfig,
 			String taskName) throws IOException {
 
-		CheckpointBarrierHandler barrierHandler;
+		CheckpointedInputGate barrierHandler;
 		if (checkpointMode == CheckpointingMode.EXACTLY_ONCE) {
 			long maxAlign = taskManagerConfig.getLong(TaskManagerOptions.TASK_CHECKPOINT_ALIGNMENT_BYTES_LIMIT);
 			if (!(maxAlign == -1 || maxAlign > 0)) {
@@ -67,7 +67,10 @@ public class InputProcessorUtil {
 					checkpointedTask);
 			}
 		} else if (checkpointMode == CheckpointingMode.AT_LEAST_ONCE) {
-			barrierHandler = new BarrierTracker(inputGate);
+			barrierHandler = new BarrierBuffer(
+				inputGate,
+				new EmptyBufferStorage(),
+				new CheckpointBarrierTracker(inputGate.getNumberOfInputChannels(), checkpointedTask));
 		} else {
 			throw new IllegalArgumentException("Unrecognized Checkpointing Mode: " + checkpointMode);
 		}
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
index 742dbe8..58b2051 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
@@ -104,7 +104,7 @@ public class StreamInputProcessor<IN> {
 
 		InputGate inputGate = InputGateUtil.createInputGate(inputGates);
 
-		CheckpointBarrierHandler barrierHandler = InputProcessorUtil.createCheckpointBarrierHandler(
+		CheckpointedInputGate barrierHandler = InputProcessorUtil.createCheckpointBarrierHandler(
 			checkpointedTask,
 			checkpointMode,
 			ioManager,
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java
index 85e7f46..ecf88e2 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java
@@ -42,12 +42,12 @@ import java.util.concurrent.CompletableFuture;
 import static org.apache.flink.util.Preconditions.checkState;
 
 /**
- * Implementation of {@link StreamTaskInput} that wraps an input from network taken from {@link CheckpointBarrierHandler}.
+ * Implementation of {@link StreamTaskInput} that wraps an input from network taken from {@link CheckpointedInputGate}.
  */
 @Internal
 public final class StreamTaskNetworkInput implements StreamTaskInput {
 
-	private final CheckpointBarrierHandler barrierHandler;
+	private final CheckpointedInputGate barrierHandler;
 
 	private final DeserializationDelegate<StreamElement> deserializationDelegate;
 
@@ -63,7 +63,7 @@ public final class StreamTaskNetworkInput implements StreamTaskInput {
 
 	@SuppressWarnings("unchecked")
 	public StreamTaskNetworkInput(
-			CheckpointBarrierHandler barrierHandler,
+			CheckpointedInputGate barrierHandler,
 			TypeSerializer<?> inputSerializer,
 			IOManager ioManager,
 			int inputIndex) {
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java
index 4efbc7f..aa6354d 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java
@@ -86,7 +86,7 @@ public class StreamTwoInputProcessor<IN1, IN2> {
 	private final DeserializationDelegate<StreamElement> deserializationDelegate1;
 	private final DeserializationDelegate<StreamElement> deserializationDelegate2;
 
-	private final CheckpointBarrierHandler barrierHandler;
+	private final CheckpointedInputGate barrierHandler;
 
 	private final Object lock;
 
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputSelectableProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputSelectableProcessor.java
index 54ce749..d5ebf29 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputSelectableProcessor.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputSelectableProcessor.java
@@ -120,8 +120,9 @@ public class StreamTwoInputSelectableProcessor<IN1, IN2> {
 		InputGate unionedInputGate2 = InputGateUtil.createInputGate(inputGates2.toArray(new InputGate[0]));
 
 		// create a Input instance for each input
-		this.input1 = new StreamTaskNetworkInput(new BarrierDiscarder(unionedInputGate1), inputSerializer1, ioManager, 0);
-		this.input2 = new StreamTaskNetworkInput(new BarrierDiscarder(unionedInputGate2), inputSerializer2, ioManager, 1);
+		CachedBufferStorage bufferStorage = new CachedBufferStorage(unionedInputGate1.getPageSize());
+		this.input1 = new StreamTaskNetworkInput(new BarrierBuffer(unionedInputGate1, bufferStorage, new CheckpointBarrierDiscarder()), inputSerializer1, ioManager, 0);
+		this.input2 = new StreamTaskNetworkInput(new BarrierBuffer(unionedInputGate2, bufferStorage, new CheckpointBarrierDiscarder()), inputSerializer2, ioManager, 1);
 
 		this.statusWatermarkValve1 = new StatusWatermarkValve(
 			unionedInputGate1.getNumberOfInputChannels(),
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTestBase.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTestBase.java
index e6e48a8..13c4aad 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTestBase.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTestBase.java
@@ -483,7 +483,7 @@ public abstract class BarrierBufferTestBase {
 		// align checkpoint 1
 		startTs = System.nanoTime();
 		check(sequence[7], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(1L, buffer.getCurrentCheckpointId());
+		assertEquals(1L, buffer.getLatestCheckpointId());
 
 		// checkpoint done - replay buffered
 		check(sequence[5], buffer.pollNext().get(), PAGE_SIZE);
@@ -501,7 +501,7 @@ public abstract class BarrierBufferTestBase {
 
 		// checkpoint 2 aborted, checkpoint 3 started
 		check(sequence[12], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(3L, buffer.getCurrentCheckpointId());
+		assertEquals(3L, buffer.getLatestCheckpointId());
 		validateAlignmentTime(startTs, buffer.getAlignmentDurationNanos());
 		verify(toNotify).abortCheckpointOnBarrier(eq(2L),
 			argThat(new CheckpointExceptionMatcher(CheckpointFailureReason.CHECKPOINT_DECLINED_SUBSUMED)));
@@ -565,7 +565,7 @@ public abstract class BarrierBufferTestBase {
 		check(sequence[1], buffer.pollNext().get(), PAGE_SIZE);
 		check(sequence[2], buffer.pollNext().get(), PAGE_SIZE);
 		check(sequence[7], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(1L, buffer.getCurrentCheckpointId());
+		assertEquals(1L, buffer.getLatestCheckpointId());
 
 		check(sequence[5], buffer.pollNext().get(), PAGE_SIZE);
 		check(sequence[6], buffer.pollNext().get(), PAGE_SIZE);
@@ -574,7 +574,7 @@ public abstract class BarrierBufferTestBase {
 
 		// alignment of checkpoint 2
 		check(sequence[13], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(2L, buffer.getCurrentCheckpointId());
+		assertEquals(2L, buffer.getLatestCheckpointId());
 		check(sequence[15], buffer.pollNext().get(), PAGE_SIZE);
 		check(sequence[19], buffer.pollNext().get(), PAGE_SIZE);
 		check(sequence[21], buffer.pollNext().get(), PAGE_SIZE);
@@ -583,7 +583,7 @@ public abstract class BarrierBufferTestBase {
 
 		// checkpoint 2 aborted, checkpoint 4 started. replay buffered
 		check(sequence[12], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(4L, buffer.getCurrentCheckpointId());
+		assertEquals(4L, buffer.getLatestCheckpointId());
 		check(sequence[16], buffer.pollNext().get(), PAGE_SIZE);
 		check(sequence[18], buffer.pollNext().get(), PAGE_SIZE);
 		check(sequence[22], buffer.pollNext().get(), PAGE_SIZE);
@@ -651,7 +651,7 @@ public abstract class BarrierBufferTestBase {
 		check(sequence[1], buffer.pollNext().get(), PAGE_SIZE);
 		check(sequence[2], buffer.pollNext().get(), PAGE_SIZE);
 		check(sequence[7], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(1L, buffer.getCurrentCheckpointId());
+		assertEquals(1L, buffer.getLatestCheckpointId());
 		check(sequence[5], buffer.pollNext().get(), PAGE_SIZE);
 		check(sequence[6], buffer.pollNext().get(), PAGE_SIZE);
 		check(sequence[9], buffer.pollNext().get(), PAGE_SIZE);
@@ -660,7 +660,7 @@ public abstract class BarrierBufferTestBase {
 		// alignment of checkpoint 2
 		check(sequence[13], buffer.pollNext().get(), PAGE_SIZE);
 		check(sequence[22], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(2L, buffer.getCurrentCheckpointId());
+		assertEquals(2L, buffer.getLatestCheckpointId());
 
 		// checkpoint 2 completed
 		check(sequence[12], buffer.pollNext().get(), PAGE_SIZE);
@@ -669,7 +669,7 @@ public abstract class BarrierBufferTestBase {
 
 		// checkpoint 3 skipped, alignment for 4 started
 		check(sequence[18], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(4L, buffer.getCurrentCheckpointId());
+		assertEquals(4L, buffer.getLatestCheckpointId());
 		check(sequence[21], buffer.pollNext().get(), PAGE_SIZE);
 		check(sequence[24], buffer.pollNext().get(), PAGE_SIZE);
 		check(sequence[26], buffer.pollNext().get(), PAGE_SIZE);
@@ -790,24 +790,24 @@ public abstract class BarrierBufferTestBase {
 
 		// checkpoint 3 alignment
 		check(sequence[7], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(2L, buffer.getCurrentCheckpointId());
+		assertEquals(2L, buffer.getLatestCheckpointId());
 		check(sequence[8], buffer.pollNext().get(), PAGE_SIZE);
 		check(sequence[11], buffer.pollNext().get(), PAGE_SIZE);
 
 		// checkpoint 3 buffered
 		check(sequence[10], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(3L, buffer.getCurrentCheckpointId());
+		assertEquals(3L, buffer.getLatestCheckpointId());
 
 		// after checkpoint 4
 		check(sequence[15], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(4L, buffer.getCurrentCheckpointId());
+		assertEquals(4L, buffer.getLatestCheckpointId());
 		check(sequence[16], buffer.pollNext().get(), PAGE_SIZE);
 		check(sequence[17], buffer.pollNext().get(), PAGE_SIZE);
 		check(sequence[18], buffer.pollNext().get(), PAGE_SIZE);
 
 		check(sequence[19], buffer.pollNext().get(), PAGE_SIZE);
 		check(sequence[21], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(5L, buffer.getCurrentCheckpointId());
+		assertEquals(5L, buffer.getLatestCheckpointId());
 		check(sequence[22], buffer.pollNext().get(), PAGE_SIZE);
 	}
 
@@ -837,11 +837,11 @@ public abstract class BarrierBufferTestBase {
 		check(sequence[3], buffer.pollNext().get(), PAGE_SIZE);
 		check(sequence[4], buffer.pollNext().get(), PAGE_SIZE);
 		check(sequence[5], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(1L, buffer.getCurrentCheckpointId());
+		assertEquals(1L, buffer.getLatestCheckpointId());
 
 		// alignment of second checkpoint
 		check(sequence[10], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(2L, buffer.getCurrentCheckpointId());
+		assertEquals(2L, buffer.getLatestCheckpointId());
 
 		// first end-of-partition encountered: checkpoint will not be completed
 		check(sequence[12], buffer.pollNext().get(), PAGE_SIZE);
@@ -874,7 +874,7 @@ public abstract class BarrierBufferTestBase {
 		assertEquals(0L, buffer.getAlignmentDurationNanos());
 
 		check(sequence[6], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(5L, buffer.getCurrentCheckpointId());
+		assertEquals(5L, buffer.getLatestCheckpointId());
 		verify(toNotify, times(1)).triggerCheckpointOnBarrier(argThat(new CheckpointMatcher(2L)), any(CheckpointOptions.class), any(CheckpointMetrics.class));
 		verify(toNotify, times(1)).abortCheckpointOnBarrier(eq(4L),
 			argThat(new CheckpointExceptionMatcher(CheckpointFailureReason.CHECKPOINT_DECLINED_ON_CANCELLATION_BARRIER)));
@@ -882,7 +882,7 @@ public abstract class BarrierBufferTestBase {
 		assertEquals(0L, buffer.getAlignmentDurationNanos());
 
 		check(sequence[8], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(6L, buffer.getCurrentCheckpointId());
+		assertEquals(6L, buffer.getLatestCheckpointId());
 		verify(toNotify, times(1)).abortCheckpointOnBarrier(eq(6L),
 			argThat(new CheckpointExceptionMatcher(CheckpointFailureReason.CHECKPOINT_DECLINED_ON_CANCELLATION_BARRIER)));
 		assertEquals(0L, buffer.getAlignmentDurationNanos());
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java
index a8e7727..5112f63 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java
@@ -46,7 +46,7 @@ public class BarrierTrackerTest {
 
 	private static final int PAGE_SIZE = 512;
 
-	private BarrierTracker tracker;
+	private CheckpointedInputGate tracker;
 
 	@After
 	public void ensureEmpty() throws Exception {
@@ -365,16 +365,19 @@ public class BarrierTrackerTest {
 	// ------------------------------------------------------------------------
 	//  Utils
 	// ------------------------------------------------------------------------
-	private static BarrierTracker createBarrierTracker(int numberOfChannels, BufferOrEvent[] sequence) {
+	private static CheckpointedInputGate createBarrierTracker(int numberOfChannels, BufferOrEvent[] sequence) {
 		return createBarrierTracker(numberOfChannels, sequence, null);
 	}
 
-	private static BarrierTracker createBarrierTracker(
+	private static CheckpointedInputGate createBarrierTracker(
 			int numberOfChannels,
 			BufferOrEvent[] sequence,
 			@Nullable AbstractInvokable toNotifyOnCheckpoint) {
 		MockInputGate gate = new MockInputGate(PAGE_SIZE, numberOfChannels, Arrays.asList(sequence));
-		return new BarrierTracker(gate, toNotifyOnCheckpoint);
+		return new BarrierBuffer(
+			gate,
+			new CachedBufferStorage(PAGE_SIZE, -1, "Testing"),
+			new CheckpointBarrierTracker(gate.getNumberOfInputChannels(), toNotifyOnCheckpoint));
 	}
 
 	private static BufferOrEvent createBarrier(long id, int channel) {


[flink] 05/16: [hotfix][network] Do not abort the same checkpoint barrier twice when cancellation marker was lost

Posted by pn...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pnowojski pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 62927c90c5a8aa79dfc68b6bb4f05f4aeebcd2b4
Author: Piotr Nowojski <pi...@gmail.com>
AuthorDate: Sat Jun 15 10:50:24 2019 +0200

    [hotfix][network] Do not abort the same checkpoint barrier twice when cancellation marker was lost
---
 .../flink/streaming/runtime/io/BarrierBuffer.java  | 12 +--
 .../runtime/io/BarrierBufferTestBase.java          | 19 +++++
 .../streaming/runtime/io/BarrierTrackerTest.java   | 55 -------------
 .../runtime/io/CheckpointSequenceValidator.java    | 90 ++++++++++++++++++++++
 4 files changed, 112 insertions(+), 64 deletions(-)

diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
index 23717f5..0f8fa40 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
@@ -330,12 +330,6 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 				startOfAlignmentTimestamp = 0L;
 				latestAlignmentDurationNanos = 0L;
 
-				notifyAbort(currentCheckpointId,
-					new CheckpointException(
-						"Barrier id: " + barrierId,
-						CheckpointFailureReason.CHECKPOINT_DECLINED_SUBSUMED
-					));
-
 				notifyAbortOnCancellationBarrier(barrierId);
 			}
 
@@ -380,11 +374,11 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 	private void notifyCheckpoint(CheckpointBarrier checkpointBarrier) throws Exception {
 		if (toNotifyOnCheckpoint != null) {
 			CheckpointMetaData checkpointMetaData =
-					new CheckpointMetaData(checkpointBarrier.getId(), checkpointBarrier.getTimestamp());
+				new CheckpointMetaData(checkpointBarrier.getId(), checkpointBarrier.getTimestamp());
 
 			CheckpointMetrics checkpointMetrics = new CheckpointMetrics()
-					.setBytesBufferedInAlignment(bufferStorage.currentBufferedSize())
-					.setAlignmentDurationNanos(latestAlignmentDurationNanos);
+				.setBytesBufferedInAlignment(bufferStorage.currentBufferedSize())
+				.setAlignmentDurationNanos(latestAlignmentDurationNanos);
 
 			toNotifyOnCheckpoint.triggerCheckpointOnBarrier(
 				checkpointMetaData,
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTestBase.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTestBase.java
index 3b4f65f..e6e48a8 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTestBase.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTestBase.java
@@ -689,6 +689,25 @@ public abstract class BarrierBufferTestBase {
 	}
 
 	@Test
+	public void testMissingCancellationBarriers() throws Exception {
+		BufferOrEvent[] sequence = {
+			createBarrier(1L, 0),
+			createCancellationBarrier(2L, 0),
+			createCancellationBarrier(3L, 0),
+			createCancellationBarrier(3L, 1),
+			createBuffer(0)
+		};
+		AbstractInvokable validator = new CheckpointSequenceValidator(-3);
+		buffer = createBarrierBuffer(2, sequence, validator);
+
+		for (BufferOrEvent boe : sequence) {
+			if (boe.isBuffer() || (boe.getEvent().getClass() != CheckpointBarrier.class && boe.getEvent().getClass() != CancelCheckpointMarker.class)) {
+				assertEquals(boe, buffer.pollNext().get());
+			}
+		}
+	}
+
+	@Test
 	public void testEarlyCleanup() throws Exception {
 		BufferOrEvent[] sequence = {
 			createBuffer(0), createBuffer(1), createBuffer(2),
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java
index cb58837..1be2aab 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java
@@ -19,8 +19,6 @@
 package org.apache.flink.streaming.runtime.io;
 
 import org.apache.flink.core.memory.MemorySegmentFactory;
-import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
-import org.apache.flink.runtime.checkpoint.CheckpointMetrics;
 import org.apache.flink.runtime.checkpoint.CheckpointOptions;
 import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker;
 import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
@@ -28,7 +26,6 @@ import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
 import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
 import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
 
 import org.junit.After;
 import org.junit.Test;
@@ -40,7 +37,6 @@ import java.util.Arrays;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.mock;
@@ -404,55 +400,4 @@ public class BarrierTrackerTest {
 	//  Testing Mocks
 	// ------------------------------------------------------------------------
 
-	private static class CheckpointSequenceValidator extends AbstractInvokable {
-
-		private final long[] checkpointIDs;
-
-		private int i = 0;
-
-		private CheckpointSequenceValidator(long... checkpointIDs) {
-			super(new DummyEnvironment("test", 1, 0));
-			this.checkpointIDs = checkpointIDs;
-		}
-
-		@Override
-		public void invoke() {
-			throw new UnsupportedOperationException("should never be called");
-		}
-
-		@Override
-		public boolean triggerCheckpoint(CheckpointMetaData checkpointMetaData, CheckpointOptions checkpointOptions, boolean advanceToEndOfEventTime) throws Exception {
-			throw new UnsupportedOperationException("should never be called");
-		}
-
-		@Override
-		public void triggerCheckpointOnBarrier(CheckpointMetaData checkpointMetaData, CheckpointOptions checkpointOptions, CheckpointMetrics checkpointMetrics) throws Exception {
-			assertTrue("More checkpoints than expected", i < checkpointIDs.length);
-
-			final long expectedId = checkpointIDs[i++];
-			if (expectedId >= 0) {
-				assertEquals("wrong checkpoint id", expectedId, checkpointMetaData.getCheckpointId());
-				assertTrue(checkpointMetaData.getTimestamp() > 0);
-			} else {
-				fail("got 'triggerCheckpointOnBarrier()' when expecting an 'abortCheckpointOnBarrier()'");
-			}
-		}
-
-		@Override
-		public void abortCheckpointOnBarrier(long checkpointId, Throwable cause) {
-			assertTrue("More checkpoints than expected", i < checkpointIDs.length);
-
-			final long expectedId = checkpointIDs[i++];
-			if (expectedId < 0) {
-				assertEquals("wrong checkpoint id for checkpoint abort", -expectedId, checkpointId);
-			} else {
-				fail("got 'abortCheckpointOnBarrier()' when expecting an 'triggerCheckpointOnBarrier()'");
-			}
-		}
-
-		@Override
-		public void notifyCheckpointComplete(long checkpointId) throws Exception {
-			throw new UnsupportedOperationException("should never be called");
-		}
-	}
 }
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CheckpointSequenceValidator.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CheckpointSequenceValidator.java
new file mode 100644
index 0000000..83b5364
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CheckpointSequenceValidator.java
@@ -0,0 +1,90 @@
+/*
+ * 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.streaming.runtime.io;
+
+import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
+import org.apache.flink.runtime.checkpoint.CheckpointMetrics;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * {@link AbstractInvokable} that validates expected order of completed and aborted checkpoints.
+ */
+class CheckpointSequenceValidator extends AbstractInvokable {
+
+	private final long[] checkpointIDs;
+
+	private int i = 0;
+
+	CheckpointSequenceValidator(long... checkpointIDs) {
+		super(new DummyEnvironment("test", 1, 0));
+		this.checkpointIDs = checkpointIDs;
+	}
+
+	@Override
+	public void invoke() {
+		throw new UnsupportedOperationException("should never be called");
+	}
+
+	@Override
+	public boolean triggerCheckpoint(CheckpointMetaData checkpointMetaData, CheckpointOptions checkpointOptions, boolean advanceToEndOfEventTime) throws Exception {
+		throw new UnsupportedOperationException("should never be called");
+	}
+
+	@Override
+	public void triggerCheckpointOnBarrier(CheckpointMetaData checkpointMetaData, CheckpointOptions checkpointOptions, CheckpointMetrics checkpointMetrics) throws Exception {
+		assertTrue("Unexpected triggerCheckpointOnBarrier(" + checkpointMetaData.getCheckpointId() + ")", i < checkpointIDs.length);
+
+		final long expectedId = checkpointIDs[i++];
+		if (expectedId >= 0) {
+			assertEquals("wrong checkpoint id", expectedId, checkpointMetaData.getCheckpointId());
+			assertTrue(checkpointMetaData.getTimestamp() > 0);
+		} else {
+			fail(String.format(
+				"got 'triggerCheckpointOnBarrier(%d)' when expecting an 'abortCheckpointOnBarrier(%d)'",
+				checkpointMetaData.getCheckpointId(),
+				expectedId));
+		}
+	}
+
+	@Override
+	public void abortCheckpointOnBarrier(long checkpointId, Throwable cause) {
+		assertTrue("Unexpected abortCheckpointOnBarrier(" + checkpointId + ")", i < checkpointIDs.length);
+
+		final long expectedId = checkpointIDs[i++];
+		if (expectedId < 0) {
+			assertEquals("wrong checkpoint id for checkpoint abort", -expectedId, checkpointId);
+		} else {
+			fail(String.format(
+				"got 'abortCheckpointOnBarrier(%d)' when expecting an 'triggerCheckpointOnBarrier(%d)'",
+				checkpointId,
+				expectedId));
+		}
+	}
+
+	@Override
+	public void notifyCheckpointComplete(long checkpointId) throws Exception {
+		throw new UnsupportedOperationException("should never be called");
+	}
+}


[flink] 11/16: [FLINK-12777][network] Introduce LinkedBufferStorage class

Posted by pn...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pnowojski pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 76b299359fcd6463d1e7c46fc64bfd42f787daef
Author: Piotr Nowojski <pi...@gmail.com>
AuthorDate: Tue Jun 18 15:57:02 2019 +0200

    [FLINK-12777][network] Introduce LinkedBufferStorage class
---
 .../streaming/runtime/io/LinkedBufferStorage.java  |  92 ++++++++++++
 .../runtime/io/BufferStorageTestBase.java          |   4 +
 .../runtime/io/LinkedBufferStorageTest.java        | 165 +++++++++++++++++++++
 3 files changed, 261 insertions(+)

diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/LinkedBufferStorage.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/LinkedBufferStorage.java
new file mode 100644
index 0000000..aac2ba6
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/LinkedBufferStorage.java
@@ -0,0 +1,92 @@
+/*
+ * 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.streaming.runtime.io;
+
+import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
+
+import java.io.IOException;
+import java.util.Optional;
+
+/**
+ * Implementation of {@link BufferStorage} that links two {@link BufferStorage} together.
+ * Each of the linked {@link BufferStorage} will store buffers independently, but they will be
+ * linked together for {@link #rollOver()} - if one is rolled over, other will do that as well.
+ *
+ * <p>Note that only {@code mainStorage} is closed when {@link LinkedBufferStorage} instance is closed.
+ */
+public class LinkedBufferStorage implements BufferStorage {
+
+	private final BufferStorage mainStorage;
+
+	private final BufferStorage linkedStorage;
+
+	private long maxBufferedBytes;
+
+	public LinkedBufferStorage(BufferStorage mainStorage, BufferStorage linkedStorage, long maxBufferedBytes) {
+		this.mainStorage = mainStorage;
+		this.linkedStorage = linkedStorage;
+		this.maxBufferedBytes = maxBufferedBytes;
+	}
+
+	@Override
+	public void add(BufferOrEvent boe) throws IOException {
+		mainStorage.add(boe);
+	}
+
+	@Override
+	public boolean isFull() {
+		return maxBufferedBytes > 0 && (getRolledBytes() + getPendingBytes()) > maxBufferedBytes;
+	}
+
+	@Override
+	public void rollOver() throws IOException {
+		mainStorage.rollOver();
+		linkedStorage.rollOver();
+	}
+
+	@Override
+	public long getPendingBytes() {
+		return mainStorage.getPendingBytes() + linkedStorage.getPendingBytes();
+	}
+
+	@Override
+	public long getRolledBytes() {
+		return mainStorage.getRolledBytes() + linkedStorage.getRolledBytes();
+	}
+
+	@Override
+	public boolean isEmpty() {
+		return mainStorage.isEmpty();
+	}
+
+	@Override
+	public Optional<BufferOrEvent> pollNext() throws IOException {
+		return mainStorage.pollNext();
+	}
+
+	@Override
+	public long getMaxBufferedBytes() {
+		return maxBufferedBytes;
+	}
+
+	@Override
+	public void close() throws IOException {
+		mainStorage.close();
+	}
+}
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BufferStorageTestBase.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BufferStorageTestBase.java
index b23d3e9..1e219a5 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BufferStorageTestBase.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BufferStorageTestBase.java
@@ -244,6 +244,10 @@ public abstract class BufferStorageTestBase {
 		return new BufferOrEvent(evt, channelIndex);
 	}
 
+	public static BufferOrEvent generateRandomBuffer(int size) {
+		return generateRandomBuffer(size, 0);
+	}
+
 	public static BufferOrEvent generateRandomBuffer(int size, int channelIndex) {
 		MemorySegment seg = MemorySegmentFactory.allocateUnpooledSegment(PAGE_SIZE);
 		for (int i = 0; i < size; i++) {
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/LinkedBufferStorageTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/LinkedBufferStorageTest.java
new file mode 100644
index 0000000..1edae8d
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/LinkedBufferStorageTest.java
@@ -0,0 +1,165 @@
+/*
+ * 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.streaming.runtime.io;
+
+import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Optional;
+
+import static junit.framework.TestCase.assertFalse;
+import static org.apache.flink.streaming.runtime.io.BufferStorageTestBase.generateRandomBuffer;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for {@link LinkedBufferStorage}.
+ */
+public class LinkedBufferStorageTest {
+	private static final int PAGE_SIZE = 100;
+
+	private CachedBufferStorage mainStorage;
+
+	private CachedBufferStorage linkedStorage;
+
+	private LinkedBufferStorage bufferStorage;
+
+	@Before
+	public void setUp() {
+		mainStorage = new CachedBufferStorage(PAGE_SIZE);
+		linkedStorage = new CachedBufferStorage(PAGE_SIZE);
+		bufferStorage = new LinkedBufferStorage(
+			mainStorage,
+			linkedStorage,
+			700);
+	}
+
+	@After
+	public void tearDown() throws IOException {
+		bufferStorage.close();
+		mainStorage.close();
+		linkedStorage.close();
+	}
+
+	@Test
+	public void testBasicUsage() throws IOException {
+		linkedStorage.add(generateRandomBuffer(PAGE_SIZE + 0));
+		assertEquals(PAGE_SIZE, bufferStorage.getPendingBytes());
+		assertTrue(bufferStorage.isEmpty());
+
+		bufferStorage.add(generateRandomBuffer(PAGE_SIZE + 1));
+		bufferStorage.add(generateRandomBuffer(PAGE_SIZE + 2));
+
+		assertTrue(bufferStorage.isEmpty());
+		assertEquals(mainStorage.getPendingBytes() + linkedStorage.getPendingBytes(), bufferStorage.getPendingBytes());
+		assertEquals(mainStorage.getRolledBytes() + linkedStorage.getRolledBytes(), bufferStorage.getRolledBytes());
+
+		assertTrue(bufferStorage.isEmpty());
+		assertTrue(linkedStorage.isEmpty());
+
+		bufferStorage.rollOver();
+
+		assertFalse(bufferStorage.isEmpty());
+		assertFalse(linkedStorage.isEmpty());
+
+		assertEquals(mainStorage.getPendingBytes() + linkedStorage.getPendingBytes(), bufferStorage.getPendingBytes());
+		assertEquals(mainStorage.getRolledBytes() + linkedStorage.getRolledBytes(), bufferStorage.getRolledBytes());
+
+		linkedStorage.add(generateRandomBuffer(PAGE_SIZE + 3));
+		bufferStorage.add(generateRandomBuffer(PAGE_SIZE + 4));
+
+		assertEquals(mainStorage.getPendingBytes() + linkedStorage.getPendingBytes(), bufferStorage.getPendingBytes());
+		assertEquals(mainStorage.getRolledBytes() + linkedStorage.getRolledBytes(), bufferStorage.getRolledBytes());
+
+		bufferStorage.rollOver();
+
+		assertEquals(mainStorage.getPendingBytes() + linkedStorage.getPendingBytes(), bufferStorage.getPendingBytes());
+		assertEquals(mainStorage.getRolledBytes() + linkedStorage.getRolledBytes(), bufferStorage.getRolledBytes());
+
+		ArrayList<Integer> bufferSizes = drain(bufferStorage);
+
+		assertEquals(PAGE_SIZE + 4, (long) bufferSizes.get(0));
+		assertEquals(PAGE_SIZE + 1, (long) bufferSizes.get(1));
+		assertEquals(PAGE_SIZE + 2, (long) bufferSizes.get(2));
+
+		bufferSizes = drain(linkedStorage);
+
+		assertEquals(PAGE_SIZE + 3, (long) bufferSizes.get(0));
+		assertEquals(PAGE_SIZE + 0, (long) bufferSizes.get(1));
+
+		assertEquals(0, bufferStorage.getRolledBytes());
+		assertEquals(0, bufferStorage.getPendingBytes());
+	}
+
+	@Test
+	public void testPendingIsFull() throws IOException {
+		linkedStorage.add(generateRandomBuffer(PAGE_SIZE));
+		linkedStorage.add(generateRandomBuffer(PAGE_SIZE));
+		bufferStorage.add(generateRandomBuffer(PAGE_SIZE));
+		bufferStorage.add(generateRandomBuffer(PAGE_SIZE));
+		bufferStorage.add(generateRandomBuffer(PAGE_SIZE));
+		bufferStorage.add(generateRandomBuffer(PAGE_SIZE));
+		bufferStorage.add(generateRandomBuffer(PAGE_SIZE));
+
+		assertFalse(bufferStorage.isFull());
+
+		bufferStorage.add(generateRandomBuffer(PAGE_SIZE));
+
+		assertTrue(bufferStorage.isFull());
+	}
+
+	/**
+	 * This test is broken because of FLINK-12912.
+	 * https://issues.apache.org/jira/browse/FLINK-12912
+	 */
+	//@Test
+	public void testRolledIsFull() throws IOException {
+		linkedStorage.add(generateRandomBuffer(PAGE_SIZE));
+		linkedStorage.add(generateRandomBuffer(PAGE_SIZE));
+		bufferStorage.add(generateRandomBuffer(PAGE_SIZE));
+		bufferStorage.rollOver();
+		bufferStorage.add(generateRandomBuffer(PAGE_SIZE));
+		bufferStorage.add(generateRandomBuffer(PAGE_SIZE));
+		bufferStorage.rollOver();
+		linkedStorage.add(generateRandomBuffer(PAGE_SIZE));
+		bufferStorage.add(generateRandomBuffer(PAGE_SIZE));
+
+		assertFalse(bufferStorage.isFull());
+
+		bufferStorage.add(generateRandomBuffer(PAGE_SIZE));
+
+		assertTrue(bufferStorage.isFull());
+	}
+
+	private ArrayList<Integer> drain(BufferStorage bufferStorage) throws IOException {
+		ArrayList<Integer> result = new ArrayList<>();
+		while (!bufferStorage.isEmpty()) {
+			Optional<BufferOrEvent> bufferOrEvent = bufferStorage.pollNext();
+			if (bufferOrEvent.isPresent()) {
+				result.add(bufferOrEvent.get().getSize());
+			}
+		}
+		return result;
+	}
+}


[flink] 01/16: [hotfix][network] Rename BufferBlocker to BufferStorage

Posted by pn...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pnowojski pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit ff8a94612c2e80189f9ba883e58c8fd599d4bb45
Author: Piotr Nowojski <pi...@gmail.com>
AuthorDate: Thu Jun 13 13:28:13 2019 +0200

    [hotfix][network] Rename BufferBlocker to BufferStorage
---
 .../flink/streaming/runtime/io/BarrierBuffer.java  | 24 +++++++++----------
 .../flink/streaming/runtime/io/BufferSpiller.java  |  6 ++---
 .../io/{BufferBlocker.java => BufferStorage.java}  | 10 ++++----
 ...BufferBlocker.java => CachedBufferStorage.java} | 18 +++++++-------
 .../streaming/runtime/io/InputProcessorUtil.java   |  2 +-
 .../runtime/io/BarrierBufferTestBase.java          |  2 +-
 .../streaming/runtime/io/BufferSpillerTest.java    |  4 ++--
 ...kerTestBase.java => BufferStorageTestBase.java} | 28 +++++++++++-----------
 ...ockerTest.java => CachedBufferStorageTest.java} | 20 ++++++++--------
 .../runtime/io/CreditBasedBarrierBufferTest.java   |  4 ++--
 10 files changed, 60 insertions(+), 58 deletions(-)

diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
index 63fa1ac..ad62360 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
@@ -64,7 +64,7 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 	private final int totalNumberOfInputChannels;
 
 	/** To utility to write blocked data to a file channel. */
-	private final BufferBlocker bufferBlocker;
+	private final BufferStorage bufferStorage;
 
 	/**
 	 * The pending blocked buffer/event sequences. Must be consumed before requesting further data
@@ -123,11 +123,11 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 	 * <p>There is no limit to how much data may be buffered during an alignment.
 	 *
 	 * @param inputGate The input gate to draw the buffers and events from.
-	 * @param bufferBlocker The buffer blocker to hold the buffers and events for channels with barrier.
+	 * @param bufferStorage The storage to hold the buffers and events for blocked channels.
 	 */
 	@VisibleForTesting
-	BarrierBuffer(InputGate inputGate, BufferBlocker bufferBlocker) {
-		this (inputGate, bufferBlocker, -1, "Testing: No task associated");
+	BarrierBuffer(InputGate inputGate, BufferStorage bufferStorage) {
+		this (inputGate, bufferStorage, -1, "Testing: No task associated");
 	}
 
 	/**
@@ -138,11 +138,11 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 	 * checkpoint has been cancelled.
 	 *
 	 * @param inputGate The input gate to draw the buffers and events from.
-	 * @param bufferBlocker The buffer blocker to hold the buffers and events for channels with barrier.
+	 * @param bufferStorage The storage to hold the buffers and events for blocked channels.
 	 * @param maxBufferedBytes The maximum bytes to be buffered before the checkpoint aborts.
 	 * @param taskName The task name for logging.
 	 */
-	BarrierBuffer(InputGate inputGate, BufferBlocker bufferBlocker, long maxBufferedBytes, String taskName) {
+	BarrierBuffer(InputGate inputGate, BufferStorage bufferStorage, long maxBufferedBytes, String taskName) {
 		checkArgument(maxBufferedBytes == -1 || maxBufferedBytes > 0);
 
 		this.inputGate = inputGate;
@@ -150,7 +150,7 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 		this.totalNumberOfInputChannels = inputGate.getNumberOfInputChannels();
 		this.blockedChannels = new boolean[this.totalNumberOfInputChannels];
 
-		this.bufferBlocker = checkNotNull(bufferBlocker);
+		this.bufferStorage = checkNotNull(bufferStorage);
 		this.queuedBuffered = new ArrayDeque<BufferOrEventSequence>();
 
 		this.taskName = taskName;
@@ -192,7 +192,7 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 			BufferOrEvent bufferOrEvent = next.get();
 			if (isBlocked(bufferOrEvent.getChannelIndex())) {
 				// if the channel is blocked, we just store the BufferOrEvent
-				bufferBlocker.add(bufferOrEvent);
+				bufferStorage.add(bufferOrEvent);
 				checkSizeLimit();
 			}
 			else if (bufferOrEvent.isBuffer()) {
@@ -436,7 +436,7 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 	}
 
 	private void checkSizeLimit() throws Exception {
-		if (maxBufferedBytes > 0 && (numQueuedBytes + bufferBlocker.getBytesBlocked()) > maxBufferedBytes) {
+		if (maxBufferedBytes > 0 && (numQueuedBytes + bufferStorage.getBytesBlocked()) > maxBufferedBytes) {
 			// exceeded our limit - abort this checkpoint
 			LOG.info("{}: Checkpoint {} aborted because alignment volume limit ({} bytes) exceeded.",
 				taskName,
@@ -473,7 +473,7 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 
 	@Override
 	public void cleanup() throws IOException {
-		bufferBlocker.close();
+		bufferStorage.close();
 		if (currentBuffered != null) {
 			currentBuffered.cleanup();
 		}
@@ -538,7 +538,7 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 
 		if (currentBuffered == null) {
 			// common case: no more buffered data
-			currentBuffered = bufferBlocker.rollOverReusingResources();
+			currentBuffered = bufferStorage.rollOverReusingResources();
 			if (currentBuffered != null) {
 				currentBuffered.open();
 			}
@@ -550,7 +550,7 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 					"Pushing back current alignment buffers and feeding back new alignment data first.", taskName);
 
 			// since we did not fully drain the previous sequence, we need to allocate a new buffer for this one
-			BufferOrEventSequence bufferedNow = bufferBlocker.rollOverWithoutReusingResources();
+			BufferOrEventSequence bufferedNow = bufferStorage.rollOverWithoutReusingResources();
 			if (bufferedNow != null) {
 				bufferedNow.open();
 				queuedBuffered.addFirst(currentBuffered);
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java
index 5a7c496..59877a0 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java
@@ -40,7 +40,7 @@ import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.atomic.AtomicInteger;
 
 /**
- * The buffer spiller takes the buffers and events from a data stream and adds them to a spill file.
+ * The {@link BufferSpiller} takes the buffers and events from a data stream and adds them to a spill file.
  * After a number of elements have been spilled, the spiller can "roll over": It presents the spilled
  * elements as a readable sequence, and opens a new spill file.
  *
@@ -53,7 +53,7 @@ import java.util.concurrent.atomic.AtomicInteger;
  */
 @Internal
 @Deprecated
-public class BufferSpiller implements BufferBlocker {
+public class BufferSpiller implements BufferStorage {
 
 	/** Size of header in bytes (see add method). */
 	static final int HEADER_SIZE = 9;
@@ -92,7 +92,7 @@ public class BufferSpiller implements BufferBlocker {
 	private long bytesWritten;
 
 	/**
-	 * Creates a new buffer spiller, spilling to one of the I/O manager's temp directories.
+	 * Creates a new {@link BufferSpiller}, spilling to one of the I/O manager's temp directories.
 	 *
 	 * @param ioManager The I/O manager for access to the temp directories.
 	 * @param pageSize The page size used to re-create spilled buffers.
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BufferBlocker.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BufferStorage.java
similarity index 84%
rename from flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BufferBlocker.java
rename to flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BufferStorage.java
index 4d0f66f..7d4dff0 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BufferBlocker.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BufferStorage.java
@@ -24,15 +24,15 @@ import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
 import java.io.IOException;
 
 /**
- * The buffer blocker takes the buffers and events from a data stream and adds them in a sequence.
- * After a number of elements have been added, the blocker can "roll over": It presents the added
- * elements as a readable sequence, and creates a new sequence.
+ * The {@link BufferStorage} takes the buffers and events from a data stream and adds them in a sequence.
+ * After a number of elements have been added, the {@link BufferStorage} can "roll over":
+ * It presents the added elements as a readable sequence, and creates a new sequence.
  */
 @Internal
-public interface BufferBlocker {
+public interface BufferStorage {
 
 	/**
-	 * Adds a buffer or event to the blocker.
+	 * Adds a buffer or event to the {@link BufferStorage}.
 	 *
 	 * @param boe The buffer or event to be added into the blocker.
 	 */
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CachedBufferBlocker.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CachedBufferStorage.java
similarity index 84%
rename from flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CachedBufferBlocker.java
rename to flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CachedBufferStorage.java
index f91e8cc..e0a79c2 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CachedBufferBlocker.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CachedBufferStorage.java
@@ -25,14 +25,16 @@ import javax.annotation.Nullable;
 import java.util.ArrayDeque;
 
 /**
- * The cached buffer blocker takes the buffers and events from a data stream and adds them to a memory queue.
- * After a number of elements have been cached, the blocker can "roll over": It presents the cached
- * elements as a readable sequence, and creates a new memory queue.
+ * The {@link CachedBufferStorage} takes the buffers and events from a data stream and adds them to
+ * a memory queue. After a number of elements have been cached, the {@link CachedBufferStorage}
+ * can "roll over":
+ * It presents the cached elements as a readable sequence, and creates a new memory queue.
  *
- * <p>This buffer blocked can be used in credit-based flow control for better barrier alignment in exactly-once mode.
+ * <p>This {@link CachedBufferStorage} can be used in credit-based flow control for better barrier
+ * alignment in exactly-once mode.
  */
 @Internal
-public class CachedBufferBlocker implements BufferBlocker {
+public class CachedBufferStorage implements BufferStorage {
 
 	/** The page size, to estimate the total cached data size. */
 	private final int pageSize;
@@ -44,11 +46,11 @@ public class CachedBufferBlocker implements BufferBlocker {
 	private ArrayDeque<BufferOrEvent> currentBuffers;
 
 	/**
-	 * Creates a new buffer blocker, caching the buffers or events in memory queue.
+	 * Creates a new {@link CachedBufferStorage}, caching the buffers or events in memory queue.
 	 *
 	 * @param pageSize The page size used to estimate the cached size.
 	 */
-	public CachedBufferBlocker(int pageSize) {
+	public CachedBufferStorage(int pageSize) {
 		this.pageSize = pageSize;
 		this.currentBuffers = new ArrayDeque<BufferOrEvent>();
 	}
@@ -100,7 +102,7 @@ public class CachedBufferBlocker implements BufferBlocker {
 
 	/**
 	 * This class represents a sequence of cached buffers and events, created by the
-	 * {@link CachedBufferBlocker}.
+	 * {@link CachedBufferStorage}.
 	 */
 	public static class CachedBufferOrEventSequence implements BufferOrEventSequence {
 
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java
index b77c7d0..289dd1a 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java
@@ -56,7 +56,7 @@ public class InputProcessorUtil {
 			if (taskManagerConfig.getBoolean(NettyShuffleEnvironmentOptions.NETWORK_CREDIT_MODEL)) {
 				barrierHandler = new BarrierBuffer(
 					inputGate,
-					new CachedBufferBlocker(inputGate.getPageSize()),
+					new CachedBufferStorage(inputGate.getPageSize()),
 					maxAlign,
 					taskName);
 			} else {
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTestBase.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTestBase.java
index c9981b5..4bc05ff 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTestBase.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTestBase.java
@@ -58,7 +58,7 @@ import static org.mockito.Mockito.verify;
 import static org.mockito.hamcrest.MockitoHamcrest.argThat;
 
 /**
- * Tests for the behavior of the {@link BarrierBuffer} with different {@link BufferBlocker} implements.
+ * Tests for the behavior of the {@link BarrierBuffer} with different {@link BufferStorage} implements.
  */
 public abstract class BarrierBufferTestBase {
 
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java
index b70ba24..4633154 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java
@@ -38,7 +38,7 @@ import static org.junit.Assert.fail;
 /**
  * Tests for {@link BufferSpiller}.
  */
-public class BufferSpillerTest extends BufferBlockerTestBase {
+public class BufferSpillerTest extends BufferStorageTestBase {
 
 	private static IOManager ioManager;
 
@@ -76,7 +76,7 @@ public class BufferSpillerTest extends BufferBlockerTestBase {
 	}
 
 	@Override
-	public BufferBlocker createBufferBlocker() {
+	public BufferStorage createBufferStorage() {
 		return spiller;
 	}
 
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BufferBlockerTestBase.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BufferStorageTestBase.java
similarity index 93%
rename from flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BufferBlockerTestBase.java
rename to flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BufferStorageTestBase.java
index 4533a65..0485d88 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BufferBlockerTestBase.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BufferStorageTestBase.java
@@ -39,20 +39,20 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 /**
- * Tests for {@link BufferBlocker}.
+ * Tests for {@link BufferStorage}.
  */
-public abstract class BufferBlockerTestBase {
+public abstract class BufferStorageTestBase {
 
 	protected static final int PAGE_SIZE = 4096;
 
-	abstract BufferBlocker createBufferBlocker();
+	abstract BufferStorage createBufferStorage();
 
 	@Test
 	public void testRollOverEmptySequences() throws IOException {
-		BufferBlocker bufferBlocker = createBufferBlocker();
-		assertNull(bufferBlocker.rollOverReusingResources());
-		assertNull(bufferBlocker.rollOverReusingResources());
-		assertNull(bufferBlocker.rollOverReusingResources());
+		BufferStorage bufferStorage = createBufferStorage();
+		assertNull(bufferStorage.rollOverReusingResources());
+		assertNull(bufferStorage.rollOverReusingResources());
+		assertNull(bufferStorage.rollOverReusingResources());
 	}
 
 	@Test
@@ -63,7 +63,7 @@ public abstract class BufferBlockerTestBase {
 		final int maxNumEventsAndBuffers = 3000;
 		final int maxNumChannels = 1656;
 
-		BufferBlocker bufferBlocker = createBufferBlocker();
+		BufferStorage bufferStorage = createBufferStorage();
 
 		// do multiple spilling / rolling over rounds
 		for (int round = 0; round < 5; round++) {
@@ -86,13 +86,13 @@ public abstract class BufferBlockerTestBase {
 				} else {
 					evt = generateRandomBuffer(bufferRnd.nextInt(PAGE_SIZE) + 1, bufferRnd.nextInt(numberOfChannels));
 				}
-				bufferBlocker.add(evt);
+				bufferStorage.add(evt);
 			}
 
 			// reset and create reader
 			bufferRnd.setSeed(bufferSeed);
 
-			BufferOrEventSequence seq = bufferBlocker.rollOverReusingResources();
+			BufferOrEventSequence seq = bufferStorage.rollOverReusingResources();
 			seq.open();
 
 			// read and validate the sequence
@@ -136,14 +136,14 @@ public abstract class BufferBlockerTestBase {
 		int currentNumEvents = 0;
 		int currentNumRecordAndEvents = 0;
 
-		BufferBlocker bufferBlocker = createBufferBlocker();
+		BufferStorage bufferStorage = createBufferStorage();
 
 		// do multiple spilling / rolling over rounds
 		for (int round = 0; round < 2 * sequences; round++) {
 
 			if (round % 2 == 1) {
 				// make this an empty sequence
-				assertNull(bufferBlocker.rollOverReusingResources());
+				assertNull(bufferStorage.rollOverReusingResources());
 			} else {
 				// proper spilled sequence
 				final long bufferSeed = rnd.nextLong();
@@ -167,7 +167,7 @@ public abstract class BufferBlockerTestBase {
 						} else {
 							evt = generateRandomBuffer(bufferRnd.nextInt(PAGE_SIZE) + 1, bufferRnd.nextInt(numberOfChannels));
 						}
-						bufferBlocker.add(evt);
+						bufferStorage.add(evt);
 						generated++;
 					} else {
 						// consume a record
@@ -205,7 +205,7 @@ public abstract class BufferBlockerTestBase {
 
 				// done generating a sequence. queue it for consumption
 				bufferRnd.setSeed(bufferSeed);
-				BufferOrEventSequence seq = bufferBlocker.rollOverReusingResources();
+				BufferOrEventSequence seq = bufferStorage.rollOverReusingResources();
 
 				SequenceToConsume stc = new SequenceToConsume(bufferRnd, events, seq, numEventsAndBuffers, numberOfChannels);
 
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CachedBufferBlockerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CachedBufferStorageTest.java
similarity index 73%
rename from flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CachedBufferBlockerTest.java
rename to flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CachedBufferStorageTest.java
index e7bf128..d1e2cf4 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CachedBufferBlockerTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CachedBufferStorageTest.java
@@ -24,30 +24,30 @@ import org.junit.Before;
 import java.io.IOException;
 
 /**
- * Tests for {@link CachedBufferBlocker}.
+ * Tests for {@link CachedBufferStorage}.
  */
-public class CachedBufferBlockerTest extends BufferBlockerTestBase {
+public class CachedBufferStorageTest extends BufferStorageTestBase {
 
-	private CachedBufferBlocker bufferBlocker;
+	private CachedBufferStorage bufferStorage;
 
 	// ------------------------------------------------------------------------
 	//  Setup / Cleanup
 	// ------------------------------------------------------------------------
 
 	@Before
-	public void createBlocker() {
-		bufferBlocker = new CachedBufferBlocker(PAGE_SIZE);
+	public void createStorage() {
+		bufferStorage = new CachedBufferStorage(PAGE_SIZE);
 	}
 
 	@After
-	public void cleanupBlocker() throws IOException {
-		if (bufferBlocker != null) {
-			bufferBlocker.close();
+	public void cleanupStorage() throws IOException {
+		if (bufferStorage != null) {
+			bufferStorage.close();
 		}
 	}
 
 	@Override
-	public BufferBlocker createBufferBlocker() {
-		return  bufferBlocker;
+	public BufferStorage createBufferStorage() {
+		return  bufferStorage;
 	}
 }
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CreditBasedBarrierBufferTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CreditBasedBarrierBufferTest.java
index e9c87ed..da88ffb 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CreditBasedBarrierBufferTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CreditBasedBarrierBufferTest.java
@@ -26,13 +26,13 @@ import java.io.IOException;
 import static org.junit.Assert.assertEquals;
 
 /**
- * Tests for the behaviors of the {@link BarrierBuffer} with {@link CachedBufferBlocker}.
+ * Tests for the behaviors of the {@link BarrierBuffer} with {@link CachedBufferStorage}.
  */
 public class CreditBasedBarrierBufferTest extends BarrierBufferTestBase {
 
 	@Override
 	public BarrierBuffer createBarrierBuffer(InputGate gate) throws IOException {
-		return new BarrierBuffer(gate, new CachedBufferBlocker(PAGE_SIZE));
+		return new BarrierBuffer(gate, new CachedBufferStorage(PAGE_SIZE));
 	}
 
 	@Override


[flink] 10/16: [hotfix][network] Split InputProcessorUtil into smaller methods

Posted by pn...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pnowojski pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 2e5c499d6263b59183dc93d56b2bb27684eeecb1
Author: Piotr Nowojski <pi...@gmail.com>
AuthorDate: Tue Jun 18 16:57:19 2019 +0200

    [hotfix][network] Split InputProcessorUtil into smaller methods
---
 .../streaming/runtime/io/InputProcessorUtil.java   | 79 ++++++++++++++--------
 1 file changed, 49 insertions(+), 30 deletions(-)

diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java
index 7eda06c..419cf16 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java
@@ -24,8 +24,8 @@ import org.apache.flink.configuration.NettyShuffleEnvironmentOptions;
 import org.apache.flink.configuration.TaskManagerOptions;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.streaming.api.CheckpointingMode;
-import org.apache.flink.streaming.runtime.tasks.StreamTask;
 
 import java.io.IOException;
 
@@ -37,44 +37,63 @@ import java.io.IOException;
 public class InputProcessorUtil {
 
 	public static CheckpointedInputGate createCheckpointedInputGate(
-			StreamTask<?, ?> checkpointedTask,
+			AbstractInvokable toNotifyOnCheckpoint,
 			CheckpointingMode checkpointMode,
 			IOManager ioManager,
 			InputGate inputGate,
 			Configuration taskManagerConfig,
 			String taskName) throws IOException {
 
-		CheckpointedInputGate checkpointedInputGate;
-		if (checkpointMode == CheckpointingMode.EXACTLY_ONCE) {
-			long maxAlign = taskManagerConfig.getLong(TaskManagerOptions.TASK_CHECKPOINT_ALIGNMENT_BYTES_LIMIT);
-			if (!(maxAlign == -1 || maxAlign > 0)) {
-				throw new IllegalConfigurationException(
-					TaskManagerOptions.TASK_CHECKPOINT_ALIGNMENT_BYTES_LIMIT.key()
-					+ " must be positive or -1 (infinite)");
-			}
+		BufferStorage bufferStorage = createBufferStorage(
+			checkpointMode, ioManager, inputGate.getPageSize(), taskManagerConfig, taskName);
+		CheckpointBarrierHandler barrierHandler = createCheckpointBarrierHandler(
+			checkpointMode, inputGate.getNumberOfInputChannels(), taskName, toNotifyOnCheckpoint);
+		return new CheckpointedInputGate(inputGate, bufferStorage, barrierHandler);
+	}
 
-			if (taskManagerConfig.getBoolean(NettyShuffleEnvironmentOptions.NETWORK_CREDIT_MODEL)) {
-				checkpointedInputGate = new CheckpointedInputGate(
-					inputGate,
-					new CachedBufferStorage(inputGate.getPageSize(), maxAlign, taskName),
-					taskName,
-					checkpointedTask);
-			} else {
-				checkpointedInputGate = new CheckpointedInputGate(
-					inputGate,
-					new BufferSpiller(ioManager, inputGate.getPageSize(), maxAlign, taskName),
+	private static CheckpointBarrierHandler createCheckpointBarrierHandler(
+			CheckpointingMode checkpointMode,
+			int numberOfInputChannels,
+			String taskName,
+			AbstractInvokable toNotifyOnCheckpoint) {
+		switch (checkpointMode) {
+			case EXACTLY_ONCE:
+				return new CheckpointBarrierAligner(
+					numberOfInputChannels,
 					taskName,
-					checkpointedTask);
-			}
-		} else if (checkpointMode == CheckpointingMode.AT_LEAST_ONCE) {
-			checkpointedInputGate = new CheckpointedInputGate(
-				inputGate,
-				new EmptyBufferStorage(),
-				new CheckpointBarrierTracker(inputGate.getNumberOfInputChannels(), checkpointedTask));
-		} else {
-			throw new IllegalArgumentException("Unrecognized Checkpointing Mode: " + checkpointMode);
+					toNotifyOnCheckpoint);
+			case AT_LEAST_ONCE:
+				return new CheckpointBarrierTracker(numberOfInputChannels, toNotifyOnCheckpoint);
+			default:
+				throw new UnsupportedOperationException("Unrecognized Checkpointing Mode: " + checkpointMode);
 		}
+	}
 
-		return checkpointedInputGate;
+	private static BufferStorage createBufferStorage(
+			CheckpointingMode checkpointMode,
+			IOManager ioManager,
+			int pageSize,
+			Configuration taskManagerConfig,
+			String taskName) throws IOException {
+		switch (checkpointMode) {
+			case EXACTLY_ONCE: {
+				long maxAlign = taskManagerConfig.getLong(TaskManagerOptions.TASK_CHECKPOINT_ALIGNMENT_BYTES_LIMIT);
+				if (!(maxAlign == -1 || maxAlign > 0)) {
+					throw new IllegalConfigurationException(
+						TaskManagerOptions.TASK_CHECKPOINT_ALIGNMENT_BYTES_LIMIT.key()
+							+ " must be positive or -1 (infinite)");
+				}
+
+				if (taskManagerConfig.getBoolean(NettyShuffleEnvironmentOptions.NETWORK_CREDIT_MODEL)) {
+					return new CachedBufferStorage(pageSize, maxAlign, taskName);
+				} else {
+					return new BufferSpiller(ioManager, pageSize, maxAlign, taskName);
+				}
+			}
+			case AT_LEAST_ONCE:
+				return new EmptyBufferStorage();
+			default:
+				throw new UnsupportedOperationException("Unrecognized Checkpointing Mode: " + checkpointMode);
+		}
 	}
 }


[flink] 04/16: [hotfix][test] Drop unnecessary pageSize argument in BufferBarierTestBase#createBuffer

Posted by pn...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pnowojski pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit ad4404854d8fe6c75bcef301e69a867673c0ea5b
Author: Piotr Nowojski <pi...@gmail.com>
AuthorDate: Sat Jun 15 10:29:37 2019 +0200

    [hotfix][test] Drop unnecessary pageSize argument in BufferBarierTestBase#createBuffer
---
 .../runtime/io/BarrierBufferTestBase.java          | 234 ++++++++++-----------
 1 file changed, 117 insertions(+), 117 deletions(-)

diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTestBase.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTestBase.java
index 908a199..3b4f65f 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTestBase.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTestBase.java
@@ -112,8 +112,8 @@ public abstract class BarrierBufferTestBase {
 	@Test
 	public void testSingleChannelNoBarriers() throws Exception {
 		BufferOrEvent[] sequence = {
-			createBuffer(0, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
-			createBuffer(0, PAGE_SIZE), createEndOfPartition(0)
+			createBuffer(0), createBuffer(0),
+			createBuffer(0), createEndOfPartition(0)
 		};
 		buffer = createBarrierBuffer(1, sequence);
 
@@ -131,10 +131,10 @@ public abstract class BarrierBufferTestBase {
 	@Test
 	public void testMultiChannelNoBarriers() throws Exception {
 		BufferOrEvent[] sequence = {
-			createBuffer(2, PAGE_SIZE), createBuffer(2, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
-			createBuffer(1, PAGE_SIZE), createBuffer(0, PAGE_SIZE), createEndOfPartition(0),
-			createBuffer(3, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createEndOfPartition(3),
-			createBuffer(1, PAGE_SIZE), createEndOfPartition(1), createBuffer(2, PAGE_SIZE), createEndOfPartition(2)
+			createBuffer(2), createBuffer(2), createBuffer(0),
+			createBuffer(1), createBuffer(0), createEndOfPartition(0),
+			createBuffer(3), createBuffer(1), createEndOfPartition(3),
+			createBuffer(1), createEndOfPartition(1), createBuffer(2), createEndOfPartition(2)
 		};
 		buffer = createBarrierBuffer(4, sequence);
 
@@ -152,13 +152,13 @@ public abstract class BarrierBufferTestBase {
 	@Test
 	public void testSingleChannelWithBarriers() throws Exception {
 		BufferOrEvent[] sequence = {
-			createBuffer(0, PAGE_SIZE), createBuffer(0, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			createBuffer(0), createBuffer(0), createBuffer(0),
 			createBarrier(1, 0),
-			createBuffer(0, PAGE_SIZE), createBuffer(0, PAGE_SIZE), createBuffer(0, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			createBuffer(0), createBuffer(0), createBuffer(0), createBuffer(0),
 			createBarrier(2, 0), createBarrier(3, 0),
-			createBuffer(0, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			createBuffer(0), createBuffer(0),
 			createBarrier(4, 0), createBarrier(5, 0), createBarrier(6, 0),
-			createBuffer(0, PAGE_SIZE), createEndOfPartition(0)
+			createBuffer(0), createEndOfPartition(0)
 		};
 		ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler();
 		buffer = createBarrierBuffer(1, sequence, handler);
@@ -180,34 +180,34 @@ public abstract class BarrierBufferTestBase {
 	public void testMultiChannelWithBarriers() throws Exception {
 		BufferOrEvent[] sequence = {
 			// checkpoint with blocked data
-			createBuffer(0, PAGE_SIZE), createBuffer(2, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			createBuffer(0), createBuffer(2), createBuffer(0),
 			createBarrier(1, 1), createBarrier(1, 2),
-			createBuffer(2, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			createBuffer(2), createBuffer(1), createBuffer(0),
 			createBarrier(1, 0),
 
 			// checkpoint without blocked data
-			createBuffer(0, PAGE_SIZE), createBuffer(0, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createBuffer(2, PAGE_SIZE),
+			createBuffer(0), createBuffer(0), createBuffer(1), createBuffer(1), createBuffer(2),
 			createBarrier(2, 0), createBarrier(2, 1), createBarrier(2, 2),
 
 			// checkpoint with data only from one channel
-			createBuffer(2, PAGE_SIZE), createBuffer(2, PAGE_SIZE),
+			createBuffer(2), createBuffer(2),
 			createBarrier(3, 2),
-			createBuffer(2, PAGE_SIZE), createBuffer(2, PAGE_SIZE),
+			createBuffer(2), createBuffer(2),
 			createBarrier(3, 0), createBarrier(3, 1),
 
 			// empty checkpoint
 			createBarrier(4, 1), createBarrier(4, 2), createBarrier(4, 0),
 
 			// checkpoint with blocked data in mixed order
-			createBuffer(0, PAGE_SIZE), createBuffer(2, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			createBuffer(0), createBuffer(2), createBuffer(0),
 			createBarrier(5, 1),
-			createBuffer(2, PAGE_SIZE), createBuffer(0, PAGE_SIZE), createBuffer(2, PAGE_SIZE), createBuffer(1, PAGE_SIZE),
+			createBuffer(2), createBuffer(0), createBuffer(2), createBuffer(1),
 			createBarrier(5, 2),
-			createBuffer(1, PAGE_SIZE), createBuffer(0, PAGE_SIZE), createBuffer(2, PAGE_SIZE), createBuffer(1, PAGE_SIZE),
+			createBuffer(1), createBuffer(0), createBuffer(2), createBuffer(1),
 			createBarrier(5, 0),
 
 			// some trailing data
-			createBuffer(0, PAGE_SIZE),
+			createBuffer(0),
 			createEndOfPartition(0), createEndOfPartition(1), createEndOfPartition(2)
 		};
 		ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler();
@@ -294,14 +294,14 @@ public abstract class BarrierBufferTestBase {
 	@Test
 	public void testMultiChannelTrailingBlockedData() throws Exception {
 		BufferOrEvent[] sequence = {
-			createBuffer(0, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createBuffer(2, PAGE_SIZE),
+			createBuffer(0), createBuffer(1), createBuffer(2),
 			createBarrier(1, 1), createBarrier(1, 2), createBarrier(1, 0),
 
-			createBuffer(2, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			createBuffer(2), createBuffer(1), createBuffer(0),
 			createBarrier(2, 1),
-			createBuffer(1, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createEndOfPartition(1), createBuffer(0, PAGE_SIZE), createBuffer(2, PAGE_SIZE),
+			createBuffer(1), createBuffer(1), createEndOfPartition(1), createBuffer(0), createBuffer(2),
 			createBarrier(2, 2),
-			createBuffer(2, PAGE_SIZE), createEndOfPartition(2), createBuffer(0, PAGE_SIZE), createEndOfPartition(0)
+			createBuffer(2), createEndOfPartition(2), createBuffer(0), createEndOfPartition(0)
 		};
 		ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler();
 		buffer = createBarrierBuffer(3, sequence, handler);
@@ -345,38 +345,38 @@ public abstract class BarrierBufferTestBase {
 	public void testMultiChannelWithQueuedFutureBarriers() throws Exception{
 		BufferOrEvent[] sequence = {
 			// checkpoint 1 - with blocked data
-			createBuffer(0, PAGE_SIZE), createBuffer(2, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			createBuffer(0), createBuffer(2), createBuffer(0),
 			createBarrier(1, 1), createBarrier(1, 2),
-			createBuffer(2, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			createBuffer(2), createBuffer(1), createBuffer(0),
 			createBarrier(1, 0),
-			createBuffer(1, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			createBuffer(1), createBuffer(0),
 
 			// checkpoint 2 - where future checkpoint barriers come before
 			// the current checkpoint is complete
 			createBarrier(2, 1),
-			createBuffer(1, PAGE_SIZE), createBuffer(2, PAGE_SIZE), createBarrier(2, 0),
-			createBarrier(3, 0), createBuffer(0, PAGE_SIZE),
-			createBarrier(3, 1), createBuffer(0, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createBuffer(2, PAGE_SIZE),
-			createBarrier(4, 1), createBuffer(1, PAGE_SIZE), createBuffer(2, PAGE_SIZE),
+			createBuffer(1), createBuffer(2), createBarrier(2, 0),
+			createBarrier(3, 0), createBuffer(0),
+			createBarrier(3, 1), createBuffer(0), createBuffer(1), createBuffer(2),
+			createBarrier(4, 1), createBuffer(1), createBuffer(2),
 
 			// complete checkpoint 2, send a barrier for checkpoints 4 and 5
 			createBarrier(2, 2),
-			createBuffer(2, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createBuffer(2, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			createBuffer(2), createBuffer(1), createBuffer(2), createBuffer(0),
 			createBarrier(4, 0),
-			createBuffer(2, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createBuffer(2, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			createBuffer(2), createBuffer(1), createBuffer(2), createBuffer(0),
 			createBarrier(5, 1),
 
 			// complete checkpoint 3
 			createBarrier(3, 2),
-			createBuffer(2, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createBuffer(2, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			createBuffer(2), createBuffer(1), createBuffer(2), createBuffer(0),
 			createBarrier(6, 1),
 
 			// complete checkpoint 4, checkpoint 5 remains not fully triggered
 			createBarrier(4, 2),
-			createBuffer(2, PAGE_SIZE),
-			createBuffer(1, PAGE_SIZE), createEndOfPartition(1),
-			createBuffer(2, PAGE_SIZE), createEndOfPartition(2),
-			createBuffer(0, PAGE_SIZE), createEndOfPartition(0)
+			createBuffer(2),
+			createBuffer(1), createEndOfPartition(1),
+			createBuffer(2), createEndOfPartition(2),
+			createBuffer(0), createEndOfPartition(0)
 		};
 		ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler();
 		buffer = createBarrierBuffer(3, sequence, handler);
@@ -452,23 +452,23 @@ public abstract class BarrierBufferTestBase {
 	public void testMultiChannelSkippingCheckpoints() throws Exception {
 		BufferOrEvent[] sequence = {
 			// checkpoint 1 - with blocked data
-			createBuffer(0, PAGE_SIZE), createBuffer(2, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			createBuffer(0), createBuffer(2), createBuffer(0),
 			createBarrier(1, 1), createBarrier(1, 2),
-			createBuffer(2, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			createBuffer(2), createBuffer(1), createBuffer(0),
 			createBarrier(1, 0),
-			createBuffer(1, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			createBuffer(1), createBuffer(0),
 
 			// checkpoint 2 will not complete: pre-mature barrier from checkpoint 3
 			createBarrier(2, 1),
-			createBuffer(1, PAGE_SIZE), createBuffer(2, PAGE_SIZE),
+			createBuffer(1), createBuffer(2),
 			createBarrier(2, 0),
-			createBuffer(2, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			createBuffer(2), createBuffer(0),
 			createBarrier(3, 2),
 
-			createBuffer(2, PAGE_SIZE),
-			createBuffer(1, PAGE_SIZE), createEndOfPartition(1),
-			createBuffer(2, PAGE_SIZE), createEndOfPartition(2),
-			createBuffer(0, PAGE_SIZE), createEndOfPartition(0)
+			createBuffer(2),
+			createBuffer(1), createEndOfPartition(1),
+			createBuffer(2), createEndOfPartition(2),
+			createBuffer(0), createEndOfPartition(0)
 		};
 		AbstractInvokable toNotify = mock(AbstractInvokable.class);
 		buffer = createBarrierBuffer(3, sequence, toNotify);
@@ -533,27 +533,27 @@ public abstract class BarrierBufferTestBase {
 	public void testMultiChannelJumpingOverCheckpoint() throws Exception {
 		BufferOrEvent[] sequence = {
 			// checkpoint 1 - with blocked data
-			createBuffer(0, PAGE_SIZE), createBuffer(2, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			createBuffer(0), createBuffer(2), createBuffer(0),
 			createBarrier(1, 1), createBarrier(1, 2),
-			createBuffer(2, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			createBuffer(2), createBuffer(1), createBuffer(0),
 			createBarrier(1, 0),
-			createBuffer(1, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			createBuffer(1), createBuffer(0),
 
 			// checkpoint 2 will not complete: pre-mature barrier from checkpoint 3
 			createBarrier(2, 1),
-			createBuffer(1, PAGE_SIZE), createBuffer(2, PAGE_SIZE),
+			createBuffer(1), createBuffer(2),
 			createBarrier(2, 0),
-			createBuffer(2, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			createBuffer(2), createBuffer(0),
 			createBarrier(3, 1),
-			createBuffer(1, PAGE_SIZE), createBuffer(2, PAGE_SIZE),
+			createBuffer(1), createBuffer(2),
 			createBarrier(3, 0),
-			createBuffer(2, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			createBuffer(2), createBuffer(0),
 			createBarrier(4, 2),
 
-			createBuffer(2, PAGE_SIZE),
-			createBuffer(1, PAGE_SIZE), createEndOfPartition(1),
-			createBuffer(2, PAGE_SIZE), createEndOfPartition(2),
-			createBuffer(0, PAGE_SIZE), createEndOfPartition(0)
+			createBuffer(2),
+			createBuffer(1), createEndOfPartition(1),
+			createBuffer(2), createEndOfPartition(2),
+			createBuffer(0), createEndOfPartition(0)
 		};
 		ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler();
 		buffer = createBarrierBuffer(3, sequence, handler);
@@ -610,39 +610,39 @@ public abstract class BarrierBufferTestBase {
 	public void testMultiChannelSkippingCheckpointsViaBlockedInputs() throws Exception {
 		BufferOrEvent[] sequence = {
 			// checkpoint 1 - with blocked data
-			createBuffer(0, PAGE_SIZE), createBuffer(2, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			createBuffer(0), createBuffer(2), createBuffer(0),
 			createBarrier(1, 1), createBarrier(1, 2),
-			createBuffer(2, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			createBuffer(2), createBuffer(1), createBuffer(0),
 			createBarrier(1, 0),
-			createBuffer(1, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			createBuffer(1), createBuffer(0),
 
 			// checkpoint 2 will not complete: pre-mature barrier from checkpoint 3
 			createBarrier(2, 1),
-			createBuffer(1, PAGE_SIZE), createBuffer(2, PAGE_SIZE),
+			createBuffer(1), createBuffer(2),
 			createBarrier(2, 0),
-			createBuffer(1, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			createBuffer(1), createBuffer(0),
 
 			createBarrier(3, 0), // queued barrier on blocked input
-			createBuffer(0, PAGE_SIZE),
+			createBuffer(0),
 
 			createBarrier(4, 1), // pre-mature barrier on blocked input
-			createBuffer(1, PAGE_SIZE),
-			createBuffer(0, PAGE_SIZE),
-			createBuffer(2, PAGE_SIZE),
+			createBuffer(1),
+			createBuffer(0),
+			createBuffer(2),
 
 			// complete checkpoint 2
 			createBarrier(2, 2),
-			createBuffer(0, PAGE_SIZE),
+			createBuffer(0),
 
 			createBarrier(3, 2), // should be ignored
-			createBuffer(2, PAGE_SIZE),
+			createBuffer(2),
 			createBarrier(4, 0),
-			createBuffer(0, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createBuffer(2, PAGE_SIZE),
+			createBuffer(0), createBuffer(1), createBuffer(2),
 			createBarrier(4, 2),
 
-			createBuffer(1, PAGE_SIZE), createEndOfPartition(1),
-			createBuffer(2, PAGE_SIZE), createEndOfPartition(2),
-			createBuffer(0, PAGE_SIZE), createEndOfPartition(0)
+			createBuffer(1), createEndOfPartition(1),
+			createBuffer(2), createEndOfPartition(2),
+			createBuffer(0), createEndOfPartition(0)
 		};
 		buffer = createBarrierBuffer(3, sequence);
 
@@ -691,14 +691,14 @@ public abstract class BarrierBufferTestBase {
 	@Test
 	public void testEarlyCleanup() throws Exception {
 		BufferOrEvent[] sequence = {
-			createBuffer(0, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createBuffer(2, PAGE_SIZE),
+			createBuffer(0), createBuffer(1), createBuffer(2),
 			createBarrier(1, 1), createBarrier(1, 2), createBarrier(1, 0),
 
-			createBuffer(2, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			createBuffer(2), createBuffer(1), createBuffer(0),
 			createBarrier(2, 1),
-			createBuffer(1, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createEndOfPartition(1), createBuffer(0, PAGE_SIZE), createBuffer(2, PAGE_SIZE),
+			createBuffer(1), createBuffer(1), createEndOfPartition(1), createBuffer(0), createBuffer(2),
 			createBarrier(2, 2),
-			createBuffer(2, PAGE_SIZE), createEndOfPartition(2), createBuffer(0, PAGE_SIZE), createEndOfPartition(0)
+			createBuffer(2), createEndOfPartition(2), createBuffer(0), createEndOfPartition(0)
 		};
 		ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler();
 		buffer = createBarrierBuffer(3, sequence, handler);
@@ -738,26 +738,26 @@ public abstract class BarrierBufferTestBase {
 			createEndOfPartition(2), createEndOfPartition(1),
 
 			// checkpoint without blocked data
-			createBuffer(0, PAGE_SIZE), createBuffer(0, PAGE_SIZE), createBuffer(3, PAGE_SIZE),
+			createBuffer(0), createBuffer(0), createBuffer(3),
 			createBarrier(2, 3), createBarrier(2, 0),
 
 			// checkpoint with blocked data
-			createBuffer(3, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			createBuffer(3), createBuffer(0),
 			createBarrier(3, 3),
-			createBuffer(3, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			createBuffer(3), createBuffer(0),
 			createBarrier(3, 0),
 
 			// empty checkpoint
 			createBarrier(4, 0), createBarrier(4, 3),
 
 			// some data, one channel closes
-			createBuffer(0, PAGE_SIZE), createBuffer(0, PAGE_SIZE), createBuffer(3, PAGE_SIZE),
+			createBuffer(0), createBuffer(0), createBuffer(3),
 			createEndOfPartition(0),
 
 			// checkpoint on last remaining channel
-			createBuffer(3, PAGE_SIZE),
+			createBuffer(3),
 			createBarrier(5, 3),
-			createBuffer(3, PAGE_SIZE),
+			createBuffer(3),
 			createEndOfPartition(3)
 		};
 		buffer = createBarrierBuffer(4, sequence);
@@ -799,15 +799,15 @@ public abstract class BarrierBufferTestBase {
 			createBarrier(1, 0), createBarrier(1, 1), createBarrier(1, 2),
 
 			// some buffers
-			createBuffer(0, PAGE_SIZE), createBuffer(0, PAGE_SIZE), createBuffer(2, PAGE_SIZE),
+			createBuffer(0), createBuffer(0), createBuffer(2),
 
 			// start the checkpoint that will be incomplete
 			createBarrier(2, 2), createBarrier(2, 0),
-			createBuffer(0, PAGE_SIZE), createBuffer(2, PAGE_SIZE), createBuffer(1, PAGE_SIZE),
+			createBuffer(0), createBuffer(2), createBuffer(1),
 
 			// close one after the barrier one before the barrier
 			createEndOfPartition(2), createEndOfPartition(1),
-			createBuffer(0, PAGE_SIZE),
+			createBuffer(0),
 
 			// final end of stream
 			createEndOfPartition(0)
@@ -836,15 +836,15 @@ public abstract class BarrierBufferTestBase {
 	@Test
 	public void testSingleChannelAbortCheckpoint() throws Exception {
 		BufferOrEvent[] sequence = {
-			createBuffer(0, PAGE_SIZE),
+			createBuffer(0),
 			createBarrier(1, 0),
-			createBuffer(0, PAGE_SIZE),
+			createBuffer(0),
 			createBarrier(2, 0),
 			createCancellationBarrier(4, 0),
 			createBarrier(5, 0),
-			createBuffer(0, PAGE_SIZE),
+			createBuffer(0),
 			createCancellationBarrier(6, 0),
-			createBuffer(0, PAGE_SIZE)
+			createBuffer(0)
 		};
 		AbstractInvokable toNotify = mock(AbstractInvokable.class);
 		buffer = createBarrierBuffer(1, sequence, toNotify);
@@ -873,37 +873,37 @@ public abstract class BarrierBufferTestBase {
 	public void testMultiChannelAbortCheckpoint() throws Exception {
 		BufferOrEvent[] sequence = {
 				// some buffers and a successful checkpoint
-			/* 0 */ createBuffer(0, PAGE_SIZE), createBuffer(2, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			/* 0 */ createBuffer(0), createBuffer(2), createBuffer(0),
 			/* 3 */ createBarrier(1, 1), createBarrier(1, 2),
-			/* 5 */ createBuffer(2, PAGE_SIZE), createBuffer(1, PAGE_SIZE),
+			/* 5 */ createBuffer(2), createBuffer(1),
 			/* 7 */ createBarrier(1, 0),
-			/* 8 */ createBuffer(0, PAGE_SIZE), createBuffer(2, PAGE_SIZE),
+			/* 8 */ createBuffer(0), createBuffer(2),
 
 				// aborted on last barrier
 			/* 10 */ createBarrier(2, 0), createBarrier(2, 2),
-			/* 12 */ createBuffer(0, PAGE_SIZE), createBuffer(2, PAGE_SIZE),
+			/* 12 */ createBuffer(0), createBuffer(2),
 			/* 14 */ createCancellationBarrier(2, 1),
 
 				// successful checkpoint
-			/* 15 */ createBuffer(2, PAGE_SIZE), createBuffer(1, PAGE_SIZE),
+			/* 15 */ createBuffer(2), createBuffer(1),
 			/* 17 */ createBarrier(3, 1), createBarrier(3, 2), createBarrier(3, 0),
 
 				// abort on first barrier
-			/* 20 */ createBuffer(0, PAGE_SIZE), createBuffer(1, PAGE_SIZE),
+			/* 20 */ createBuffer(0), createBuffer(1),
 			/* 22 */ createCancellationBarrier(4, 1), createBarrier(4, 2),
-			/* 24 */ createBuffer(0, PAGE_SIZE),
+			/* 24 */ createBuffer(0),
 			/* 25 */ createBarrier(4, 0),
 
 				// another successful checkpoint
-			/* 26 */ createBuffer(0, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createBuffer(2, PAGE_SIZE),
+			/* 26 */ createBuffer(0), createBuffer(1), createBuffer(2),
 			/* 29 */ createBarrier(5, 2), createBarrier(5, 1), createBarrier(5, 0),
-			/* 32 */ createBuffer(0, PAGE_SIZE), createBuffer(1, PAGE_SIZE),
+			/* 32 */ createBuffer(0), createBuffer(1),
 
 				// abort multiple cancellations and a barrier after the cancellations
 			/* 34 */ createCancellationBarrier(6, 1), createCancellationBarrier(6, 2),
 			/* 36 */ createBarrier(6, 0),
 
-			/* 37 */ createBuffer(0, PAGE_SIZE)
+			/* 37 */ createBuffer(0)
 		};
 		AbstractInvokable toNotify = mock(AbstractInvokable.class);
 		buffer = createBarrierBuffer(3, sequence, toNotify);
@@ -968,28 +968,28 @@ public abstract class BarrierBufferTestBase {
 	public void testAbortViaQueuedBarriers() throws Exception {
 		BufferOrEvent[] sequence = {
 				// starting a checkpoint
-			/* 0 */ createBuffer(1, PAGE_SIZE),
+			/* 0 */ createBuffer(1),
 			/* 1 */ createBarrier(1, 1), createBarrier(1, 2),
-			/* 3 */ createBuffer(2, PAGE_SIZE), createBuffer(0, PAGE_SIZE), createBuffer(1, PAGE_SIZE),
+			/* 3 */ createBuffer(2), createBuffer(0), createBuffer(1),
 
 				// queued barrier and cancellation barrier
 			/* 6 */ createCancellationBarrier(2, 2),
 			/* 7 */ createBarrier(2, 1),
 
 				// some intermediate buffers (some queued)
-			/* 8 */ createBuffer(0, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createBuffer(2, PAGE_SIZE),
+			/* 8 */ createBuffer(0), createBuffer(1), createBuffer(2),
 
 				// complete initial checkpoint
 			/* 11 */ createBarrier(1, 0),
 
 				// some buffers (none queued, since checkpoint is aborted)
-			/* 12 */ createBuffer(2, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			/* 12 */ createBuffer(2), createBuffer(1), createBuffer(0),
 
 				// final barrier of aborted checkpoint
 			/* 15 */ createBarrier(2, 0),
 
 				// some more buffers
-			/* 16 */ createBuffer(0, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createBuffer(2, PAGE_SIZE)
+			/* 16 */ createBuffer(0), createBuffer(1), createBuffer(2)
 		};
 		AbstractInvokable toNotify = mock(AbstractInvokable.class);
 		buffer = createBarrierBuffer(3, sequence, toNotify);
@@ -1045,33 +1045,33 @@ public abstract class BarrierBufferTestBase {
 	public void testAbortWhileHavingQueuedBarriers() throws Exception {
 		BufferOrEvent[] sequence = {
 				// starting a checkpoint
-			/*  0 */ createBuffer(1, PAGE_SIZE),
+			/*  0 */ createBuffer(1),
 			/*  1 */ createBarrier(1, 1),
-			/*  2 */ createBuffer(2, PAGE_SIZE), createBuffer(0, PAGE_SIZE), createBuffer(1, PAGE_SIZE),
+			/*  2 */ createBuffer(2), createBuffer(0), createBuffer(1),
 
 				// queued barrier and cancellation barrier
 			/*  5 */ createBarrier(2, 1),
 
 				// some queued buffers
-			/*  6 */ createBuffer(2, PAGE_SIZE), createBuffer(1, PAGE_SIZE),
+			/*  6 */ createBuffer(2), createBuffer(1),
 
 				// cancel the initial checkpoint
 			/*  8 */ createCancellationBarrier(1, 0),
 
 				// some more buffers
-			/*  9 */ createBuffer(2, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			/*  9 */ createBuffer(2), createBuffer(1), createBuffer(0),
 
 				// ignored barrier - already canceled and moved to next checkpoint
 			/* 12 */ createBarrier(1, 2),
 
 				// some more buffers
-			/* 13 */ createBuffer(0, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createBuffer(2, PAGE_SIZE),
+			/* 13 */ createBuffer(0), createBuffer(1), createBuffer(2),
 
 				// complete next checkpoint regularly
 			/* 16 */ createBarrier(2, 0), createBarrier(2, 2),
 
 				// some more buffers
-			/* 18 */ createBuffer(0, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createBuffer(2, PAGE_SIZE)
+			/* 18 */ createBuffer(0), createBuffer(1), createBuffer(2)
 		};
 		AbstractInvokable toNotify = mock(AbstractInvokable.class);
 		buffer = createBarrierBuffer(3, sequence, toNotify);
@@ -1126,27 +1126,27 @@ public abstract class BarrierBufferTestBase {
 	public void testIgnoreCancelBarrierIfCheckpointSubsumed() throws Exception {
 		BufferOrEvent[] sequence = {
 				// starting a checkpoint
-			/*  0 */ createBuffer(2, PAGE_SIZE),
+			/*  0 */ createBuffer(2),
 			/*  1 */ createBarrier(3, 1), createBarrier(3, 0),
-			/*  3 */ createBuffer(0, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createBuffer(2, PAGE_SIZE),
+			/*  3 */ createBuffer(0), createBuffer(1), createBuffer(2),
 
 				// newer checkpoint barrier cancels/subsumes pending checkpoint
 			/*  6 */ createBarrier(5, 2),
 
 				// some queued buffers
-			/*  7 */ createBuffer(2, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createBuffer(0, PAGE_SIZE),
+			/*  7 */ createBuffer(2), createBuffer(1), createBuffer(0),
 
 				// cancel barrier the initial checkpoint /it is already canceled)
 			/* 10 */ createCancellationBarrier(3, 2),
 
 				// some more buffers
-			/* 11 */ createBuffer(2, PAGE_SIZE), createBuffer(0, PAGE_SIZE), createBuffer(1, PAGE_SIZE),
+			/* 11 */ createBuffer(2), createBuffer(0), createBuffer(1),
 
 				// complete next checkpoint regularly
 			/* 14 */ createBarrier(5, 0), createBarrier(5, 1),
 
 				// some more buffers
-			/* 16 */ createBuffer(0, PAGE_SIZE), createBuffer(1, PAGE_SIZE), createBuffer(2, PAGE_SIZE)
+			/* 16 */ createBuffer(0), createBuffer(1), createBuffer(2)
 		};
 		AbstractInvokable toNotify = mock(AbstractInvokable.class);
 		buffer = createBarrierBuffer(3, sequence, toNotify);
@@ -1202,12 +1202,12 @@ public abstract class BarrierBufferTestBase {
 		return new BufferOrEvent(new CancelCheckpointMarker(checkpointId), channel);
 	}
 
-	private static BufferOrEvent createBuffer(int channel, int pageSize) {
+	private static BufferOrEvent createBuffer(int channel) {
 		final int size = sizeCounter++;
 		byte[] bytes = new byte[size];
 		RND.nextBytes(bytes);
 
-		MemorySegment memory = MemorySegmentFactory.allocateUnpooledSegment(pageSize);
+		MemorySegment memory = MemorySegmentFactory.allocateUnpooledSegment(PAGE_SIZE);
 		memory.put(0, bytes);
 
 		Buffer buf = new NetworkBuffer(memory, FreeingBufferRecycler.INSTANCE);


[flink] 09/16: [FLINK-12777][network] Rename existing classes to make them in sync with the refactor

Posted by pn...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pnowojski pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 0d502b6addffdc23a4826796c630bf7f9dbae718
Author: Piotr Nowojski <pi...@gmail.com>
AuthorDate: Tue Jun 18 10:41:11 2019 +0200

    [FLINK-12777][network] Rename existing classes to make them in sync with the refactor
    
    1. Rename BarrierBuffer to CheckpointedInputGate
    CheckpointedInputGate was an interface, while BarrierBuffer was
    it's implementation. This rename means that we are dropping the interface
    and keeping only the concrete class.
    
    2. Rename BarrierBuffer and BarrierTracker tests to match this rename
    and previous refactorings.
---
 .../flink/streaming/runtime/io/BarrierBuffer.java  | 243 --------
 .../streaming/runtime/io/CachedBufferStorage.java  |   2 +-
 .../runtime/io/CheckpointedInputGate.java          | 228 ++++++-
 .../streaming/runtime/io/InputProcessorUtil.java   |  12 +-
 .../streaming/runtime/io/StreamInputProcessor.java |   2 +-
 .../runtime/io/StreamTaskNetworkInput.java         |  22 +-
 .../runtime/io/StreamTwoInputProcessor.java        |   2 +-
 .../io/StreamTwoInputSelectableProcessor.java      |   4 +-
 ...heckpointBarrierAlignerAlignmentLimitTest.java} |  15 +-
 ...CheckpointBarrierAlignerMassiveRandomTest.java} |   6 +-
 ....java => CheckpointBarrierAlignerTestBase.java} | 660 ++++++++++-----------
 ...Test.java => CheckpointBarrierTrackerTest.java} |  54 +-
 ...> CreditBasedCheckpointBarrierAlignerTest.java} |   8 +-
 ...a => SpillingCheckpointBarrierAlignerTest.java} |   8 +-
 .../tasks/StreamTaskCancellationBarrierTest.java   |   7 +-
 15 files changed, 614 insertions(+), 659 deletions(-)

diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
deleted file mode 100644
index 8dcc005..0000000
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
+++ /dev/null
@@ -1,243 +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.streaming.runtime.io;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker;
-import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
-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.jobgraph.tasks.AbstractInvokable;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.annotation.Nullable;
-
-import java.io.IOException;
-import java.util.Optional;
-import java.util.concurrent.CompletableFuture;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * The barrier buffer is {@link CheckpointedInputGate} that blocks inputs with barriers until
- * all inputs have received the barrier for a given checkpoint.
- *
- * <p>To avoid back-pressuring the input streams (which may cause distributed deadlocks), the
- * BarrierBuffer continues receiving buffers from the blocked channels and stores them internally until
- * the blocks are released.
- */
-@Internal
-public class BarrierBuffer implements CheckpointedInputGate {
-
-	private static final Logger LOG = LoggerFactory.getLogger(BarrierBuffer.class);
-
-	private final CheckpointBarrierHandler barrierHandler;
-
-	/** The gate that the buffer draws its input from. */
-	private final InputGate inputGate;
-
-	private final BufferStorage bufferStorage;
-
-	/** Flag to indicate whether we have drawn all available input. */
-	private boolean endOfInputGate;
-
-	/** Indicate end of the input. Set to true after encountering {@link #endOfInputGate} and depleting
-	 * {@link #bufferStorage}. */
-	private boolean isFinished;
-
-	/**
-	 * Creates a new checkpoint stream aligner.
-	 *
-	 * <p>There is no limit to how much data may be buffered during an alignment.
-	 *
-	 * @param inputGate The input gate to draw the buffers and events from.
-	 * @param bufferStorage The storage to hold the buffers and events for blocked channels.
-	 */
-	@VisibleForTesting
-	BarrierBuffer(InputGate inputGate, BufferStorage bufferStorage) {
-		this (inputGate, bufferStorage, "Testing: No task associated", null);
-	}
-
-	BarrierBuffer(
-			InputGate inputGate,
-			BufferStorage bufferStorage,
-			String taskName,
-			@Nullable AbstractInvokable toNotifyOnCheckpoint) {
-		this(
-			inputGate,
-			bufferStorage,
-			new CheckpointBarrierAligner(
-				inputGate.getNumberOfInputChannels(),
-				taskName,
-				toNotifyOnCheckpoint)
-		);
-	}
-
-	/**
-	 * Creates a new checkpoint stream aligner.
-	 *
-	 * <p>The aligner will allow only alignments that buffer up to the given number of bytes.
-	 * When that number is exceeded, it will stop the alignment and notify the task that the
-	 * checkpoint has been cancelled.
-	 *
-	 * @param inputGate The input gate to draw the buffers and events from.
-	 * @param bufferStorage The storage to hold the buffers and events for blocked channels.
-	 * @param barrierHandler Handler that controls which channels are blocked.
-	 */
-	BarrierBuffer(
-			InputGate inputGate,
-			BufferStorage bufferStorage,
-			CheckpointBarrierHandler barrierHandler) {
-		this.inputGate = inputGate;
-		this.bufferStorage = checkNotNull(bufferStorage);
-		this.barrierHandler = barrierHandler;
-	}
-
-	@Override
-	public CompletableFuture<?> isAvailable() {
-		if (bufferStorage.isEmpty()) {
-			return inputGate.isAvailable();
-		}
-		return AVAILABLE;
-	}
-
-	@Override
-	public Optional<BufferOrEvent> pollNext() throws Exception {
-		while (true) {
-			// process buffered BufferOrEvents before grabbing new ones
-			Optional<BufferOrEvent> next;
-			if (bufferStorage.isEmpty()) {
-				next = inputGate.pollNext();
-			}
-			else {
-				// TODO: FLINK-12536 for non credit-based flow control, getNext method is blocking
-				next = bufferStorage.pollNext();
-				if (!next.isPresent()) {
-					return pollNext();
-				}
-			}
-
-			if (!next.isPresent()) {
-				return handleEmptyBuffer();
-			}
-
-			BufferOrEvent bufferOrEvent = next.get();
-			if (barrierHandler.isBlocked(bufferOrEvent.getChannelIndex())) {
-				// if the channel is blocked, we just store the BufferOrEvent
-				bufferStorage.add(bufferOrEvent);
-				if (bufferStorage.isFull()) {
-					barrierHandler.checkpointSizeLimitExceeded(bufferStorage.getMaxBufferedBytes());
-					bufferStorage.rollOver();
-				}
-			}
-			else if (bufferOrEvent.isBuffer()) {
-				return next;
-			}
-			else if (bufferOrEvent.getEvent().getClass() == CheckpointBarrier.class) {
-				CheckpointBarrier checkpointBarrier = (CheckpointBarrier) bufferOrEvent.getEvent();
-				if (!endOfInputGate) {
-					// process barriers only if there is a chance of the checkpoint completing
-					if (barrierHandler.processBarrier(checkpointBarrier, bufferOrEvent.getChannelIndex(), bufferStorage.getPendingBytes())) {
-						bufferStorage.rollOver();
-					}
-				}
-			}
-			else if (bufferOrEvent.getEvent().getClass() == CancelCheckpointMarker.class) {
-				if (barrierHandler.processCancellationBarrier((CancelCheckpointMarker) bufferOrEvent.getEvent())) {
-					bufferStorage.rollOver();
-				}
-			}
-			else {
-				if (bufferOrEvent.getEvent().getClass() == EndOfPartitionEvent.class) {
-					if (barrierHandler.processEndOfPartition()) {
-						bufferStorage.rollOver();
-					}
-				}
-				return next;
-			}
-		}
-	}
-
-	private Optional<BufferOrEvent> handleEmptyBuffer() throws Exception {
-		if (!inputGate.isFinished()) {
-			return Optional.empty();
-		}
-
-		if (endOfInputGate) {
-			isFinished = true;
-			return Optional.empty();
-		} else {
-			// end of input stream. stream continues with the buffered data
-			endOfInputGate = true;
-			barrierHandler.releaseBlocksAndResetBarriers();
-			bufferStorage.rollOver();
-			return pollNext();
-		}
-	}
-
-	@Override
-	public boolean isEmpty() {
-		return bufferStorage.isEmpty();
-	}
-
-	@Override
-	public boolean isFinished() {
-		return isFinished;
-	}
-
-	@Override
-	public void cleanup() throws IOException {
-		bufferStorage.close();
-	}
-
-	// ------------------------------------------------------------------------
-	//  Properties
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Gets the ID defining the current pending, or just completed, checkpoint.
-	 *
-	 * @return The ID of the pending of completed checkpoint.
-	 */
-	public long getLatestCheckpointId() {
-		return barrierHandler.getLatestCheckpointId();
-	}
-
-	@Override
-	public long getAlignmentDurationNanos() {
-		return barrierHandler.getAlignmentDurationNanos();
-	}
-
-	@Override
-	public int getNumberOfInputChannels() {
-		return inputGate.getNumberOfInputChannels();
-	}
-
-	// ------------------------------------------------------------------------
-	// Utilities
-	// ------------------------------------------------------------------------
-
-	@Override
-	public String toString() {
-		return barrierHandler.toString();
-	}
-}
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CachedBufferStorage.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CachedBufferStorage.java
index 628a69c..4927c35 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CachedBufferStorage.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CachedBufferStorage.java
@@ -118,7 +118,7 @@ public class CachedBufferStorage extends AbstractBufferStorage {
 	 */
 	public static class CachedBufferOrEventSequence implements BufferOrEventSequence {
 
-		/** The sequence of buffers and events to be consumed by {@link BarrierBuffer}.*/
+		/** The sequence of buffers and events to be consumed by {@link CheckpointedInputGate}.*/
 		private final ArrayDeque<BufferOrEvent> queuedBuffers;
 
 		/** The total size of the cached data. */
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java
index cdbbfbc..7604d0a 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java
@@ -1,13 +1,12 @@
 /*
- * 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
+ * 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
+ *    http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -19,29 +18,213 @@
 package org.apache.flink.streaming.runtime.io;
 
 import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.runtime.io.AsyncDataInput;
+import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker;
+import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
+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.jobgraph.tasks.AbstractInvokable;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
 
 import java.io.IOException;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
  * The {@link CheckpointedInputGate} uses {@link CheckpointBarrierHandler} to handle incoming
- * {@link org.apache.flink.runtime.io.network.api.CheckpointBarrier} from the {@link org.apache.flink.runtime.io.network.partition.consumer.InputGate}.
+ * {@link CheckpointBarrier} from the {@link InputGate}.
  */
 @Internal
-public interface CheckpointedInputGate extends AsyncDataInput<BufferOrEvent> {
+public class CheckpointedInputGate implements AsyncDataInput<BufferOrEvent> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(CheckpointedInputGate.class);
+
+	private final CheckpointBarrierHandler barrierHandler;
+
+	/** The gate that the buffer draws its input from. */
+	private final InputGate inputGate;
+
+	private final BufferStorage bufferStorage;
+
+	/** Flag to indicate whether we have drawn all available input. */
+	private boolean endOfInputGate;
+
+	/** Indicate end of the input. Set to true after encountering {@link #endOfInputGate} and depleting
+	 * {@link #bufferStorage}. */
+	private boolean isFinished;
+
 	/**
-	 * Cleans up all internally held resources.
+	 * Creates a new checkpoint stream aligner.
 	 *
-	 * @throws IOException Thrown if the cleanup of I/O resources failed.
+	 * <p>There is no limit to how much data may be buffered during an alignment.
+	 *
+	 * @param inputGate The input gate to draw the buffers and events from.
+	 * @param bufferStorage The storage to hold the buffers and events for blocked channels.
 	 */
-	void cleanup() throws IOException;
+	@VisibleForTesting
+	CheckpointedInputGate(InputGate inputGate, BufferStorage bufferStorage) {
+		this (inputGate, bufferStorage, "Testing: No task associated", null);
+	}
+
+	public CheckpointedInputGate(
+			InputGate inputGate,
+			BufferStorage bufferStorage,
+			String taskName,
+			@Nullable AbstractInvokable toNotifyOnCheckpoint) {
+		this(
+			inputGate,
+			bufferStorage,
+			new CheckpointBarrierAligner(
+				inputGate.getNumberOfInputChannels(),
+				taskName,
+				toNotifyOnCheckpoint)
+		);
+	}
+
+	/**
+	 * Creates a new checkpoint stream aligner.
+	 *
+	 * <p>The aligner will allow only alignments that buffer up to the given number of bytes.
+	 * When that number is exceeded, it will stop the alignment and notify the task that the
+	 * checkpoint has been cancelled.
+	 *
+	 * @param inputGate The input gate to draw the buffers and events from.
+	 * @param bufferStorage The storage to hold the buffers and events for blocked channels.
+	 * @param barrierHandler Handler that controls which channels are blocked.
+	 */
+	public CheckpointedInputGate(
+			InputGate inputGate,
+			BufferStorage bufferStorage,
+			CheckpointBarrierHandler barrierHandler) {
+		this.inputGate = inputGate;
+		this.bufferStorage = checkNotNull(bufferStorage);
+		this.barrierHandler = barrierHandler;
+	}
+
+	@Override
+	public CompletableFuture<?> isAvailable() {
+		if (bufferStorage.isEmpty()) {
+			return inputGate.isAvailable();
+		}
+		return AVAILABLE;
+	}
+
+	@Override
+	public Optional<BufferOrEvent> pollNext() throws Exception {
+		while (true) {
+			// process buffered BufferOrEvents before grabbing new ones
+			Optional<BufferOrEvent> next;
+			if (bufferStorage.isEmpty()) {
+				next = inputGate.pollNext();
+			}
+			else {
+				// TODO: FLINK-12536 for non credit-based flow control, getNext method is blocking
+				next = bufferStorage.pollNext();
+				if (!next.isPresent()) {
+					return pollNext();
+				}
+			}
+
+			if (!next.isPresent()) {
+				return handleEmptyBuffer();
+			}
+
+			BufferOrEvent bufferOrEvent = next.get();
+			if (barrierHandler.isBlocked(bufferOrEvent.getChannelIndex())) {
+				// if the channel is blocked, we just store the BufferOrEvent
+				bufferStorage.add(bufferOrEvent);
+				if (bufferStorage.isFull()) {
+					barrierHandler.checkpointSizeLimitExceeded(bufferStorage.getMaxBufferedBytes());
+					bufferStorage.rollOver();
+				}
+			}
+			else if (bufferOrEvent.isBuffer()) {
+				return next;
+			}
+			else if (bufferOrEvent.getEvent().getClass() == CheckpointBarrier.class) {
+				CheckpointBarrier checkpointBarrier = (CheckpointBarrier) bufferOrEvent.getEvent();
+				if (!endOfInputGate) {
+					// process barriers only if there is a chance of the checkpoint completing
+					if (barrierHandler.processBarrier(checkpointBarrier, bufferOrEvent.getChannelIndex(), bufferStorage.getPendingBytes())) {
+						bufferStorage.rollOver();
+					}
+				}
+			}
+			else if (bufferOrEvent.getEvent().getClass() == CancelCheckpointMarker.class) {
+				if (barrierHandler.processCancellationBarrier((CancelCheckpointMarker) bufferOrEvent.getEvent())) {
+					bufferStorage.rollOver();
+				}
+			}
+			else {
+				if (bufferOrEvent.getEvent().getClass() == EndOfPartitionEvent.class) {
+					if (barrierHandler.processEndOfPartition()) {
+						bufferStorage.rollOver();
+					}
+				}
+				return next;
+			}
+		}
+	}
+
+	private Optional<BufferOrEvent> handleEmptyBuffer() throws Exception {
+		if (!inputGate.isFinished()) {
+			return Optional.empty();
+		}
+
+		if (endOfInputGate) {
+			isFinished = true;
+			return Optional.empty();
+		} else {
+			// end of input stream. stream continues with the buffered data
+			endOfInputGate = true;
+			barrierHandler.releaseBlocksAndResetBarriers();
+			bufferStorage.rollOver();
+			return pollNext();
+		}
+	}
 
 	/**
 	 * Checks if the barrier handler has buffered any data internally.
 	 * @return {@code True}, if no data is buffered internally, {@code false} otherwise.
 	 */
-	boolean isEmpty();
+	public boolean isEmpty() {
+		return bufferStorage.isEmpty();
+	}
+
+	@Override
+	public boolean isFinished() {
+		return isFinished;
+	}
+
+	/**
+	 * Cleans up all internally held resources.
+	 *
+	 * @throws IOException Thrown if the cleanup of I/O resources failed.
+	 */
+	public void cleanup() throws IOException {
+		bufferStorage.close();
+	}
+
+	// ------------------------------------------------------------------------
+	//  Properties
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Gets the ID defining the current pending, or just completed, checkpoint.
+	 *
+	 * @return The ID of the pending of completed checkpoint.
+	 */
+	public long getLatestCheckpointId() {
+		return barrierHandler.getLatestCheckpointId();
+	}
 
 	/**
 	 * Gets the time that the latest alignment took, in nanoseconds.
@@ -50,10 +233,23 @@ public interface CheckpointedInputGate extends AsyncDataInput<BufferOrEvent> {
 	 *
 	 * @return The duration in nanoseconds
 	 */
-	long getAlignmentDurationNanos();
+	public long getAlignmentDurationNanos() {
+		return barrierHandler.getAlignmentDurationNanos();
+	}
 
 	/**
 	 * @return number of underlying input channels.
 	 */
-	int getNumberOfInputChannels();
+	public int getNumberOfInputChannels() {
+		return inputGate.getNumberOfInputChannels();
+	}
+
+	// ------------------------------------------------------------------------
+	// Utilities
+	// ------------------------------------------------------------------------
+
+	@Override
+	public String toString() {
+		return barrierHandler.toString();
+	}
 }
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java
index 75926b9..7eda06c 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java
@@ -36,7 +36,7 @@ import java.io.IOException;
 @Internal
 public class InputProcessorUtil {
 
-	public static CheckpointedInputGate createCheckpointBarrierHandler(
+	public static CheckpointedInputGate createCheckpointedInputGate(
 			StreamTask<?, ?> checkpointedTask,
 			CheckpointingMode checkpointMode,
 			IOManager ioManager,
@@ -44,7 +44,7 @@ public class InputProcessorUtil {
 			Configuration taskManagerConfig,
 			String taskName) throws IOException {
 
-		CheckpointedInputGate barrierHandler;
+		CheckpointedInputGate checkpointedInputGate;
 		if (checkpointMode == CheckpointingMode.EXACTLY_ONCE) {
 			long maxAlign = taskManagerConfig.getLong(TaskManagerOptions.TASK_CHECKPOINT_ALIGNMENT_BYTES_LIMIT);
 			if (!(maxAlign == -1 || maxAlign > 0)) {
@@ -54,20 +54,20 @@ public class InputProcessorUtil {
 			}
 
 			if (taskManagerConfig.getBoolean(NettyShuffleEnvironmentOptions.NETWORK_CREDIT_MODEL)) {
-				barrierHandler = new BarrierBuffer(
+				checkpointedInputGate = new CheckpointedInputGate(
 					inputGate,
 					new CachedBufferStorage(inputGate.getPageSize(), maxAlign, taskName),
 					taskName,
 					checkpointedTask);
 			} else {
-				barrierHandler = new BarrierBuffer(
+				checkpointedInputGate = new CheckpointedInputGate(
 					inputGate,
 					new BufferSpiller(ioManager, inputGate.getPageSize(), maxAlign, taskName),
 					taskName,
 					checkpointedTask);
 			}
 		} else if (checkpointMode == CheckpointingMode.AT_LEAST_ONCE) {
-			barrierHandler = new BarrierBuffer(
+			checkpointedInputGate = new CheckpointedInputGate(
 				inputGate,
 				new EmptyBufferStorage(),
 				new CheckpointBarrierTracker(inputGate.getNumberOfInputChannels(), checkpointedTask));
@@ -75,6 +75,6 @@ public class InputProcessorUtil {
 			throw new IllegalArgumentException("Unrecognized Checkpointing Mode: " + checkpointMode);
 		}
 
-		return barrierHandler;
+		return checkpointedInputGate;
 	}
 }
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
index 58b2051..d6fcad2 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
@@ -104,7 +104,7 @@ public class StreamInputProcessor<IN> {
 
 		InputGate inputGate = InputGateUtil.createInputGate(inputGates);
 
-		CheckpointedInputGate barrierHandler = InputProcessorUtil.createCheckpointBarrierHandler(
+		CheckpointedInputGate barrierHandler = InputProcessorUtil.createCheckpointedInputGate(
 			checkpointedTask,
 			checkpointMode,
 			ioManager,
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java
index ecf88e2..8c37141 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java
@@ -47,7 +47,7 @@ import static org.apache.flink.util.Preconditions.checkState;
 @Internal
 public final class StreamTaskNetworkInput implements StreamTaskInput {
 
-	private final CheckpointedInputGate barrierHandler;
+	private final CheckpointedInputGate checkpointedInputGate;
 
 	private final DeserializationDelegate<StreamElement> deserializationDelegate;
 
@@ -63,16 +63,16 @@ public final class StreamTaskNetworkInput implements StreamTaskInput {
 
 	@SuppressWarnings("unchecked")
 	public StreamTaskNetworkInput(
-			CheckpointedInputGate barrierHandler,
+			CheckpointedInputGate checkpointedInputGate,
 			TypeSerializer<?> inputSerializer,
 			IOManager ioManager,
 			int inputIndex) {
-		this.barrierHandler = barrierHandler;
+		this.checkpointedInputGate = checkpointedInputGate;
 		this.deserializationDelegate = new NonReusingDeserializationDelegate<>(
 			new StreamElementSerializer<>(inputSerializer));
 
 		// Initialize one deserializer per input channel
-		this.recordDeserializers = new SpillingAdaptiveSpanningRecordDeserializer[barrierHandler.getNumberOfInputChannels()];
+		this.recordDeserializers = new SpillingAdaptiveSpanningRecordDeserializer[checkpointedInputGate.getNumberOfInputChannels()];
 		for (int i = 0; i < recordDeserializers.length; i++) {
 			recordDeserializers[i] = new SpillingAdaptiveSpanningRecordDeserializer<>(
 				ioManager.getSpillingDirectoriesPaths());
@@ -99,14 +99,14 @@ public final class StreamTaskNetworkInput implements StreamTaskInput {
 				}
 			}
 
-			Optional<BufferOrEvent> bufferOrEvent = barrierHandler.pollNext();
+			Optional<BufferOrEvent> bufferOrEvent = checkpointedInputGate.pollNext();
 			if (bufferOrEvent.isPresent()) {
 				processBufferOrEvent(bufferOrEvent.get());
 			} else {
-				if (barrierHandler.isFinished()) {
+				if (checkpointedInputGate.isFinished()) {
 					isFinished = true;
-					checkState(barrierHandler.isAvailable().isDone(), "Finished BarrierHandler should be available");
-					if (!barrierHandler.isEmpty()) {
+					checkState(checkpointedInputGate.isAvailable().isDone(), "Finished BarrierHandler should be available");
+					if (!checkpointedInputGate.isEmpty()) {
 						throw new IllegalStateException("Trailing data in checkpoint barrier handler.");
 					}
 				}
@@ -124,7 +124,7 @@ public final class StreamTaskNetworkInput implements StreamTaskInput {
 		else {
 			// Event received
 			final AbstractEvent event = bufferOrEvent.getEvent();
-			// TODO: with barrierHandler.isFinished() we might not need to support any events on this level.
+			// TODO: with checkpointedInputGate.isFinished() we might not need to support any events on this level.
 			if (event.getClass() != EndOfPartitionEvent.class) {
 				throw new IOException("Unexpected event: " + event);
 			}
@@ -148,7 +148,7 @@ public final class StreamTaskNetworkInput implements StreamTaskInput {
 
 	@Override
 	public CompletableFuture<?> isAvailable() {
-		return barrierHandler.isAvailable();
+		return checkpointedInputGate.isAvailable();
 	}
 
 	@Override
@@ -162,6 +162,6 @@ public final class StreamTaskNetworkInput implements StreamTaskInput {
 			deserializer.clear();
 		}
 
-		barrierHandler.cleanup();
+		checkpointedInputGate.cleanup();
 	}
 }
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java
index aa6354d..f989e57 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java
@@ -154,7 +154,7 @@ public class StreamTwoInputProcessor<IN1, IN2> {
 
 		final InputGate inputGate = InputGateUtil.createInputGate(inputGates1, inputGates2);
 
-		this.barrierHandler = InputProcessorUtil.createCheckpointBarrierHandler(
+		this.barrierHandler = InputProcessorUtil.createCheckpointedInputGate(
 			checkpointedTask,
 			checkpointMode,
 			ioManager,
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputSelectableProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputSelectableProcessor.java
index d5ebf29..37c17db 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputSelectableProcessor.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputSelectableProcessor.java
@@ -121,8 +121,8 @@ public class StreamTwoInputSelectableProcessor<IN1, IN2> {
 
 		// create a Input instance for each input
 		CachedBufferStorage bufferStorage = new CachedBufferStorage(unionedInputGate1.getPageSize());
-		this.input1 = new StreamTaskNetworkInput(new BarrierBuffer(unionedInputGate1, bufferStorage, new CheckpointBarrierDiscarder()), inputSerializer1, ioManager, 0);
-		this.input2 = new StreamTaskNetworkInput(new BarrierBuffer(unionedInputGate2, bufferStorage, new CheckpointBarrierDiscarder()), inputSerializer2, ioManager, 1);
+		this.input1 = new StreamTaskNetworkInput(new CheckpointedInputGate(unionedInputGate1, bufferStorage, new CheckpointBarrierDiscarder()), inputSerializer1, ioManager, 0);
+		this.input2 = new StreamTaskNetworkInput(new CheckpointedInputGate(unionedInputGate2, bufferStorage, new CheckpointBarrierDiscarder()), inputSerializer2, ioManager, 1);
 
 		this.statusWatermarkValve1 = new StatusWatermarkValve(
 			unionedInputGate1.getNumberOfInputChannels(),
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferAlignmentLimitTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierAlignerAlignmentLimitTest.java
similarity index 94%
rename from flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferAlignmentLimitTest.java
rename to flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierAlignerAlignmentLimitTest.java
index 2eb3f5c..0621179 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferAlignmentLimitTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierAlignerAlignmentLimitTest.java
@@ -32,6 +32,7 @@ import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
 import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
 import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.streaming.runtime.io.CheckpointBarrierAlignerTestBase.CheckpointExceptionMatcher;
 
 import org.hamcrest.BaseMatcher;
 import org.hamcrest.Description;
@@ -56,9 +57,9 @@ import static org.mockito.Mockito.verify;
 import static org.mockito.hamcrest.MockitoHamcrest.argThat;
 
 /**
- * Tests for the barrier buffer's maximum limit of buffered/spilled bytes.
+ * Tests for the {@link CheckpointBarrierAligner}'s maximum limit of buffered/spilled bytes.
  */
-public class BarrierBufferAlignmentLimitTest {
+public class CheckpointBarrierAlignerAlignmentLimitTest {
 
 	private static final int PAGE_SIZE = 512;
 
@@ -116,7 +117,7 @@ public class BarrierBufferAlignmentLimitTest {
 		// the barrier buffer has a limit that only 1000 bytes may be spilled in alignment
 		MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence));
 		AbstractInvokable toNotify = mock(AbstractInvokable.class);
-		BarrierBuffer buffer = new BarrierBuffer(
+		CheckpointedInputGate buffer = new CheckpointedInputGate(
 			gate,
 			new BufferSpiller(ioManager, gate.getPageSize(), 1000),
 			"Testing",
@@ -139,7 +140,7 @@ public class BarrierBufferAlignmentLimitTest {
 		check(sequence[5], buffer.pollNext().get());
 		validateAlignmentTime(startTs, buffer);
 		verify(toNotify, times(1)).abortCheckpointOnBarrier(eq(7L),
-			argThat(new BarrierBufferTestBase.CheckpointExceptionMatcher(CheckpointFailureReason.CHECKPOINT_DECLINED_ALIGNMENT_LIMIT_EXCEEDED)));
+			argThat(new CheckpointBarrierAlignerTestBase.CheckpointExceptionMatcher(CheckpointFailureReason.CHECKPOINT_DECLINED_ALIGNMENT_LIMIT_EXCEEDED)));
 
 		// playing back buffered events
 		check(sequence[7], buffer.pollNext().get());
@@ -213,7 +214,7 @@ public class BarrierBufferAlignmentLimitTest {
 		// the barrier buffer has a limit that only 1000 bytes may be spilled in alignment
 		MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence));
 		AbstractInvokable toNotify = mock(AbstractInvokable.class);
-		BarrierBuffer buffer = new BarrierBuffer(
+		CheckpointedInputGate buffer = new CheckpointedInputGate(
 			gate,
 			new BufferSpiller(ioManager, gate.getPageSize(), 500),
 			"Testing",
@@ -237,7 +238,7 @@ public class BarrierBufferAlignmentLimitTest {
 		check(sequence[4], buffer.pollNext().get());
 		validateAlignmentTime(startTs, buffer);
 		verify(toNotify, times(1)).abortCheckpointOnBarrier(eq(3L),
-			argThat(new BarrierBufferTestBase.CheckpointExceptionMatcher(CheckpointFailureReason.CHECKPOINT_DECLINED_ALIGNMENT_LIMIT_EXCEEDED)));
+			argThat(new CheckpointExceptionMatcher(CheckpointFailureReason.CHECKPOINT_DECLINED_ALIGNMENT_LIMIT_EXCEEDED)));
 
 		// replay buffered data - in the middle, the alignment for checkpoint 4 starts
 		check(sequence[6], buffer.pollNext().get());
@@ -314,7 +315,7 @@ public class BarrierBufferAlignmentLimitTest {
 		}
 	}
 
-	private static void validateAlignmentTime(long startTimestamp, BarrierBuffer buffer) {
+	private static void validateAlignmentTime(long startTimestamp, CheckpointedInputGate buffer) {
 		final long elapsed = System.nanoTime() - startTimestamp;
 		assertTrue("wrong alignment time", buffer.getAlignmentDurationNanos() <= elapsed);
 	}
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/CheckpointBarrierAlignerMassiveRandomTest.java
similarity index 95%
rename from flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java
rename to flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierAlignerMassiveRandomTest.java
index 7fc8a5d..7da0aa3 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/CheckpointBarrierAlignerMassiveRandomTest.java
@@ -41,7 +41,7 @@ import static org.junit.Assert.fail;
  * and randomly generate checkpoint barriers. The two streams are very
  * unaligned, putting heavy work on the BarrierBuffer.
  */
-public class BarrierBufferMassiveRandomTest {
+public class CheckpointBarrierAlignerMassiveRandomTest {
 
 	private static final int PAGE_SIZE = 1024;
 
@@ -62,10 +62,10 @@ public class BarrierBufferMassiveRandomTest {
 					new BufferPool[] { pool1, pool2 },
 					new BarrierGenerator[] { new CountBarrier(100000), new RandomBarrier(100000) });
 
-			BarrierBuffer barrierBuffer = new BarrierBuffer(myIG, new BufferSpiller(ioMan, myIG.getPageSize()));
+			CheckpointedInputGate checkpointedInputGate = new CheckpointedInputGate(myIG, new BufferSpiller(ioMan, myIG.getPageSize()));
 
 			for (int i = 0; i < 2000000; i++) {
-				BufferOrEvent boe = barrierBuffer.pollNext().get();
+				BufferOrEvent boe = checkpointedInputGate.pollNext().get();
 				if (boe.isBuffer()) {
 					boe.getBuffer().recycleBuffer();
 				}
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTestBase.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierAlignerTestBase.java
similarity index 63%
rename from flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTestBase.java
rename to flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierAlignerTestBase.java
index 13c4aad..687c95d 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTestBase.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierAlignerTestBase.java
@@ -60,9 +60,9 @@ import static org.mockito.Mockito.verify;
 import static org.mockito.hamcrest.MockitoHamcrest.argThat;
 
 /**
- * Tests for the behavior of the {@link BarrierBuffer} with different {@link BufferStorage} implements.
+ * Tests for the behavior of the {@link CheckpointBarrierAligner} with different {@link BufferStorage} implements.
  */
-public abstract class BarrierBufferTestBase {
+public abstract class CheckpointBarrierAlignerTestBase {
 
 	protected static final int PAGE_SIZE = 512;
 
@@ -70,9 +70,9 @@ public abstract class BarrierBufferTestBase {
 
 	private static int sizeCounter = 1;
 
-	BarrierBuffer buffer;
+	CheckpointedInputGate inputGate;
 
-	protected BarrierBuffer createBarrierBuffer(
+	protected CheckpointedInputGate createBarrierBuffer(
 		int numberOfChannels,
 		BufferOrEvent[] sequence,
 		@Nullable AbstractInvokable toNotify) throws IOException {
@@ -80,25 +80,25 @@ public abstract class BarrierBufferTestBase {
 		return createBarrierBuffer(gate, toNotify);
 	}
 
-	protected BarrierBuffer createBarrierBuffer(int numberOfChannels, BufferOrEvent[] sequence) throws IOException {
+	protected CheckpointedInputGate createBarrierBuffer(int numberOfChannels, BufferOrEvent[] sequence) throws IOException {
 		return createBarrierBuffer(numberOfChannels, sequence, null);
 	}
 
-	protected BarrierBuffer createBarrierBuffer(InputGate gate) throws IOException {
+	protected CheckpointedInputGate createBarrierBuffer(InputGate gate) throws IOException {
 		return createBarrierBuffer(gate, null);
 	}
 
-	abstract BarrierBuffer createBarrierBuffer(InputGate gate, @Nullable AbstractInvokable toNotify) throws IOException;
+	abstract CheckpointedInputGate createBarrierBuffer(InputGate gate, @Nullable AbstractInvokable toNotify) throws IOException;
 
 	abstract void validateAlignmentBuffered(long actualBytesBuffered, BufferOrEvent... sequence);
 
 	@After
 	public void ensureEmpty() throws Exception {
-		assertFalse(buffer.pollNext().isPresent());
-		assertTrue(buffer.isFinished());
-		assertTrue(buffer.isEmpty());
+		assertFalse(inputGate.pollNext().isPresent());
+		assertTrue(inputGate.isFinished());
+		assertTrue(inputGate.isEmpty());
 
-		buffer.cleanup();
+		inputGate.cleanup();
 	}
 
 	// ------------------------------------------------------------------------
@@ -115,13 +115,13 @@ public abstract class BarrierBufferTestBase {
 			createBuffer(0), createBuffer(0),
 			createBuffer(0), createEndOfPartition(0)
 		};
-		buffer = createBarrierBuffer(1, sequence);
+		inputGate = createBarrierBuffer(1, sequence);
 
 		for (BufferOrEvent boe : sequence) {
-			assertEquals(boe, buffer.pollNext().get());
+			assertEquals(boe, inputGate.pollNext().get());
 		}
 
-		assertEquals(0L, buffer.getAlignmentDurationNanos());
+		assertEquals(0L, inputGate.getAlignmentDurationNanos());
 	}
 
 	/**
@@ -136,13 +136,13 @@ public abstract class BarrierBufferTestBase {
 			createBuffer(3), createBuffer(1), createEndOfPartition(3),
 			createBuffer(1), createEndOfPartition(1), createBuffer(2), createEndOfPartition(2)
 		};
-		buffer = createBarrierBuffer(4, sequence);
+		inputGate = createBarrierBuffer(4, sequence);
 
 		for (BufferOrEvent boe : sequence) {
-			assertEquals(boe, buffer.pollNext().get());
+			assertEquals(boe, inputGate.pollNext().get());
 		}
 
-		assertEquals(0L, buffer.getAlignmentDurationNanos());
+		assertEquals(0L, inputGate.getAlignmentDurationNanos());
 	}
 
 	/**
@@ -161,13 +161,13 @@ public abstract class BarrierBufferTestBase {
 			createBuffer(0), createEndOfPartition(0)
 		};
 		ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler();
-		buffer = createBarrierBuffer(1, sequence, handler);
+		inputGate = createBarrierBuffer(1, sequence, handler);
 
 		handler.setNextExpectedCheckpointId(1L);
 
 		for (BufferOrEvent boe : sequence) {
 			if (boe.isBuffer() || boe.getEvent().getClass() != CheckpointBarrier.class) {
-				assertEquals(boe, buffer.pollNext().get());
+				assertEquals(boe, inputGate.pollNext().get());
 			}
 		}
 	}
@@ -211,81 +211,81 @@ public abstract class BarrierBufferTestBase {
 			createEndOfPartition(0), createEndOfPartition(1), createEndOfPartition(2)
 		};
 		ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler();
-		buffer = createBarrierBuffer(3, sequence, handler);
+		inputGate = createBarrierBuffer(3, sequence, handler);
 
 		handler.setNextExpectedCheckpointId(1L);
 
 		// pre checkpoint 1
-		check(sequence[0], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[1], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[2], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[0], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[1], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[2], inputGate.pollNext().get(), PAGE_SIZE);
 		assertEquals(1L, handler.getNextExpectedCheckpointId());
 
 		long startTs = System.nanoTime();
 
 		// blocking while aligning for checkpoint 1
-		check(sequence[7], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[7], inputGate.pollNext().get(), PAGE_SIZE);
 		assertEquals(1L, handler.getNextExpectedCheckpointId());
 
 		// checkpoint 1 done, returning buffered data
-		check(sequence[5], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[5], inputGate.pollNext().get(), PAGE_SIZE);
 		assertEquals(2L, handler.getNextExpectedCheckpointId());
-		validateAlignmentTime(startTs, buffer.getAlignmentDurationNanos());
+		validateAlignmentTime(startTs, inputGate.getAlignmentDurationNanos());
 		validateAlignmentBuffered(handler.getLastReportedBytesBufferedInAlignment(), sequence[5], sequence[6]);
 
-		check(sequence[6], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[6], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// pre checkpoint 2
-		check(sequence[9], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[10], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[11], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[12], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[13], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[9], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[10], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[11], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[12], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[13], inputGate.pollNext().get(), PAGE_SIZE);
 		assertEquals(2L, handler.getNextExpectedCheckpointId());
 
 		// checkpoint 2 barriers come together
 		startTs = System.nanoTime();
-		check(sequence[17], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[17], inputGate.pollNext().get(), PAGE_SIZE);
 		assertEquals(3L, handler.getNextExpectedCheckpointId());
-		validateAlignmentTime(startTs, buffer.getAlignmentDurationNanos());
+		validateAlignmentTime(startTs, inputGate.getAlignmentDurationNanos());
 		validateAlignmentBuffered(handler.getLastReportedBytesBufferedInAlignment());
 
-		check(sequence[18], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[18], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// checkpoint 3 starts, data buffered
-		check(sequence[20], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[20], inputGate.pollNext().get(), PAGE_SIZE);
 		validateAlignmentBuffered(handler.getLastReportedBytesBufferedInAlignment(), sequence[20], sequence[21]);
 		assertEquals(4L, handler.getNextExpectedCheckpointId());
-		check(sequence[21], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[21], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// checkpoint 4 happens without extra data
 
 		// pre checkpoint 5
-		check(sequence[27], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[27], inputGate.pollNext().get(), PAGE_SIZE);
 
 		validateAlignmentBuffered(handler.getLastReportedBytesBufferedInAlignment());
 		assertEquals(5L, handler.getNextExpectedCheckpointId());
 
-		check(sequence[28], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[29], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[28], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[29], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// checkpoint 5 aligning
-		check(sequence[31], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[32], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[33], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[37], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[31], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[32], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[33], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[37], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// buffered data from checkpoint 5 alignment
-		check(sequence[34], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[36], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[38], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[39], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[34], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[36], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[38], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[39], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// remaining data
-		check(sequence[41], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[42], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[43], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[44], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[41], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[42], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[43], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[44], inputGate.pollNext().get(), PAGE_SIZE);
 
 		validateAlignmentBuffered(handler.getLastReportedBytesBufferedInAlignment(),
 			sequence[34], sequence[36], sequence[38], sequence[39]);
@@ -304,36 +304,36 @@ public abstract class BarrierBufferTestBase {
 			createBuffer(2), createEndOfPartition(2), createBuffer(0), createEndOfPartition(0)
 		};
 		ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler();
-		buffer = createBarrierBuffer(3, sequence, handler);
+		inputGate = createBarrierBuffer(3, sequence, handler);
 
 		handler.setNextExpectedCheckpointId(1L);
 
 		// pre-checkpoint 1
-		check(sequence[0], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[1], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[2], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[0], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[1], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[2], inputGate.pollNext().get(), PAGE_SIZE);
 		assertEquals(1L, handler.getNextExpectedCheckpointId());
 
 		// pre-checkpoint 2
-		check(sequence[6], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[6], inputGate.pollNext().get(), PAGE_SIZE);
 		assertEquals(2L, handler.getNextExpectedCheckpointId());
-		check(sequence[7], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[8], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[7], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[8], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// checkpoint 2 alignment
 		long startTs = System.nanoTime();
-		check(sequence[13], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[14], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[18], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[19], buffer.pollNext().get(), PAGE_SIZE);
-		validateAlignmentTime(startTs, buffer.getAlignmentDurationNanos());
+		check(sequence[13], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[14], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[18], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[19], inputGate.pollNext().get(), PAGE_SIZE);
+		validateAlignmentTime(startTs, inputGate.getAlignmentDurationNanos());
 
 		// end of stream: remaining buffered contents
-		check(sequence[10], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[11], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[12], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[16], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[17], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[10], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[11], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[12], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[16], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[17], inputGate.pollNext().get(), PAGE_SIZE);
 	}
 
 	/**
@@ -379,69 +379,69 @@ public abstract class BarrierBufferTestBase {
 			createBuffer(0), createEndOfPartition(0)
 		};
 		ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler();
-		buffer = createBarrierBuffer(3, sequence, handler);
+		inputGate = createBarrierBuffer(3, sequence, handler);
 
 		handler.setNextExpectedCheckpointId(1L);
 
 		// around checkpoint 1
-		check(sequence[0], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[1], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[2], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[7], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[0], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[1], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[2], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[7], inputGate.pollNext().get(), PAGE_SIZE);
 
-		check(sequence[5], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[5], inputGate.pollNext().get(), PAGE_SIZE);
 		assertEquals(2L, handler.getNextExpectedCheckpointId());
-		check(sequence[6], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[9], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[10], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[6], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[9], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[10], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// alignment of checkpoint 2 - buffering also some barriers for
 		// checkpoints 3 and 4
 		long startTs = System.nanoTime();
-		check(sequence[13], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[20], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[23], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[13], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[20], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[23], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// checkpoint 2 completed
-		check(sequence[12], buffer.pollNext().get(), PAGE_SIZE);
-		validateAlignmentTime(startTs, buffer.getAlignmentDurationNanos());
-		check(sequence[25], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[27], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[30], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[32], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[12], inputGate.pollNext().get(), PAGE_SIZE);
+		validateAlignmentTime(startTs, inputGate.getAlignmentDurationNanos());
+		check(sequence[25], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[27], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[30], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[32], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// checkpoint 3 completed (emit buffered)
-		check(sequence[16], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[18], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[19], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[28], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[16], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[18], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[19], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[28], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// past checkpoint 3
-		check(sequence[36], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[38], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[36], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[38], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// checkpoint 4 completed (emit buffered)
-		check(sequence[22], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[26], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[31], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[33], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[39], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[22], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[26], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[31], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[33], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[39], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// past checkpoint 4, alignment for checkpoint 5
-		check(sequence[42], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[45], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[46], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[42], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[45], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[46], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// abort checkpoint 5 (end of partition)
-		check(sequence[37], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[37], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// start checkpoint 6 alignment
-		check(sequence[47], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[48], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[47], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[48], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// end of input, emit remainder
-		check(sequence[43], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[44], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[43], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[44], inputGate.pollNext().get(), PAGE_SIZE);
 	}
 
 	/**
@@ -471,58 +471,58 @@ public abstract class BarrierBufferTestBase {
 			createBuffer(0), createEndOfPartition(0)
 		};
 		AbstractInvokable toNotify = mock(AbstractInvokable.class);
-		buffer = createBarrierBuffer(3, sequence, toNotify);
+		inputGate = createBarrierBuffer(3, sequence, toNotify);
 
 		long startTs;
 
 		// initial data
-		check(sequence[0], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[1], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[2], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[0], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[1], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[2], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// align checkpoint 1
 		startTs = System.nanoTime();
-		check(sequence[7], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(1L, buffer.getLatestCheckpointId());
+		check(sequence[7], inputGate.pollNext().get(), PAGE_SIZE);
+		assertEquals(1L, inputGate.getLatestCheckpointId());
 
 		// checkpoint done - replay buffered
-		check(sequence[5], buffer.pollNext().get(), PAGE_SIZE);
-		validateAlignmentTime(startTs, buffer.getAlignmentDurationNanos());
+		check(sequence[5], inputGate.pollNext().get(), PAGE_SIZE);
+		validateAlignmentTime(startTs, inputGate.getAlignmentDurationNanos());
 		verify(toNotify).triggerCheckpointOnBarrier(argThat(new CheckpointMatcher(1L)), any(CheckpointOptions.class), any(CheckpointMetrics.class));
-		check(sequence[6], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[6], inputGate.pollNext().get(), PAGE_SIZE);
 
-		check(sequence[9], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[10], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[9], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[10], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// alignment of checkpoint 2
 		startTs = System.nanoTime();
-		check(sequence[13], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[15], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[13], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[15], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// checkpoint 2 aborted, checkpoint 3 started
-		check(sequence[12], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(3L, buffer.getLatestCheckpointId());
-		validateAlignmentTime(startTs, buffer.getAlignmentDurationNanos());
+		check(sequence[12], inputGate.pollNext().get(), PAGE_SIZE);
+		assertEquals(3L, inputGate.getLatestCheckpointId());
+		validateAlignmentTime(startTs, inputGate.getAlignmentDurationNanos());
 		verify(toNotify).abortCheckpointOnBarrier(eq(2L),
 			argThat(new CheckpointExceptionMatcher(CheckpointFailureReason.CHECKPOINT_DECLINED_SUBSUMED)));
-		check(sequence[16], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[16], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// checkpoint 3 alignment in progress
-		check(sequence[19], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[19], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// checkpoint 3 aborted (end of partition)
-		check(sequence[20], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[20], inputGate.pollNext().get(), PAGE_SIZE);
 		verify(toNotify).abortCheckpointOnBarrier(eq(3L),
 			argThat(new CheckpointExceptionMatcher(CheckpointFailureReason.CHECKPOINT_DECLINED_INPUT_END_OF_STREAM)));
 
 		// replay buffered data from checkpoint 3
-		check(sequence[18], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[18], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// all the remaining messages
-		check(sequence[21], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[22], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[23], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[24], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[21], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[22], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[23], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[24], inputGate.pollNext().get(), PAGE_SIZE);
 	}
 
 	/**
@@ -556,50 +556,50 @@ public abstract class BarrierBufferTestBase {
 			createBuffer(0), createEndOfPartition(0)
 		};
 		ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler();
-		buffer = createBarrierBuffer(3, sequence, handler);
+		inputGate = createBarrierBuffer(3, sequence, handler);
 
 		handler.setNextExpectedCheckpointId(1L);
 
 		// checkpoint 1
-		check(sequence[0], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[1], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[2], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[7], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(1L, buffer.getLatestCheckpointId());
+		check(sequence[0], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[1], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[2], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[7], inputGate.pollNext().get(), PAGE_SIZE);
+		assertEquals(1L, inputGate.getLatestCheckpointId());
 
-		check(sequence[5], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[6], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[9], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[10], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[5], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[6], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[9], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[10], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// alignment of checkpoint 2
-		check(sequence[13], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(2L, buffer.getLatestCheckpointId());
-		check(sequence[15], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[19], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[21], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[13], inputGate.pollNext().get(), PAGE_SIZE);
+		assertEquals(2L, inputGate.getLatestCheckpointId());
+		check(sequence[15], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[19], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[21], inputGate.pollNext().get(), PAGE_SIZE);
 
 		long startTs = System.nanoTime();
 
 		// checkpoint 2 aborted, checkpoint 4 started. replay buffered
-		check(sequence[12], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(4L, buffer.getLatestCheckpointId());
-		check(sequence[16], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[18], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[22], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[12], inputGate.pollNext().get(), PAGE_SIZE);
+		assertEquals(4L, inputGate.getLatestCheckpointId());
+		check(sequence[16], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[18], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[22], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// align checkpoint 4 remainder
-		check(sequence[25], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[26], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[25], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[26], inputGate.pollNext().get(), PAGE_SIZE);
 
-		validateAlignmentTime(startTs, buffer.getAlignmentDurationNanos());
+		validateAlignmentTime(startTs, inputGate.getAlignmentDurationNanos());
 
 		// checkpoint 4 aborted (due to end of partition)
-		check(sequence[24], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[27], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[28], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[29], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[30], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[24], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[27], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[28], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[29], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[30], inputGate.pollNext().get(), PAGE_SIZE);
 	}
 
 	/**
@@ -644,48 +644,48 @@ public abstract class BarrierBufferTestBase {
 			createBuffer(2), createEndOfPartition(2),
 			createBuffer(0), createEndOfPartition(0)
 		};
-		buffer = createBarrierBuffer(3, sequence);
+		inputGate = createBarrierBuffer(3, sequence);
 
 		// checkpoint 1
-		check(sequence[0], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[1], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[2], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[7], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(1L, buffer.getLatestCheckpointId());
-		check(sequence[5], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[6], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[9], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[10], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[0], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[1], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[2], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[7], inputGate.pollNext().get(), PAGE_SIZE);
+		assertEquals(1L, inputGate.getLatestCheckpointId());
+		check(sequence[5], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[6], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[9], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[10], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// alignment of checkpoint 2
-		check(sequence[13], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[22], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(2L, buffer.getLatestCheckpointId());
+		check(sequence[13], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[22], inputGate.pollNext().get(), PAGE_SIZE);
+		assertEquals(2L, inputGate.getLatestCheckpointId());
 
 		// checkpoint 2 completed
-		check(sequence[12], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[15], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[16], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[12], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[15], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[16], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// checkpoint 3 skipped, alignment for 4 started
-		check(sequence[18], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(4L, buffer.getLatestCheckpointId());
-		check(sequence[21], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[24], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[26], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[30], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[18], inputGate.pollNext().get(), PAGE_SIZE);
+		assertEquals(4L, inputGate.getLatestCheckpointId());
+		check(sequence[21], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[24], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[26], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[30], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// checkpoint 4 completed
-		check(sequence[20], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[28], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[29], buffer.pollNext().get(), PAGE_SIZE);
-
-		check(sequence[32], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[33], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[34], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[35], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[36], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[37], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[20], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[28], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[29], inputGate.pollNext().get(), PAGE_SIZE);
+
+		check(sequence[32], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[33], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[34], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[35], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[36], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[37], inputGate.pollNext().get(), PAGE_SIZE);
 	}
 
 	@Test
@@ -698,11 +698,11 @@ public abstract class BarrierBufferTestBase {
 			createBuffer(0)
 		};
 		AbstractInvokable validator = new CheckpointSequenceValidator(-3);
-		buffer = createBarrierBuffer(2, sequence, validator);
+		inputGate = createBarrierBuffer(2, sequence, validator);
 
 		for (BufferOrEvent boe : sequence) {
 			if (boe.isBuffer() || (boe.getEvent().getClass() != CheckpointBarrier.class && boe.getEvent().getClass() != CancelCheckpointMarker.class)) {
-				assertEquals(boe, buffer.pollNext().get());
+				assertEquals(boe, inputGate.pollNext().get());
 			}
 		}
 	}
@@ -720,34 +720,34 @@ public abstract class BarrierBufferTestBase {
 			createBuffer(2), createEndOfPartition(2), createBuffer(0), createEndOfPartition(0)
 		};
 		ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler();
-		buffer = createBarrierBuffer(3, sequence, handler);
+		inputGate = createBarrierBuffer(3, sequence, handler);
 
 		handler.setNextExpectedCheckpointId(1L);
 
 		// pre-checkpoint 1
-		check(sequence[0], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[1], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[2], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[0], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[1], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[2], inputGate.pollNext().get(), PAGE_SIZE);
 		assertEquals(1L, handler.getNextExpectedCheckpointId());
 
 		// pre-checkpoint 2
-		check(sequence[6], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[6], inputGate.pollNext().get(), PAGE_SIZE);
 		assertEquals(2L, handler.getNextExpectedCheckpointId());
-		check(sequence[7], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[8], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[7], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[8], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// checkpoint 2 alignment
-		check(sequence[13], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[14], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[18], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[19], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[13], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[14], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[18], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[19], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// drain buffer
-		buffer.pollNext().get();
-		buffer.pollNext().get();
-		buffer.pollNext().get();
-		buffer.pollNext().get();
-		buffer.pollNext().get();
+		inputGate.pollNext().get();
+		inputGate.pollNext().get();
+		inputGate.pollNext().get();
+		inputGate.pollNext().get();
+		inputGate.pollNext().get();
 	}
 
 	@Test
@@ -779,36 +779,36 @@ public abstract class BarrierBufferTestBase {
 			createBuffer(3),
 			createEndOfPartition(3)
 		};
-		buffer = createBarrierBuffer(4, sequence);
+		inputGate = createBarrierBuffer(4, sequence);
 
 		// pre checkpoint 2
-		check(sequence[0], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[1], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[2], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[3], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[4], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[0], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[1], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[2], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[3], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[4], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// checkpoint 3 alignment
-		check(sequence[7], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(2L, buffer.getLatestCheckpointId());
-		check(sequence[8], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[11], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[7], inputGate.pollNext().get(), PAGE_SIZE);
+		assertEquals(2L, inputGate.getLatestCheckpointId());
+		check(sequence[8], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[11], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// checkpoint 3 buffered
-		check(sequence[10], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(3L, buffer.getLatestCheckpointId());
+		check(sequence[10], inputGate.pollNext().get(), PAGE_SIZE);
+		assertEquals(3L, inputGate.getLatestCheckpointId());
 
 		// after checkpoint 4
-		check(sequence[15], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(4L, buffer.getLatestCheckpointId());
-		check(sequence[16], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[17], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[18], buffer.pollNext().get(), PAGE_SIZE);
-
-		check(sequence[19], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[21], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(5L, buffer.getLatestCheckpointId());
-		check(sequence[22], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[15], inputGate.pollNext().get(), PAGE_SIZE);
+		assertEquals(4L, inputGate.getLatestCheckpointId());
+		check(sequence[16], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[17], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[18], inputGate.pollNext().get(), PAGE_SIZE);
+
+		check(sequence[19], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[21], inputGate.pollNext().get(), PAGE_SIZE);
+		assertEquals(5L, inputGate.getLatestCheckpointId());
+		check(sequence[22], inputGate.pollNext().get(), PAGE_SIZE);
 	}
 
 	@Test
@@ -831,25 +831,25 @@ public abstract class BarrierBufferTestBase {
 			// final end of stream
 			createEndOfPartition(0)
 		};
-		buffer = createBarrierBuffer(3, sequence);
+		inputGate = createBarrierBuffer(3, sequence);
 
 		// data after first checkpoint
-		check(sequence[3], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[4], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[5], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(1L, buffer.getLatestCheckpointId());
+		check(sequence[3], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[4], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[5], inputGate.pollNext().get(), PAGE_SIZE);
+		assertEquals(1L, inputGate.getLatestCheckpointId());
 
 		// alignment of second checkpoint
-		check(sequence[10], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(2L, buffer.getLatestCheckpointId());
+		check(sequence[10], inputGate.pollNext().get(), PAGE_SIZE);
+		assertEquals(2L, inputGate.getLatestCheckpointId());
 
 		// first end-of-partition encountered: checkpoint will not be completed
-		check(sequence[12], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[8], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[9], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[11], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[13], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[14], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[12], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[8], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[9], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[11], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[13], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[14], inputGate.pollNext().get(), PAGE_SIZE);
 	}
 
 	@Test
@@ -866,26 +866,26 @@ public abstract class BarrierBufferTestBase {
 			createBuffer(0)
 		};
 		AbstractInvokable toNotify = mock(AbstractInvokable.class);
-		buffer = createBarrierBuffer(1, sequence, toNotify);
+		inputGate = createBarrierBuffer(1, sequence, toNotify);
 
-		check(sequence[0], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[2], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[0], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[2], inputGate.pollNext().get(), PAGE_SIZE);
 		verify(toNotify, times(1)).triggerCheckpointOnBarrier(argThat(new CheckpointMatcher(1L)), any(CheckpointOptions.class), any(CheckpointMetrics.class));
-		assertEquals(0L, buffer.getAlignmentDurationNanos());
+		assertEquals(0L, inputGate.getAlignmentDurationNanos());
 
-		check(sequence[6], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(5L, buffer.getLatestCheckpointId());
+		check(sequence[6], inputGate.pollNext().get(), PAGE_SIZE);
+		assertEquals(5L, inputGate.getLatestCheckpointId());
 		verify(toNotify, times(1)).triggerCheckpointOnBarrier(argThat(new CheckpointMatcher(2L)), any(CheckpointOptions.class), any(CheckpointMetrics.class));
 		verify(toNotify, times(1)).abortCheckpointOnBarrier(eq(4L),
 			argThat(new CheckpointExceptionMatcher(CheckpointFailureReason.CHECKPOINT_DECLINED_ON_CANCELLATION_BARRIER)));
 		verify(toNotify, times(1)).triggerCheckpointOnBarrier(argThat(new CheckpointMatcher(5L)), any(CheckpointOptions.class), any(CheckpointMetrics.class));
-		assertEquals(0L, buffer.getAlignmentDurationNanos());
+		assertEquals(0L, inputGate.getAlignmentDurationNanos());
 
-		check(sequence[8], buffer.pollNext().get(), PAGE_SIZE);
-		assertEquals(6L, buffer.getLatestCheckpointId());
+		check(sequence[8], inputGate.pollNext().get(), PAGE_SIZE);
+		assertEquals(6L, inputGate.getLatestCheckpointId());
 		verify(toNotify, times(1)).abortCheckpointOnBarrier(eq(6L),
 			argThat(new CheckpointExceptionMatcher(CheckpointFailureReason.CHECKPOINT_DECLINED_ON_CANCELLATION_BARRIER)));
-		assertEquals(0L, buffer.getAlignmentDurationNanos());
+		assertEquals(0L, inputGate.getAlignmentDurationNanos());
 	}
 
 	@Test
@@ -925,62 +925,62 @@ public abstract class BarrierBufferTestBase {
 			/* 37 */ createBuffer(0)
 		};
 		AbstractInvokable toNotify = mock(AbstractInvokable.class);
-		buffer = createBarrierBuffer(3, sequence, toNotify);
+		inputGate = createBarrierBuffer(3, sequence, toNotify);
 
 		long startTs;
 
 		// successful first checkpoint, with some aligned buffers
-		check(sequence[0], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[1], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[2], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[0], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[1], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[2], inputGate.pollNext().get(), PAGE_SIZE);
 		startTs = System.nanoTime();
-		check(sequence[5], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[5], inputGate.pollNext().get(), PAGE_SIZE);
 		verify(toNotify, times(1)).triggerCheckpointOnBarrier(argThat(new CheckpointMatcher(1L)), any(CheckpointOptions.class), any(CheckpointMetrics.class));
-		validateAlignmentTime(startTs, buffer.getAlignmentDurationNanos());
+		validateAlignmentTime(startTs, inputGate.getAlignmentDurationNanos());
 
-		check(sequence[6], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[8], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[9], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[6], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[8], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[9], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// canceled checkpoint on last barrier
 		startTs = System.nanoTime();
-		check(sequence[12], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[12], inputGate.pollNext().get(), PAGE_SIZE);
 		verify(toNotify, times(1)).abortCheckpointOnBarrier(eq(2L),
 			argThat(new CheckpointExceptionMatcher(CheckpointFailureReason.CHECKPOINT_DECLINED_ON_CANCELLATION_BARRIER)));
-		validateAlignmentTime(startTs, buffer.getAlignmentDurationNanos());
-		check(sequence[13], buffer.pollNext().get(), PAGE_SIZE);
+		validateAlignmentTime(startTs, inputGate.getAlignmentDurationNanos());
+		check(sequence[13], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// one more successful checkpoint
-		check(sequence[15], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[16], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[15], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[16], inputGate.pollNext().get(), PAGE_SIZE);
 		startTs = System.nanoTime();
-		check(sequence[20], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[20], inputGate.pollNext().get(), PAGE_SIZE);
 		verify(toNotify, times(1)).triggerCheckpointOnBarrier(argThat(new CheckpointMatcher(3L)), any(CheckpointOptions.class), any(CheckpointMetrics.class));
-		validateAlignmentTime(startTs, buffer.getAlignmentDurationNanos());
-		check(sequence[21], buffer.pollNext().get(), PAGE_SIZE);
+		validateAlignmentTime(startTs, inputGate.getAlignmentDurationNanos());
+		check(sequence[21], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// this checkpoint gets immediately canceled
-		check(sequence[24], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[24], inputGate.pollNext().get(), PAGE_SIZE);
 		verify(toNotify, times(1)).abortCheckpointOnBarrier(eq(4L),
 			argThat(new CheckpointExceptionMatcher(CheckpointFailureReason.CHECKPOINT_DECLINED_ON_CANCELLATION_BARRIER)));
-		assertEquals(0L, buffer.getAlignmentDurationNanos());
+		assertEquals(0L, inputGate.getAlignmentDurationNanos());
 
 		// some buffers
-		check(sequence[26], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[27], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[28], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[26], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[27], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[28], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// a simple successful checkpoint
 		startTs = System.nanoTime();
-		check(sequence[32], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[32], inputGate.pollNext().get(), PAGE_SIZE);
 		verify(toNotify, times(1)).triggerCheckpointOnBarrier(argThat(new CheckpointMatcher(5L)), any(CheckpointOptions.class), any(CheckpointMetrics.class));
-		validateAlignmentTime(startTs, buffer.getAlignmentDurationNanos());
-		check(sequence[33], buffer.pollNext().get(), PAGE_SIZE);
+		validateAlignmentTime(startTs, inputGate.getAlignmentDurationNanos());
+		check(sequence[33], inputGate.pollNext().get(), PAGE_SIZE);
 
-		check(sequence[37], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[37], inputGate.pollNext().get(), PAGE_SIZE);
 		verify(toNotify, times(1)).abortCheckpointOnBarrier(eq(6L),
 			argThat(new CheckpointExceptionMatcher(CheckpointFailureReason.CHECKPOINT_DECLINED_ON_CANCELLATION_BARRIER)));
-		assertEquals(0L, buffer.getAlignmentDurationNanos());
+		assertEquals(0L, inputGate.getAlignmentDurationNanos());
 	}
 
 	@Test
@@ -1011,41 +1011,41 @@ public abstract class BarrierBufferTestBase {
 			/* 16 */ createBuffer(0), createBuffer(1), createBuffer(2)
 		};
 		AbstractInvokable toNotify = mock(AbstractInvokable.class);
-		buffer = createBarrierBuffer(3, sequence, toNotify);
+		inputGate = createBarrierBuffer(3, sequence, toNotify);
 
 		long startTs;
 
-		check(sequence[0], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[0], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// starting first checkpoint
 		startTs = System.nanoTime();
-		check(sequence[4], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[8], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[4], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[8], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// finished first checkpoint
-		check(sequence[3], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[3], inputGate.pollNext().get(), PAGE_SIZE);
 		verify(toNotify, times(1)).triggerCheckpointOnBarrier(argThat(new CheckpointMatcher(1L)), any(CheckpointOptions.class), any(CheckpointMetrics.class));
-		validateAlignmentTime(startTs, buffer.getAlignmentDurationNanos());
+		validateAlignmentTime(startTs, inputGate.getAlignmentDurationNanos());
 
-		check(sequence[5], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[5], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// re-read the queued cancellation barriers
-		check(sequence[9], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[9], inputGate.pollNext().get(), PAGE_SIZE);
 		verify(toNotify, times(1)).abortCheckpointOnBarrier(eq(2L),
 			argThat(new CheckpointExceptionMatcher(CheckpointFailureReason.CHECKPOINT_DECLINED_ON_CANCELLATION_BARRIER)));
-		assertEquals(0L, buffer.getAlignmentDurationNanos());
+		assertEquals(0L, inputGate.getAlignmentDurationNanos());
 
-		check(sequence[10], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[12], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[13], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[14], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[10], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[12], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[13], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[14], inputGate.pollNext().get(), PAGE_SIZE);
 
-		check(sequence[16], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[17], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[18], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[16], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[17], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[18], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// no further alignment should have happened
-		assertEquals(0L, buffer.getAlignmentDurationNanos());
+		assertEquals(0L, inputGate.getAlignmentDurationNanos());
 
 		// no further checkpoint (abort) notifications
 		verify(toNotify, times(1)).triggerCheckpointOnBarrier(any(CheckpointMetaData.class), any(CheckpointOptions.class), any(CheckpointMetrics.class));
@@ -1093,44 +1093,44 @@ public abstract class BarrierBufferTestBase {
 			/* 18 */ createBuffer(0), createBuffer(1), createBuffer(2)
 		};
 		AbstractInvokable toNotify = mock(AbstractInvokable.class);
-		buffer = createBarrierBuffer(3, sequence, toNotify);
+		inputGate = createBarrierBuffer(3, sequence, toNotify);
 
 		long startTs;
 
-		check(sequence[0], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[0], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// starting first checkpoint
 		startTs = System.nanoTime();
-		check(sequence[2], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[3], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[6], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[2], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[3], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[6], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// cancelled by cancellation barrier
-		check(sequence[4], buffer.pollNext().get(), PAGE_SIZE);
-		validateAlignmentTime(startTs, buffer.getAlignmentDurationNanos());
+		check(sequence[4], inputGate.pollNext().get(), PAGE_SIZE);
+		validateAlignmentTime(startTs, inputGate.getAlignmentDurationNanos());
 		verify(toNotify).abortCheckpointOnBarrier(eq(1L),
 			argThat(new CheckpointExceptionMatcher(CheckpointFailureReason.CHECKPOINT_DECLINED_ON_CANCELLATION_BARRIER)));
 
 		// the next checkpoint alignment starts now
 		startTs = System.nanoTime();
-		check(sequence[9], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[11], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[13], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[15], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[9], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[11], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[13], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[15], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// checkpoint done
-		check(sequence[7], buffer.pollNext().get(), PAGE_SIZE);
-		validateAlignmentTime(startTs, buffer.getAlignmentDurationNanos());
+		check(sequence[7], inputGate.pollNext().get(), PAGE_SIZE);
+		validateAlignmentTime(startTs, inputGate.getAlignmentDurationNanos());
 		verify(toNotify).triggerCheckpointOnBarrier(argThat(new CheckpointMatcher(2L)), any(CheckpointOptions.class), any(CheckpointMetrics.class));
 
 		// queued data
-		check(sequence[10], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[14], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[10], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[14], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// trailing data
-		check(sequence[18], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[19], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[20], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[18], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[19], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[20], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// check overall notifications
 		verify(toNotify, times(1)).triggerCheckpointOnBarrier(any(CheckpointMetaData.class), any(CheckpointOptions.class), any(CheckpointMetrics.class));
@@ -1168,40 +1168,40 @@ public abstract class BarrierBufferTestBase {
 			/* 16 */ createBuffer(0), createBuffer(1), createBuffer(2)
 		};
 		AbstractInvokable toNotify = mock(AbstractInvokable.class);
-		buffer = createBarrierBuffer(3, sequence, toNotify);
+		inputGate = createBarrierBuffer(3, sequence, toNotify);
 
 		long startTs;
 
 		// validate the sequence
 
-		check(sequence[0], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[0], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// beginning of first checkpoint
-		check(sequence[5], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[5], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// future barrier aborts checkpoint
 		startTs = System.nanoTime();
-		check(sequence[3], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[3], inputGate.pollNext().get(), PAGE_SIZE);
 		verify(toNotify, times(1)).abortCheckpointOnBarrier(eq(3L),
 			argThat(new CheckpointExceptionMatcher(CheckpointFailureReason.CHECKPOINT_DECLINED_SUBSUMED)));
-		check(sequence[4], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[4], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// alignment of next checkpoint
-		check(sequence[8], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[9], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[12], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[13], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[8], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[9], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[12], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[13], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// checkpoint finished
-		check(sequence[7], buffer.pollNext().get(), PAGE_SIZE);
-		validateAlignmentTime(startTs, buffer.getAlignmentDurationNanos());
+		check(sequence[7], inputGate.pollNext().get(), PAGE_SIZE);
+		validateAlignmentTime(startTs, inputGate.getAlignmentDurationNanos());
 		verify(toNotify, times(1)).triggerCheckpointOnBarrier(argThat(new CheckpointMatcher(5L)), any(CheckpointOptions.class), any(CheckpointMetrics.class));
-		check(sequence[11], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[11], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// remaining data
-		check(sequence[16], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[17], buffer.pollNext().get(), PAGE_SIZE);
-		check(sequence[18], buffer.pollNext().get(), PAGE_SIZE);
+		check(sequence[16], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[17], inputGate.pollNext().get(), PAGE_SIZE);
+		check(sequence[18], inputGate.pollNext().get(), PAGE_SIZE);
 
 		// check overall notifications
 		verify(toNotify, times(1)).triggerCheckpointOnBarrier(any(CheckpointMetaData.class), any(CheckpointOptions.class), any(CheckpointMetrics.class));
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierTrackerTest.java
similarity index 89%
rename from flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java
rename to flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierTrackerTest.java
index 5112f63..2218680 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierTrackerTest.java
@@ -42,26 +42,26 @@ import static org.junit.Assert.assertTrue;
 /**
  * Tests for the behavior of the barrier tracker.
  */
-public class BarrierTrackerTest {
+public class CheckpointBarrierTrackerTest {
 
 	private static final int PAGE_SIZE = 512;
 
-	private CheckpointedInputGate tracker;
+	private CheckpointedInputGate inputGate;
 
 	@After
 	public void ensureEmpty() throws Exception {
-		assertFalse(tracker.pollNext().isPresent());
-		assertTrue(tracker.isFinished());
-		assertTrue(tracker.isEmpty());
+		assertFalse(inputGate.pollNext().isPresent());
+		assertTrue(inputGate.isFinished());
+		assertTrue(inputGate.isEmpty());
 	}
 
 	@Test
 	public void testSingleChannelNoBarriers() throws Exception {
 		BufferOrEvent[] sequence = { createBuffer(0), createBuffer(0), createBuffer(0) };
-		tracker = createBarrierTracker(1, sequence);
+		inputGate = createBarrierTracker(1, sequence);
 
 		for (BufferOrEvent boe : sequence) {
-			assertEquals(boe, tracker.pollNext().get());
+			assertEquals(boe, inputGate.pollNext().get());
 		}
 	}
 
@@ -71,10 +71,10 @@ public class BarrierTrackerTest {
 				createBuffer(1), createBuffer(0), createBuffer(3),
 				createBuffer(1), createBuffer(1), createBuffer(2)
 		};
-		tracker = createBarrierTracker(4, sequence);
+		inputGate = createBarrierTracker(4, sequence);
 
 		for (BufferOrEvent boe : sequence) {
-			assertEquals(boe, tracker.pollNext().get());
+			assertEquals(boe, inputGate.pollNext().get());
 		}
 	}
 
@@ -91,11 +91,11 @@ public class BarrierTrackerTest {
 		};
 		CheckpointSequenceValidator validator =
 			new CheckpointSequenceValidator(1, 2, 3, 4, 5, 6);
-		tracker = createBarrierTracker(1, sequence, validator);
+		inputGate = createBarrierTracker(1, sequence, validator);
 
 		for (BufferOrEvent boe : sequence) {
 			if (boe.isBuffer() || boe.getEvent().getClass() != CheckpointBarrier.class) {
-				assertEquals(boe, tracker.pollNext().get());
+				assertEquals(boe, inputGate.pollNext().get());
 			}
 		}
 	}
@@ -113,11 +113,11 @@ public class BarrierTrackerTest {
 		};
 		CheckpointSequenceValidator validator =
 			new CheckpointSequenceValidator(1, 3, 4, 6, 7, 10);
-		tracker = createBarrierTracker(1, sequence, validator);
+		inputGate = createBarrierTracker(1, sequence, validator);
 
 		for (BufferOrEvent boe : sequence) {
 			if (boe.isBuffer() || boe.getEvent().getClass() != CheckpointBarrier.class) {
-				assertEquals(boe, tracker.pollNext().get());
+				assertEquals(boe, inputGate.pollNext().get());
 			}
 		}
 	}
@@ -144,11 +144,11 @@ public class BarrierTrackerTest {
 		};
 		CheckpointSequenceValidator validator =
 			new CheckpointSequenceValidator(1, 2, 3, 4);
-		tracker = createBarrierTracker(3, sequence, validator);
+		inputGate = createBarrierTracker(3, sequence, validator);
 
 		for (BufferOrEvent boe : sequence) {
 			if (boe.isBuffer() || boe.getEvent().getClass() != CheckpointBarrier.class) {
-				assertEquals(boe, tracker.pollNext().get());
+				assertEquals(boe, inputGate.pollNext().get());
 			}
 		}
 	}
@@ -179,11 +179,11 @@ public class BarrierTrackerTest {
 		};
 		CheckpointSequenceValidator validator =
 			new CheckpointSequenceValidator(1, 2, 4);
-		tracker = createBarrierTracker(3, sequence, validator);
+		inputGate = createBarrierTracker(3, sequence, validator);
 
 		for (BufferOrEvent boe : sequence) {
 			if (boe.isBuffer() || boe.getEvent().getClass() != CheckpointBarrier.class) {
-				assertEquals(boe, tracker.pollNext().get());
+				assertEquals(boe, inputGate.pollNext().get());
 			}
 		}
 	}
@@ -253,11 +253,11 @@ public class BarrierTrackerTest {
 		};
 		CheckpointSequenceValidator validator =
 			new CheckpointSequenceValidator(2, 3, 4, 5, 7, 8, 9, 10);
-		tracker = createBarrierTracker(3, sequence, validator);
+		inputGate = createBarrierTracker(3, sequence, validator);
 
 		for (BufferOrEvent boe : sequence) {
 			if (boe.isBuffer() || boe.getEvent().getClass() != CheckpointBarrier.class) {
-				assertEquals(boe, tracker.pollNext().get());
+				assertEquals(boe, inputGate.pollNext().get());
 			}
 		}
 	}
@@ -278,13 +278,13 @@ public class BarrierTrackerTest {
 		// negative values mean an expected cancellation call!
 		CheckpointSequenceValidator validator =
 			new CheckpointSequenceValidator(1, 2, -4, 5, -6);
-		tracker = createBarrierTracker(1, sequence, validator);
+		inputGate = createBarrierTracker(1, sequence, validator);
 
 		for (BufferOrEvent boe : sequence) {
 			if (boe.isBuffer()) {
-				assertEquals(boe, tracker.pollNext().get());
+				assertEquals(boe, inputGate.pollNext().get());
 			}
-			assertTrue(tracker.isEmpty());
+			assertTrue(inputGate.isEmpty());
 		}
 	}
 
@@ -327,11 +327,11 @@ public class BarrierTrackerTest {
 		// negative values mean an expected cancellation call!
 		CheckpointSequenceValidator validator =
 			new CheckpointSequenceValidator(1, -2, 3, -4, 5, -6);
-		tracker = createBarrierTracker(3, sequence, validator);
+		inputGate = createBarrierTracker(3, sequence, validator);
 
 		for (BufferOrEvent boe : sequence) {
 			if (boe.isBuffer()) {
-				assertEquals(boe, tracker.pollNext().get());
+				assertEquals(boe, inputGate.pollNext().get());
 			}
 		}
 	}
@@ -353,11 +353,11 @@ public class BarrierTrackerTest {
 		};
 		CheckpointSequenceValidator validator =
 			new CheckpointSequenceValidator(-1, -2);
-		tracker = createBarrierTracker(3, sequence, validator);
+		inputGate = createBarrierTracker(3, sequence, validator);
 
 		for (BufferOrEvent boe : sequence) {
 			if (boe.isBuffer() || (boe.getEvent().getClass() != CheckpointBarrier.class && boe.getEvent().getClass() != CancelCheckpointMarker.class)) {
-				assertEquals(boe, tracker.pollNext().get());
+				assertEquals(boe, inputGate.pollNext().get());
 			}
 		}
 	}
@@ -374,7 +374,7 @@ public class BarrierTrackerTest {
 			BufferOrEvent[] sequence,
 			@Nullable AbstractInvokable toNotifyOnCheckpoint) {
 		MockInputGate gate = new MockInputGate(PAGE_SIZE, numberOfChannels, Arrays.asList(sequence));
-		return new BarrierBuffer(
+		return new CheckpointedInputGate(
 			gate,
 			new CachedBufferStorage(PAGE_SIZE, -1, "Testing"),
 			new CheckpointBarrierTracker(gate.getNumberOfInputChannels(), toNotifyOnCheckpoint));
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CreditBasedBarrierBufferTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CreditBasedCheckpointBarrierAlignerTest.java
similarity index 79%
rename from flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CreditBasedBarrierBufferTest.java
rename to flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CreditBasedCheckpointBarrierAlignerTest.java
index 3db884d..5b942a6 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CreditBasedBarrierBufferTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CreditBasedCheckpointBarrierAlignerTest.java
@@ -27,13 +27,13 @@ import javax.annotation.Nullable;
 import static org.junit.Assert.assertEquals;
 
 /**
- * Tests for the behaviors of the {@link BarrierBuffer} with {@link CachedBufferStorage}.
+ * Tests for the behaviors of the {@link CheckpointedInputGate} with {@link CachedBufferStorage}.
  */
-public class CreditBasedBarrierBufferTest extends BarrierBufferTestBase {
+public class CreditBasedCheckpointBarrierAlignerTest extends CheckpointBarrierAlignerTestBase {
 
 	@Override
-	BarrierBuffer createBarrierBuffer(InputGate gate, @Nullable AbstractInvokable toNotify) {
-		return new BarrierBuffer(gate, new CachedBufferStorage(PAGE_SIZE), "Testing", toNotify);
+	CheckpointedInputGate createBarrierBuffer(InputGate gate, @Nullable AbstractInvokable toNotify) {
+		return new CheckpointedInputGate(gate, new CachedBufferStorage(PAGE_SIZE), "Testing", toNotify);
 	}
 
 	@Override
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/SpillingBarrierBufferTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/SpillingCheckpointBarrierAlignerTest.java
similarity index 85%
rename from flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/SpillingBarrierBufferTest.java
rename to flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/SpillingCheckpointBarrierAlignerTest.java
index f9541a9..c892073 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/SpillingBarrierBufferTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/SpillingCheckpointBarrierAlignerTest.java
@@ -36,9 +36,9 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
 /**
- * Tests for the behavior of the {@link BarrierBuffer} with {@link BufferSpiller}.
+ * Tests for the behavior of the {@link CheckpointedInputGate} with {@link BufferSpiller}.
  */
-public class SpillingBarrierBufferTest extends BarrierBufferTestBase {
+public class SpillingCheckpointBarrierAlignerTest extends CheckpointBarrierAlignerTestBase {
 
 	private static IOManager ioManager;
 
@@ -67,8 +67,8 @@ public class SpillingBarrierBufferTest extends BarrierBufferTestBase {
 	}
 
 	@Override
-	BarrierBuffer createBarrierBuffer(InputGate gate, @Nullable AbstractInvokable toNotify) throws IOException {
-		return new BarrierBuffer(gate, new BufferSpiller(ioManager, PAGE_SIZE), "Testing", toNotify);
+	CheckpointedInputGate createBarrierBuffer(InputGate gate, @Nullable AbstractInvokable toNotify) throws IOException {
+		return new CheckpointedInputGate(gate, new BufferSpiller(ioManager, PAGE_SIZE), "Testing", toNotify);
 	}
 
 	@Override
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 56c3889..456aea5 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
@@ -31,7 +31,8 @@ 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.apache.flink.streaming.runtime.io.BarrierBufferTestBase;
+import org.apache.flink.streaming.runtime.io.CheckpointBarrierAlignerTestBase;
+import org.apache.flink.streaming.runtime.io.CheckpointBarrierAlignerTestBase.CheckpointExceptionMatcher;
 
 import org.junit.Test;
 
@@ -110,7 +111,7 @@ public class StreamTaskCancellationBarrierTest {
 
 		// the decline call should go to the coordinator
 		verify(environment, times(1)).declineCheckpoint(eq(2L),
-			argThat(new BarrierBufferTestBase.CheckpointExceptionMatcher(CheckpointFailureReason.CHECKPOINT_DECLINED_ON_CANCELLATION_BARRIER)));
+			argThat(new CheckpointExceptionMatcher(CheckpointFailureReason.CHECKPOINT_DECLINED_ON_CANCELLATION_BARRIER)));
 
 		// a cancellation barrier should be downstream
 		Object result = testHarness.getOutput().poll();
@@ -155,7 +156,7 @@ public class StreamTaskCancellationBarrierTest {
 
 		// the decline call should go to the coordinator
 		verify(environment, times(1)).declineCheckpoint(eq(2L),
-			argThat(new BarrierBufferTestBase.CheckpointExceptionMatcher(CheckpointFailureReason.CHECKPOINT_DECLINED_ON_CANCELLATION_BARRIER)));
+			argThat(new CheckpointBarrierAlignerTestBase.CheckpointExceptionMatcher(CheckpointFailureReason.CHECKPOINT_DECLINED_ON_CANCELLATION_BARRIER)));
 
 		// a cancellation barrier should be downstream
 		Object result = testHarness.getOutput().poll();


[flink] 07/16: [FLINK-12777][network] Extract CheckpointBarrierAligner from BarrierBuffer

Posted by pn...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pnowojski pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit aa5f0618ef88161b23097b45df4055a58ca11685
Author: Piotr Nowojski <pi...@gmail.com>
AuthorDate: Thu Jun 13 14:42:14 2019 +0200

    [FLINK-12777][network] Extract CheckpointBarrierAligner from BarrierBuffer
---
 .../runtime/io/AbstractBufferStorage.java          |   5 -
 .../flink/streaming/runtime/io/BarrierBuffer.java  | 369 ++------------------
 .../flink/streaming/runtime/io/BufferStorage.java  |   2 -
 ...erBuffer.java => CheckpointBarrierAligner.java} | 379 ++++++---------------
 4 files changed, 144 insertions(+), 611 deletions(-)

diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/AbstractBufferStorage.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/AbstractBufferStorage.java
index f7e4dd7..5eb30cf 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/AbstractBufferStorage.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/AbstractBufferStorage.java
@@ -164,11 +164,6 @@ public abstract class AbstractBufferStorage implements BufferStorage {
 	}
 
 	@Override
-	public long currentBufferedSize() {
-		return currentBuffered != null ? currentBuffered.size() : 0L;
-	}
-
-	@Override
 	public long getMaxBufferedBytes() {
 		return maxBufferedBytes;
 	}
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
index 0f8fa40..1594389 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
@@ -19,10 +19,6 @@ package org.apache.flink.streaming.runtime.io;
 
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.runtime.checkpoint.CheckpointException;
-import org.apache.flink.runtime.checkpoint.CheckpointFailureReason;
-import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
-import org.apache.flink.runtime.checkpoint.CheckpointMetrics;
 import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker;
 import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
 import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
@@ -54,45 +50,17 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 
 	private static final Logger LOG = LoggerFactory.getLogger(BarrierBuffer.class);
 
+	private final CheckpointBarrierAligner barrierAligner;
+
 	/** The gate that the buffer draws its input from. */
 	private final InputGate inputGate;
 
-	/** Flags that indicate whether a channel is currently blocked/buffered. */
-	private final boolean[] blockedChannels;
-
-	/** The total number of channels that this buffer handles data from. */
-	private final int totalNumberOfInputChannels;
-
-	/** To utility to write blocked data to a file channel. */
 	private final BufferStorage bufferStorage;
 
-	private final String taskName;
-
-	@Nullable
-	private final AbstractInvokable toNotifyOnCheckpoint;
-
-	/** The ID of the checkpoint for which we expect barriers. */
-	private long currentCheckpointId = -1L;
-
-	/**
-	 * The number of received barriers (= number of blocked/buffered channels) IMPORTANT: A canceled
-	 * checkpoint must always have 0 barriers.
-	 */
-	private int numBarriersReceived;
-
-	/** The number of already closed channels. */
-	private int numClosedChannels;
-
-	/** The timestamp as in {@link System#nanoTime()} at which the last alignment started. */
-	private long startOfAlignmentTimestamp;
-
-	/** The time (in nanoseconds) that the latest alignment took. */
-	private long latestAlignmentDurationNanos;
-
 	/** Flag to indicate whether we have drawn all available input. */
-	private boolean endOfStream;
+	private boolean endOfInputGate;
 
-	/** Indicate end of the input. Set to true after encountering {@link #endOfStream} and depleting
+	/** Indicate end of the input. Set to true after encountering {@link #endOfInputGate} and depleting
 	 * {@link #bufferStorage}. */
 	private boolean isFinished;
 
@@ -122,19 +90,16 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 	 * @param toNotifyOnCheckpoint optional Handler that receives the checkpoint notifications.
 	 */
 	BarrierBuffer(
-		InputGate inputGate,
-		BufferStorage bufferStorage,
-		String taskName,
-		@Nullable AbstractInvokable toNotifyOnCheckpoint) {
-
+			InputGate inputGate,
+			BufferStorage bufferStorage,
+			String taskName,
+			@Nullable AbstractInvokable toNotifyOnCheckpoint) {
 		this.inputGate = inputGate;
-		this.totalNumberOfInputChannels = inputGate.getNumberOfInputChannels();
-		this.blockedChannels = new boolean[this.totalNumberOfInputChannels];
-
 		this.bufferStorage = checkNotNull(bufferStorage);
-
-		this.taskName = taskName;
-		this.toNotifyOnCheckpoint = toNotifyOnCheckpoint;
+		this.barrierAligner = new CheckpointBarrierAligner(
+			inputGate.getNumberOfInputChannels(),
+			taskName,
+			toNotifyOnCheckpoint);
 	}
 
 	@Override
@@ -145,10 +110,6 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 		return AVAILABLE;
 	}
 
-	// ------------------------------------------------------------------------
-	//  Buffer and barrier handling
-	// ------------------------------------------------------------------------
-
 	@Override
 	public Optional<BufferOrEvent> pollNext() throws Exception {
 		while (true) {
@@ -170,28 +131,36 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 			}
 
 			BufferOrEvent bufferOrEvent = next.get();
-			if (isBlocked(bufferOrEvent.getChannelIndex())) {
+			if (barrierAligner.isBlocked(bufferOrEvent.getChannelIndex())) {
 				// if the channel is blocked, we just store the BufferOrEvent
 				bufferStorage.add(bufferOrEvent);
 				if (bufferStorage.isFull()) {
-					sizeLimitExceeded();
+					barrierAligner.checkpointSizeLimitExceeded(bufferStorage.getMaxBufferedBytes());
+					bufferStorage.rollOver();
 				}
 			}
 			else if (bufferOrEvent.isBuffer()) {
 				return next;
 			}
 			else if (bufferOrEvent.getEvent().getClass() == CheckpointBarrier.class) {
-				if (!endOfStream) {
+				CheckpointBarrier checkpointBarrier = (CheckpointBarrier) bufferOrEvent.getEvent();
+				if (!endOfInputGate) {
 					// process barriers only if there is a chance of the checkpoint completing
-					processBarrier((CheckpointBarrier) bufferOrEvent.getEvent(), bufferOrEvent.getChannelIndex());
+					if (barrierAligner.processBarrier(checkpointBarrier, bufferOrEvent.getChannelIndex(), bufferStorage.getPendingBytes())) {
+						bufferStorage.rollOver();
+					}
 				}
 			}
 			else if (bufferOrEvent.getEvent().getClass() == CancelCheckpointMarker.class) {
-				processCancellationBarrier((CancelCheckpointMarker) bufferOrEvent.getEvent());
+				if (barrierAligner.processCancellationBarrier((CancelCheckpointMarker) bufferOrEvent.getEvent())) {
+					bufferStorage.rollOver();
+				}
 			}
 			else {
 				if (bufferOrEvent.getEvent().getClass() == EndOfPartitionEvent.class) {
-					processEndOfPartition();
+					if (barrierAligner.processEndOfPartition()) {
+						bufferStorage.rollOver();
+					}
 				}
 				return next;
 			}
@@ -203,216 +172,18 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 			return Optional.empty();
 		}
 
-		if (endOfStream) {
+		if (endOfInputGate) {
 			isFinished = true;
 			return Optional.empty();
 		} else {
 			// end of input stream. stream continues with the buffered data
-			endOfStream = true;
-			releaseBlocksAndResetBarriers();
+			endOfInputGate = true;
+			barrierAligner.releaseBlocksAndResetBarriers();
+			bufferStorage.rollOver();
 			return pollNext();
 		}
 	}
 
-	private void processBarrier(CheckpointBarrier receivedBarrier, int channelIndex) throws Exception {
-		final long barrierId = receivedBarrier.getId();
-
-		// fast path for single channel cases
-		if (totalNumberOfInputChannels == 1) {
-			if (barrierId > currentCheckpointId) {
-				// new checkpoint
-				currentCheckpointId = barrierId;
-				notifyCheckpoint(receivedBarrier);
-			}
-			return;
-		}
-
-		// -- general code path for multiple input channels --
-
-		if (numBarriersReceived > 0) {
-			// this is only true if some alignment is already progress and was not canceled
-
-			if (barrierId == currentCheckpointId) {
-				// regular case
-				onBarrier(channelIndex);
-			}
-			else if (barrierId > currentCheckpointId) {
-				// we did not complete the current checkpoint, another started before
-				LOG.warn("{}: Received checkpoint barrier for checkpoint {} before completing current checkpoint {}. " +
-						"Skipping current checkpoint.",
-					taskName,
-					barrierId,
-					currentCheckpointId);
-
-				// let the task know we are not completing this
-				notifyAbort(currentCheckpointId,
-					new CheckpointException(
-						"Barrier id: " + barrierId,
-						CheckpointFailureReason.CHECKPOINT_DECLINED_SUBSUMED));
-
-				// abort the current checkpoint
-				releaseBlocksAndResetBarriers();
-
-				// begin a the new checkpoint
-				beginNewAlignment(barrierId, channelIndex);
-			}
-			else {
-				// ignore trailing barrier from an earlier checkpoint (obsolete now)
-				return;
-			}
-		}
-		else if (barrierId > currentCheckpointId) {
-			// first barrier of a new checkpoint
-			beginNewAlignment(barrierId, channelIndex);
-		}
-		else {
-			// either the current checkpoint was canceled (numBarriers == 0) or
-			// this barrier is from an old subsumed checkpoint
-			return;
-		}
-
-		// check if we have all barriers - since canceled checkpoints always have zero barriers
-		// this can only happen on a non canceled checkpoint
-		if (numBarriersReceived + numClosedChannels == totalNumberOfInputChannels) {
-			// actually trigger checkpoint
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("{}: Received all barriers, triggering checkpoint {} at {}.",
-					taskName,
-					receivedBarrier.getId(),
-					receivedBarrier.getTimestamp());
-			}
-
-			releaseBlocksAndResetBarriers();
-			notifyCheckpoint(receivedBarrier);
-		}
-	}
-
-	private void processCancellationBarrier(CancelCheckpointMarker cancelBarrier) throws Exception {
-		final long barrierId = cancelBarrier.getCheckpointId();
-
-		// fast path for single channel cases
-		if (totalNumberOfInputChannels == 1) {
-			if (barrierId > currentCheckpointId) {
-				// new checkpoint
-				currentCheckpointId = barrierId;
-				notifyAbortOnCancellationBarrier(barrierId);
-			}
-			return;
-		}
-
-		// -- general code path for multiple input channels --
-
-		if (numBarriersReceived > 0) {
-			// this is only true if some alignment is in progress and nothing was canceled
-
-			if (barrierId == currentCheckpointId) {
-				// cancel this alignment
-				if (LOG.isDebugEnabled()) {
-					LOG.debug("{}: Checkpoint {} canceled, aborting alignment.", taskName, barrierId);
-				}
-
-				releaseBlocksAndResetBarriers();
-				notifyAbortOnCancellationBarrier(barrierId);
-			}
-			else if (barrierId > currentCheckpointId) {
-				// we canceled the next which also cancels the current
-				LOG.warn("{}: Received cancellation barrier for checkpoint {} before completing current checkpoint {}. " +
-						"Skipping current checkpoint.",
-					taskName,
-					barrierId,
-					currentCheckpointId);
-
-				// this stops the current alignment
-				releaseBlocksAndResetBarriers();
-
-				// the next checkpoint starts as canceled
-				currentCheckpointId = barrierId;
-				startOfAlignmentTimestamp = 0L;
-				latestAlignmentDurationNanos = 0L;
-
-				notifyAbortOnCancellationBarrier(barrierId);
-			}
-
-			// else: ignore trailing (cancellation) barrier from an earlier checkpoint (obsolete now)
-
-		}
-		else if (barrierId > currentCheckpointId) {
-			// first barrier of a new checkpoint is directly a cancellation
-
-			// by setting the currentCheckpointId to this checkpoint while keeping the numBarriers
-			// at zero means that no checkpoint barrier can start a new alignment
-			currentCheckpointId = barrierId;
-
-			startOfAlignmentTimestamp = 0L;
-			latestAlignmentDurationNanos = 0L;
-
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("{}: Checkpoint {} canceled, skipping alignment.", taskName, barrierId);
-			}
-
-			notifyAbortOnCancellationBarrier(barrierId);
-		}
-
-		// else: trailing barrier from either
-		//   - a previous (subsumed) checkpoint
-		//   - the current checkpoint if it was already canceled
-	}
-
-	private void processEndOfPartition() throws Exception {
-		numClosedChannels++;
-
-		if (numBarriersReceived > 0) {
-			// let the task know we skip a checkpoint
-			notifyAbort(currentCheckpointId,
-				new CheckpointException(CheckpointFailureReason.CHECKPOINT_DECLINED_INPUT_END_OF_STREAM));
-
-			// no chance to complete this checkpoint
-			releaseBlocksAndResetBarriers();
-		}
-	}
-
-	private void notifyCheckpoint(CheckpointBarrier checkpointBarrier) throws Exception {
-		if (toNotifyOnCheckpoint != null) {
-			CheckpointMetaData checkpointMetaData =
-				new CheckpointMetaData(checkpointBarrier.getId(), checkpointBarrier.getTimestamp());
-
-			CheckpointMetrics checkpointMetrics = new CheckpointMetrics()
-				.setBytesBufferedInAlignment(bufferStorage.currentBufferedSize())
-				.setAlignmentDurationNanos(latestAlignmentDurationNanos);
-
-			toNotifyOnCheckpoint.triggerCheckpointOnBarrier(
-				checkpointMetaData,
-				checkpointBarrier.getCheckpointOptions(),
-				checkpointMetrics);
-		}
-	}
-
-	private void notifyAbortOnCancellationBarrier(long checkpointId) throws Exception {
-		notifyAbort(checkpointId,
-			new CheckpointException(CheckpointFailureReason.CHECKPOINT_DECLINED_ON_CANCELLATION_BARRIER));
-	}
-
-	private void notifyAbort(long checkpointId, CheckpointException cause) throws Exception {
-		if (toNotifyOnCheckpoint != null) {
-			toNotifyOnCheckpoint.abortCheckpointOnBarrier(checkpointId, cause);
-		}
-	}
-
-	private void sizeLimitExceeded() throws Exception {
-		long maxBufferedBytes = bufferStorage.getMaxBufferedBytes();
-		// exceeded our limit - abort this checkpoint
-		LOG.info("{}: Checkpoint {} aborted because alignment volume limit ({} bytes) exceeded.",
-			taskName,
-			currentCheckpointId,
-			maxBufferedBytes);
-
-		releaseBlocksAndResetBarriers();
-		notifyAbort(currentCheckpointId,
-			new CheckpointException(
-				"Max buffered bytes: " + maxBufferedBytes,
-				CheckpointFailureReason.CHECKPOINT_DECLINED_ALIGNMENT_LIMIT_EXCEEDED));
-	}
-
 	@Override
 	public boolean isEmpty() {
 		return bufferStorage.isEmpty();
@@ -428,69 +199,6 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 		bufferStorage.close();
 	}
 
-	private void beginNewAlignment(long checkpointId, int channelIndex) throws IOException {
-		currentCheckpointId = checkpointId;
-		onBarrier(channelIndex);
-
-		startOfAlignmentTimestamp = System.nanoTime();
-
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("{}: Starting stream alignment for checkpoint {}.", taskName, checkpointId);
-		}
-	}
-
-	/**
-	 * Checks whether the channel with the given index is blocked.
-	 *
-	 * @param channelIndex The channel index to check.
-	 * @return True if the channel is blocked, false if not.
-	 */
-	private boolean isBlocked(int channelIndex) {
-		return blockedChannels[channelIndex];
-	}
-
-	/**
-	 * Blocks the given channel index, from which a barrier has been received.
-	 *
-	 * @param channelIndex The channel index to block.
-	 */
-	private void onBarrier(int channelIndex) throws IOException {
-		if (!blockedChannels[channelIndex]) {
-			blockedChannels[channelIndex] = true;
-
-			numBarriersReceived++;
-
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("{}: Received barrier from channel {}.", taskName, channelIndex);
-			}
-		}
-		else {
-			throw new IOException("Stream corrupt: Repeated barrier for same checkpoint on input " + channelIndex);
-		}
-	}
-
-	/**
-	 * Releases the blocks on all channels and resets the barrier count.
-	 * Makes sure the just written data is the next to be consumed.
-	 */
-	private void releaseBlocksAndResetBarriers() throws IOException {
-		LOG.debug("{}: End of stream alignment, feeding buffered data back.", taskName);
-
-		for (int i = 0; i < blockedChannels.length; i++) {
-			blockedChannels[i] = false;
-		}
-
-		bufferStorage.rollOver();
-
-		// the next barrier that comes must assume it is the first
-		numBarriersReceived = 0;
-
-		if (startOfAlignmentTimestamp > 0) {
-			latestAlignmentDurationNanos = System.nanoTime() - startOfAlignmentTimestamp;
-			startOfAlignmentTimestamp = 0;
-		}
-	}
-
 	// ------------------------------------------------------------------------
 	//  Properties
 	// ------------------------------------------------------------------------
@@ -501,22 +209,17 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 	 * @return The ID of the pending of completed checkpoint.
 	 */
 	public long getCurrentCheckpointId() {
-		return this.currentCheckpointId;
+		return barrierAligner.getCurrentCheckpointId();
 	}
 
 	@Override
 	public long getAlignmentDurationNanos() {
-		long start = this.startOfAlignmentTimestamp;
-		if (start <= 0) {
-			return latestAlignmentDurationNanos;
-		} else {
-			return System.nanoTime() - start;
-		}
+		return barrierAligner.getAlignmentDurationNanos();
 	}
 
 	@Override
 	public int getNumberOfInputChannels() {
-		return totalNumberOfInputChannels;
+		return inputGate.getNumberOfInputChannels();
 	}
 
 	// ------------------------------------------------------------------------
@@ -525,10 +228,6 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 
 	@Override
 	public String toString() {
-		return String.format("%s: last checkpoint: %d, current barriers: %d, closed channels: %d",
-			taskName,
-			currentCheckpointId,
-			numBarriersReceived,
-			numClosedChannels);
+		return barrierAligner.toString();
 	}
 }
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BufferStorage.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BufferStorage.java
index 8e6194d..4ad7eac 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BufferStorage.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BufferStorage.java
@@ -67,8 +67,6 @@ public interface BufferStorage extends AutoCloseable {
 
 	Optional<BufferOrEvent> pollNext() throws IOException;
 
-	long currentBufferedSize();
-
 	long getMaxBufferedBytes();
 
 	/**
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierAligner.java
similarity index 55%
copy from flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
copy to flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierAligner.java
index 0f8fa40..30e05c1 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierAligner.java
@@ -1,12 +1,13 @@
 /*
- * 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
+ * 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
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -18,16 +19,12 @@
 package org.apache.flink.streaming.runtime.io;
 
 import org.apache.flink.annotation.Internal;
-import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.runtime.checkpoint.CheckpointException;
 import org.apache.flink.runtime.checkpoint.CheckpointFailureReason;
 import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
 import org.apache.flink.runtime.checkpoint.CheckpointMetrics;
 import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker;
 import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
-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.jobgraph.tasks.AbstractInvokable;
 
 import org.slf4j.Logger;
@@ -36,26 +33,16 @@ import org.slf4j.LoggerFactory;
 import javax.annotation.Nullable;
 
 import java.io.IOException;
-import java.util.Optional;
-import java.util.concurrent.CompletableFuture;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * The barrier buffer is {@link CheckpointBarrierHandler} that blocks inputs with barriers until
- * all inputs have received the barrier for a given checkpoint.
- *
- * <p>To avoid back-pressuring the input streams (which may cause distributed deadlocks), the
- * BarrierBuffer continues receiving buffers from the blocked channels and stores them internally until
- * the blocks are released.
+ * {@link CheckpointBarrierAligner} keep tracks of received {@link CheckpointBarrier} on given
+ * channels and controls the alignment, by deciding which channels should be blocked and when to
+ * release blocked channels.
  */
 @Internal
-public class BarrierBuffer implements CheckpointBarrierHandler {
-
-	private static final Logger LOG = LoggerFactory.getLogger(BarrierBuffer.class);
+public class CheckpointBarrierAligner {
 
-	/** The gate that the buffer draws its input from. */
-	private final InputGate inputGate;
+	private static final Logger LOG = LoggerFactory.getLogger(CheckpointBarrierAligner.class);
 
 	/** Flags that indicate whether a channel is currently blocked/buffered. */
 	private final boolean[] blockedChannels;
@@ -63,9 +50,6 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 	/** The total number of channels that this buffer handles data from. */
 	private final int totalNumberOfInputChannels;
 
-	/** To utility to write blocked data to a file channel. */
-	private final BufferStorage bufferStorage;
-
 	private final String taskName;
 
 	@Nullable
@@ -89,132 +73,47 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 	/** The time (in nanoseconds) that the latest alignment took. */
 	private long latestAlignmentDurationNanos;
 
-	/** Flag to indicate whether we have drawn all available input. */
-	private boolean endOfStream;
-
-	/** Indicate end of the input. Set to true after encountering {@link #endOfStream} and depleting
-	 * {@link #bufferStorage}. */
-	private boolean isFinished;
-
-	/**
-	 * Creates a new checkpoint stream aligner.
-	 *
-	 * <p>There is no limit to how much data may be buffered during an alignment.
-	 *
-	 * @param inputGate The input gate to draw the buffers and events from.
-	 * @param bufferStorage The storage to hold the buffers and events for blocked channels.
-	 */
-	@VisibleForTesting
-	BarrierBuffer(InputGate inputGate, BufferStorage bufferStorage) {
-		this (inputGate, bufferStorage, "Testing: No task associated", null);
-	}
-
-	/**
-	 * Creates a new checkpoint stream aligner.
-	 *
-	 * <p>The aligner will allow only alignments that buffer up to the given number of bytes.
-	 * When that number is exceeded, it will stop the alignment and notify the task that the
-	 * checkpoint has been cancelled.
-	 *
-	 * @param inputGate The input gate to draw the buffers and events from.
-	 * @param bufferStorage The storage to hold the buffers and events for blocked channels.
-	 * @param taskName The task name for logging.
-	 * @param toNotifyOnCheckpoint optional Handler that receives the checkpoint notifications.
-	 */
-	BarrierBuffer(
-		InputGate inputGate,
-		BufferStorage bufferStorage,
-		String taskName,
-		@Nullable AbstractInvokable toNotifyOnCheckpoint) {
-
-		this.inputGate = inputGate;
-		this.totalNumberOfInputChannels = inputGate.getNumberOfInputChannels();
-		this.blockedChannels = new boolean[this.totalNumberOfInputChannels];
-
-		this.bufferStorage = checkNotNull(bufferStorage);
-
+	CheckpointBarrierAligner(
+			int totalNumberOfInputChannels,
+			String taskName,
+			@Nullable AbstractInvokable toNotifyOnCheckpoint) {
+		this.totalNumberOfInputChannels = totalNumberOfInputChannels;
 		this.taskName = taskName;
 		this.toNotifyOnCheckpoint = toNotifyOnCheckpoint;
-	}
 
-	@Override
-	public CompletableFuture<?> isAvailable() {
-		if (bufferStorage.isEmpty()) {
-			return inputGate.isAvailable();
-		}
-		return AVAILABLE;
+		this.blockedChannels = new boolean[totalNumberOfInputChannels];
 	}
 
-	// ------------------------------------------------------------------------
-	//  Buffer and barrier handling
-	// ------------------------------------------------------------------------
+	public void releaseBlocksAndResetBarriers() throws IOException {
+		LOG.debug("{}: End of stream alignment, feeding buffered data back.", taskName);
 
-	@Override
-	public Optional<BufferOrEvent> pollNext() throws Exception {
-		while (true) {
-			// process buffered BufferOrEvents before grabbing new ones
-			Optional<BufferOrEvent> next;
-			if (bufferStorage.isEmpty()) {
-				next = inputGate.pollNext();
-			}
-			else {
-				// TODO: FLINK-12536 for non credit-based flow control, getNext method is blocking
-				next = bufferStorage.pollNext();
-				if (!next.isPresent()) {
-					return pollNext();
-				}
-			}
+		for (int i = 0; i < blockedChannels.length; i++) {
+			blockedChannels[i] = false;
+		}
 
-			if (!next.isPresent()) {
-				return handleEmptyBuffer();
-			}
+		// the next barrier that comes must assume it is the first
+		numBarriersReceived = 0;
 
-			BufferOrEvent bufferOrEvent = next.get();
-			if (isBlocked(bufferOrEvent.getChannelIndex())) {
-				// if the channel is blocked, we just store the BufferOrEvent
-				bufferStorage.add(bufferOrEvent);
-				if (bufferStorage.isFull()) {
-					sizeLimitExceeded();
-				}
-			}
-			else if (bufferOrEvent.isBuffer()) {
-				return next;
-			}
-			else if (bufferOrEvent.getEvent().getClass() == CheckpointBarrier.class) {
-				if (!endOfStream) {
-					// process barriers only if there is a chance of the checkpoint completing
-					processBarrier((CheckpointBarrier) bufferOrEvent.getEvent(), bufferOrEvent.getChannelIndex());
-				}
-			}
-			else if (bufferOrEvent.getEvent().getClass() == CancelCheckpointMarker.class) {
-				processCancellationBarrier((CancelCheckpointMarker) bufferOrEvent.getEvent());
-			}
-			else {
-				if (bufferOrEvent.getEvent().getClass() == EndOfPartitionEvent.class) {
-					processEndOfPartition();
-				}
-				return next;
-			}
+		if (startOfAlignmentTimestamp > 0) {
+			latestAlignmentDurationNanos = System.nanoTime() - startOfAlignmentTimestamp;
+			startOfAlignmentTimestamp = 0;
 		}
 	}
 
-	private Optional<BufferOrEvent> handleEmptyBuffer() throws Exception {
-		if (!inputGate.isFinished()) {
-			return Optional.empty();
-		}
-
-		if (endOfStream) {
-			isFinished = true;
-			return Optional.empty();
-		} else {
-			// end of input stream. stream continues with the buffered data
-			endOfStream = true;
-			releaseBlocksAndResetBarriers();
-			return pollNext();
-		}
+	/**
+	 * Checks whether the channel with the given index is blocked.
+	 *
+	 * @param channelIndex The channel index to check.
+	 * @return True if the channel is blocked, false if not.
+	 */
+	public boolean isBlocked(int channelIndex) {
+		return blockedChannels[channelIndex];
 	}
 
-	private void processBarrier(CheckpointBarrier receivedBarrier, int channelIndex) throws Exception {
+	/**
+	 * @return true if some blocked data should be unblocked/rolled over.
+	 */
+	public boolean processBarrier(CheckpointBarrier receivedBarrier, int channelIndex, long bufferedBytes) throws Exception {
 		final long barrierId = receivedBarrier.getId();
 
 		// fast path for single channel cases
@@ -222,11 +121,13 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 			if (barrierId > currentCheckpointId) {
 				// new checkpoint
 				currentCheckpointId = barrierId;
-				notifyCheckpoint(receivedBarrier);
+				notifyCheckpoint(receivedBarrier, bufferedBytes);
 			}
-			return;
+			return false;
 		}
 
+		boolean checkpointAborted = false;
+
 		// -- general code path for multiple input channels --
 
 		if (numBarriersReceived > 0) {
@@ -252,13 +153,14 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 
 				// abort the current checkpoint
 				releaseBlocksAndResetBarriers();
+				checkpointAborted = true;
 
 				// begin a the new checkpoint
 				beginNewAlignment(barrierId, channelIndex);
 			}
 			else {
 				// ignore trailing barrier from an earlier checkpoint (obsolete now)
-				return;
+				return false;
 			}
 		}
 		else if (barrierId > currentCheckpointId) {
@@ -268,7 +170,7 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 		else {
 			// either the current checkpoint was canceled (numBarriers == 0) or
 			// this barrier is from an old subsumed checkpoint
-			return;
+			return false;
 		}
 
 		// check if we have all barriers - since canceled checkpoints always have zero barriers
@@ -283,11 +185,47 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 			}
 
 			releaseBlocksAndResetBarriers();
-			notifyCheckpoint(receivedBarrier);
+			notifyCheckpoint(receivedBarrier, bufferedBytes);
+			return true;
+		}
+		return checkpointAborted;
+	}
+
+	protected void beginNewAlignment(long checkpointId, int channelIndex) throws IOException {
+		currentCheckpointId = checkpointId;
+		onBarrier(channelIndex);
+
+		startOfAlignmentTimestamp = System.nanoTime();
+
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("{}: Starting stream alignment for checkpoint {}.", taskName, checkpointId);
 		}
 	}
 
-	private void processCancellationBarrier(CancelCheckpointMarker cancelBarrier) throws Exception {
+	/**
+	 * Blocks the given channel index, from which a barrier has been received.
+	 *
+	 * @param channelIndex The channel index to block.
+	 */
+	protected void onBarrier(int channelIndex) throws IOException {
+		if (!blockedChannels[channelIndex]) {
+			blockedChannels[channelIndex] = true;
+
+			numBarriersReceived++;
+
+			if (LOG.isDebugEnabled()) {
+				LOG.debug("{}: Received barrier from channel {}.", taskName, channelIndex);
+			}
+		}
+		else {
+			throw new IOException("Stream corrupt: Repeated barrier for same checkpoint on input " + channelIndex);
+		}
+	}
+
+	/**
+	 * @return true if some blocked data should be unblocked/rolled over.
+	 */
+	public boolean processCancellationBarrier(CancelCheckpointMarker cancelBarrier) throws Exception {
 		final long barrierId = cancelBarrier.getCheckpointId();
 
 		// fast path for single channel cases
@@ -297,7 +235,7 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 				currentCheckpointId = barrierId;
 				notifyAbortOnCancellationBarrier(barrierId);
 			}
-			return;
+			return false;
 		}
 
 		// -- general code path for multiple input channels --
@@ -313,6 +251,7 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 
 				releaseBlocksAndResetBarriers();
 				notifyAbortOnCancellationBarrier(barrierId);
+				return true;
 			}
 			else if (barrierId > currentCheckpointId) {
 				// we canceled the next which also cancels the current
@@ -331,6 +270,7 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 				latestAlignmentDurationNanos = 0L;
 
 				notifyAbortOnCancellationBarrier(barrierId);
+				return true;
 			}
 
 			// else: ignore trailing (cancellation) barrier from an earlier checkpoint (obsolete now)
@@ -351,33 +291,39 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 			}
 
 			notifyAbortOnCancellationBarrier(barrierId);
+			return false;
 		}
 
 		// else: trailing barrier from either
 		//   - a previous (subsumed) checkpoint
 		//   - the current checkpoint if it was already canceled
+		return false;
 	}
 
-	private void processEndOfPartition() throws Exception {
+	/**
+	 * @return true if some blocked data should be unblocked/rolled over.
+	 */
+	public boolean processEndOfPartition() throws Exception {
 		numClosedChannels++;
 
 		if (numBarriersReceived > 0) {
 			// let the task know we skip a checkpoint
 			notifyAbort(currentCheckpointId,
 				new CheckpointException(CheckpointFailureReason.CHECKPOINT_DECLINED_INPUT_END_OF_STREAM));
-
 			// no chance to complete this checkpoint
 			releaseBlocksAndResetBarriers();
+			return true;
 		}
+		return false;
 	}
 
-	private void notifyCheckpoint(CheckpointBarrier checkpointBarrier) throws Exception {
+	private void notifyCheckpoint(CheckpointBarrier checkpointBarrier, long bufferedBytes) throws Exception {
 		if (toNotifyOnCheckpoint != null) {
 			CheckpointMetaData checkpointMetaData =
 				new CheckpointMetaData(checkpointBarrier.getId(), checkpointBarrier.getTimestamp());
 
 			CheckpointMetrics checkpointMetrics = new CheckpointMetrics()
-				.setBytesBufferedInAlignment(bufferStorage.currentBufferedSize())
+				.setBytesBufferedInAlignment(bufferedBytes)
 				.setAlignmentDurationNanos(latestAlignmentDurationNanos);
 
 			toNotifyOnCheckpoint.triggerCheckpointOnBarrier(
@@ -398,132 +344,19 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 		}
 	}
 
-	private void sizeLimitExceeded() throws Exception {
-		long maxBufferedBytes = bufferStorage.getMaxBufferedBytes();
-		// exceeded our limit - abort this checkpoint
-		LOG.info("{}: Checkpoint {} aborted because alignment volume limit ({} bytes) exceeded.",
-			taskName,
-			currentCheckpointId,
-			maxBufferedBytes);
-
-		releaseBlocksAndResetBarriers();
-		notifyAbort(currentCheckpointId,
-			new CheckpointException(
-				"Max buffered bytes: " + maxBufferedBytes,
-				CheckpointFailureReason.CHECKPOINT_DECLINED_ALIGNMENT_LIMIT_EXCEEDED));
-	}
-
-	@Override
-	public boolean isEmpty() {
-		return bufferStorage.isEmpty();
-	}
-
-	@Override
-	public boolean isFinished() {
-		return isFinished;
-	}
-
-	@Override
-	public void cleanup() throws IOException {
-		bufferStorage.close();
-	}
-
-	private void beginNewAlignment(long checkpointId, int channelIndex) throws IOException {
-		currentCheckpointId = checkpointId;
-		onBarrier(channelIndex);
-
-		startOfAlignmentTimestamp = System.nanoTime();
-
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("{}: Starting stream alignment for checkpoint {}.", taskName, checkpointId);
-		}
-	}
-
-	/**
-	 * Checks whether the channel with the given index is blocked.
-	 *
-	 * @param channelIndex The channel index to check.
-	 * @return True if the channel is blocked, false if not.
-	 */
-	private boolean isBlocked(int channelIndex) {
-		return blockedChannels[channelIndex];
-	}
-
-	/**
-	 * Blocks the given channel index, from which a barrier has been received.
-	 *
-	 * @param channelIndex The channel index to block.
-	 */
-	private void onBarrier(int channelIndex) throws IOException {
-		if (!blockedChannels[channelIndex]) {
-			blockedChannels[channelIndex] = true;
-
-			numBarriersReceived++;
-
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("{}: Received barrier from channel {}.", taskName, channelIndex);
-			}
-		}
-		else {
-			throw new IOException("Stream corrupt: Repeated barrier for same checkpoint on input " + channelIndex);
-		}
-	}
-
-	/**
-	 * Releases the blocks on all channels and resets the barrier count.
-	 * Makes sure the just written data is the next to be consumed.
-	 */
-	private void releaseBlocksAndResetBarriers() throws IOException {
-		LOG.debug("{}: End of stream alignment, feeding buffered data back.", taskName);
-
-		for (int i = 0; i < blockedChannels.length; i++) {
-			blockedChannels[i] = false;
-		}
-
-		bufferStorage.rollOver();
-
-		// the next barrier that comes must assume it is the first
-		numBarriersReceived = 0;
-
-		if (startOfAlignmentTimestamp > 0) {
-			latestAlignmentDurationNanos = System.nanoTime() - startOfAlignmentTimestamp;
-			startOfAlignmentTimestamp = 0;
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Properties
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Gets the ID defining the current pending, or just completed, checkpoint.
-	 *
-	 * @return The ID of the pending of completed checkpoint.
-	 */
 	public long getCurrentCheckpointId() {
-		return this.currentCheckpointId;
+		return currentCheckpointId;
 	}
 
-	@Override
 	public long getAlignmentDurationNanos() {
-		long start = this.startOfAlignmentTimestamp;
-		if (start <= 0) {
+		if (startOfAlignmentTimestamp <= 0) {
 			return latestAlignmentDurationNanos;
 		} else {
-			return System.nanoTime() - start;
+			return System.nanoTime() - startOfAlignmentTimestamp;
 		}
 	}
 
 	@Override
-	public int getNumberOfInputChannels() {
-		return totalNumberOfInputChannels;
-	}
-
-	// ------------------------------------------------------------------------
-	// Utilities
-	// ------------------------------------------------------------------------
-
-	@Override
 	public String toString() {
 		return String.format("%s: last checkpoint: %d, current barriers: %d, closed channels: %d",
 			taskName,
@@ -531,4 +364,12 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 			numBarriersReceived,
 			numClosedChannels);
 	}
+
+	public void checkpointSizeLimitExceeded(long maxBufferedBytes) throws Exception {
+		releaseBlocksAndResetBarriers();
+		notifyAbort(currentCheckpointId,
+			new CheckpointException(
+				"Max buffered bytes: " + maxBufferedBytes,
+				CheckpointFailureReason.CHECKPOINT_DECLINED_ALIGNMENT_LIMIT_EXCEEDED));
+	}
 }


[flink] 12/16: [hotfix][operator] Fix checkpointing lock in StreamTwoInputSelectableProcessor

Posted by pn...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pnowojski pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 530a5e445785df9a85260b2654b776839fc74708
Author: Piotr Nowojski <pi...@gmail.com>
AuthorDate: Thu Jun 20 20:07:47 2019 +0200

    [hotfix][operator] Fix checkpointing lock in StreamTwoInputSelectableProcessor
---
 .../flink/streaming/runtime/tasks/TwoInputSelectableStreamTask.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputSelectableStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputSelectableStreamTask.java
index 1199901..b577b20 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputSelectableStreamTask.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputSelectableStreamTask.java
@@ -49,7 +49,7 @@ public class TwoInputSelectableStreamTask<IN1, IN2, OUT> extends AbstractTwoInpu
 		this.inputProcessor = new StreamTwoInputSelectableProcessor<>(
 			inputGates1, inputGates2,
 			inputDeserializer1, inputDeserializer2,
-			this,
+			getCheckpointLock(),
 			getEnvironment().getIOManager(),
 			getStreamStatusMaintainer(),
 			this.headOperator,


[flink] 03/16: [hotfix][network] Move queuedBuffered and currentBuffered fields to BufferStorage

Posted by pn...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pnowojski pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit d6093ebbcb008914e8dec6e75645a0e628428568
Author: Piotr Nowojski <pi...@gmail.com>
AuthorDate: Thu Jun 13 15:29:15 2019 +0200

    [hotfix][network] Move queuedBuffered and currentBuffered fields to BufferStorage
    
    This makes BufferStorage contract more complete. Now it takes care of the whole process
    of storing and returning the data with simpler interface (single #rollOver method
    vs two different as it was before).
---
 .../runtime/io/AbstractBufferStorage.java          | 175 ++++++++++++++++++
 .../flink/streaming/runtime/io/BarrierBuffer.java  | 130 +++-----------
 .../flink/streaming/runtime/io/BufferSpiller.java  |  24 ++-
 .../flink/streaming/runtime/io/BufferStorage.java  |  49 +++--
 .../streaming/runtime/io/CachedBufferStorage.java  |  22 ++-
 .../streaming/runtime/io/InputProcessorUtil.java   |   6 +-
 .../io/BarrierBufferAlignmentLimitTest.java        |   6 +-
 .../streaming/runtime/io/BufferSpillerTest.java    |   2 +-
 .../runtime/io/BufferStorageTestBase.java          | 197 ++++++++-------------
 .../runtime/io/CreditBasedBarrierBufferTest.java   |   2 +-
 .../runtime/io/SpillingBarrierBufferTest.java      |   2 +-
 11 files changed, 344 insertions(+), 271 deletions(-)

diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/AbstractBufferStorage.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/AbstractBufferStorage.java
new file mode 100644
index 0000000..f7e4dd7
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/AbstractBufferStorage.java
@@ -0,0 +1,175 @@
+/*
+ * 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.streaming.runtime.io;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.Optional;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/**
+ * A default abstract based class for {@link BufferStorage} implementations.
+ */
+@Internal
+public abstract class AbstractBufferStorage implements BufferStorage {
+
+	protected static final Logger LOG = LoggerFactory.getLogger(AbstractBufferStorage.class);
+
+	/**
+	 * The pending blocked buffer/event sequences. Must be consumed before requesting further data
+	 * from the input gate.
+	 */
+	protected final ArrayDeque<BufferOrEventSequence> queuedBuffered = new ArrayDeque<>();
+
+	protected final long maxBufferedBytes;
+
+	protected final String taskName;
+
+	/**
+	 * The sequence of buffers/events that has been unblocked and must now be consumed before
+	 * requesting further data from the input gate.
+	 */
+	protected BufferOrEventSequence currentBuffered;
+
+	/** The number of bytes in the queued spilled sequences. */
+	protected long rolledBytes;
+
+	protected AbstractBufferStorage(long maxBufferedBytes, String taskName) {
+		checkArgument(maxBufferedBytes == -1 || maxBufferedBytes > 0);
+
+		this.maxBufferedBytes = maxBufferedBytes;
+		this.taskName = taskName;
+	}
+
+	@Override
+	public boolean isFull() {
+		return maxBufferedBytes > 0 && (getRolledBytes() + getPendingBytes()) > maxBufferedBytes;
+	}
+
+	@Override
+	public void rollOver() throws IOException {
+		if (currentBuffered == null) {
+			// common case: no more buffered data
+			currentBuffered = rollOverReusingResources();
+			if (currentBuffered != null) {
+				currentBuffered.open();
+			}
+		}
+		else {
+			// uncommon case: buffered data pending
+			// push back the pending data, if we have any
+			LOG.debug("{}: Checkpoint skipped via buffered data:" +
+				"Pushing back current alignment buffers and feeding back new alignment data first.", taskName);
+
+			// since we did not fully drain the previous sequence, we need to allocate a new buffer for this one
+			BufferOrEventSequence bufferedNow = rollOverWithoutReusingResources();
+			if (bufferedNow != null) {
+				bufferedNow.open();
+				queuedBuffered.addFirst(currentBuffered);
+				rolledBytes += currentBuffered.size();
+				currentBuffered = bufferedNow;
+			}
+		}
+
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("{}: Size of buffered data: {} bytes",
+				taskName,
+				currentBuffered == null ? 0L : currentBuffered.size());
+		}
+	}
+
+	/**
+	 * Starts a new sequence of buffers and event without reusing the same resources and
+	 * returns the current sequence of buffers for reading.
+	 *
+	 * @return The readable sequence of buffers and events, or 'null', if nothing was added.
+	 */
+	protected abstract BufferOrEventSequence rollOverWithoutReusingResources() throws IOException;
+
+	/**
+	 * Starts a new sequence of buffers and event reusing the same resources and
+	 * returns the current sequence of buffers for reading.
+	 *
+	 * @return The readable sequence of buffers and events, or 'null', if nothing was added.
+	 */
+	protected abstract BufferOrEventSequence rollOverReusingResources() throws IOException;
+
+	@Override
+	public void close() throws IOException {
+		if (currentBuffered != null) {
+			currentBuffered.cleanup();
+		}
+		for (BufferOrEventSequence seq : queuedBuffered) {
+			seq.cleanup();
+		}
+		queuedBuffered.clear();
+		rolledBytes = 0L;
+	}
+
+	@Override
+	public long getRolledBytes() {
+		return rolledBytes;
+	}
+
+	@Override
+	public boolean isEmpty() {
+		return currentBuffered == null;
+	}
+
+	@Override
+	public Optional<BufferOrEvent> pollNext() throws IOException {
+		if (currentBuffered == null) {
+			return Optional.empty();
+		}
+		// TODO: FLINK-12536 for non credit-based flow control, getNext method is blocking
+		Optional<BufferOrEvent> next = Optional.ofNullable(currentBuffered.getNext());
+		if (!next.isPresent()) {
+			completeBufferedSequence();
+		}
+		return next;
+	}
+
+	protected void completeBufferedSequence() throws IOException {
+		LOG.debug("{}: Finished feeding back buffered data.", taskName);
+
+		currentBuffered.cleanup();
+		currentBuffered = queuedBuffered.pollFirst();
+		if (currentBuffered != null) {
+			currentBuffered.open();
+			rolledBytes -= currentBuffered.size();
+		}
+	}
+
+	@Override
+	public long currentBufferedSize() {
+		return currentBuffered != null ? currentBuffered.size() : 0L;
+	}
+
+	@Override
+	public long getMaxBufferedBytes() {
+		return maxBufferedBytes;
+	}
+}
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
index b2e6ea1..23717f5 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
@@ -36,11 +36,9 @@ import org.slf4j.LoggerFactory;
 import javax.annotation.Nullable;
 
 import java.io.IOException;
-import java.util.ArrayDeque;
 import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 
-import static org.apache.flink.util.Preconditions.checkArgument;
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
@@ -68,26 +66,8 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 	/** To utility to write blocked data to a file channel. */
 	private final BufferStorage bufferStorage;
 
-	/**
-	 * The pending blocked buffer/event sequences. Must be consumed before requesting further data
-	 * from the input gate.
-	 */
-	private final ArrayDeque<BufferOrEventSequence> queuedBuffered;
-
-	/**
-	 * The maximum number of bytes that may be buffered before an alignment is broken. -1 means
-	 * unlimited.
-	 */
-	private final long maxBufferedBytes;
-
 	private final String taskName;
 
-	/**
-	 * The sequence of buffers/events that has been unblocked and must now be consumed before
-	 * requesting further data from the input gate.
-	 */
-	private BufferOrEventSequence currentBuffered;
-
 	@Nullable
 	private final AbstractInvokable toNotifyOnCheckpoint;
 
@@ -103,9 +83,6 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 	/** The number of already closed channels. */
 	private int numClosedChannels;
 
-	/** The number of bytes in the queued spilled sequences. */
-	private long numQueuedBytes;
-
 	/** The timestamp as in {@link System#nanoTime()} at which the last alignment started. */
 	private long startOfAlignmentTimestamp;
 
@@ -116,7 +93,7 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 	private boolean endOfStream;
 
 	/** Indicate end of the input. Set to true after encountering {@link #endOfStream} and depleting
-	 * {@link #currentBuffered}. */
+	 * {@link #bufferStorage}. */
 	private boolean isFinished;
 
 	/**
@@ -129,7 +106,7 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 	 */
 	@VisibleForTesting
 	BarrierBuffer(InputGate inputGate, BufferStorage bufferStorage) {
-		this (inputGate, bufferStorage, -1, "Testing: No task associated", null);
+		this (inputGate, bufferStorage, "Testing: No task associated", null);
 	}
 
 	/**
@@ -141,25 +118,20 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 	 *
 	 * @param inputGate The input gate to draw the buffers and events from.
 	 * @param bufferStorage The storage to hold the buffers and events for blocked channels.
-	 * @param maxBufferedBytes The maximum bytes to be buffered before the checkpoint aborts.
 	 * @param taskName The task name for logging.
 	 * @param toNotifyOnCheckpoint optional Handler that receives the checkpoint notifications.
 	 */
 	BarrierBuffer(
-			InputGate inputGate,
-			BufferStorage bufferStorage,
-			long maxBufferedBytes,
-			String taskName,
-			@Nullable AbstractInvokable toNotifyOnCheckpoint) {
-		checkArgument(maxBufferedBytes == -1 || maxBufferedBytes > 0);
+		InputGate inputGate,
+		BufferStorage bufferStorage,
+		String taskName,
+		@Nullable AbstractInvokable toNotifyOnCheckpoint) {
 
 		this.inputGate = inputGate;
-		this.maxBufferedBytes = maxBufferedBytes;
 		this.totalNumberOfInputChannels = inputGate.getNumberOfInputChannels();
 		this.blockedChannels = new boolean[this.totalNumberOfInputChannels];
 
 		this.bufferStorage = checkNotNull(bufferStorage);
-		this.queuedBuffered = new ArrayDeque<BufferOrEventSequence>();
 
 		this.taskName = taskName;
 		this.toNotifyOnCheckpoint = toNotifyOnCheckpoint;
@@ -167,7 +139,7 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 
 	@Override
 	public CompletableFuture<?> isAvailable() {
-		if (currentBuffered == null) {
+		if (bufferStorage.isEmpty()) {
 			return inputGate.isAvailable();
 		}
 		return AVAILABLE;
@@ -182,14 +154,13 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 		while (true) {
 			// process buffered BufferOrEvents before grabbing new ones
 			Optional<BufferOrEvent> next;
-			if (currentBuffered == null) {
+			if (bufferStorage.isEmpty()) {
 				next = inputGate.pollNext();
 			}
 			else {
 				// TODO: FLINK-12536 for non credit-based flow control, getNext method is blocking
-				next = Optional.ofNullable(currentBuffered.getNext());
+				next = bufferStorage.pollNext();
 				if (!next.isPresent()) {
-					completeBufferedSequence();
 					return pollNext();
 				}
 			}
@@ -202,7 +173,9 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 			if (isBlocked(bufferOrEvent.getChannelIndex())) {
 				// if the channel is blocked, we just store the BufferOrEvent
 				bufferStorage.add(bufferOrEvent);
-				checkSizeLimit();
+				if (bufferStorage.isFull()) {
+					sizeLimitExceeded();
+				}
 			}
 			else if (bufferOrEvent.isBuffer()) {
 				return next;
@@ -241,17 +214,6 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 		}
 	}
 
-	private void completeBufferedSequence() throws IOException {
-		LOG.debug("{}: Finished feeding back buffered data.", taskName);
-
-		currentBuffered.cleanup();
-		currentBuffered = queuedBuffered.pollFirst();
-		if (currentBuffered != null) {
-			currentBuffered.open();
-			numQueuedBytes -= currentBuffered.size();
-		}
-	}
-
 	private void processBarrier(CheckpointBarrier receivedBarrier, int channelIndex) throws Exception {
 		final long barrierId = receivedBarrier.getId();
 
@@ -420,10 +382,8 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 			CheckpointMetaData checkpointMetaData =
 					new CheckpointMetaData(checkpointBarrier.getId(), checkpointBarrier.getTimestamp());
 
-			long bytesBuffered = currentBuffered != null ? currentBuffered.size() : 0L;
-
 			CheckpointMetrics checkpointMetrics = new CheckpointMetrics()
-					.setBytesBufferedInAlignment(bytesBuffered)
+					.setBytesBufferedInAlignment(bufferStorage.currentBufferedSize())
 					.setAlignmentDurationNanos(latestAlignmentDurationNanos);
 
 			toNotifyOnCheckpoint.triggerCheckpointOnBarrier(
@@ -444,25 +404,24 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 		}
 	}
 
-	private void checkSizeLimit() throws Exception {
-		if (maxBufferedBytes > 0 && (numQueuedBytes + bufferStorage.getBytesBlocked()) > maxBufferedBytes) {
-			// exceeded our limit - abort this checkpoint
-			LOG.info("{}: Checkpoint {} aborted because alignment volume limit ({} bytes) exceeded.",
-				taskName,
-				currentCheckpointId,
-				maxBufferedBytes);
+	private void sizeLimitExceeded() throws Exception {
+		long maxBufferedBytes = bufferStorage.getMaxBufferedBytes();
+		// exceeded our limit - abort this checkpoint
+		LOG.info("{}: Checkpoint {} aborted because alignment volume limit ({} bytes) exceeded.",
+			taskName,
+			currentCheckpointId,
+			maxBufferedBytes);
 
-			releaseBlocksAndResetBarriers();
-			notifyAbort(currentCheckpointId,
-				new CheckpointException(
-					"Max buffered bytes: " + maxBufferedBytes,
-					CheckpointFailureReason.CHECKPOINT_DECLINED_ALIGNMENT_LIMIT_EXCEEDED));
-		}
+		releaseBlocksAndResetBarriers();
+		notifyAbort(currentCheckpointId,
+			new CheckpointException(
+				"Max buffered bytes: " + maxBufferedBytes,
+				CheckpointFailureReason.CHECKPOINT_DECLINED_ALIGNMENT_LIMIT_EXCEEDED));
 	}
 
 	@Override
 	public boolean isEmpty() {
-		return currentBuffered == null;
+		return bufferStorage.isEmpty();
 	}
 
 	@Override
@@ -473,14 +432,6 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 	@Override
 	public void cleanup() throws IOException {
 		bufferStorage.close();
-		if (currentBuffered != null) {
-			currentBuffered.cleanup();
-		}
-		for (BufferOrEventSequence seq : queuedBuffered) {
-			seq.cleanup();
-		}
-		queuedBuffered.clear();
-		numQueuedBytes = 0L;
 	}
 
 	private void beginNewAlignment(long checkpointId, int channelIndex) throws IOException {
@@ -535,34 +486,7 @@ public class BarrierBuffer implements CheckpointBarrierHandler {
 			blockedChannels[i] = false;
 		}
 
-		if (currentBuffered == null) {
-			// common case: no more buffered data
-			currentBuffered = bufferStorage.rollOverReusingResources();
-			if (currentBuffered != null) {
-				currentBuffered.open();
-			}
-		}
-		else {
-			// uncommon case: buffered data pending
-			// push back the pending data, if we have any
-			LOG.debug("{}: Checkpoint skipped via buffered data:" +
-					"Pushing back current alignment buffers and feeding back new alignment data first.", taskName);
-
-			// since we did not fully drain the previous sequence, we need to allocate a new buffer for this one
-			BufferOrEventSequence bufferedNow = bufferStorage.rollOverWithoutReusingResources();
-			if (bufferedNow != null) {
-				bufferedNow.open();
-				queuedBuffered.addFirst(currentBuffered);
-				numQueuedBytes += currentBuffered.size();
-				currentBuffered = bufferedNow;
-			}
-		}
-
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("{}: Size of buffered data: {} bytes",
-				taskName,
-				currentBuffered == null ? 0L : currentBuffered.size());
-		}
+		bufferStorage.rollOver();
 
 		// the next barrier that comes must assume it is the first
 		numBarriersReceived = 0;
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java
index 59877a0..4fbfaee 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java
@@ -53,7 +53,7 @@ import java.util.concurrent.atomic.AtomicInteger;
  */
 @Internal
 @Deprecated
-public class BufferSpiller implements BufferStorage {
+public class BufferSpiller extends AbstractBufferStorage {
 
 	/** Size of header in bytes (see add method). */
 	static final int HEADER_SIZE = 9;
@@ -91,14 +91,25 @@ public class BufferSpiller implements BufferStorage {
 	/** The number of bytes written since the last roll over. */
 	private long bytesWritten;
 
+	public BufferSpiller(IOManager ioManager, int pageSize) throws IOException {
+		this(ioManager, pageSize, -1);
+	}
+
+	public BufferSpiller(IOManager ioManager, int pageSize, long maxBufferedBytes) throws IOException {
+		this(ioManager, pageSize, maxBufferedBytes, "Unknown");
+	}
+
 	/**
 	 * Creates a new {@link BufferSpiller}, spilling to one of the I/O manager's temp directories.
 	 *
 	 * @param ioManager The I/O manager for access to the temp directories.
 	 * @param pageSize The page size used to re-create spilled buffers.
+	 * @param maxBufferedBytes The maximum bytes to be buffered before the checkpoint aborts.
+	 * @param taskName The task name for logging.
 	 * @throws IOException Thrown if the temp files for spilling cannot be initialized.
 	 */
-	public BufferSpiller(IOManager ioManager, int pageSize) throws IOException {
+	public BufferSpiller(IOManager ioManager, int pageSize, long maxBufferedBytes, String taskName) throws IOException {
+		super(maxBufferedBytes, taskName);
 		this.pageSize = pageSize;
 
 		this.readBuffer = ByteBuffer.allocateDirect(READ_BUFFER_SIZE);
@@ -118,12 +129,6 @@ public class BufferSpiller implements BufferStorage {
 		createSpillingChannel();
 	}
 
-	/**
-	 * Adds a buffer or event to the sequence of spilled buffers and events.
-	 *
-	 * @param boe The buffer or event to add and spill.
-	 * @throws IOException Thrown, if the buffer of event could not be spilled.
-	 */
 	@Override
 	public void add(BufferOrEvent boe) throws IOException {
 		try {
@@ -222,6 +227,7 @@ public class BufferSpiller implements BufferStorage {
 		if (!currentSpillFile.delete()) {
 			throw new IOException("Cannot delete spill file");
 		}
+		super.close();
 	}
 
 	/**
@@ -230,7 +236,7 @@ public class BufferSpiller implements BufferStorage {
 	 * @return the number of bytes written in the current spill file
 	 */
 	@Override
-	public long getBytesBlocked() {
+	public long getPendingBytes() {
 		return bytesWritten;
 	}
 
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BufferStorage.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BufferStorage.java
index 7d4dff0..8e6194d 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BufferStorage.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BufferStorage.java
@@ -22,14 +22,15 @@ import org.apache.flink.annotation.Internal;
 import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
 
 import java.io.IOException;
+import java.util.Optional;
 
 /**
  * The {@link BufferStorage} takes the buffers and events from a data stream and adds them in a sequence.
- * After a number of elements have been added, the {@link BufferStorage} can "roll over":
- * It presents the added elements as a readable sequence, and creates a new sequence.
+ * After a number of elements have been added, the {@link BufferStorage} can {@link #rollOver() "roll over"}.
+ * After rolling over, previously stored buffers are available for reading via {@link #pollNext()}.
  */
 @Internal
-public interface BufferStorage {
+public interface BufferStorage extends AutoCloseable {
 
 	/**
 	 * Adds a buffer or event to the {@link BufferStorage}.
@@ -39,30 +40,40 @@ public interface BufferStorage {
 	void add(BufferOrEvent boe) throws IOException;
 
 	/**
-	 * Starts a new sequence of buffers and event without reusing the same resources and
-	 * returns the current sequence of buffers for reading.
-	 *
-	 * @return The readable sequence of buffers and events, or 'null', if nothing was added.
+	 * @return true if size limit was exceeded.
 	 */
-	BufferOrEventSequence rollOverWithoutReusingResources() throws IOException;
+	boolean isFull();
 
 	/**
-	 * Starts a new sequence of buffers and event reusing the same resources and
-	 * returns the current sequence of buffers for reading.
-	 *
-	 * @return The readable sequence of buffers and events, or 'null', if nothing was added.
+	 * Start returning next sequence of stored {@link BufferOrEvent}s.
 	 */
-	BufferOrEventSequence rollOverReusingResources() throws IOException;
+	void rollOver() throws IOException;
 
 	/**
-	 * Cleans up all the resources in the current sequence.
+	 * @return the number of pending bytes blocked in the current sequence - bytes that are have not
+	 * been yet rolled, but are already blocked.
 	 */
-	void close() throws IOException;
+	long getPendingBytes();
 
 	/**
-	 * Gets the number of bytes blocked in the current sequence.
-	 *
-	 * @return the number of bytes blocked in the current sequence.
+	 * @return the number of already rolled bytes in in blocked sequences.
+	 */
+	long getRolledBytes();
+
+	/**
+	 * @return true if this {@link BufferStorage} doesn't store and data.
 	 */
-	long getBytesBlocked();
+	boolean isEmpty();
+
+	Optional<BufferOrEvent> pollNext() throws IOException;
+
+	long currentBufferedSize();
+
+	long getMaxBufferedBytes();
+
+	/**
+	 * Cleans up all the resources in the current sequence.
+	 */
+	@Override
+	void close() throws IOException;
 }
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CachedBufferStorage.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CachedBufferStorage.java
index e0a79c2..628a69c 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CachedBufferStorage.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CachedBufferStorage.java
@@ -22,6 +22,7 @@ import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
 
 import javax.annotation.Nullable;
 
+import java.io.IOException;
 import java.util.ArrayDeque;
 
 /**
@@ -34,7 +35,7 @@ import java.util.ArrayDeque;
  * alignment in exactly-once mode.
  */
 @Internal
-public class CachedBufferStorage implements BufferStorage {
+public class CachedBufferStorage extends AbstractBufferStorage {
 
 	/** The page size, to estimate the total cached data size. */
 	private final int pageSize;
@@ -46,13 +47,23 @@ public class CachedBufferStorage implements BufferStorage {
 	private ArrayDeque<BufferOrEvent> currentBuffers;
 
 	/**
-	 * Creates a new {@link CachedBufferStorage}, caching the buffers or events in memory queue.
+	 * Create a new {@link CachedBufferStorage} with unlimited storage.
 	 *
 	 * @param pageSize The page size used to estimate the cached size.
 	 */
 	public CachedBufferStorage(int pageSize) {
+		this(pageSize, -1, "Unknown");
+	}
+
+	/**
+	 * Creates a new {@link CachedBufferStorage}, caching the buffers or events in memory queue.
+	 *
+	 * @param pageSize The page size used to estimate the cached size.
+	 */
+	public CachedBufferStorage(int pageSize, long maxBufferedBytes, String taskName) {
+		super(maxBufferedBytes, taskName);
 		this.pageSize = pageSize;
-		this.currentBuffers = new ArrayDeque<BufferOrEvent>();
+		this.currentBuffers = new ArrayDeque<>();
 	}
 
 	@Override
@@ -84,17 +95,18 @@ public class CachedBufferStorage implements BufferStorage {
 	}
 
 	@Override
-	public void close() {
+	public void close() throws IOException {
 		BufferOrEvent boe;
 		while ((boe = currentBuffers.poll()) != null) {
 			if (boe.isBuffer()) {
 				boe.getBuffer().recycleBuffer();
 			}
 		}
+		super.close();
 	}
 
 	@Override
-	public long getBytesBlocked() {
+	public long getPendingBytes() {
 		return bytesBlocked;
 	}
 
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java
index ebef48f..c9ec6bf 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java
@@ -56,15 +56,13 @@ public class InputProcessorUtil {
 			if (taskManagerConfig.getBoolean(NettyShuffleEnvironmentOptions.NETWORK_CREDIT_MODEL)) {
 				barrierHandler = new BarrierBuffer(
 					inputGate,
-					new CachedBufferStorage(inputGate.getPageSize()),
-					maxAlign,
+					new CachedBufferStorage(inputGate.getPageSize(), maxAlign, taskName),
 					taskName,
 					checkpointedTask);
 			} else {
 				barrierHandler = new BarrierBuffer(
 					inputGate,
-					new BufferSpiller(ioManager, inputGate.getPageSize()),
-					maxAlign,
+					new BufferSpiller(ioManager, inputGate.getPageSize(), maxAlign, taskName),
 					taskName,
 					checkpointedTask);
 			}
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferAlignmentLimitTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferAlignmentLimitTest.java
index 8c97938..2eb3f5c 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferAlignmentLimitTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferAlignmentLimitTest.java
@@ -118,8 +118,7 @@ public class BarrierBufferAlignmentLimitTest {
 		AbstractInvokable toNotify = mock(AbstractInvokable.class);
 		BarrierBuffer buffer = new BarrierBuffer(
 			gate,
-			new BufferSpiller(ioManager, gate.getPageSize()),
-			1000,
+			new BufferSpiller(ioManager, gate.getPageSize(), 1000),
 			"Testing",
 			toNotify);
 
@@ -216,8 +215,7 @@ public class BarrierBufferAlignmentLimitTest {
 		AbstractInvokable toNotify = mock(AbstractInvokable.class);
 		BarrierBuffer buffer = new BarrierBuffer(
 			gate,
-			new BufferSpiller(ioManager, gate.getPageSize()),
-			500,
+			new BufferSpiller(ioManager, gate.getPageSize(), 500),
 			"Testing",
 			toNotify);
 
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java
index 4633154..4d46451 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java
@@ -92,7 +92,7 @@ public class BufferSpillerTest extends BufferStorageTestBase {
 		assertEquals(
 			"Changed the header format, but did not adjust the HEADER_SIZE field",
 			BufferSpiller.HEADER_SIZE + size,
-			spiller.getBytesBlocked());
+			spiller.getPendingBytes());
 	}
 
 	private static void checkNoTempFilesRemain() {
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BufferStorageTestBase.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BufferStorageTestBase.java
index 0485d88..b23d3e9 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BufferStorageTestBase.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BufferStorageTestBase.java
@@ -30,11 +30,11 @@ import org.junit.Test;
 import java.io.IOException;
 import java.util.ArrayDeque;
 import java.util.ArrayList;
+import java.util.Optional;
 import java.util.Random;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -50,9 +50,12 @@ public abstract class BufferStorageTestBase {
 	@Test
 	public void testRollOverEmptySequences() throws IOException {
 		BufferStorage bufferStorage = createBufferStorage();
-		assertNull(bufferStorage.rollOverReusingResources());
-		assertNull(bufferStorage.rollOverReusingResources());
-		assertNull(bufferStorage.rollOverReusingResources());
+		bufferStorage.rollOver();
+		assertFalse(bufferStorage.pollNext().isPresent());
+		bufferStorage.rollOver();
+		assertFalse(bufferStorage.pollNext().isPresent());
+		bufferStorage.rollOver();
+		assertFalse(bufferStorage.pollNext().isPresent());
 	}
 
 	@Test
@@ -92,31 +95,35 @@ public abstract class BufferStorageTestBase {
 			// reset and create reader
 			bufferRnd.setSeed(bufferSeed);
 
-			BufferOrEventSequence seq = bufferStorage.rollOverReusingResources();
-			seq.open();
+			bufferStorage.rollOver();
 
 			// read and validate the sequence
 
 			int numEvent = 0;
 			for (int i = 0; i < numEventsAndBuffers; i++) {
-				BufferOrEvent next = seq.getNext();
-				assertNotNull(next);
-				if (next.isEvent()) {
+				assertFalse(bufferStorage.isEmpty());
+
+				Optional<BufferOrEvent> next = bufferStorage.pollNext();
+				assertTrue(next.isPresent());
+				BufferOrEvent bufferOrEvent = next.get();
+
+				if (bufferOrEvent.isEvent()) {
 					BufferOrEvent expected = events.get(numEvent++);
-					assertEquals(expected.getEvent(), next.getEvent());
-					assertEquals(expected.getChannelIndex(), next.getChannelIndex());
+					assertEquals(expected.getEvent(), bufferOrEvent.getEvent());
+					assertEquals(expected.getChannelIndex(), bufferOrEvent.getChannelIndex());
 				} else {
-					validateBuffer(next, bufferRnd.nextInt(PAGE_SIZE) + 1, bufferRnd.nextInt(numberOfChannels));
+					validateBuffer(
+						bufferOrEvent,
+						bufferRnd.nextInt(PAGE_SIZE) + 1, bufferRnd.nextInt(numberOfChannels));
 				}
 			}
 
 			// no further data
-			assertNull(seq.getNext());
+			assertFalse(bufferStorage.pollNext().isPresent());
+			assertTrue(bufferStorage.isEmpty());
 
 			// all events need to be consumed
 			assertEquals(events.size(), numEvent);
-
-			seq.cleanup();
 		}
 	}
 
@@ -126,15 +133,11 @@ public abstract class BufferStorageTestBase {
 
 		final Random rnd = new Random();
 
-		final int maxNumEventsAndBuffers = 30000;
+		final int maxNumEventsAndBuffers = 300;
 		final int maxNumChannels = 1656;
 
-		int sequencesConsumed = 0;
-
-		ArrayDeque<SequenceToConsume> pendingSequences = new ArrayDeque<SequenceToConsume>();
-		SequenceToConsume currentSequence = null;
-		int currentNumEvents = 0;
-		int currentNumRecordAndEvents = 0;
+		ArrayDeque<ArrayDeque<BufferOrEvent>> expectedRolledSequences = new ArrayDeque<>();
+		ArrayDeque<BufferOrEvent> expectedPendingSequence = new ArrayDeque<>();
 
 		BufferStorage bufferStorage = createBufferStorage();
 
@@ -143,7 +146,9 @@ public abstract class BufferStorageTestBase {
 
 			if (round % 2 == 1) {
 				// make this an empty sequence
-				assertNull(bufferStorage.rollOverReusingResources());
+				bufferStorage.rollOver();
+				expectedRolledSequences.addFirst(expectedPendingSequence);
+				expectedPendingSequence = new ArrayDeque<>();
 			} else {
 				// proper spilled sequence
 				final long bufferSeed = rnd.nextLong();
@@ -152,12 +157,12 @@ public abstract class BufferStorageTestBase {
 				final int numEventsAndBuffers = rnd.nextInt(maxNumEventsAndBuffers) + 1;
 				final int numberOfChannels = rnd.nextInt(maxNumChannels) + 1;
 
-				final ArrayList<BufferOrEvent> events = new ArrayList<BufferOrEvent>(128);
+				final ArrayList<BufferOrEvent> events = new ArrayList<>(128);
 
 				int generated = 0;
 				while (generated < numEventsAndBuffers) {
 
-					if (currentSequence == null || rnd.nextDouble() < 0.5) {
+					if (rnd.nextDouble() < 0.5) {
 						// add a new record
 						boolean isEvent = rnd.nextDouble() < 0.05;
 						BufferOrEvent evt;
@@ -168,97 +173,66 @@ public abstract class BufferStorageTestBase {
 							evt = generateRandomBuffer(bufferRnd.nextInt(PAGE_SIZE) + 1, bufferRnd.nextInt(numberOfChannels));
 						}
 						bufferStorage.add(evt);
+
+						expectedPendingSequence.addLast(evt);
 						generated++;
 					} else {
 						// consume a record
-						BufferOrEvent next = currentSequence.sequence.getNext();
-						assertNotNull(next);
-						if (next.isEvent()) {
-							BufferOrEvent expected = currentSequence.events.get(currentNumEvents++);
-							assertEquals(expected.getEvent(), next.getEvent());
-							assertEquals(expected.getChannelIndex(), next.getChannelIndex());
-						} else {
-							Random validationRnd = currentSequence.bufferRnd;
-							validateBuffer(next, validationRnd.nextInt(PAGE_SIZE) + 1, validationRnd.nextInt(currentSequence.numberOfChannels));
-						}
-
-						currentNumRecordAndEvents++;
-						if (currentNumRecordAndEvents == currentSequence.numBuffersAndEvents) {
-							// done with the sequence
-							currentSequence.sequence.cleanup();
-							sequencesConsumed++;
-
-							// validate we had all events
-							assertEquals(currentSequence.events.size(), currentNumEvents);
+						bufferStorage.rollOver();
+						expectedRolledSequences.addFirst(expectedPendingSequence);
+						expectedPendingSequence = new ArrayDeque<>();
 
-							// reset
-							currentSequence = pendingSequences.pollFirst();
-							if (currentSequence != null) {
-								currentSequence.sequence.open();
-							}
-
-							currentNumRecordAndEvents = 0;
-							currentNumEvents = 0;
-						}
+						assertNextBufferOrEvent(expectedRolledSequences, bufferStorage);
 					}
 				}
-
-				// done generating a sequence. queue it for consumption
-				bufferRnd.setSeed(bufferSeed);
-				BufferOrEventSequence seq = bufferStorage.rollOverReusingResources();
-
-				SequenceToConsume stc = new SequenceToConsume(bufferRnd, events, seq, numEventsAndBuffers, numberOfChannels);
-
-				if (currentSequence == null) {
-					currentSequence = stc;
-					stc.sequence.open();
-				} else {
-					pendingSequences.addLast(stc);
-				}
+				bufferStorage.rollOver();
+				expectedRolledSequences.addFirst(expectedPendingSequence);
+				expectedPendingSequence = new ArrayDeque<>();
 			}
 		}
 
 		// consume all the remainder
-		while (currentSequence != null) {
-			// consume a record
-			BufferOrEvent next = currentSequence.sequence.getNext();
-			assertNotNull(next);
-			if (next.isEvent()) {
-				BufferOrEvent expected = currentSequence.events.get(currentNumEvents++);
-				assertEquals(expected.getEvent(), next.getEvent());
-				assertEquals(expected.getChannelIndex(), next.getChannelIndex());
-			} else {
-				Random validationRnd = currentSequence.bufferRnd;
-				validateBuffer(next, validationRnd.nextInt(PAGE_SIZE) + 1, validationRnd.nextInt(currentSequence.numberOfChannels));
-			}
+		while (!expectedRolledSequences.isEmpty()) {
+			assertNextBufferOrEvent(expectedRolledSequences, bufferStorage);
+		}
+	}
 
-			currentNumRecordAndEvents++;
-			if (currentNumRecordAndEvents == currentSequence.numBuffersAndEvents) {
-				// done with the sequence
-				currentSequence.sequence.cleanup();
-				sequencesConsumed++;
+	// ------------------------------------------------------------------------
+	//  Utils
+	// ------------------------------------------------------------------------
 
-				// validate we had all events
-				assertEquals(currentSequence.events.size(), currentNumEvents);
+	private static void assertNextBufferOrEvent(
+			ArrayDeque<ArrayDeque<BufferOrEvent>> expectedRolledSequence,
+			BufferStorage bufferStorage) throws IOException {
+		while (!expectedRolledSequence.isEmpty() && expectedRolledSequence.peekFirst().isEmpty()) {
+			expectedRolledSequence.pollFirst();
+		}
 
-				// reset
-				currentSequence = pendingSequences.pollFirst();
-				if (currentSequence != null) {
-					currentSequence.sequence.open();
-				}
+		Optional<BufferOrEvent> next = bufferStorage.pollNext();
+		if (expectedRolledSequence.isEmpty()) {
+			assertFalse(next.isPresent());
+			return;
+		}
 
-				currentNumRecordAndEvents = 0;
-				currentNumEvents = 0;
-			}
+		while (!next.isPresent() && !bufferStorage.isEmpty()) {
+			next = bufferStorage.pollNext();
 		}
 
-		assertEquals(sequences, sequencesConsumed);
+		assertTrue(next.isPresent());
+		BufferOrEvent actualBufferOrEvent = next.get();
+		BufferOrEvent expectedBufferOrEvent = expectedRolledSequence.peekFirst().pollFirst();
+
+		if (expectedBufferOrEvent.isEvent()) {
+			assertEquals(expectedBufferOrEvent.getChannelIndex(), actualBufferOrEvent.getChannelIndex());
+			assertEquals(expectedBufferOrEvent.getEvent(), actualBufferOrEvent.getEvent());
+		} else {
+			validateBuffer(
+				actualBufferOrEvent,
+				expectedBufferOrEvent.getSize(),
+				expectedBufferOrEvent.getChannelIndex());
+		}
 	}
 
-	// ------------------------------------------------------------------------
-	//  Utils
-	// ------------------------------------------------------------------------
-
 	private static BufferOrEvent generateRandomEvent(Random rnd, int numberOfChannels) {
 		long magicNumber = rnd.nextLong();
 		byte[] data = new byte[rnd.nextInt(1000)];
@@ -297,29 +271,4 @@ public abstract class BufferStorageTestBase {
 			}
 		}
 	}
-
-	/**
-	 * Wrappers the buffered sequence and related elements for consuming and validation.
-	 */
-	private static class SequenceToConsume {
-
-		final BufferOrEventSequence sequence;
-		final ArrayList<BufferOrEvent> events;
-		final Random bufferRnd;
-		final int numBuffersAndEvents;
-		final int numberOfChannels;
-
-		private SequenceToConsume(
-				Random bufferRnd,
-				ArrayList<BufferOrEvent> events,
-				BufferOrEventSequence sequence,
-				int numBuffersAndEvents,
-				int numberOfChannels) {
-			this.bufferRnd = bufferRnd;
-			this.events = events;
-			this.sequence = sequence;
-			this.numBuffersAndEvents = numBuffersAndEvents;
-			this.numberOfChannels = numberOfChannels;
-		}
-	}
 }
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CreditBasedBarrierBufferTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CreditBasedBarrierBufferTest.java
index bbfe8b6..3db884d 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CreditBasedBarrierBufferTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CreditBasedBarrierBufferTest.java
@@ -33,7 +33,7 @@ public class CreditBasedBarrierBufferTest extends BarrierBufferTestBase {
 
 	@Override
 	BarrierBuffer createBarrierBuffer(InputGate gate, @Nullable AbstractInvokable toNotify) {
-		return new BarrierBuffer(gate, new CachedBufferStorage(PAGE_SIZE), -1, "Testing", toNotify);
+		return new BarrierBuffer(gate, new CachedBufferStorage(PAGE_SIZE), "Testing", toNotify);
 	}
 
 	@Override
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/SpillingBarrierBufferTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/SpillingBarrierBufferTest.java
index 2101f40..f9541a9 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/SpillingBarrierBufferTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/SpillingBarrierBufferTest.java
@@ -68,7 +68,7 @@ public class SpillingBarrierBufferTest extends BarrierBufferTestBase {
 
 	@Override
 	BarrierBuffer createBarrierBuffer(InputGate gate, @Nullable AbstractInvokable toNotify) throws IOException {
-		return new BarrierBuffer(gate, new BufferSpiller(ioManager, PAGE_SIZE), -1, "Testing", toNotify);
+		return new BarrierBuffer(gate, new BufferSpiller(ioManager, PAGE_SIZE), "Testing", toNotify);
 	}
 
 	@Override


[flink] 13/16: [FLINK-12777][operator] Use CheckpointedInputGate StreamTwoInputSelectableProcessor

Posted by pn...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pnowojski pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 6c7a69a0442d6bfb3b5d86006db462756a51ec7c
Author: Piotr Nowojski <pi...@gmail.com>
AuthorDate: Tue Jun 18 17:26:47 2019 +0200

    [FLINK-12777][operator] Use CheckpointedInputGate StreamTwoInputSelectableProcessor
---
 .../runtime/io/CheckpointBarrierDiscarder.java     | 74 ----------------------
 .../runtime/io/CheckpointedInputGate.java          | 23 ++++++-
 .../streaming/runtime/io/InputProcessorUtil.java   | 41 ++++++++++++
 .../io/StreamTwoInputSelectableProcessor.java      | 23 +++++--
 .../tasks/TwoInputSelectableStreamTask.java        |  7 +-
 5 files changed, 86 insertions(+), 82 deletions(-)

diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierDiscarder.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierDiscarder.java
deleted file mode 100644
index 4c6cdab..0000000
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierDiscarder.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.io;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker;
-import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
-
-import java.io.IOException;
-
-/**
- * The {@link CheckpointBarrierDiscarder} discards checkpoint barriers have been received from which input channels.
- */
-@Internal
-public class CheckpointBarrierDiscarder extends CheckpointBarrierHandler {
-	public CheckpointBarrierDiscarder() {
-		super(null);
-	}
-
-	@Override
-	public void releaseBlocksAndResetBarriers() throws IOException {
-	}
-
-	@Override
-	public boolean isBlocked(int channelIndex) {
-		return false;
-	}
-
-	@Override
-	public boolean processBarrier(CheckpointBarrier receivedBarrier, int channelIndex, long bufferedBytes) throws Exception {
-		return false;
-	}
-
-	@Override
-	public boolean processCancellationBarrier(CancelCheckpointMarker cancelBarrier) throws Exception {
-		return false;
-	}
-
-	@Override
-	public boolean processEndOfPartition() throws Exception {
-		return false;
-	}
-
-	@Override
-	public long getLatestCheckpointId() {
-		return 0;
-	}
-
-	@Override
-	public long getAlignmentDurationNanos() {
-		return 0;
-	}
-
-	@Override
-	public void checkpointSizeLimitExceeded(long maxBufferedBytes) throws Exception {
-
-	}
-}
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java
index 7604d0a..ce80e30 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java
@@ -52,6 +52,8 @@ public class CheckpointedInputGate implements AsyncDataInput<BufferOrEvent> {
 	/** The gate that the buffer draws its input from. */
 	private final InputGate inputGate;
 
+	private final int channelIndexOffset;
+
 	private final BufferStorage bufferStorage;
 
 	/** Flag to indicate whether we have drawn all available input. */
@@ -89,6 +91,13 @@ public class CheckpointedInputGate implements AsyncDataInput<BufferOrEvent> {
 		);
 	}
 
+	public CheckpointedInputGate(
+			InputGate inputGate,
+			BufferStorage bufferStorage,
+			CheckpointBarrierHandler barrierHandler) {
+		this(inputGate, bufferStorage, barrierHandler, 0);
+	}
+
 	/**
 	 * Creates a new checkpoint stream aligner.
 	 *
@@ -99,12 +108,16 @@ public class CheckpointedInputGate implements AsyncDataInput<BufferOrEvent> {
 	 * @param inputGate The input gate to draw the buffers and events from.
 	 * @param bufferStorage The storage to hold the buffers and events for blocked channels.
 	 * @param barrierHandler Handler that controls which channels are blocked.
+	 * @param channelIndexOffset Optional offset added to channelIndex returned from the inputGate
+	 *                           before passing it to the barrierHandler.
 	 */
 	public CheckpointedInputGate(
 			InputGate inputGate,
 			BufferStorage bufferStorage,
-			CheckpointBarrierHandler barrierHandler) {
+			CheckpointBarrierHandler barrierHandler,
+			int channelIndexOffset) {
 		this.inputGate = inputGate;
+		this.channelIndexOffset = channelIndexOffset;
 		this.bufferStorage = checkNotNull(bufferStorage);
 		this.barrierHandler = barrierHandler;
 	}
@@ -138,7 +151,7 @@ public class CheckpointedInputGate implements AsyncDataInput<BufferOrEvent> {
 			}
 
 			BufferOrEvent bufferOrEvent = next.get();
-			if (barrierHandler.isBlocked(bufferOrEvent.getChannelIndex())) {
+			if (barrierHandler.isBlocked(offsetChannelIndex(bufferOrEvent.getChannelIndex()))) {
 				// if the channel is blocked, we just store the BufferOrEvent
 				bufferStorage.add(bufferOrEvent);
 				if (bufferStorage.isFull()) {
@@ -153,7 +166,7 @@ public class CheckpointedInputGate implements AsyncDataInput<BufferOrEvent> {
 				CheckpointBarrier checkpointBarrier = (CheckpointBarrier) bufferOrEvent.getEvent();
 				if (!endOfInputGate) {
 					// process barriers only if there is a chance of the checkpoint completing
-					if (barrierHandler.processBarrier(checkpointBarrier, bufferOrEvent.getChannelIndex(), bufferStorage.getPendingBytes())) {
+					if (barrierHandler.processBarrier(checkpointBarrier, offsetChannelIndex(bufferOrEvent.getChannelIndex()), bufferStorage.getPendingBytes())) {
 						bufferStorage.rollOver();
 					}
 				}
@@ -174,6 +187,10 @@ public class CheckpointedInputGate implements AsyncDataInput<BufferOrEvent> {
 		}
 	}
 
+	private int offsetChannelIndex(int channelIndex) {
+		return channelIndex + channelIndexOffset;
+	}
+
 	private Optional<BufferOrEvent> handleEmptyBuffer() throws Exception {
 		if (!inputGate.isFinished()) {
 			return Optional.empty();
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java
index 419cf16..800c33e 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java
@@ -29,6 +29,8 @@ import org.apache.flink.streaming.api.CheckpointingMode;
 
 import java.io.IOException;
 
+import static org.apache.flink.util.Preconditions.checkState;
+
 /**
  * Utility for creating {@link CheckpointedInputGate} based on checkpoint mode
  * for {@link StreamInputProcessor} and {@link StreamTwoInputProcessor}.
@@ -51,6 +53,45 @@ public class InputProcessorUtil {
 		return new CheckpointedInputGate(inputGate, bufferStorage, barrierHandler);
 	}
 
+	/**
+	 * @return a pair of {@link CheckpointedInputGate} created for two corresponding
+	 * {@link InputGate}s supplied as parameters.
+	 */
+	public static CheckpointedInputGate[] createCheckpointedInputGatePair(
+			AbstractInvokable toNotifyOnCheckpoint,
+			CheckpointingMode checkpointMode,
+			IOManager ioManager,
+			InputGate inputGate1,
+			InputGate inputGate2,
+			Configuration taskManagerConfig,
+			String taskName) throws IOException {
+
+		BufferStorage mainBufferStorage1 = createBufferStorage(
+			checkpointMode, ioManager, inputGate1.getPageSize(), taskManagerConfig, taskName);
+		BufferStorage mainBufferStorage2 = createBufferStorage(
+			checkpointMode, ioManager, inputGate2.getPageSize(), taskManagerConfig, taskName);
+		checkState(mainBufferStorage1.getMaxBufferedBytes() == mainBufferStorage2.getMaxBufferedBytes());
+
+		BufferStorage linkedBufferStorage1 = new LinkedBufferStorage(
+			mainBufferStorage1,
+			mainBufferStorage2,
+			mainBufferStorage1.getMaxBufferedBytes());
+		BufferStorage linkedBufferStorage2 = new LinkedBufferStorage(
+			mainBufferStorage2,
+			mainBufferStorage1,
+			mainBufferStorage1.getMaxBufferedBytes());
+
+		CheckpointBarrierHandler barrierHandler = createCheckpointBarrierHandler(
+			checkpointMode,
+			inputGate1.getNumberOfInputChannels() + inputGate2.getNumberOfInputChannels(),
+			taskName,
+			toNotifyOnCheckpoint);
+		return new CheckpointedInputGate[] {
+			new CheckpointedInputGate(inputGate1, linkedBufferStorage1, barrierHandler),
+			new CheckpointedInputGate(inputGate2, linkedBufferStorage2, barrierHandler, inputGate1.getNumberOfInputChannels())
+		};
+	}
+
 	private static CheckpointBarrierHandler createCheckpointBarrierHandler(
 			CheckpointingMode checkpointMode,
 			int numberOfInputChannels,
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputSelectableProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputSelectableProcessor.java
index 37c17db..d5172ac 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputSelectableProcessor.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputSelectableProcessor.java
@@ -20,11 +20,13 @@ package org.apache.flink.streaming.runtime.io;
 
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.configuration.Configuration;
 import org.apache.flink.metrics.Counter;
 import org.apache.flink.metrics.SimpleCounter;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
 import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup;
+import org.apache.flink.streaming.api.CheckpointingMode;
 import org.apache.flink.streaming.api.operators.InputSelectable;
 import org.apache.flink.streaming.api.operators.InputSelection;
 import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
@@ -36,6 +38,7 @@ import org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve;
 import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;
 import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer;
 import org.apache.flink.streaming.runtime.tasks.OperatorChain;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
 import org.apache.flink.util.ExceptionUtils;
 
 import org.slf4j.Logger;
@@ -101,13 +104,17 @@ public class StreamTwoInputSelectableProcessor<IN1, IN2> {
 		Collection<InputGate> inputGates2,
 		TypeSerializer<IN1> inputSerializer1,
 		TypeSerializer<IN2> inputSerializer2,
+		StreamTask<?, ?> streamTask,
+		CheckpointingMode checkpointingMode,
 		Object lock,
 		IOManager ioManager,
+		Configuration taskManagerConfig,
 		StreamStatusMaintainer streamStatusMaintainer,
 		TwoInputStreamOperator<IN1, IN2, ?> streamOperator,
 		WatermarkGauge input1WatermarkGauge,
 		WatermarkGauge input2WatermarkGauge,
-		OperatorChain<?, ?> operatorChain) {
+		String taskName,
+		OperatorChain<?, ?> operatorChain) throws IOException {
 
 		checkState(streamOperator instanceof InputSelectable);
 
@@ -120,9 +127,17 @@ public class StreamTwoInputSelectableProcessor<IN1, IN2> {
 		InputGate unionedInputGate2 = InputGateUtil.createInputGate(inputGates2.toArray(new InputGate[0]));
 
 		// create a Input instance for each input
-		CachedBufferStorage bufferStorage = new CachedBufferStorage(unionedInputGate1.getPageSize());
-		this.input1 = new StreamTaskNetworkInput(new CheckpointedInputGate(unionedInputGate1, bufferStorage, new CheckpointBarrierDiscarder()), inputSerializer1, ioManager, 0);
-		this.input2 = new StreamTaskNetworkInput(new CheckpointedInputGate(unionedInputGate2, bufferStorage, new CheckpointBarrierDiscarder()), inputSerializer2, ioManager, 1);
+		CheckpointedInputGate[] checkpointedInputGates = InputProcessorUtil.createCheckpointedInputGatePair(
+			streamTask,
+			checkpointingMode,
+			ioManager,
+			unionedInputGate1,
+			unionedInputGate2,
+			taskManagerConfig,
+			taskName);
+		checkState(checkpointedInputGates.length == 2);
+		this.input1 = new StreamTaskNetworkInput(checkpointedInputGates[0], inputSerializer1, ioManager, 0);
+		this.input2 = new StreamTaskNetworkInput(checkpointedInputGates[1], inputSerializer2, ioManager, 1);
 
 		this.statusWatermarkValve1 = new StatusWatermarkValve(
 			unionedInputGate1.getNumberOfInputChannels(),
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputSelectableStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputSelectableStreamTask.java
index b577b20..cde5a5a 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputSelectableStreamTask.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputSelectableStreamTask.java
@@ -24,6 +24,7 @@ import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
 import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
 import org.apache.flink.streaming.runtime.io.StreamTwoInputSelectableProcessor;
 
+import java.io.IOException;
 import java.util.Collection;
 
 /**
@@ -44,17 +45,21 @@ public class TwoInputSelectableStreamTask<IN1, IN2, OUT> extends AbstractTwoInpu
 		Collection<InputGate> inputGates1,
 		Collection<InputGate> inputGates2,
 		TypeSerializer<IN1> inputDeserializer1,
-		TypeSerializer<IN2> inputDeserializer2) {
+		TypeSerializer<IN2> inputDeserializer2) throws IOException {
 
 		this.inputProcessor = new StreamTwoInputSelectableProcessor<>(
 			inputGates1, inputGates2,
 			inputDeserializer1, inputDeserializer2,
+			this,
+			getConfiguration().getCheckpointMode(),
 			getCheckpointLock(),
 			getEnvironment().getIOManager(),
+			getEnvironment().getTaskManagerInfo().getConfiguration(),
 			getStreamStatusMaintainer(),
 			this.headOperator,
 			input1WatermarkGauge,
 			input2WatermarkGauge,
+			getTaskNameWithSubtaskAndId(),
 			operatorChain);
 	}