You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by se...@apache.org on 2019/05/10 23:33:59 UTC

[flink] branch master updated (446e711 -> a7cf243)

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

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


    from 446e711  [hotfix][travis] Retain existing MAVEN_OPTS in Travis maven setup
     new b50896a  [FLINK-12378][docs] Consolidate FileSystem Documentation
     new 4dae94d  [FLINK-8513][docs] Add documentation for connecting to non-AWS S3 endpoints
     new 7d45b75  [FLINK-10249][docs] Document hadoop/presto s3 file system configuration forwarding
     new 4c0bbc4  [hotfix][docs] Some fixes to FileSystem Documentation
     new d14fc9e  [hotfix] [docs] Minor cleanup in filesystem docs
     new 754cd71  [FLINK-12388][docs] Update the production readiness checklist
     new 541e413  [hotfix] Expand JavaDoc of MemorySegment.wrap()
     new 826daab  [hotfix] [network] Release unpooled buffer for events.
     new e94ec4f  [hotfix] [tests] Remove unnecessary timeouts from SingleInputGate Tests
     new dfd08ab  [hotfix] [tests] Network and Partition Tests pass in Testing IOManager rather than dysfunctional mock
     new 8a76952  [hotfix] [tests] Consolidate mocking of ResultPartition in one utility class
     new e5a6a94  [hotfix] [tests] Remove redundant test assertions
     new 422f7b5  [hotfix] [tests] Move utility methods into correct test class.
     new a7cf243  [FLINK-12070] [network] Change Bounded Blocking Subpartition Implementation to Memory Mapped Files

The 14 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:
 docs/ops/deployment/aws.md                         | 387 ----------
 docs/ops/deployment/aws.zh.md                      | 387 ----------
 docs/ops/deployment/oss.md                         | 233 ------
 docs/ops/deployment/oss.zh.md                      | 233 ------
 docs/ops/filesystems.md                            | 177 -----
 docs/ops/filesystems.zh.md                         | 177 -----
 docs/ops/filesystems/common.md                     |  69 ++
 docs/ops/filesystems/common.zh.md                  |  69 ++
 docs/ops/filesystems/index.md                      |  94 +++
 docs/ops/filesystems/index.zh.md                   |  94 +++
 docs/ops/filesystems/oss.md                        |  78 ++
 docs/ops/filesystems/oss.zh.md                     |  78 ++
 docs/ops/filesystems/s3.md                         | 136 ++++
 docs/ops/filesystems/s3.zh.md                      | 136 ++++
 docs/ops/production_ready.md                       |  87 +--
 docs/ops/production_ready.zh.md                    |  88 +--
 docs/redirects/filesystems.md                      |  24 +
 docs/redirects/oss.md                              |  24 +
 .../apache/flink/core/memory/MemorySegment.java    |   3 +-
 .../flink/core/memory/MemorySegmentFactory.java    |  14 +
 .../runtime/io/network/buffer/BufferConsumer.java  |  11 +-
 .../partition/BoundedBlockingSubpartition.java     | 274 +++++++
 .../BoundedBlockingSubpartitionReader.java         | 144 ++++
 .../io/network/partition/BufferToByteBuffer.java   | 130 ++++
 .../io/network/partition/MemoryMappedBuffers.java  | 279 +++++++
 .../runtime/io/network/partition/PageSizeUtil.java | 113 +++
 .../network/partition/PipelinedSubpartition.java   |  70 ++
 .../io/network/partition/ResultPartition.java      |  38 +-
 .../io/network/partition/ResultSubpartition.java   |  83 +--
 .../network/partition/SpillableSubpartition.java   | 312 --------
 .../partition/SpillableSubpartitionView.java       | 280 --------
 .../network/partition/SpilledSubpartitionView.java | 303 --------
 .../partition/consumer/SingleInputGate.java        |   8 +-
 .../runtime/io/disk/iomanager/NoOpIOManager.java   |  73 ++
 .../runtime/io/network/NetworkEnvironmentTest.java |  21 +-
 .../io/network/buffer/BufferBuilderTestUtils.java  |  29 +
 .../partition/BoundedBlockingSubpartitionTest.java | 105 +++
 .../BoundedBlockingSubpartitionWriteReadTest.java  | 199 +++++
 .../network/partition/BufferToByteBufferTest.java  |  80 +++
 .../network/partition/MemoryMappedBuffersTest.java | 162 +++++
 .../io/network/partition/PartitionTestUtils.java   |  74 ++
 .../partition/PipelinedSubpartitionTest.java       |  47 +-
 .../PipelinedSubpartitionWithReadViewTest.java     |  93 ++-
 .../io/network/partition/ResultPartitionTest.java  |  38 +-
 .../partition/SpillableSubpartitionTest.java       | 800 ---------------------
 .../io/network/partition/SubpartitionTestBase.java | 197 ++---
 .../partition/consumer/SingleInputGateTest.java    |   8 +-
 .../partition/consumer/TestInputChannel.java       |  32 +-
 48 files changed, 2928 insertions(+), 3663 deletions(-)
 delete mode 100644 docs/ops/deployment/oss.md
 delete mode 100644 docs/ops/deployment/oss.zh.md
 delete mode 100644 docs/ops/filesystems.md
 delete mode 100644 docs/ops/filesystems.zh.md
 create mode 100644 docs/ops/filesystems/common.md
 create mode 100644 docs/ops/filesystems/common.zh.md
 create mode 100644 docs/ops/filesystems/index.md
 create mode 100644 docs/ops/filesystems/index.zh.md
 create mode 100644 docs/ops/filesystems/oss.md
 create mode 100644 docs/ops/filesystems/oss.zh.md
 create mode 100644 docs/ops/filesystems/s3.md
 create mode 100644 docs/ops/filesystems/s3.zh.md
 create mode 100644 docs/redirects/filesystems.md
 create mode 100644 docs/redirects/oss.md
 create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartition.java
 create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionReader.java
 create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferToByteBuffer.java
 create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/MemoryMappedBuffers.java
 create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PageSizeUtil.java
 delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java
 delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java
 delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionView.java
 create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/NoOpIOManager.java
 create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionTest.java
 create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionWriteReadTest.java
 create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BufferToByteBufferTest.java
 create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MemoryMappedBuffersTest.java
 create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionTestUtils.java
 delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java


[flink] 03/14: [FLINK-10249][docs] Document hadoop/presto s3 file system configuration forwarding

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

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

commit 7d45b756b8bccbbacf6b7b9ca50a7ede8ed01a5c
Author: Seth Wiesman <sj...@gmail.com>
AuthorDate: Tue Apr 30 16:57:28 2019 -0500

    [FLINK-10249][docs] Document hadoop/presto s3 file system configuration forwarding
---
 docs/ops/filesystems/s3.md    | 4 ++--
 docs/ops/filesystems/s3.zh.md | 4 ++--
 2 files changed, 4 insertions(+), 4 deletions(-)

diff --git a/docs/ops/filesystems/s3.md b/docs/ops/filesystems/s3.md
index 4efcca2..b668ac9 100644
--- a/docs/ops/filesystems/s3.md
+++ b/docs/ops/filesystems/s3.md
@@ -64,10 +64,10 @@ There is no need to add Hadoop to the classpath to use them.
 Both internally use some Hadoop code, but "shade away" all classes to avoid any dependency conflicts.
 
   - `flink-s3-fs-presto`, registered under the scheme *"s3://"* and *"s3p://"*, is based on code from the [Presto project](https://prestodb.io/).
-  You can configure it the same way you can [configure the Presto file system](https://prestodb.io/docs/0.187/connector/hive.html#amazon-s3-configuration).
+  You can configure it the same way you can [configure the Presto file system](https://prestodb.io/docs/0.187/connector/hive.html#amazon-s3-configuration) by placing adding the configurations to your `flink-conf.yaml`.
       
   - `flink-s3-fs-hadoop`, registered under *"s3://"* and *"s3a://"*, based on code from the [Hadoop Project](https://hadoop.apache.org/).
-  The file system can be [configured exactly like Hadoop's s3a](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/index.html#S3A).
+  The file system can be [configured exactly like Hadoop's s3a](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/index.html#S3A) by placing adding the configurations to your `flink-conf.yaml`.
     
 Both `flink-s3-fs-hadoop` and `flink-s3-fs-presto` register default FileSystem
 wrappers for URIs with the `s3://` scheme, `flink-s3-fs-hadoop` also registers
diff --git a/docs/ops/filesystems/s3.zh.md b/docs/ops/filesystems/s3.zh.md
index 4efcca2..b668ac9 100644
--- a/docs/ops/filesystems/s3.zh.md
+++ b/docs/ops/filesystems/s3.zh.md
@@ -64,10 +64,10 @@ There is no need to add Hadoop to the classpath to use them.
 Both internally use some Hadoop code, but "shade away" all classes to avoid any dependency conflicts.
 
   - `flink-s3-fs-presto`, registered under the scheme *"s3://"* and *"s3p://"*, is based on code from the [Presto project](https://prestodb.io/).
-  You can configure it the same way you can [configure the Presto file system](https://prestodb.io/docs/0.187/connector/hive.html#amazon-s3-configuration).
+  You can configure it the same way you can [configure the Presto file system](https://prestodb.io/docs/0.187/connector/hive.html#amazon-s3-configuration) by placing adding the configurations to your `flink-conf.yaml`.
       
   - `flink-s3-fs-hadoop`, registered under *"s3://"* and *"s3a://"*, based on code from the [Hadoop Project](https://hadoop.apache.org/).
-  The file system can be [configured exactly like Hadoop's s3a](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/index.html#S3A).
+  The file system can be [configured exactly like Hadoop's s3a](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/index.html#S3A) by placing adding the configurations to your `flink-conf.yaml`.
     
 Both `flink-s3-fs-hadoop` and `flink-s3-fs-presto` register default FileSystem
 wrappers for URIs with the `s3://` scheme, `flink-s3-fs-hadoop` also registers


[flink] 08/14: [hotfix] [network] Release unpooled buffer for events.

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

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

commit 826daab2c3cc52d100c38480b67393e067472b8f
Author: Stephan Ewen <se...@apache.org>
AuthorDate: Mon Apr 15 21:02:51 2019 +0200

    [hotfix] [network] Release unpooled buffer for events.
    
    So far, these buffers never needed to be released, because they do not come from a buffer pool.
    They were simply garbage collected.
    
    When changing the blocking partitions to use memory mapped files, these buffer were refering
    for a short time to an unmapped memory region (after the partition is released). Because the buffers
    were not accessed any more by any code, it did not matter when regularly running Flink.
    
    But, it did segfault the JVM when attaching a debugger and exploring just that part of the code.
    This happens because the debugger calls toString() on the buffer object as part of its rendering of the current
    stack frame. The toString() method access the buffer contents, which is an unmapped region of memory,
    and boom!
---
 .../partition/consumer/SingleInputGate.java        |  8 +++++-
 .../partition/consumer/SingleInputGateTest.java    |  4 +++
 .../partition/consumer/TestInputChannel.java       | 32 +++++++++++++++++++++-
 3 files changed, 42 insertions(+), 2 deletions(-)

diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
index 19912b2..a584a21 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
@@ -593,7 +593,13 @@ public class SingleInputGate extends InputGate {
 			return new BufferOrEvent(buffer, currentChannel.getChannelIndex(), moreAvailable);
 		}
 		else {
-			final AbstractEvent event = EventSerializer.fromBuffer(buffer, getClass().getClassLoader());
+			final AbstractEvent event;
+			try {
+				event = EventSerializer.fromBuffer(buffer, getClass().getClassLoader());
+			}
+			finally {
+				buffer.recycleBuffer();
+			}
 
 			if (event.getClass() == EndOfPartitionEvent.class) {
 				channelsWithEndOfPartitionEvents.set(currentChannel.getChannelIndex());
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java
index a6f824d..5dda8db 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java
@@ -114,6 +114,10 @@ public class SingleInputGateTest extends InputGateTestBase {
 
 		// Return null when the input gate has received all end-of-partition events
 		assertTrue(inputGate.isFinished());
+
+		for (TestInputChannel ic : inputChannels) {
+			ic.assertReturnedEventsAreRecycled();
+		}
 	}
 
 	@Test
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java
index ac3f0ff..96a5db2 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java
@@ -27,12 +27,15 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Optional;
 import java.util.Queue;
 import java.util.concurrent.ConcurrentLinkedQueue;
 
 import static org.apache.flink.util.Preconditions.checkArgument;
 import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.junit.Assert.fail;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -43,6 +46,8 @@ public class TestInputChannel extends InputChannel {
 
 	private final Queue<BufferAndAvailabilityProvider> buffers = new ConcurrentLinkedQueue<>();
 
+	private final Collection<Buffer> allReturnedBuffers = new ArrayList<>();
+
 	private BufferAndAvailabilityProvider lastProvider = null;
 
 	private boolean isReleased = false;
@@ -125,7 +130,9 @@ public class TestInputChannel extends InputChannel {
 
 		if (provider != null) {
 			lastProvider = provider;
-			return provider.getBufferAvailability();
+			Optional<BufferAndAvailability> baa = provider.getBufferAvailability();
+			baa.ifPresent((v) -> allReturnedBuffers.add(v.buffer()));
+			return baa;
 		} else if (lastProvider != null) {
 			return lastProvider.getBufferAvailability();
 		} else {
@@ -162,6 +169,29 @@ public class TestInputChannel extends InputChannel {
 
 	}
 
+	public void assertReturnedDataBuffersAreRecycled() {
+		assertReturnedBuffersAreRecycled(true, false);
+	}
+
+	public void assertReturnedEventsAreRecycled() {
+		assertReturnedBuffersAreRecycled(false, true);
+	}
+
+	public void assertAllReturnedBuffersAreRecycled() {
+		assertReturnedBuffersAreRecycled(true, true);
+	}
+
+	private void assertReturnedBuffersAreRecycled(boolean assertBuffers, boolean assertEvents) {
+		for (Buffer b : allReturnedBuffers) {
+			if (b.isBuffer() && assertBuffers && !b.isRecycled()) {
+				fail("Data Buffer " + b + " not recycled");
+			}
+			if (!b.isBuffer() && assertEvents && !b.isRecycled()) {
+				fail("Event Buffer " + b + " not recycled");
+			}
+		}
+	}
+
 	interface BufferAndAvailabilityProvider {
 		Optional<BufferAndAvailability> getBufferAvailability() throws IOException, InterruptedException;
 	}


[flink] 12/14: [hotfix] [tests] Remove redundant test assertions

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

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

commit e5a6a9410148ef064b4f78ee4020fb7b451f7030
Author: Stephan Ewen <se...@apache.org>
AuthorDate: Thu Apr 25 12:43:30 2019 +0200

    [hotfix] [tests] Remove redundant test assertions
    
    These seem to be remainders of a copy/paste error.
---
 .../flink/runtime/io/network/partition/SubpartitionTestBase.java     | 5 -----
 1 file changed, 5 deletions(-)

diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java
index 9f5e6d0..e4006be 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java
@@ -60,9 +60,6 @@ public abstract class SubpartitionTestBase extends TestLogger {
 		try {
 			subpartition.finish();
 			assertEquals(1, subpartition.getTotalNumberOfBuffers());
-			assertEquals(0, subpartition.getTotalNumberOfBytes()); // only updated after consuming the buffers
-
-			assertEquals(1, subpartition.getTotalNumberOfBuffers());
 			assertEquals(0, subpartition.getBuffersInBacklog());
 			assertEquals(0, subpartition.getTotalNumberOfBytes()); // only updated after consuming the buffers
 
@@ -85,8 +82,6 @@ public abstract class SubpartitionTestBase extends TestLogger {
 
 		try {
 			subpartition.release();
-			assertEquals(0, subpartition.getTotalNumberOfBuffers());
-			assertEquals(0, subpartition.getTotalNumberOfBytes());
 
 			assertEquals(0, subpartition.getTotalNumberOfBuffers());
 			assertEquals(0, subpartition.getBuffersInBacklog());


[flink] 14/14: [FLINK-12070] [network] Change Bounded Blocking Subpartition Implementation to Memory Mapped Files

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

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

commit a7cf24383be9f310fb5ccc5a032721421fa45791
Author: Stephan Ewen <se...@apache.org>
AuthorDate: Thu Apr 25 13:09:40 2019 +0200

    [FLINK-12070] [network] Change Bounded Blocking Subpartition Implementation to Memory Mapped Files
    
    This commit consists of multiple steps (originally individual commits) that are squashed into
    one commit after review, to form a self-contained (compiling / test passing) commit.
    
    Part 1: Make tests that were specific to pipelined (and buffer storing) implementations into the
        specific test classes.
    
        Several assumptions are tied to specific implementations of the partitions, rather than testing
        required behavior:
    
          - The availability of statistics after disposing partitions is not necessary and requires extra
            effort to guarantee in certain implementations
          - The fact that the number of bytes in a partition only update on consumption seems wrong and
            can only apply on "consume once" implementations. This should not be assumed in a test base.
    
    Part 2: Make "release parent releases readers" test specific to pipelined partitions.
    
        For pipelined partitions, the release call on the SubPartition causes immedately releasing the reader (view).
        For bounded partitions, this is not required or even desirable, because too eager release can segfault in case
        of direct byte buffers and memory mapped files.
    
    Part 3: Remove old SpillableSubpartition and SpillableSubpartitionView
    
    Part 4: Move code specific to pipelined subpartitions into PipelinedSubpartition class.
    
    Part 5: Implement new BoundedBlockingSubpartition
    
    Part 6: Remove no longer applicable memory release test for blocking partitions
    
    Part 7: Add tests for BoundedBlockingSubpartition
---
 .../flink/core/memory/MemorySegmentFactory.java    |  14 +
 .../runtime/io/network/buffer/BufferConsumer.java  |  11 +-
 .../partition/BoundedBlockingSubpartition.java     | 274 +++++++
 .../BoundedBlockingSubpartitionReader.java         | 144 ++++
 .../io/network/partition/BufferToByteBuffer.java   | 130 ++++
 .../io/network/partition/MemoryMappedBuffers.java  | 279 +++++++
 .../runtime/io/network/partition/PageSizeUtil.java | 113 +++
 .../network/partition/PipelinedSubpartition.java   |  70 ++
 .../io/network/partition/ResultPartition.java      |  38 +-
 .../io/network/partition/ResultSubpartition.java   |  83 +--
 .../network/partition/SpillableSubpartition.java   | 312 --------
 .../partition/SpillableSubpartitionView.java       | 280 --------
 .../network/partition/SpilledSubpartitionView.java | 303 --------
 .../io/network/buffer/BufferBuilderTestUtils.java  |  29 +
 .../partition/BoundedBlockingSubpartitionTest.java | 105 +++
 .../BoundedBlockingSubpartitionWriteReadTest.java  | 199 +++++
 .../network/partition/BufferToByteBufferTest.java  |  80 +++
 .../network/partition/MemoryMappedBuffersTest.java | 162 +++++
 .../partition/PipelinedSubpartitionTest.java       |  45 ++
 .../io/network/partition/ResultPartitionTest.java  |   5 -
 .../partition/SpillableSubpartitionTest.java       | 800 ---------------------
 .../io/network/partition/SubpartitionTestBase.java | 109 ++-
 22 files changed, 1778 insertions(+), 1807 deletions(-)

diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java
index 48b9a20..9a20e0e 100644
--- a/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java
+++ b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java
@@ -116,4 +116,18 @@ public final class MemorySegmentFactory {
 		return new HybridMemorySegment(memory, owner);
 	}
 
+	/**
+	 * Creates a memory segment that wraps the off-heap memory backing the given ByteBuffer.
+	 * Note that the ByteBuffer needs to be a <i>direct ByteBuffer</i>.
+	 *
+	 * <p>This method is intended to be used for components which pool memory and create
+	 * memory segments around long-lived memory regions.
+	 *
+	 * @param memory The byte buffer with the off-heap memory to be represented by the memory segment.
+	 * @return A new memory segment representing the given off-heap memory.
+	 */
+	public static MemorySegment wrapOffHeapMemory(ByteBuffer memory) {
+		return new HybridMemorySegment(memory);
+	}
+
 }
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferConsumer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferConsumer.java
index affcc74..b58a627 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferConsumer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferConsumer.java
@@ -61,9 +61,16 @@ public class BufferConsumer implements Closeable {
 	 * Constructs {@link BufferConsumer} instance with static content.
 	 */
 	public BufferConsumer(MemorySegment memorySegment, BufferRecycler recycler, boolean isBuffer) {
+		this(memorySegment, recycler, memorySegment.size(), isBuffer);
+	}
+
+	/**
+	 * Constructs {@link BufferConsumer} instance with static content of a certain size.
+	 */
+	public BufferConsumer(MemorySegment memorySegment, BufferRecycler recycler, int size, boolean isBuffer) {
 		this(new NetworkBuffer(checkNotNull(memorySegment), checkNotNull(recycler), isBuffer),
-			() -> -memorySegment.size(),
-			0);
+				() -> -size,
+				0);
 		checkState(memorySegment.size() > 0);
 		checkState(isFinished(), "BufferConsumer with static size must be finished after construction!");
 	}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartition.java
new file mode 100644
index 0000000..76c7a2d
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartition.java
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferConsumer;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * An implementation of the ResultSubpartition for a bounded result transferred
+ * in a blocking manner: The result is first produced, then consumed.
+ * The result can be consumed possibly multiple times.
+ *
+ * <p>The implementation creates a temporary memory mapped file and writes all buffers to that
+ * memory and serves the result from that memory. The kernel backs the mapped memory region
+ * with physical memory and file space incrementally as new pages are filled.
+ *
+ * <h2>Important Notes on Thread Safety</h2>
+ *
+ * <p>This class does not synchronize every buffer access. It assumes the threading model of the
+ * Flink network stack and is not thread-safe beyond that.
+ *
+ * <p>This class assumes a single writer thread that adds buffers, flushes, and finishes the write
+ * phase. That same thread is also assumed to perform the partition release, if the release happens
+ * during the write phase.
+ *
+ * <p>The implementation supports multiple concurrent readers, but assumes a single
+ * thread per reader. That same thread must also release the reader. In particular, after the reader
+ * was released, no buffers obtained from this reader may be accessed any more, or segmentation
+ * faults might occur.
+ *
+ * <p>The method calls to create readers, dispose readers, and dispose the partition are
+ * thread-safe vis-a-vis each other.
+ */
+final class BoundedBlockingSubpartition extends ResultSubpartition {
+
+	/** This lock guards the creation of readers and disposal of the memory mapped file. */
+	private final Object lock = new Object();
+
+	/** The current buffer, may be filled further over time. */
+	@Nullable
+	private BufferConsumer currentBuffer;
+
+	/** The memory that we store the data in, via a memory mapped file. */
+	private final MemoryMappedBuffers memory;
+
+	/** All created and not yet released readers. */
+	@GuardedBy("lock")
+	private final Set<BoundedBlockingSubpartitionReader> readers;
+
+	/** Counter for the number of data buffers (not events!) written. */
+	private int numDataBuffersWritten;
+
+	/** The counter for the number of data buffers and events. */
+	private int numBuffersAndEventsWritten;
+
+	/** Flag indicating whether the writing has finished and this is now available for read. */
+	private boolean isFinished;
+
+	/** Flag indicating whether the subpartition has been released. */
+	private boolean isReleased;
+
+	/**
+	 * Common constructor.
+	 */
+	public BoundedBlockingSubpartition(
+			int index,
+			ResultPartition parent,
+			Path filePath) throws IOException {
+
+		this(index, parent, MemoryMappedBuffers.create(filePath));
+	}
+
+	/**
+	 * Constructor for testing, to pass in custom MemoryMappedBuffers.
+	 */
+	@VisibleForTesting
+	BoundedBlockingSubpartition(
+			int index,
+			ResultPartition parent,
+			MemoryMappedBuffers memory) throws IOException {
+
+		super(index, parent);
+
+		this.memory = checkNotNull(memory);
+		this.readers = new HashSet<>();
+	}
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Checks if writing is finished.
+	 * Readers cannot be created until writing is finished, and no further writes can happen after that.
+	 */
+	public boolean isFinished() {
+		return isFinished;
+	}
+
+	@Override
+	public boolean isReleased() {
+		return isReleased;
+	}
+
+	@Override
+	public boolean add(BufferConsumer bufferConsumer) throws IOException {
+		if (isFinished()) {
+			bufferConsumer.close();
+			return false;
+		}
+
+		flushCurrentBuffer();
+		currentBuffer = bufferConsumer;
+		return true;
+	}
+
+	@Override
+	public void flush() {
+		// unfortunately, the signature of flush does not allow for any exceptions, so we
+		// need to do this discouraged pattern of runtime exception wrapping
+		try {
+			flushCurrentBuffer();
+		}
+		catch (IOException e) {
+			throw new FlinkRuntimeException(e.getMessage(), e);
+		}
+	}
+
+	private void flushCurrentBuffer() throws IOException {
+		if (currentBuffer != null) {
+			writeAndCloseBufferConsumer(currentBuffer);
+			currentBuffer = null;
+		}
+	}
+
+	private void writeAndCloseBufferConsumer(BufferConsumer bufferConsumer) throws IOException {
+		try {
+			final Buffer buffer = bufferConsumer.build();
+			try {
+				memory.writeBuffer(buffer);
+
+				numBuffersAndEventsWritten++;
+				if (buffer.isBuffer()) {
+					numDataBuffersWritten++;
+				}
+			}
+			finally {
+				buffer.recycleBuffer();
+			}
+		}
+		finally {
+			bufferConsumer.close();
+		}
+	}
+
+	@Override
+	public void finish() throws IOException {
+		checkState(!isReleased, "data partition already released");
+		checkState(!isFinished, "data partition already finished");
+
+		isFinished = true;
+		flushCurrentBuffer();
+		writeAndCloseBufferConsumer(EventSerializer.toBufferConsumer(EndOfPartitionEvent.INSTANCE));
+		memory.finishWrite();
+	}
+
+	@Override
+	public void release() throws IOException {
+		synchronized (lock) {
+			if (isReleased) {
+				return;
+			}
+
+			isReleased = true;
+			isFinished = true; // for fail fast writes
+
+			checkReaderReferencesAndDispose();
+		}
+	}
+
+	@Override
+	public ResultSubpartitionView createReadView(BufferAvailabilityListener availability) throws IOException {
+		synchronized (lock) {
+			checkState(!isReleased, "data partition already released");
+			checkState(isFinished, "writing of blocking partition not yet finished");
+
+			availability.notifyDataAvailable();
+
+			final MemoryMappedBuffers.BufferSlicer memoryReader = memory.getFullBuffers();
+			final BoundedBlockingSubpartitionReader reader = new BoundedBlockingSubpartitionReader(
+					this, memoryReader, numDataBuffersWritten);
+			readers.add(reader);
+			return reader;
+		}
+	}
+
+	void releaseReaderReference(BoundedBlockingSubpartitionReader reader) throws IOException {
+		synchronized (lock) {
+			if (readers.remove(reader) && isReleased) {
+				checkReaderReferencesAndDispose();
+			}
+		}
+	}
+
+	@GuardedBy("lock")
+	private void checkReaderReferencesAndDispose() throws IOException {
+		assert Thread.holdsLock(lock);
+
+		// To avoid lingering memory mapped files (large resource footprint), we don't
+		// wait for GC to unmap the files, but use a Netty utility to directly unmap the file.
+		// To avoid segmentation faults, we need to wait until all readers have been released.
+
+		if (readers.isEmpty()) {
+			memory.close();
+		}
+	}
+
+	// ------------------------------ legacy ----------------------------------
+
+	@Override
+	public int releaseMemory() throws IOException {
+		return 0;
+	}
+
+	// ---------------------------- statistics --------------------------------
+
+	@Override
+	public int unsynchronizedGetNumberOfQueuedBuffers() {
+		return 0;
+	}
+
+	@Override
+	protected long getTotalNumberOfBuffers() {
+		return numBuffersAndEventsWritten;
+	}
+
+	@Override
+	protected long getTotalNumberOfBytes() {
+		return memory.getSize();
+	}
+
+	int getBuffersInBacklog() {
+		return numDataBuffersWritten;
+	}
+}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionReader.java
new file mode 100644
index 0000000..d6c6834
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionReader.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition;
+
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.partition.MemoryMappedBuffers.BufferSlicer;
+import org.apache.flink.runtime.io.network.partition.ResultSubpartition.BufferAndBacklog;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The reader (read view) of a BoundedBlockingSubpartition.
+ */
+final class BoundedBlockingSubpartitionReader implements ResultSubpartitionView {
+
+	/** The result subpartition that we read. */
+	private final BoundedBlockingSubpartition parent;
+
+	/** The next buffer (look ahead). Null once the data is depleted or reader is disposed. */
+	@Nullable
+	private Buffer nextBuffer;
+
+	/** The reader/decoder to the memory mapped region with the data we currently read from.
+	 * Null once the reader empty or disposed.*/
+	@Nullable
+	private BufferSlicer memory;
+
+	/** The remaining number of data buffers (not events) in the result. */
+	private int dataBufferBacklog;
+
+	/** Flag whether this reader is released. Atomic, to avoid double release. */
+	private boolean isReleased;
+
+	/**
+	 * Convenience constructor that takes a single buffer.
+	 */
+	BoundedBlockingSubpartitionReader(
+			BoundedBlockingSubpartition parent,
+			BufferSlicer memory,
+			int numDataBuffers) {
+
+		checkArgument(numDataBuffers >= 0);
+
+		this.parent = checkNotNull(parent);
+		this.memory = checkNotNull(memory);
+		this.dataBufferBacklog = numDataBuffers;
+
+		this.nextBuffer = memory.sliceNextBuffer();
+	}
+
+	@Nullable
+	@Override
+	public BufferAndBacklog getNextBuffer() throws IOException {
+		final Buffer current = nextBuffer; // copy reference to stack
+
+		if (current == null) {
+			// as per contract, we must return null when the reader is empty,
+			// but also in case the reader is disposed (rather than throwing an exception)
+			return null;
+		}
+		if (current.isBuffer()) {
+			dataBufferBacklog--;
+		}
+
+		assert memory != null;
+		nextBuffer = memory.sliceNextBuffer();
+
+		return BufferAndBacklog.fromBufferAndLookahead(current, nextBuffer, dataBufferBacklog);
+	}
+
+	@Override
+	public void notifyDataAvailable() {
+		throw new IllegalStateException("No data should become available on a blocking partition during consumption.");
+	}
+
+	@Override
+	public void notifySubpartitionConsumed() throws IOException {
+		parent.onConsumedSubpartition();
+	}
+
+	@Override
+	public void releaseAllResources() throws IOException {
+		// it is not a problem if this method executes multiple times
+		isReleased = true;
+
+		// nulling these fields means thet read method and will fail fast
+		nextBuffer = null;
+		memory = null;
+
+		// Notify the parent that this one is released. This allows the parent to
+		// eventually release all resources (when all readers are done and the
+		// parent is disposed).
+		parent.releaseReaderReference(this);
+	}
+
+	@Override
+	public boolean isReleased() {
+		return isReleased;
+	}
+
+	@Override
+	public boolean nextBufferIsEvent() {
+		return nextBuffer != null && !nextBuffer.isBuffer();
+	}
+
+	@Override
+	public boolean isAvailable() {
+		return nextBuffer != null;
+	}
+
+	@Override
+	public Throwable getFailureCause() {
+		// we can never throw an error after this was created
+		return null;
+	}
+
+	@Override
+	public String toString() {
+		return String.format("Blocking Subpartition Reader: ID=%s, index=%d",
+				parent.parent.getPartitionId(),
+				parent.index);
+	}
+}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferToByteBuffer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferToByteBuffer.java
new file mode 100644
index 0000000..4cdf41a
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferToByteBuffer.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition;
+
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+
+import javax.annotation.Nullable;
+
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/**
+ * Putting and getting of a sequence of buffers to/from a ByteBuffer.
+ * This class handles the headers, length encoding, memory slicing.
+ */
+final class BufferToByteBuffer {
+
+	// all fields and methods below here have package-private access to avoid bridge
+	// methods when accessing them from the nested classes
+
+	static final int HEADER_LENGTH = 8;
+
+	static final int HEADER_VALUE_IS_BUFFER = 0;
+
+	static final int HEADER_VALUE_IS_EVENT = 1;
+
+	static ByteBuffer checkAndConfigureByteBuffer(ByteBuffer buffer) {
+		checkArgument(buffer.position() == 0);
+		checkArgument(buffer.capacity() > 8);
+		checkArgument(buffer.limit() == buffer.capacity());
+
+		return buffer.order(ByteOrder.nativeOrder());
+	}
+
+	// ------------------------------------------------------------------------
+
+	static final class Writer {
+
+		private final ByteBuffer memory;
+
+		Writer(ByteBuffer memory) {
+			this.memory = checkAndConfigureByteBuffer(memory);
+		}
+
+		public boolean writeBuffer(Buffer buffer) {
+			final ByteBuffer memory = this.memory;
+			final int bufferSize = buffer.getSize();
+
+			if (memory.remaining() < bufferSize + HEADER_LENGTH) {
+				return false;
+			}
+
+			memory.putInt(buffer.isBuffer() ? HEADER_VALUE_IS_BUFFER : HEADER_VALUE_IS_EVENT);
+			memory.putInt(bufferSize);
+			memory.put(buffer.getNioBufferReadable());
+			return true;
+		}
+
+		public ByteBuffer complete() {
+			memory.flip();
+			return memory;
+		}
+
+		public int getNumBytes() {
+			return memory.position();
+		}
+	}
+
+	static final class Reader {
+
+		private final ByteBuffer memory;
+
+		Reader(ByteBuffer memory) {
+			this.memory = checkAndConfigureByteBuffer(memory);
+		}
+
+		@Nullable
+		public Buffer sliceNextBuffer() {
+			final ByteBuffer memory = this.memory;
+			final int remaining = memory.remaining();
+
+			// we only check the correct case where data is exhausted
+			// all other cases can only occur if our write logic is wrong and will already throw
+			// buffer underflow exceptions which will cause the read to fail.
+			if (remaining == 0) {
+				return null;
+			}
+
+			final int header = memory.getInt();
+			final int size = memory.getInt();
+
+			memory.limit(memory.position() + size);
+			ByteBuffer buf = memory.slice();
+			memory.position(memory.limit());
+			memory.limit(memory.capacity());
+
+			MemorySegment memorySegment = MemorySegmentFactory.wrapOffHeapMemory(buf);
+			Buffer buffer = new NetworkBuffer(memorySegment, FreeingBufferRecycler.INSTANCE);
+			buffer.setSize(size);
+
+			if (header == HEADER_VALUE_IS_EVENT) {
+				buffer.tagAsEvent();
+			}
+
+			return buffer;
+		}
+	}
+}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/MemoryMappedBuffers.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/MemoryMappedBuffers.java
new file mode 100644
index 0000000..6bb031e
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/MemoryMappedBuffers.java
@@ -0,0 +1,279 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition;
+
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.partition.BufferToByteBuffer.Writer;
+import org.apache.flink.util.IOUtils;
+
+import org.apache.flink.shaded.netty4.io.netty.util.internal.PlatformDependent;
+
+import javax.annotation.Nullable;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.channels.FileChannel.MapMode;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/**
+ * This class is largely a workaround for the fact that a memory mapped region in Java can cannot
+ * be larger than 2GB (== signed 32 bit int max value). The class takes {@link Buffer Buffers} and
+ * writes them to several memory mapped region, using the {@link BufferToByteBuffer}
+ * class.
+ *
+ * <h2>Useage</h2>
+ *
+ * <p>The class assumes in the first phase that data is written by repeatedly calling
+ * {@link #writeBuffer(Buffer)}. That puts the data into the memory region of the memory
+ * mapped file. After writing, one must call {@link #finishWrite()}.
+ *
+ * <p>After that, the class can produce multiple {@link BufferSlicer} instances to re-read
+ * the data from the memory regions. Multiple slicers can read concurrently, but each slicer
+ * should be read from by a single thread.
+ *
+ * <p>Eventually, the resources must be disposed via {@link #close()}. After that,
+ * no reading can happen any more.
+ *
+ * <h2>Important!</h2>
+ *
+ * <p>This class performs absolutely no synchronization and relies on single threaded access
+ * or externally synchronized access. Concurrent access around disposal may cause
+ * segmentation faults!
+ *
+ * <p>This class does limited sanity checks and assumes correct use from {@link BoundedBlockingSubpartition}
+ * and {@link BoundedBlockingSubpartitionReader}, such as writing first and rading after.
+ * Not obeying these contracts throws NullPointerExceptions.
+ */
+class MemoryMappedBuffers implements Closeable {
+
+	/** Memory mappings should be at the granularity of page sizes, for efficiency. */
+	private static final int PAGE_SIZE = PageSizeUtil.getSystemPageSizeOrConservativeMultiple();
+
+	/** The encoder to the current memory mapped region we are writing to.
+	 * This value is null once writing has finished or the buffers are disposed. */
+	@Nullable
+	private BufferToByteBuffer.Writer currentBuffer;
+
+	/** All memory mapped regions that are already full (completed). */
+	private final ArrayList<ByteBuffer> fullBuffers;
+
+	/** The file channel backing the memory mapped file. */
+	private final FileChannel file;
+
+	/** The path of the memory mapped file. */
+	private final Path filePath;
+
+	/** The offset where the next mapped region should start. */
+	private long nextMappingOffset;
+
+	/** The size of each mapped region. */
+	private final long mappingSize;
+
+	MemoryMappedBuffers(
+			Path filePath,
+			FileChannel fileChannel,
+			int maxSizePerByteBuffer) throws IOException {
+
+		this.filePath = filePath;
+		this.file = fileChannel;
+		this.mappingSize = alignSize(maxSizePerByteBuffer);
+		this.fullBuffers = new ArrayList<>(4);
+
+		rollOverToNextBuffer();
+	}
+
+	void writeBuffer(Buffer buffer) throws IOException {
+		assert currentBuffer != null;
+
+		if (currentBuffer.writeBuffer(buffer)) {
+			return;
+		}
+
+		rollOverToNextBuffer();
+
+		if (!currentBuffer.writeBuffer(buffer)) {
+			throwTooLargeBuffer(buffer);
+		}
+	}
+
+	BufferSlicer getFullBuffers() {
+		assert currentBuffer == null;
+
+		final List<ByteBuffer> buffers = fullBuffers.stream()
+				.map(ByteBuffer::slice)
+				.collect(Collectors.toList());
+
+		return new BufferSlicer(buffers);
+	}
+
+	/**
+	 * Finishes the current region and prevents further writes.
+	 * After calling this method, further calls to {@link #writeBuffer(Buffer)} will fail.
+	 */
+	void finishWrite() throws IOException {
+		assert currentBuffer != null;
+
+		fullBuffers.add(currentBuffer.complete());
+		currentBuffer = null; // fail further writes fast
+		file.close(); // won't map further regions from now on
+	}
+
+	/**
+	 * Unmaps the file from memory and deletes the file.
+	 * After calling this method, access to any ByteBuffer obtained from this instance
+	 * will cause a segmentation fault.
+	 */
+	public void close() throws IOException {
+		IOUtils.closeQuietly(file); // in case we dispose before finishing writes
+
+		for (ByteBuffer bb : fullBuffers) {
+			PlatformDependent.freeDirectBuffer(bb);
+		}
+		fullBuffers.clear();
+
+		if (currentBuffer != null) {
+			PlatformDependent.freeDirectBuffer(currentBuffer.complete());
+			currentBuffer = null;
+		}
+
+		// To make this compatible with all versions of Windows, we must wait with
+		// deleting the file until it is unmapped.
+		// See also https://stackoverflow.com/questions/11099295/file-flag-delete-on-close-and-memory-mapped-files/51649618#51649618
+
+		Files.delete(filePath);
+	}
+
+	/**
+	 * Gets the number of bytes of all written data (including the metadata in the buffer headers).
+	 */
+	long getSize() {
+		long size = 0L;
+		for (ByteBuffer bb : fullBuffers) {
+			size += bb.remaining();
+		}
+		if (currentBuffer != null) {
+			size += currentBuffer.getNumBytes();
+		}
+		return size;
+	}
+
+	private void rollOverToNextBuffer() throws IOException {
+		if (currentBuffer != null) {
+			// we need to remember the original buffers, not any slices.
+			// slices have no cleaner, which we need to trigger explicit unmapping
+			fullBuffers.add(currentBuffer.complete());
+		}
+
+		final ByteBuffer mapped = file.map(MapMode.READ_WRITE, nextMappingOffset, mappingSize);
+		currentBuffer = new Writer(mapped);
+		nextMappingOffset += mappingSize;
+	}
+
+	private void throwTooLargeBuffer(Buffer buffer) throws IOException {
+		throw new IOException(String.format(
+				"The buffer (%d bytes) is larger than the maximum size of a memory buffer (%d bytes)",
+				buffer.getSize(), mappingSize));
+	}
+
+	/**
+	 * Rounds the size down to the next multiple of the {@link #PAGE_SIZE}.
+	 * We need to round down here to not exceed the original maximum size value.
+	 * Otherwise, values like INT_MAX would round up to overflow the valid maximum
+	 * size of a memory mapping region in Java.
+	 */
+	private static int alignSize(int maxRegionSize) {
+		checkArgument(maxRegionSize >= PAGE_SIZE);
+		return maxRegionSize - (maxRegionSize % PAGE_SIZE);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Reader
+	// ------------------------------------------------------------------------
+
+	/**
+	 * The "reader" for the memory region. It slices a sequence of buffers from the
+	 * sequence of mapped ByteBuffers.
+	 */
+	static final class BufferSlicer {
+
+		/** The reader/decoder to the memory mapped region with the data we currently read from.
+		 * Max 2GB large. Further regions may be in the {@link #furtherData} field. */
+		private BufferToByteBuffer.Reader data;
+
+		/** Further byte buffers, to handle cases where there is more data than fits into
+		 * one mapped byte buffer (2GB = Integer.MAX_VALUE). */
+		private final Iterator<ByteBuffer> furtherData;
+
+		BufferSlicer(Iterable<ByteBuffer> data) {
+			this.furtherData = data.iterator();
+			this.data = new BufferToByteBuffer.Reader(furtherData.next());
+		}
+
+		@Nullable
+		public Buffer sliceNextBuffer() {
+			// should only be null once empty or disposed, in which case this method
+			// should not be called any more
+			assert data != null;
+
+			final Buffer next = data.sliceNextBuffer();
+			if (next != null) {
+				return next;
+			}
+
+			if (!furtherData.hasNext()) {
+				return null;
+			}
+
+			data = new BufferToByteBuffer.Reader(furtherData.next());
+			return sliceNextBuffer();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Factories
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates new MemoryMappedBuffers, creating a memory mapped file at the given path.
+	 */
+	public static MemoryMappedBuffers create(Path memMappedFilePath) throws IOException {
+		return createWithRegionSize(memMappedFilePath, Integer.MAX_VALUE);
+	}
+
+	/**
+	 * Creates new MemoryMappedBuffers, creating a memory mapped file at the given path.
+	 * Each mapped region (= ByteBuffer) will be of the given size.
+	 */
+	public static MemoryMappedBuffers createWithRegionSize(Path memMappedFilePath, int regionSize) throws IOException {
+		final FileChannel fileChannel = FileChannel.open(memMappedFilePath,
+				StandardOpenOption.READ, StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW);
+
+		return new MemoryMappedBuffers(memMappedFilePath, fileChannel, regionSize);
+	}
+}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PageSizeUtil.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PageSizeUtil.java
new file mode 100644
index 0000000..1ce1a76
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PageSizeUtil.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition;
+
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.netty4.io.netty.util.internal.PlatformDependent;
+import org.apache.flink.shaded.netty4.io.netty.util.internal.shaded.org.jctools.util.UnsafeAccess;
+
+import sun.misc.Unsafe;
+
+import javax.annotation.Nullable;
+
+import java.security.AccessController;
+import java.security.PrivilegedAction;
+
+/**
+ * Utility for accessing the system page size.
+ */
+public final class PageSizeUtil {
+
+	/** Value indicating an unknown page size. */
+	public static final int PAGE_SIZE_UNKNOWN = -1;
+
+	/** The default page size on most systems. */
+	public static final int DEFAULT_PAGE_SIZE = 4 * 1024;
+
+	/** A conservative fallback value (64 KiBytes) that should be a multiple of the page size even
+	 * in some uncommon cases of servers installations with larger-than-usual page sizes. */
+	public static final int CONSERVATIVE_PAGE_SIZE_MULTIPLE = 64 * 1024;
+
+	/**
+	 * Tries to get the system page size. If the page size cannot be determined, this
+	 * returns -1.
+	 *
+	 * <p>This internally relies on the presence of "unsafe" and the resolution via some
+	 * Netty utilities.
+	 */
+	public static int getSystemPageSize() {
+		try {
+			return PageSizeUtilInternal.getSystemPageSize();
+		}
+		catch (Throwable t) {
+			ExceptionUtils.rethrowIfFatalError(t);
+			return PAGE_SIZE_UNKNOWN;
+		}
+	}
+
+	/**
+	 * Tries to get the system page size. If the page size cannot be determined, this
+	 * returns the {@link #DEFAULT_PAGE_SIZE}.
+	 */
+	public static int getSystemPageSizeOrDefault() {
+		final int pageSize = getSystemPageSize();
+		return pageSize == PAGE_SIZE_UNKNOWN ? DEFAULT_PAGE_SIZE : pageSize;
+	}
+
+	/**
+	 * Tries to get the system page size. If the page size cannot be determined, this
+	 * returns the {@link #CONSERVATIVE_PAGE_SIZE_MULTIPLE}.
+	 */
+	public static int getSystemPageSizeOrConservativeMultiple() {
+		final int pageSize = getSystemPageSize();
+		return pageSize == PAGE_SIZE_UNKNOWN ? CONSERVATIVE_PAGE_SIZE_MULTIPLE : pageSize;
+	}
+
+	// ------------------------------------------------------------------------
+
+	/** This class is not meant to be instantiated. */
+	private PageSizeUtil() {}
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * All unsafe related code must be in a separate class, so that loading the outer class
+	 * does not implicitly try to resolve the unsafe class.
+	 */
+	@SuppressWarnings("all")
+	private static final class PageSizeUtilInternal {
+
+		static int getSystemPageSize() {
+			Unsafe unsafe = unsafe();
+			return unsafe == null ? PAGE_SIZE_UNKNOWN : unsafe.pageSize();
+		}
+
+		@Nullable
+		private static Unsafe unsafe() {
+			if (PlatformDependent.hasUnsafe()) {
+				return (Unsafe) AccessController.doPrivileged(
+						(PrivilegedAction<Object>) () -> UnsafeAccess.UNSAFE);
+			}
+			else {
+				return null;
+			}
+		}
+	}
+}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
index fe27d97..7394e6e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.runtime.io.network.partition;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
 import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
@@ -30,6 +31,7 @@ import javax.annotation.Nullable;
 import javax.annotation.concurrent.GuardedBy;
 
 import java.io.IOException;
+import java.util.ArrayDeque;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 import static org.apache.flink.util.Preconditions.checkState;
@@ -56,6 +58,13 @@ class PipelinedSubpartition extends ResultSubpartition {
 
 	// ------------------------------------------------------------------------
 
+	/** All buffers of this subpartition. Access to the buffers is synchronized on this object. */
+	private final ArrayDeque<BufferConsumer> buffers = new ArrayDeque<>();
+
+	/** The number of non-event buffers currently in this subpartition. */
+	@GuardedBy("buffers")
+	private int buffersInBacklog;
+
 	/** The read view to consume this subpartition. */
 	private PipelinedSubpartitionView readView;
 
@@ -68,6 +77,12 @@ class PipelinedSubpartition extends ResultSubpartition {
 	/** Flag indicating whether the subpartition has been released. */
 	private volatile boolean isReleased;
 
+	/** The total number of buffers (both data and event buffers). */
+	private long totalNumberOfBuffers;
+
+	/** The total number of bytes (both data and event buffers). */
+	private long totalNumberOfBytes;
+
 	// ------------------------------------------------------------------------
 
 	PipelinedSubpartition(int index, ResultPartition parent) {
@@ -300,6 +315,61 @@ class PipelinedSubpartition extends ResultSubpartition {
 		}
 	}
 
+	@Override
+	protected long getTotalNumberOfBuffers() {
+		return totalNumberOfBuffers;
+	}
+
+	@Override
+	protected long getTotalNumberOfBytes() {
+		return totalNumberOfBytes;
+	}
+
+	Throwable getFailureCause() {
+		return parent.getFailureCause();
+	}
+
+	private void updateStatistics(BufferConsumer buffer) {
+		totalNumberOfBuffers++;
+	}
+
+	private void updateStatistics(Buffer buffer) {
+		totalNumberOfBytes += buffer.getSize();
+	}
+
+	@GuardedBy("buffers")
+	private void decreaseBuffersInBacklogUnsafe(boolean isBuffer) {
+		assert Thread.holdsLock(buffers);
+		if (isBuffer) {
+			buffersInBacklog--;
+		}
+	}
+
+	/**
+	 * Increases the number of non-event buffers by one after adding a non-event
+	 * buffer into this subpartition.
+	 */
+	@GuardedBy("buffers")
+	private void increaseBuffersInBacklog(BufferConsumer buffer) {
+		assert Thread.holdsLock(buffers);
+
+		if (buffer != null && buffer.isBuffer()) {
+			buffersInBacklog++;
+		}
+	}
+
+	/**
+	 * Gets the number of non-event buffers in this subpartition.
+	 *
+	 * <p><strong>Beware:</strong> This method should only be used in tests in non-concurrent access
+	 * scenarios since it does not make any concurrency guarantees.
+	 */
+	@SuppressWarnings("FieldAccessNotGuarded")
+	@VisibleForTesting
+	public int getBuffersInBacklog() {
+		return buffersInBacklog;
+	}
+
 	private boolean shouldNotifyDataAvailable() {
 		// Notify only when we added first finished buffer.
 		return readView != null && !flushRequested && getNumberOfFinishedBuffers() == 1;
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java
index 24ce27e..1ff1ec5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java
@@ -32,6 +32,8 @@ import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
 import org.apache.flink.runtime.taskexecutor.TaskExecutor;
 import org.apache.flink.runtime.taskmanager.TaskActions;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkRuntimeException;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -150,10 +152,7 @@ public class ResultPartition implements ResultPartitionWriter, BufferPoolOwner {
 		// Create the subpartitions.
 		switch (partitionType) {
 			case BLOCKING:
-				for (int i = 0; i < subpartitions.length; i++) {
-					subpartitions[i] = new SpillableSubpartition(i, this, ioManager);
-				}
-
+				initializeBoundedBlockingPartitions(subpartitions, this, ioManager);
 				break;
 
 			case PIPELINED:
@@ -466,4 +465,35 @@ public class ResultPartition implements ResultPartitionWriter, BufferPoolOwner {
 			hasNotifiedPipelinedConsumers = true;
 		}
 	}
+
+	private static void initializeBoundedBlockingPartitions(
+			ResultSubpartition[] subpartitions,
+			ResultPartition parent,
+			IOManager ioManager) {
+
+		int i = 0;
+		try {
+			for (; i < subpartitions.length; i++) {
+				subpartitions[i] = new BoundedBlockingSubpartition(
+						i, parent, ioManager.createChannel().getPathFile().toPath());
+			}
+		}
+		catch (IOException e) {
+			// undo all the work so that a failed constructor does not leave any resources
+			// in need of disposal
+			releasePartitionsQuietly(subpartitions, i);
+
+			// this is not good, we should not be forced to wrap this in a runtime exception.
+			// the fact that the ResultPartition and Task constructor (which calls this) do not tolerate any exceptions
+			// is incompatible with eager initialization of resources (RAII).
+			throw new FlinkRuntimeException(e);
+		}
+	}
+
+	private static void releasePartitionsQuietly(ResultSubpartition[] partitions, int until) {
+		for (int i = 0; i < until; i++) {
+			final ResultSubpartition subpartition = partitions[i];
+			ExceptionUtils.suppressExceptions(subpartition::release);
+		}
+	}
 }
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java
index 58a1402..920ce8d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java
@@ -22,10 +22,7 @@ import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
 import org.apache.flink.runtime.io.network.buffer.BufferConsumer;
 
-import javax.annotation.concurrent.GuardedBy;
-
 import java.io.IOException;
-import java.util.ArrayDeque;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
@@ -40,41 +37,19 @@ public abstract class ResultSubpartition {
 	/** The parent partition this subpartition belongs to. */
 	protected final ResultPartition parent;
 
-	/** All buffers of this subpartition. Access to the buffers is synchronized on this object. */
-	protected final ArrayDeque<BufferConsumer> buffers = new ArrayDeque<>();
-
-	/** The number of non-event buffers currently in this subpartition. */
-	@GuardedBy("buffers")
-	private int buffersInBacklog;
-
 	// - Statistics ----------------------------------------------------------
 
-	/** The total number of buffers (both data and event buffers). */
-	private long totalNumberOfBuffers;
-
-	/** The total number of bytes (both data and event buffers). */
-	private long totalNumberOfBytes;
-
 	public ResultSubpartition(int index, ResultPartition parent) {
 		this.index = index;
 		this.parent = parent;
 	}
 
-	protected void updateStatistics(BufferConsumer buffer) {
-		totalNumberOfBuffers++;
-	}
-
-	protected void updateStatistics(Buffer buffer) {
-		totalNumberOfBytes += buffer.getSize();
-	}
-
-	protected long getTotalNumberOfBuffers() {
-		return totalNumberOfBuffers;
-	}
+	/**
+	 * Gets the total numbers of buffers (data buffers plus events).
+	 */
+	protected abstract long getTotalNumberOfBuffers();
 
-	protected long getTotalNumberOfBytes() {
-		return totalNumberOfBytes;
-	}
+	protected abstract long getTotalNumberOfBytes();
 
 	/**
 	 * Notifies the parent partition about a consumed {@link ResultSubpartitionView}.
@@ -83,10 +58,6 @@ public abstract class ResultSubpartition {
 		parent.onConsumedSubpartition(index);
 	}
 
-	protected Throwable getFailureCause() {
-		return parent.getFailureCause();
-	}
-
 	/**
 	 * Adds the given buffer.
 	 *
@@ -123,9 +94,7 @@ public abstract class ResultSubpartition {
 	 * scenarios since it does not make any concurrency guarantees.
 	 */
 	@VisibleForTesting
-	public int getBuffersInBacklog() {
-		return buffersInBacklog;
-	}
+	abstract int getBuffersInBacklog();
 
 	/**
 	 * Makes a best effort to get the current size of the queue.
@@ -134,38 +103,6 @@ public abstract class ResultSubpartition {
 	 */
 	public abstract int unsynchronizedGetNumberOfQueuedBuffers();
 
-	/**
-	 * Decreases the number of non-event buffers by one after fetching a non-event
-	 * buffer from this subpartition (for access by the subpartition views).
-	 *
-	 * @return backlog after the operation
-	 */
-	public int decreaseBuffersInBacklog(Buffer buffer) {
-		synchronized (buffers) {
-			return decreaseBuffersInBacklogUnsafe(buffer != null && buffer.isBuffer());
-		}
-	}
-
-	protected int decreaseBuffersInBacklogUnsafe(boolean isBuffer) {
-		assert Thread.holdsLock(buffers);
-		if (isBuffer) {
-			buffersInBacklog--;
-		}
-		return buffersInBacklog;
-	}
-
-	/**
-	 * Increases the number of non-event buffers by one after adding a non-event
-	 * buffer into this subpartition.
-	 */
-	protected void increaseBuffersInBacklog(BufferConsumer buffer) {
-		assert Thread.holdsLock(buffers);
-
-		if (buffer != null && buffer.isBuffer()) {
-			buffersInBacklog++;
-		}
-	}
-
 	// ------------------------------------------------------------------------
 
 	/**
@@ -201,6 +138,14 @@ public abstract class ResultSubpartition {
 		public boolean nextBufferIsEvent() {
 			return nextBufferIsEvent;
 		}
+
+		public static BufferAndBacklog fromBufferAndLookahead(Buffer current, Buffer lookahead, int backlog) {
+			return new BufferAndBacklog(
+					current,
+					lookahead != null,
+					backlog,
+					lookahead != null && !lookahead.isBuffer());
+		}
 	}
 
 }
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java
deleted file mode 100644
index 9f696ad..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java
+++ /dev/null
@@ -1,312 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.io.network.partition;
-
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.configuration.TaskManagerOptions;
-import org.apache.flink.runtime.io.disk.iomanager.BufferFileWriter;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
-import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.buffer.BufferConsumer;
-import org.apache.flink.runtime.io.network.buffer.BufferPool;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * A spillable sub partition starts out in-memory and spills to disk if asked
- * to do so.
- *
- * <p>Buffers for the partition come from a {@link BufferPool}. The buffer pool
- * is also responsible to trigger the release of the buffers if it needs them
- * back. At this point, the spillable sub partition will write all in-memory
- * buffers to disk. All added buffers after that point directly go to disk.
- *
- * <p>This partition type is used for {@link ResultPartitionType#BLOCKING}
- * results that are fully produced before they can be consumed. At the point
- * when they are consumed, the buffers are (i) all in-memory, (ii) currently
- * being spilled to disk, or (iii) completely spilled to disk. Depending on
- * this state, different reader variants are returned (see
- * {@link SpillableSubpartitionView} and {@link SpilledSubpartitionView}).
- *
- * <p>Since the network buffer pool size for outgoing partitions is usually
- * quite small, e.g. via the {@link TaskManagerOptions#NETWORK_BUFFERS_PER_CHANNEL}
- * and {@link TaskManagerOptions#NETWORK_EXTRA_BUFFERS_PER_GATE} parameters
- * for bounded channels or from the default values of
- * {@link TaskManagerOptions#NETWORK_BUFFERS_MEMORY_FRACTION},
- * {@link TaskManagerOptions#NETWORK_BUFFERS_MEMORY_MIN}, and
- * {@link TaskManagerOptions#NETWORK_BUFFERS_MEMORY_MAX}, most spillable partitions
- * will be spilled for real-world data sets.
- *
- * <p>Note on thread safety. Synchronizing on {@code buffers} is used to synchronize
- * writes and reads. Synchronizing on {@code this} is used against concurrent
- * {@link #add(BufferConsumer)} and clean ups {@link #release()} / {@link #finish()} which
- * also are touching {@code spillWriter}. Since we do not want to block reads during
- * spilling, we need those two synchronization. Probably this model could be simplified.
- */
-class SpillableSubpartition extends ResultSubpartition {
-
-	private static final Logger LOG = LoggerFactory.getLogger(SpillableSubpartition.class);
-
-	/** The I/O manager used for spilling buffers to disk. */
-	private final IOManager ioManager;
-
-	/** The writer used for spilling. As long as this is null, we are in-memory. */
-	private BufferFileWriter spillWriter;
-
-	/** Flag indicating whether the subpartition has been finished. */
-	private boolean isFinished;
-
-	/** Flag indicating whether the subpartition has been released. */
-	private volatile boolean isReleased;
-
-	/** The read view to consume this subpartition. */
-	private ResultSubpartitionView readView;
-
-	SpillableSubpartition(int index, ResultPartition parent, IOManager ioManager) {
-		super(index, parent);
-
-		this.ioManager = checkNotNull(ioManager);
-	}
-
-	@Override
-	public synchronized boolean add(BufferConsumer bufferConsumer) throws IOException {
-		return add(bufferConsumer, false);
-	}
-
-	private boolean add(BufferConsumer bufferConsumer, boolean forceFinishRemainingBuffers)
-			throws IOException {
-		checkNotNull(bufferConsumer);
-
-		synchronized (buffers) {
-			if (isFinished || isReleased) {
-				bufferConsumer.close();
-				return false;
-			}
-
-			buffers.add(bufferConsumer);
-			// The number of buffers are needed later when creating
-			// the read views. If you ever remove this line here,
-			// make sure to still count the number of buffers.
-			updateStatistics(bufferConsumer);
-			increaseBuffersInBacklog(bufferConsumer);
-
-			if (spillWriter != null) {
-				spillFinishedBufferConsumers(forceFinishRemainingBuffers);
-			}
-		}
-		return true;
-	}
-
-	@Override
-	public void flush() {
-		synchronized (buffers) {
-			if (readView != null) {
-				readView.notifyDataAvailable();
-			}
-		}
-	}
-
-	@Override
-	public synchronized void finish() throws IOException {
-		synchronized (buffers) {
-			if (add(EventSerializer.toBufferConsumer(EndOfPartitionEvent.INSTANCE), true)) {
-				isFinished = true;
-			}
-
-			flush();
-		}
-
-		// If we are spilling/have spilled, wait for the writer to finish
-		if (spillWriter != null) {
-			spillWriter.close();
-		}
-		LOG.debug("{}: Finished {}.", parent.getOwningTaskName(), this);
-	}
-
-	@Override
-	public synchronized void release() throws IOException {
-		// view reference accessible outside the lock, but assigned inside the locked scope
-		final ResultSubpartitionView view;
-
-		synchronized (buffers) {
-			if (isReleased) {
-				return;
-			}
-
-			// Release all available buffers
-			for (BufferConsumer buffer : buffers) {
-				buffer.close();
-			}
-			buffers.clear();
-
-			view = readView;
-
-			// No consumer yet, we are responsible to clean everything up. If
-			// one is available, the view is responsible is to clean up (see
-			// below).
-			if (view == null) {
-
-				// TODO This can block until all buffers are written out to
-				// disk if a spill is in-progress before deleting the file.
-				// It is possibly called from the Netty event loop threads,
-				// which can bring down the network.
-				if (spillWriter != null) {
-					spillWriter.closeAndDelete();
-				}
-			}
-
-			isReleased = true;
-		}
-
-		LOG.debug("{}: Released {}.", parent.getOwningTaskName(), this);
-
-		if (view != null) {
-			view.releaseAllResources();
-		}
-	}
-
-	@Override
-	public ResultSubpartitionView createReadView(BufferAvailabilityListener availabilityListener) throws IOException {
-		synchronized (buffers) {
-			if (!isFinished) {
-				throw new IllegalStateException("Subpartition has not been finished yet, " +
-					"but blocking subpartitions can only be consumed after they have " +
-					"been finished.");
-			}
-
-			if (readView != null) {
-				throw new IllegalStateException("Subpartition is being or already has been " +
-					"consumed, but we currently allow subpartitions to only be consumed once.");
-			}
-
-			if (spillWriter != null) {
-				readView = new SpilledSubpartitionView(
-					this,
-					parent.getBufferProvider().getMemorySegmentSize(),
-					spillWriter,
-					getTotalNumberOfBuffers(),
-					availabilityListener);
-			} else {
-				readView = new SpillableSubpartitionView(
-					this,
-					buffers,
-					ioManager,
-					parent.getBufferProvider().getMemorySegmentSize(),
-					availabilityListener);
-			}
-			return readView;
-		}
-	}
-
-	@Override
-	public int releaseMemory() throws IOException {
-		synchronized (buffers) {
-			ResultSubpartitionView view = readView;
-
-			if (view != null && view.getClass() == SpillableSubpartitionView.class) {
-				// If there is a spillable view, it's the responsibility of the
-				// view to release memory.
-				SpillableSubpartitionView spillableView = (SpillableSubpartitionView) view;
-				return spillableView.releaseMemory();
-			} else if (spillWriter == null) {
-				// No view and in-memory => spill to disk
-				spillWriter = ioManager.createBufferFileWriter(ioManager.createChannel());
-
-				int numberOfBuffers = buffers.size();
-				long spilledBytes = spillFinishedBufferConsumers(isFinished);
-				int spilledBuffers = numberOfBuffers - buffers.size();
-
-				LOG.debug("{}: Spilling {} bytes ({} buffers} for sub partition {} of {}.",
-					parent.getOwningTaskName(), spilledBytes, spilledBuffers, index, parent.getPartitionId());
-
-				return spilledBuffers;
-			}
-		}
-
-		// Else: We have already spilled and don't hold any buffers
-		return 0;
-	}
-
-	@VisibleForTesting
-	long spillFinishedBufferConsumers(boolean forceFinishRemainingBuffers) throws IOException {
-		long spilledBytes = 0;
-
-		while (!buffers.isEmpty()) {
-			BufferConsumer bufferConsumer = buffers.getFirst();
-			Buffer buffer = bufferConsumer.build();
-			updateStatistics(buffer);
-			int bufferSize = buffer.getSize();
-			spilledBytes += bufferSize;
-
-			// NOTE we may be in the process of finishing the subpartition where any buffer should
-			// be treated as if it was finished!
-			if (bufferConsumer.isFinished() || forceFinishRemainingBuffers) {
-				if (bufferSize > 0) {
-					spillWriter.writeBlock(buffer);
-				} else {
-					// If we skip a buffer for the spill writer, we need to adapt the backlog accordingly
-					decreaseBuffersInBacklog(buffer);
-					buffer.recycleBuffer();
-				}
-				bufferConsumer.close();
-				buffers.poll();
-			} else {
-				// If there is already data, we need to spill it anyway, since we do not get this
-				// slice from the buffer consumer again during the next build.
-				// BEWARE: by doing so, we increase the actual number of buffers in the spill writer!
-				if (bufferSize > 0) {
-					spillWriter.writeBlock(buffer);
-					increaseBuffersInBacklog(bufferConsumer);
-				} else {
-					buffer.recycleBuffer();
-				}
-
-				return spilledBytes;
-			}
-		}
-		return spilledBytes;
-	}
-
-	@Override
-	public boolean isReleased() {
-		return isReleased;
-	}
-
-	@Override
-	public int unsynchronizedGetNumberOfQueuedBuffers() {
-		// since we do not synchronize, the size may actually be lower than 0!
-		return Math.max(buffers.size(), 0);
-	}
-
-	@Override
-	public String toString() {
-		return String.format("SpillableSubpartition#%d [%d number of buffers (%d bytes)," +
-				"%d number of buffers in backlog, finished? %s, read view? %s, spilled? %s]",
-			index, getTotalNumberOfBuffers(), getTotalNumberOfBytes(),
-			getBuffersInBacklog(), isFinished, readView != null, spillWriter != null);
-	}
-
-}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java
deleted file mode 100644
index 65790d7..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java
+++ /dev/null
@@ -1,280 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.io.network.partition;
-
-import org.apache.flink.runtime.io.disk.iomanager.BufferFileWriter;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.buffer.BufferConsumer;
-import org.apache.flink.runtime.io.network.partition.ResultSubpartition.BufferAndBacklog;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.annotation.Nullable;
-
-import java.io.IOException;
-import java.util.ArrayDeque;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-import static org.apache.flink.util.Preconditions.checkState;
-
-class SpillableSubpartitionView implements ResultSubpartitionView {
-
-	private static final Logger LOG = LoggerFactory.getLogger(SpillableSubpartitionView.class);
-
-	/** The subpartition this view belongs to. */
-	private final SpillableSubpartition parent;
-
-	/** All buffers of this subpartition. Access to the buffers is synchronized on this object. */
-	private final ArrayDeque<BufferConsumer> buffers;
-
-	/** IO manager if we need to spill (for spilled case). */
-	private final IOManager ioManager;
-
-	/** Size of memory segments (for spilled case). */
-	private final int memorySegmentSize;
-
-	/**
-	 * The buffer availability listener. As long as in-memory, notifications
-	 * happen on a buffer per buffer basis as spilling may happen after a
-	 * notification has been sent out.
-	 */
-	private final BufferAvailabilityListener listener;
-
-	private final AtomicBoolean isReleased = new AtomicBoolean(false);
-
-	/** Remember the number of buffers this view was created with. */
-	private final long numBuffers;
-
-	/**
-	 * The next buffer to hand out. Everytime this is set to a non-null value,
-	 * a listener notification happens.
-	 */
-	private BufferConsumer nextBuffer;
-
-	private volatile SpilledSubpartitionView spilledView;
-
-	SpillableSubpartitionView(
-		SpillableSubpartition parent,
-		ArrayDeque<BufferConsumer> buffers,
-		IOManager ioManager,
-		int memorySegmentSize,
-		BufferAvailabilityListener listener) {
-
-		this.parent = checkNotNull(parent);
-		this.buffers = checkNotNull(buffers);
-		this.ioManager = checkNotNull(ioManager);
-		this.memorySegmentSize = memorySegmentSize;
-		this.listener = checkNotNull(listener);
-
-		synchronized (buffers) {
-			numBuffers = buffers.size();
-			nextBuffer = buffers.poll();
-		}
-
-		if (nextBuffer != null) {
-			listener.notifyDataAvailable();
-		}
-	}
-
-	int releaseMemory() throws IOException {
-		synchronized (buffers) {
-			if (spilledView != null || nextBuffer == null) {
-				// Already spilled or nothing in-memory
-				return 0;
-			} else {
-				// We don't touch next buffer, because a notification has
-				// already been sent for it. Only when it is consumed, will
-				// it be recycled.
-
-				// Create the spill writer and write all buffers to disk
-				BufferFileWriter spillWriter = ioManager.createBufferFileWriter(ioManager.createChannel());
-
-				long spilledBytes = 0;
-
-				int numBuffers = buffers.size();
-				for (int i = 0; i < numBuffers; i++) {
-					try (BufferConsumer bufferConsumer = buffers.remove()) {
-						Buffer buffer = bufferConsumer.build();
-						checkState(bufferConsumer.isFinished(), "BufferConsumer must be finished before " +
-							"spilling. Otherwise we would not be able to simply remove it from the queue. This should " +
-							"be guaranteed by creating ResultSubpartitionView only once Subpartition isFinished.");
-						parent.updateStatistics(buffer);
-						spilledBytes += buffer.getSize();
-						spillWriter.writeBlock(buffer);
-					}
-				}
-
-				spilledView = new SpilledSubpartitionView(
-					parent,
-					memorySegmentSize,
-					spillWriter,
-					numBuffers,
-					listener);
-
-				LOG.debug("Spilling {} bytes for sub partition {} of {}.",
-					spilledBytes,
-					parent.index,
-					parent.parent.getPartitionId());
-
-				return numBuffers;
-			}
-		}
-	}
-
-	@Nullable
-	@Override
-	public BufferAndBacklog getNextBuffer() throws IOException, InterruptedException {
-		Buffer current = null;
-		boolean nextBufferIsEvent = false;
-		int newBacklog = 0; // this is always correct if current is non-null!
-		boolean isMoreAvailable = false;
-
-		synchronized (buffers) {
-			if (isReleased.get()) {
-				return null;
-			} else if (nextBuffer != null) {
-				current = nextBuffer.build();
-				checkState(nextBuffer.isFinished(),
-					"We can only read from SpillableSubpartition after it was finished");
-
-				newBacklog = parent.decreaseBuffersInBacklogUnsafe(nextBuffer.isBuffer());
-				nextBuffer.close();
-				nextBuffer = buffers.poll();
-
-				if (nextBuffer != null) {
-					nextBufferIsEvent = !nextBuffer.isBuffer();
-					isMoreAvailable = true;
-				}
-
-				parent.updateStatistics(current);
-				// if we are spilled (but still process a non-spilled nextBuffer), we don't know the
-				// state of nextBufferIsEvent or whether more buffers are available
-				if (spilledView == null) {
-					return new BufferAndBacklog(current, isMoreAvailable, newBacklog, nextBufferIsEvent);
-				}
-			}
-		} // else: spilled
-
-		SpilledSubpartitionView spilled = spilledView;
-		if (spilled != null) {
-			if (current != null) {
-				return new BufferAndBacklog(current, spilled.isAvailable(), newBacklog, spilled.nextBufferIsEvent());
-			} else {
-				return spilled.getNextBuffer();
-			}
-		} else {
-			throw new IllegalStateException("No in-memory buffers available, but also nothing spilled.");
-		}
-	}
-
-	@Override
-	public void notifyDataAvailable() {
-		// We do the availability listener notification one by one
-	}
-
-	@Override
-	public void releaseAllResources() throws IOException {
-		if (isReleased.compareAndSet(false, true)) {
-			SpilledSubpartitionView spilled = spilledView;
-			if (spilled != null) {
-				spilled.releaseAllResources();
-			}
-			// we are never giving this buffer out in getNextBuffer(), so we need to clean it up
-			synchronized (buffers) {
-				if (nextBuffer != null) {
-					nextBuffer.close();
-					nextBuffer = null;
-				}
-			}
-		}
-	}
-
-	@Override
-	public void notifySubpartitionConsumed() throws IOException {
-		SpilledSubpartitionView spilled = spilledView;
-		if (spilled != null) {
-			spilled.notifySubpartitionConsumed();
-		} else {
-			parent.onConsumedSubpartition();
-		}
-	}
-
-	@Override
-	public boolean isReleased() {
-		SpilledSubpartitionView spilled = spilledView;
-		if (spilled != null) {
-			return spilled.isReleased();
-		} else {
-			return parent.isReleased() || isReleased.get();
-		}
-	}
-
-	@Override
-	public boolean nextBufferIsEvent() {
-		synchronized (buffers) {
-			if (isReleased.get()) {
-				return false;
-			} else if (nextBuffer != null) {
-				return !nextBuffer.isBuffer();
-			}
-		} // else: spilled
-
-		checkState(spilledView != null, "No in-memory buffers available, but also nothing spilled.");
-
-		return spilledView.nextBufferIsEvent();
-	}
-
-	@Override
-	public boolean isAvailable() {
-		synchronized (buffers) {
-			if (nextBuffer != null) {
-				return true;
-			}
-			else if (spilledView == null) {
-				return false;
-			}
-		} // else: spilled
-
-		return spilledView.isAvailable();
-	}
-
-	@Override
-	public Throwable getFailureCause() {
-		SpilledSubpartitionView spilled = spilledView;
-		if (spilled != null) {
-			return spilled.getFailureCause();
-		} else {
-			return parent.getFailureCause();
-		}
-	}
-
-	@Override
-	public String toString() {
-		boolean hasSpilled = spilledView != null;
-
-		return String.format("SpillableSubpartitionView(index: %d, buffers: %d, spilled? %b) of ResultPartition %s",
-			parent.index,
-			numBuffers,
-			hasSpilled,
-			parent.parent.getPartitionId());
-	}
-}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionView.java
deleted file mode 100644
index f941e20..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionView.java
+++ /dev/null
@@ -1,303 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.io.network.partition;
-
-import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.core.memory.MemorySegmentFactory;
-import org.apache.flink.runtime.io.disk.iomanager.BufferFileReader;
-import org.apache.flink.runtime.io.disk.iomanager.BufferFileWriter;
-import org.apache.flink.runtime.io.disk.iomanager.SynchronousBufferFileReader;
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
-import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
-import org.apache.flink.runtime.io.network.partition.ResultSubpartition.BufferAndBacklog;
-import org.apache.flink.runtime.util.event.NotificationListener;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.annotation.Nullable;
-import javax.annotation.concurrent.GuardedBy;
-
-import java.io.IOException;
-import java.util.ArrayDeque;
-import java.util.Queue;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import static org.apache.flink.util.Preconditions.checkArgument;
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * Reader for a spilled sub partition.
- *
- * <p>The partition availability listener is notified about available buffers
- * only when the spilling is done. Spilling is done async and if it is still
- * in progress, we wait with the notification until the spilling is done.
- *
- * <p>Reads of the spilled file are done in synchronously.
- */
-class SpilledSubpartitionView implements ResultSubpartitionView, NotificationListener {
-
-	private static final Logger LOG = LoggerFactory.getLogger(SpilledSubpartitionView.class);
-
-	/** The subpartition this view belongs to. */
-	private final SpillableSubpartition parent;
-
-	/** Writer for spills. */
-	private final BufferFileWriter spillWriter;
-
-	/** The synchronous file reader to do the actual I/O. */
-	@GuardedBy("this")
-	private final BufferFileReader fileReader;
-
-	/** The buffer pool to read data into. */
-	private final SpillReadBufferPool bufferPool;
-
-	/** Buffer availability listener. */
-	private final BufferAvailabilityListener availabilityListener;
-
-	/** The total number of spilled buffers. */
-	private final long numberOfSpilledBuffers;
-
-	/** Flag indicating whether all resources have been released. */
-	private AtomicBoolean isReleased = new AtomicBoolean();
-
-	/** The next buffer to hand out. */
-	@GuardedBy("this")
-	private Buffer nextBuffer;
-
-	/** Flag indicating whether a spill is still in progress. */
-	private volatile boolean isSpillInProgress = true;
-
-	SpilledSubpartitionView(
-		SpillableSubpartition parent,
-		int memorySegmentSize,
-		BufferFileWriter spillWriter,
-		long numberOfSpilledBuffers,
-		BufferAvailabilityListener availabilityListener) throws IOException {
-
-		this.parent = checkNotNull(parent);
-		this.bufferPool = new SpillReadBufferPool(2, memorySegmentSize);
-		this.spillWriter = checkNotNull(spillWriter);
-		this.fileReader = new SynchronousBufferFileReader(spillWriter.getChannelID(), false);
-		checkArgument(numberOfSpilledBuffers >= 0);
-		this.numberOfSpilledBuffers = numberOfSpilledBuffers;
-		this.availabilityListener = checkNotNull(availabilityListener);
-
-		// Check whether async spilling is still in progress. If not, this returns
-		// false and we can notify our availability listener about all available buffers.
-		// Otherwise, we notify only when the spill writer callback happens.
-		if (!spillWriter.registerAllRequestsProcessedListener(this)) {
-			isSpillInProgress = false;
-			availabilityListener.notifyDataAvailable();
-			LOG.debug("No spilling in progress. Notified about {} available buffers.", numberOfSpilledBuffers);
-		} else {
-			LOG.debug("Spilling in progress. Waiting with notification about {} available buffers.", numberOfSpilledBuffers);
-		}
-	}
-
-	/**
-	 * This is the call back method for the spill writer. If a spill is still
-	 * in progress when this view is created we wait until this method is called
-	 * before we notify the availability listener.
-	 */
-	@Override
-	public void onNotification() {
-		isSpillInProgress = false;
-		availabilityListener.notifyDataAvailable();
-		LOG.debug("Finished spilling. Notified about {} available buffers.", numberOfSpilledBuffers);
-	}
-
-	@Nullable
-	@Override
-	public BufferAndBacklog getNextBuffer() throws IOException, InterruptedException {
-		if (isSpillInProgress) {
-			return null;
-		}
-
-		Buffer current;
-		boolean nextBufferIsEvent;
-		synchronized (this) {
-			if (nextBuffer == null) {
-				current = requestAndFillBuffer();
-			} else {
-				current = nextBuffer;
-			}
-			nextBuffer = requestAndFillBuffer();
-			nextBufferIsEvent = nextBuffer != null && !nextBuffer.isBuffer();
-		}
-
-		if (current == null) {
-			return null;
-		}
-
-		int newBacklog = parent.decreaseBuffersInBacklog(current);
-		return new BufferAndBacklog(current, newBacklog > 0 || nextBufferIsEvent, newBacklog, nextBufferIsEvent);
-	}
-
-	@Nullable
-	private Buffer requestAndFillBuffer() throws IOException, InterruptedException {
-		assert Thread.holdsLock(this);
-
-		if (fileReader.hasReachedEndOfFile()) {
-			return null;
-		}
-		// TODO This is fragile as we implicitly expect that multiple calls to
-		// this method don't happen before recycling buffers returned earlier.
-		Buffer buffer = bufferPool.requestBufferBlocking();
-		fileReader.readInto(buffer);
-		return buffer;
-	}
-
-	@Override
-	public void notifyDataAvailable() {
-		// We do the availability listener notification either directly on
-		// construction of this view (when everything has been spilled) or
-		// as soon as spilling is done and we are notified about it in the
-		// #onNotification callback.
-	}
-
-	@Override
-	public void notifySubpartitionConsumed() throws IOException {
-		parent.onConsumedSubpartition();
-	}
-
-	@Override
-	public void releaseAllResources() throws IOException {
-		if (isReleased.compareAndSet(false, true)) {
-			// TODO This can block until all buffers are written out to
-			// disk if a spill is in-progress before deleting the file.
-			// It is possibly called from the Netty event loop threads,
-			// which can bring down the network.
-			spillWriter.closeAndDelete();
-
-			synchronized (this) {
-				fileReader.close();
-				if (nextBuffer != null) {
-					nextBuffer.recycleBuffer();
-					nextBuffer = null;
-				}
-			}
-
-			bufferPool.destroy();
-		}
-	}
-
-	@Override
-	public boolean isReleased() {
-		return parent.isReleased() || isReleased.get();
-	}
-
-	@Override
-	public boolean nextBufferIsEvent() {
-		synchronized (this) {
-			if (nextBuffer == null) {
-				try {
-					nextBuffer = requestAndFillBuffer();
-				} catch (Exception e) {
-					// we can ignore this here (we will get it again once getNextBuffer() is called)
-					return false;
-				}
-			}
-			return nextBuffer != null && !nextBuffer.isBuffer();
-		}
-	}
-
-	@Override
-	public synchronized boolean isAvailable() {
-		if (nextBuffer != null) {
-			return true;
-		}
-		return !fileReader.hasReachedEndOfFile();
-	}
-
-	@Override
-	public Throwable getFailureCause() {
-		return parent.getFailureCause();
-	}
-
-	@Override
-	public String toString() {
-		return String.format("SpilledSubpartitionView(index: %d, buffers: %d) of ResultPartition %s",
-			parent.index,
-			numberOfSpilledBuffers,
-			parent.parent.getPartitionId());
-	}
-
-	/**
-	 * A buffer pool to provide buffer to read the file into.
-	 *
-	 * <p>This pool ensures that a consuming input gate makes progress in all cases, even when all
-	 * buffers of the input gate buffer pool have been requested by remote input channels.
-	 */
-	private static class SpillReadBufferPool implements BufferRecycler {
-
-		private final Queue<Buffer> buffers;
-
-		private boolean isDestroyed;
-
-		SpillReadBufferPool(int numberOfBuffers, int memorySegmentSize) {
-			this.buffers = new ArrayDeque<>(numberOfBuffers);
-
-			synchronized (buffers) {
-				for (int i = 0; i < numberOfBuffers; i++) {
-					buffers.add(new NetworkBuffer(MemorySegmentFactory.allocateUnpooledOffHeapMemory(
-						memorySegmentSize, null), this));
-				}
-			}
-		}
-
-		@Override
-		public void recycle(MemorySegment memorySegment) {
-			synchronized (buffers) {
-				if (isDestroyed) {
-					memorySegment.free();
-				} else {
-					buffers.add(new NetworkBuffer(memorySegment, this));
-					buffers.notifyAll();
-				}
-			}
-		}
-
-		private Buffer requestBufferBlocking() throws InterruptedException {
-			synchronized (buffers) {
-				while (true) {
-					if (isDestroyed) {
-						return null;
-					}
-
-					Buffer buffer = buffers.poll();
-
-					if (buffer != null) {
-						return buffer;
-					}
-					// Else: wait for a buffer
-					buffers.wait();
-				}
-			}
-		}
-
-		private void destroy() {
-			synchronized (buffers) {
-				isDestroyed = true;
-				buffers.notifyAll();
-			}
-		}
-	}
-}
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderTestUtils.java
index 9706a86..7a68368 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderTestUtils.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderTestUtils.java
@@ -18,11 +18,14 @@
 
 package org.apache.flink.runtime.io.network.buffer;
 
+import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.core.memory.MemorySegmentFactory;
 
 import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
 
 import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.junit.Assert.assertEquals;
 
 /**
  * Utility class for create not-pooled {@link BufferBuilder}.
@@ -83,4 +86,30 @@ public class BufferBuilderTestUtils {
 			FreeingBufferRecycler.INSTANCE,
 			false);
 	}
+
+	public static Buffer buildBufferWithAscendingInts(int bufferSize, int numInts, int nextValue) {
+		final MemorySegment seg = MemorySegmentFactory.allocateUnpooledSegment(bufferSize);
+		for (int i = 0; i < numInts; i++) {
+			seg.putIntLittleEndian(4 * i, nextValue++);
+		}
+
+		return new NetworkBuffer(seg, MemorySegment::free, true, 4 * numInts);
+	}
+
+	public static void validateBufferWithAscendingInts(Buffer buffer, int numInts, int nextValue) {
+		final ByteBuffer bb = buffer.getNioBufferReadable().order(ByteOrder.LITTLE_ENDIAN);
+
+		for (int i = 0; i < numInts; i++) {
+			assertEquals(nextValue++, bb.getInt());
+		}
+	}
+
+	public static Buffer buildSomeBuffer() {
+		return buildSomeBuffer(1024);
+	}
+
+	public static Buffer buildSomeBuffer(int size) {
+		final MemorySegment seg = MemorySegmentFactory.allocateUnpooledSegment(size);
+		return new NetworkBuffer(seg, MemorySegment::free, true, size);
+	}
 }
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionTest.java
new file mode 100644
index 0000000..5051b90
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionTest.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition;
+
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+
+import static org.junit.Assert.fail;
+
+/**
+ * Behavior tests for the {@link BoundedBlockingSubpartition} and the
+ * {@link BoundedBlockingSubpartitionReader}.
+ *
+ * <p>Full read / write tests for the partition and the reader are in
+ * {@link BoundedBlockingSubpartitionWriteReadTest}.
+ */
+public class BoundedBlockingSubpartitionTest extends SubpartitionTestBase {
+
+	@ClassRule
+	public static final TemporaryFolder TMP_DIR = new TemporaryFolder();
+
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testCreateReaderBeforeFinished() throws Exception {
+		final ResultSubpartition partition = createSubpartition();
+
+		try {
+			partition.createReadView(new NoOpBufferAvailablityListener());
+			fail("expected exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		partition.release();
+	}
+
+	// ------------------------------------------------------------------------
+
+	@Override
+	ResultSubpartition createSubpartition() throws Exception {
+		final ResultPartition resultPartition = PartitionTestUtils.createPartition(ResultPartitionType.BLOCKING);
+		return new BoundedBlockingSubpartition(0, resultPartition, tmpPath());
+	}
+
+	@Override
+	ResultSubpartition createFailingWritesSubpartition() throws Exception {
+		final ResultPartition resultPartition = PartitionTestUtils.createPartition(ResultPartitionType.BLOCKING);
+
+		return new BoundedBlockingSubpartition(
+				0,
+				resultPartition,
+				FailingMemory.create());
+	}
+
+	// ------------------------------------------------------------------------
+
+	static Path tmpPath() throws IOException {
+		return new File(TMP_DIR.newFolder(), "subpartition").toPath();
+	}
+
+	// ------------------------------------------------------------------------
+
+	private static class FailingMemory extends MemoryMappedBuffers {
+
+		FailingMemory(Path path, FileChannel fc) throws IOException {
+			super(path, fc, Integer.MAX_VALUE);
+		}
+
+		@Override
+		void writeBuffer(Buffer buffer) throws IOException {
+			throw new IOException("test");
+		}
+
+		static FailingMemory create() throws IOException {
+			Path p = tmpPath();
+			FileChannel fc = FileChannel.open(p, StandardOpenOption.WRITE, StandardOpenOption.READ, StandardOpenOption.CREATE);
+			return new FailingMemory(p, fc);
+		}
+	}
+}
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionWriteReadTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionWriteReadTest.java
new file mode 100644
index 0000000..46dbb05
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionWriteReadTest.java
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition;
+
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.runtime.io.network.buffer.BufferConsumer;
+import org.apache.flink.runtime.io.network.partition.ResultSubpartition.BufferAndBacklog;
+
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests that read the BoundedBlockingSubpartition with multiple threads in parallel.
+ */
+public class BoundedBlockingSubpartitionWriteReadTest {
+
+	@ClassRule
+	public static final TemporaryFolder TMP_FOLDER = new TemporaryFolder();
+
+	@Test
+	public void testWriteAndReadData() throws Exception {
+		final int numLongs = 15_000_000; // roughly 115 MiBytes
+
+		// setup
+		final BoundedBlockingSubpartition subpartition = createAndFillPartition(numLongs);
+
+		// test & check
+		final ResultSubpartitionView reader = subpartition.createReadView(() -> {});
+		readLongs(reader, numLongs, subpartition.getBuffersInBacklog());
+
+		// cleanup
+		reader.releaseAllResources();
+		subpartition.release();
+	}
+
+	@Test
+	public void testRead10ConsumersSequential() throws Exception {
+		final int numLongs = 10_000_000;
+
+		// setup
+		final BoundedBlockingSubpartition subpartition = createAndFillPartition(numLongs);
+
+		// test & check
+		for (int i = 0; i < 10; i++) {
+			final ResultSubpartitionView reader = subpartition.createReadView(() -> {});
+			readLongs(reader, numLongs, subpartition.getBuffersInBacklog());
+			reader.releaseAllResources();
+		}
+
+		// cleanup
+		subpartition.release();
+	}
+
+	@Test
+	public void testRead10ConsumersConcurrent() throws Exception {
+		final int numLongs = 15_000_000;
+
+		// setup
+		final BoundedBlockingSubpartition subpartition = createAndFillPartition(numLongs);
+
+		// test
+		final LongReader[] readerThreads = createSubpartitionLongReaders(
+				subpartition, 10, numLongs, subpartition.getBuffersInBacklog());
+		for (CheckedThread t : readerThreads) {
+			t.start();
+		}
+
+		// check
+		for (CheckedThread t : readerThreads) {
+			t.sync(); // this propagates assertion errors out from the threads
+		}
+
+		// cleanup
+		subpartition.release();
+	}
+
+	// ------------------------------------------------------------------------
+	//  common test passes
+	// ------------------------------------------------------------------------
+
+	private static void readLongs(ResultSubpartitionView reader, long numLongs, int numBuffers) throws Exception {
+		BufferAndBacklog next;
+		long expectedNextLong = 0L;
+		int nextExpectedBacklog = numBuffers - 1;
+
+		while ((next = reader.getNextBuffer()) != null && next.buffer().isBuffer()) {
+			assertTrue(next.isMoreAvailable());
+			assertEquals(nextExpectedBacklog, next.buffersInBacklog());
+
+			ByteBuffer buffer = next.buffer().getNioBufferReadable();
+			while (buffer.hasRemaining()) {
+				assertEquals(expectedNextLong++, buffer.getLong());
+			}
+
+			nextExpectedBacklog--;
+		}
+
+		assertEquals(numLongs, expectedNextLong);
+		assertEquals(-1, nextExpectedBacklog);
+	}
+
+	// ------------------------------------------------------------------------
+	//  utils
+	// ------------------------------------------------------------------------
+
+	private static void writeLongs(BoundedBlockingSubpartition partition, long nums) throws IOException {
+		final MemorySegment memory = MemorySegmentFactory.allocateUnpooledSegment(1024 * 1024);
+
+		long l = 0;
+		while (nums > 0) {
+			int pos = 0;
+			for (; nums > 0 && pos <= memory.size() - 8; pos += 8) {
+				memory.putLongBigEndian(pos, l++);
+				nums--;
+			}
+
+			partition.add(new BufferConsumer(memory, (ignored) -> {}, pos, true));
+
+			// we need to flush after every buffer as long as the add() contract is that
+			// buffer are immediately added and can be filled further after that (for low latency
+			// streaming data exchanges)
+			partition.flush();
+		}
+	}
+
+	private static BoundedBlockingSubpartition createAndFillPartition(long numLongs) throws IOException {
+		BoundedBlockingSubpartition subpartition = createSubpartition();
+		writeLongs(subpartition, numLongs);
+		subpartition.finish();
+		return subpartition;
+	}
+
+	private static BoundedBlockingSubpartition createSubpartition() throws IOException {
+		return new BoundedBlockingSubpartition(
+				0,
+				PartitionTestUtils.createPartition(ResultPartitionType.BLOCKING),
+				new File(TMP_FOLDER.newFolder(), "partitiondata").toPath());
+	}
+
+	private static LongReader[] createSubpartitionLongReaders(
+			BoundedBlockingSubpartition subpartition,
+			int numReaders,
+			int numLongs,
+			int numBuffers) throws IOException {
+
+		final LongReader[] readerThreads = new LongReader[numReaders];
+		for (int i = 0; i < numReaders; i++) {
+			ResultSubpartitionView reader = subpartition.createReadView(() -> {});
+			readerThreads[i] = new LongReader(reader, numLongs, numBuffers);
+		}
+		return readerThreads;
+	}
+
+	private static final class LongReader extends CheckedThread {
+
+		private final ResultSubpartitionView reader;
+
+		private final long numLongs;
+
+		private final int numBuffers;
+
+		LongReader(ResultSubpartitionView reader, long numLongs, int numBuffers) {
+			this.reader = reader;
+			this.numLongs = numLongs;
+			this.numBuffers = numBuffers;
+		}
+
+		@Override
+		public void go() throws Exception {
+			readLongs(reader, numLongs, numBuffers);
+		}
+	}
+}
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BufferToByteBufferTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BufferToByteBufferTest.java
new file mode 100644
index 0000000..55fa496
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BufferToByteBufferTest.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition;
+
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils;
+import org.apache.flink.runtime.io.network.partition.BufferToByteBuffer.Reader;
+import org.apache.flink.runtime.io.network.partition.BufferToByteBuffer.Writer;
+
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertSame;
+
+/**
+ * Tests for the {@link BufferToByteBuffer}.
+ */
+public class BufferToByteBufferTest {
+
+	@Test
+	public void testCompleteIsSameBufferAsOriginal() {
+		final ByteBuffer bb = ByteBuffer.allocateDirect(128);
+		final BufferToByteBuffer.Writer writer = new BufferToByteBuffer.Writer(bb);
+
+		final ByteBuffer result = writer.complete();
+
+		assertSame(bb, result);
+	}
+
+	@Test
+	public void testWriteReadMatchesCapacity() {
+		final ByteBuffer bb = ByteBuffer.allocateDirect(1200);
+		testWriteAndReadMultipleBuffers(bb, 100);
+	}
+
+	@Test
+	public void testWriteReadWithLeftoverCapacity() {
+		final ByteBuffer bb = ByteBuffer.allocateDirect(1177);
+		testWriteAndReadMultipleBuffers(bb, 100);
+	}
+
+	private void testWriteAndReadMultipleBuffers(ByteBuffer buffer, int numIntsPerBuffer) {
+		final Writer writer = new Writer(buffer);
+
+		int numBuffers = 0;
+		while (writer.writeBuffer(BufferBuilderTestUtils.buildBufferWithAscendingInts(1024, numIntsPerBuffer, 0))) {
+			numBuffers++;
+		}
+
+		final ByteBuffer bb = writer.complete().slice();
+
+		final Reader reader = new Reader(bb);
+		Buffer buf;
+		while ((buf = reader.sliceNextBuffer()) != null) {
+			BufferBuilderTestUtils.validateBufferWithAscendingInts(buf, numIntsPerBuffer, 0);
+			numBuffers--;
+		}
+
+		assertEquals(0, numBuffers);
+	}
+
+}
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MemoryMappedBuffersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MemoryMappedBuffersTest.java
new file mode 100644
index 0000000..eec7dba
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MemoryMappedBuffersTest.java
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition;
+
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils;
+import org.apache.flink.runtime.io.network.partition.MemoryMappedBuffers.BufferSlicer;
+
+import org.hamcrest.Matchers;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests that read the BoundedBlockingSubpartition with multiple threads in parallel.
+ */
+public class MemoryMappedBuffersTest {
+
+	@ClassRule
+	public static final TemporaryFolder TMP_FOLDER = new TemporaryFolder();
+
+	@Test
+	public void testWriteAndReadData() throws Exception {
+		testWriteAndReadData(10_000_000, Integer.MAX_VALUE);
+	}
+
+	@Test
+	public void testWriteAndReadDataAcrossRegions() throws Exception {
+		testWriteAndReadData(10_000_000, 1_276_347);
+	}
+
+	private static void testWriteAndReadData(int numInts, int regionSize) throws Exception {
+		try (MemoryMappedBuffers memory = MemoryMappedBuffers.createWithRegionSize(createTempPath(), regionSize)) {
+			final int numBuffers = writeInts(memory, numInts);
+			memory.finishWrite();
+
+			readInts(memory.getFullBuffers(), numBuffers, numInts);
+		}
+	}
+
+	@Test
+	public void returnNullAfterEmpty() throws Exception {
+		try (MemoryMappedBuffers memory = MemoryMappedBuffers.create(createTempPath())) {
+			memory.writeBuffer(BufferBuilderTestUtils.buildSomeBuffer());
+			memory.finishWrite();
+
+			final BufferSlicer reader = memory.getFullBuffers();
+			assertNotNull(reader.sliceNextBuffer());
+
+			// check that multiple calls now return empty buffers
+			assertNull(reader.sliceNextBuffer());
+			assertNull(reader.sliceNextBuffer());
+			assertNull(reader.sliceNextBuffer());
+		}
+	}
+
+	@Test
+	public void testDeleteFileOnClose() throws Exception {
+		final Path path = createTempPath();
+		final MemoryMappedBuffers mmb = MemoryMappedBuffers.create(path);
+		assertTrue(Files.exists(path));
+
+		mmb.close();
+
+		assertFalse(Files.exists(path));
+	}
+
+	@Test
+	public void testGetSizeSingleRegion() throws Exception {
+		testGetSize(Integer.MAX_VALUE);
+	}
+
+	@Test
+	public void testGetSizeMultipleRegions() throws Exception {
+		testGetSize(100_000);
+	}
+
+	private static void testGetSize(int regionSize) throws Exception {
+		final int bufferSize1 = 60_787;
+		final int bufferSize2 = 76_687;
+		final int expectedSize1 = bufferSize1 + BufferToByteBuffer.HEADER_LENGTH;
+		final int expectedSizeFinal = bufferSize1 + bufferSize2 + 2 * BufferToByteBuffer.HEADER_LENGTH;
+
+		try (MemoryMappedBuffers memory = MemoryMappedBuffers.createWithRegionSize(createTempPath(), regionSize)) {
+
+			memory.writeBuffer(BufferBuilderTestUtils.buildSomeBuffer(bufferSize1));
+			assertEquals(expectedSize1, memory.getSize());
+
+			memory.writeBuffer(BufferBuilderTestUtils.buildSomeBuffer(bufferSize2));
+			assertEquals(expectedSizeFinal, memory.getSize());
+
+			memory.finishWrite();
+			assertEquals(expectedSizeFinal, memory.getSize());
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  utils
+	// ------------------------------------------------------------------------
+
+	private static int writeInts(MemoryMappedBuffers memory, int numInts) throws IOException {
+		final int bufferSize = 1024 * 1024; // 1 MiByte
+		final int numIntsInBuffer = bufferSize / 4;
+		int numBuffers = 0;
+
+		for (int nextValue = 0; nextValue < numInts; nextValue += numIntsInBuffer) {
+			Buffer buffer = BufferBuilderTestUtils.buildBufferWithAscendingInts(bufferSize, numIntsInBuffer, nextValue);
+			memory.writeBuffer(buffer);
+			numBuffers++;
+		}
+
+		return numBuffers;
+	}
+
+	private static void readInts(MemoryMappedBuffers.BufferSlicer memory, int numBuffersExpected, int numInts) throws IOException {
+		Buffer b;
+		int nextValue = 0;
+		int numBuffers = 0;
+
+		while ((b = memory.sliceNextBuffer()) != null) {
+			final int numIntsInBuffer = b.getSize() / 4;
+			BufferBuilderTestUtils.validateBufferWithAscendingInts(b, numIntsInBuffer, nextValue);
+			nextValue += numIntsInBuffer;
+			numBuffers++;
+		}
+
+		assertEquals(numBuffersExpected, numBuffers);
+		assertThat(nextValue, Matchers.greaterThanOrEqualTo(numInts));
+	}
+
+	private static Path createTempPath() throws IOException {
+		return new File(TMP_FOLDER.newFolder(), "subpartitiondata").toPath();
+	}
+}
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java
index 36cd353..ff15b42 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java
@@ -24,6 +24,7 @@ import org.apache.flink.runtime.concurrent.FutureUtils;
 import org.apache.flink.runtime.event.AbstractEvent;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
 import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils;
 import org.apache.flink.runtime.io.network.buffer.BufferConsumer;
 import org.apache.flink.runtime.io.network.buffer.BufferProvider;
 import org.apache.flink.runtime.io.network.util.TestConsumerCallback;
@@ -35,6 +36,7 @@ import org.apache.flink.util.function.CheckedSupplier;
 
 import org.junit.AfterClass;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Test;
 
 import java.nio.ByteBuffer;
@@ -78,6 +80,13 @@ public class PipelinedSubpartitionTest extends SubpartitionTestBase {
 		return new PipelinedSubpartition(0, parent);
 	}
 
+	@Override
+	ResultSubpartition createFailingWritesSubpartition() throws Exception {
+		// the tests relating to this are currently not supported by the PipelinedSubpartition
+		Assume.assumeTrue(false);
+		return null;
+	}
+
 	@Test
 	public void testIllegalReadViewRequest() throws Exception {
 		final PipelinedSubpartition subpartition = createSubpartition();
@@ -280,4 +289,40 @@ public class PipelinedSubpartitionTest extends SubpartitionTestBase {
 		assertEquals(2, partition.getTotalNumberOfBuffers());
 		assertEquals(0, partition.getTotalNumberOfBytes()); // buffer data is never consumed
 	}
+
+	@Test
+	public void testReleaseParent() throws Exception {
+		final ResultSubpartition partition = createSubpartition();
+		verifyViewReleasedAfterParentRelease(partition);
+	}
+
+	@Test
+	public void testReleaseParentAfterSpilled() throws Exception {
+		final ResultSubpartition partition = createSubpartition();
+		partition.releaseMemory();
+
+		verifyViewReleasedAfterParentRelease(partition);
+	}
+
+	private void verifyViewReleasedAfterParentRelease(ResultSubpartition partition) throws Exception {
+		// Add a bufferConsumer
+		BufferConsumer bufferConsumer = createFilledBufferConsumer(BufferBuilderTestUtils.BUFFER_SIZE);
+		partition.add(bufferConsumer);
+		partition.finish();
+
+		// Create the view
+		BufferAvailabilityListener listener = mock(BufferAvailabilityListener.class);
+		ResultSubpartitionView view = partition.createReadView(listener);
+
+		// The added bufferConsumer and end-of-partition event
+		assertNotNull(view.getNextBuffer());
+		assertNotNull(view.getNextBuffer());
+
+		// Release the parent
+		assertFalse(view.isReleased());
+		partition.release();
+
+		// Verify that parent release is reflected at partition view
+		assertTrue(view.isReleased());
+	}
 }
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java
index c911df7..5846d6f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java
@@ -202,11 +202,6 @@ public class ResultPartitionTest {
 	}
 
 	@Test
-	public void testReleaseMemoryOnBlockingPartition() throws Exception {
-		testReleaseMemory(ResultPartitionType.BLOCKING);
-	}
-
-	@Test
 	public void testReleaseMemoryOnPipelinedPartition() throws Exception {
 		testReleaseMemory(ResultPartitionType.PIPELINED);
 	}
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java
deleted file mode 100644
index 71dbc2b..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java
+++ /dev/null
@@ -1,800 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.io.network.partition;
-
-import org.apache.flink.runtime.io.disk.iomanager.AsynchronousBufferFileWriter;
-import org.apache.flink.runtime.io.disk.iomanager.BufferFileWriter;
-import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
-import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsyncWithNoOpBufferFileWriter;
-import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker;
-import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
-import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
-import org.apache.flink.runtime.io.network.buffer.BufferConsumer;
-import org.apache.flink.runtime.io.network.buffer.BufferProvider;
-
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.concurrent.Callable;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-
-import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createBufferBuilder;
-import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledBufferConsumer;
-import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.fillBufferBuilder;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.ArgumentMatchers.nullable;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.doNothing;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.when;
-
-/**
- * Tests for {@link SpillableSubpartition}.
- */
-public class SpillableSubpartitionTest extends SubpartitionTestBase {
-	private static final int BUFFER_DATA_SIZE = 4096;
-
-	@Rule
-	public ExpectedException exception = ExpectedException.none();
-
-	/** Executor service for concurrent produce/consume tests. */
-	private static final ExecutorService executorService = Executors.newCachedThreadPool();
-
-	/** Asynchronous I/O manager. */
-	private static IOManager ioManager;
-
-	@BeforeClass
-	public static void setup() {
-		ioManager = new IOManagerAsync();
-	}
-
-	@AfterClass
-	public static void shutdown() {
-		executorService.shutdownNow();
-		ioManager.shutdown();
-	}
-
-	@Override
-	SpillableSubpartition createSubpartition() {
-		return createSubpartition(ioManager);
-	}
-
-	private static SpillableSubpartition createSubpartition(IOManager ioManager) {
-		ResultPartition parent = mock(ResultPartition.class);
-		BufferProvider bufferProvider = mock(BufferProvider.class);
-		when(parent.getBufferProvider()).thenReturn(bufferProvider);
-		when(bufferProvider.getMemorySegmentSize()).thenReturn(32 * 1024);
-		return new SpillableSubpartition(0, parent, ioManager);
-	}
-
-	/**
-	 * Tests a fix for FLINK-2384.
-	 *
-	 * @see <a href="https://issues.apache.org/jira/browse/FLINK-2384">FLINK-2384</a>
-	 */
-	@Test
-	public void testConcurrentFinishAndReleaseMemory() throws Exception {
-		// Latches to blocking
-		final CountDownLatch doneLatch = new CountDownLatch(1);
-		final CountDownLatch blockLatch = new CountDownLatch(1);
-
-		// Blocking spill writer (blocks on the close call)
-		AsynchronousBufferFileWriter spillWriter = mock(AsynchronousBufferFileWriter.class);
-		doAnswer(new Answer<Void>() {
-			@Override
-			public Void answer(InvocationOnMock invocation) throws Throwable {
-				blockLatch.countDown();
-				doneLatch.await();
-				return null;
-			}
-		}).when(spillWriter).close();
-
-		// Mock I/O manager returning the blocking spill writer
-		IOManager ioManager = mock(IOManager.class);
-		when(ioManager.createBufferFileWriter(nullable(FileIOChannel.ID.class)))
-			.thenReturn(spillWriter);
-
-		// The partition
-		final SpillableSubpartition partition = new SpillableSubpartition(
-			0, mock(ResultPartition.class), ioManager);
-
-		// Spill the partition initially (creates the spill writer)
-		assertEquals(0, partition.releaseMemory());
-
-		ExecutorService executor = Executors.newSingleThreadExecutor();
-
-		// Finish the partition (this blocks because of the mock blocking writer)
-		Future<Void> blockingFinish = executor.submit(new Callable<Void>() {
-			@Override
-			public Void call() throws Exception {
-				partition.finish();
-				return null;
-			}
-		});
-
-		// Ensure that the blocking call has been made
-		blockLatch.await();
-
-		// This call needs to go through. FLINK-2384 discovered a bug, in
-		// which the finish call was holding a lock, which was leading to a
-		// deadlock when another operation on the partition was happening.
-		partition.releaseMemory();
-
-		// Check that the finish call succeeded w/o problems as well to avoid
-		// false test successes.
-		doneLatch.countDown();
-		blockingFinish.get();
-	}
-
-	/**
-	 * Tests a fix for FLINK-2412.
-	 *
-	 * @see <a href="https://issues.apache.org/jira/browse/FLINK-2412">FLINK-2412</a>
-	 */
-	@Test
-	public void testReleasePartitionAndGetNext() throws Exception {
-		// Create partition and add some buffers
-		SpillableSubpartition partition = createSubpartition();
-
-		partition.finish();
-
-		// Create the read view
-		ResultSubpartitionView readView = spy(partition
-			.createReadView(new NoOpBufferAvailablityListener()));
-
-		// The released state check (of the parent) needs to be independent
-		// of the released state of the view.
-		doNothing().when(readView).releaseAllResources();
-
-		// Release the partition, but the view does not notice yet.
-		partition.release();
-
-		assertNull(readView.getNextBuffer());
-	}
-
-	/**
-	 * Tests that a spilled partition is correctly read back in via a spilled
-	 * read view.
-	 */
-	@Test
-	public void testConsumeSpilledPartition() throws Exception {
-		SpillableSubpartition partition = createSubpartition();
-
-		BufferConsumer bufferConsumer = createFilledBufferConsumer(BUFFER_DATA_SIZE, BUFFER_DATA_SIZE);
-		BufferConsumer eventBufferConsumer =
-			EventSerializer.toBufferConsumer(new CancelCheckpointMarker(1));
-		final int eventSize = eventBufferConsumer.getWrittenBytes();
-
-		partition.add(bufferConsumer.copy());
-		partition.add(bufferConsumer.copy());
-		partition.add(eventBufferConsumer);
-		partition.add(bufferConsumer);
-
-		assertEquals(4, partition.getTotalNumberOfBuffers());
-		assertEquals(3, partition.getBuffersInBacklog());
-		assertEquals(0, partition.getTotalNumberOfBytes()); // only updated when getting/releasing the buffers
-
-		assertFalse(bufferConsumer.isRecycled());
-		assertEquals(4, partition.releaseMemory());
-		// now the bufferConsumer may be freed, depending on the timing of the write operation
-		// -> let's do this check at the end of the test (to save some time)
-		// still same statistics
-		assertEquals(4, partition.getTotalNumberOfBuffers());
-		assertEquals(3, partition.getBuffersInBacklog());
-		assertEquals(BUFFER_DATA_SIZE * 3 + eventSize, partition.getTotalNumberOfBytes());
-
-		partition.finish();
-		// + one EndOfPartitionEvent
-		assertEquals(5, partition.getTotalNumberOfBuffers());
-		assertEquals(3, partition.getBuffersInBacklog());
-		assertEquals(BUFFER_DATA_SIZE * 3 + eventSize + 4, partition.getTotalNumberOfBytes());
-
-		AwaitableBufferAvailablityListener listener = new AwaitableBufferAvailablityListener();
-		SpilledSubpartitionView reader = (SpilledSubpartitionView) partition.createReadView(listener);
-
-		assertEquals(1, listener.getNumNotifications());
-		assertFalse(reader.nextBufferIsEvent()); // buffer
-
-		assertNextBuffer(reader, BUFFER_DATA_SIZE, true, 2, false, true);
-		assertEquals(2, partition.getBuffersInBacklog());
-
-		assertNextBuffer(reader, BUFFER_DATA_SIZE, true, 1, true, true);
-		assertEquals(1, partition.getBuffersInBacklog());
-
-		assertNextEvent(reader, eventSize, CancelCheckpointMarker.class, true, 1, false, true);
-		assertEquals(1, partition.getBuffersInBacklog());
-
-		assertNextBuffer(reader, BUFFER_DATA_SIZE, true, 0, true, true);
-		assertEquals(0, partition.getBuffersInBacklog());
-
-		assertNextEvent(reader, 4, EndOfPartitionEvent.class, false, 0, false, true);
-		assertEquals(0, partition.getBuffersInBacklog());
-
-		// finally check that the bufferConsumer has been freed after a successful (or failed) write
-		final long deadline = System.currentTimeMillis() + 30_000L; // 30 secs
-		while (!bufferConsumer.isRecycled() && System.currentTimeMillis() < deadline) {
-			Thread.sleep(1);
-		}
-		assertTrue(bufferConsumer.isRecycled());
-	}
-
-	/**
-	 * Tests that a spilled partition is correctly read back in via a spilled read view. The
-	 * partition went into spilled state before adding buffers and the access pattern resembles
-	 * the actual use of {@link org.apache.flink.runtime.io.network.api.writer.RecordWriter}.
-	 */
-	@Test
-	public void testConsumeSpilledPartitionSpilledBeforeAdd() throws Exception {
-		SpillableSubpartition partition = createSubpartition();
-		assertEquals(0, partition.releaseMemory()); // <---- SPILL to disk
-
-		BufferBuilder[] bufferBuilders = new BufferBuilder[] {
-			createBufferBuilder(BUFFER_DATA_SIZE),
-			createBufferBuilder(BUFFER_DATA_SIZE),
-			createBufferBuilder(BUFFER_DATA_SIZE),
-			createBufferBuilder(BUFFER_DATA_SIZE)
-		};
-		BufferConsumer[] bufferConsumers = Arrays.stream(bufferBuilders).map(
-			BufferBuilder::createBufferConsumer
-		).toArray(BufferConsumer[]::new);
-
-		BufferConsumer eventBufferConsumer =
-			EventSerializer.toBufferConsumer(new CancelCheckpointMarker(1));
-		final int eventSize = eventBufferConsumer.getWrittenBytes();
-
-		// note: only the newest buffer may be unfinished!
-		partition.add(bufferConsumers[0]);
-		fillBufferBuilder(bufferBuilders[0], BUFFER_DATA_SIZE).finish();
-		partition.add(bufferConsumers[1]);
-		fillBufferBuilder(bufferBuilders[1], BUFFER_DATA_SIZE).finish();
-		partition.add(eventBufferConsumer);
-		partition.add(bufferConsumers[2]);
-		bufferBuilders[2].finish(); // remains empty
-		partition.add(bufferConsumers[3]);
-		// last one: partially filled, unfinished
-		fillBufferBuilder(bufferBuilders[3], BUFFER_DATA_SIZE / 2);
-		// finished buffers only:
-		int expectedSize = BUFFER_DATA_SIZE * 2 + eventSize;
-
-		// now the bufferConsumer may be freed, depending on the timing of the write operation
-		// -> let's do this check at the end of the test (to save some time)
-		// still same statistics
-		assertEquals(5, partition.getTotalNumberOfBuffers());
-		assertEquals(3, partition.getBuffersInBacklog());
-		assertEquals(expectedSize, partition.getTotalNumberOfBytes());
-
-		partition.finish();
-		expectedSize += BUFFER_DATA_SIZE / 2; // previously unfinished buffer
-		expectedSize += 4; // + one EndOfPartitionEvent
-		assertEquals(6, partition.getTotalNumberOfBuffers());
-		assertEquals(3, partition.getBuffersInBacklog());
-		assertEquals(expectedSize, partition.getTotalNumberOfBytes());
-		Arrays.stream(bufferConsumers).forEach(bufferConsumer -> assertTrue(bufferConsumer.isRecycled()));
-
-		AwaitableBufferAvailablityListener listener = new AwaitableBufferAvailablityListener();
-		SpilledSubpartitionView reader = (SpilledSubpartitionView) partition.createReadView(listener);
-
-		assertEquals(1, listener.getNumNotifications());
-		assertFalse(reader.nextBufferIsEvent()); // full buffer
-
-		assertNextBuffer(reader, BUFFER_DATA_SIZE, true, 2, false, true);
-		assertEquals(2, partition.getBuffersInBacklog());
-
-		assertNextBuffer(reader, BUFFER_DATA_SIZE, true, 1, true, true);
-		assertEquals(1, partition.getBuffersInBacklog());
-
-		assertNextEvent(reader, eventSize, CancelCheckpointMarker.class, true, 1, false, true);
-		assertEquals(1, partition.getBuffersInBacklog());
-
-		assertNextBuffer(reader, BUFFER_DATA_SIZE / 2, true, 0, true, true);
-		assertEquals(0, partition.getBuffersInBacklog());
-
-		assertNextEvent(reader, 4, EndOfPartitionEvent.class, false, 0, false, true);
-		assertEquals(0, partition.getBuffersInBacklog());
-
-		//close buffer consumers
-		Arrays.stream(bufferConsumers).forEach(bufferConsumer -> bufferConsumer.close());
-	}
-
-	/**
-	 * Tests that a spilled partition is correctly read back in via a spilled
-	 * read view.
-	 */
-	@Test
-	public void testConsumeSpillablePartitionSpilledDuringConsume() throws Exception {
-		SpillableSubpartition partition = createSubpartition();
-
-		BufferConsumer bufferConsumer = createFilledBufferConsumer(BUFFER_DATA_SIZE, BUFFER_DATA_SIZE);
-		BufferConsumer eventBufferConsumer =
-			EventSerializer.toBufferConsumer(new CancelCheckpointMarker(1));
-		final int eventSize = eventBufferConsumer.getWrittenBytes();
-
-		partition.add(bufferConsumer.copy());
-		partition.add(bufferConsumer.copy());
-		partition.add(eventBufferConsumer);
-		partition.add(bufferConsumer);
-		partition.finish();
-
-		assertEquals(5, partition.getTotalNumberOfBuffers());
-		assertEquals(3, partition.getBuffersInBacklog());
-		assertEquals(0, partition.getTotalNumberOfBytes()); // only updated when getting/spilling the buffers
-
-		AwaitableBufferAvailablityListener listener = new AwaitableBufferAvailablityListener();
-		SpillableSubpartitionView reader = (SpillableSubpartitionView) partition.createReadView(listener);
-
-		// Initial notification
-		assertEquals(1, listener.getNumNotifications());
-		assertFalse(bufferConsumer.isRecycled());
-
-		assertFalse(reader.nextBufferIsEvent());
-
-		// first buffer (non-spilled)
-		assertNextBuffer(reader, BUFFER_DATA_SIZE, true, 2, false, false);
-		assertEquals(BUFFER_DATA_SIZE, partition.getTotalNumberOfBytes()); // only updated when getting/spilling the buffers
-		assertEquals(2, partition.getBuffersInBacklog());
-		assertEquals(1, listener.getNumNotifications()); // since isMoreAvailable is set to true, no need for notification
-		assertFalse(bufferConsumer.isRecycled());
-
-		// Spill now
-		assertEquals(3, partition.releaseMemory());
-		assertFalse(bufferConsumer.isRecycled()); // still one in the reader!
-		// still same statistics:
-		assertEquals(5, partition.getTotalNumberOfBuffers());
-		assertEquals(2, partition.getBuffersInBacklog());
-		// only updated when getting/spilling the buffers but without the nextBuffer (kept in memory)
-		assertEquals(BUFFER_DATA_SIZE * 2 + eventSize + 4, partition.getTotalNumberOfBytes());
-
-		// wait for successfully spilling all buffers (before that we may not access any spilled buffer and cannot rely on isMoreAvailable!)
-		listener.awaitNotifications(2, 30_000);
-		// Spiller finished
-		assertEquals(2, listener.getNumNotifications());
-
-		// after consuming and releasing the next buffer, the bufferConsumer may be freed,
-		// depending on the timing of the last write operation
-		// -> retain once so that we can check below
-		Buffer buffer = bufferConsumer.build();
-		buffer.retainBuffer();
-
-		// second buffer (retained in SpillableSubpartition#nextBuffer)
-		assertNextBuffer(reader, BUFFER_DATA_SIZE, true, 1, true, false);
-		assertEquals(BUFFER_DATA_SIZE * 3 + eventSize + 4, partition.getTotalNumberOfBytes()); // finally integrates the nextBuffer statistics
-		assertEquals(1, partition.getBuffersInBacklog());
-
-		bufferConsumer.close(); // recycle the retained buffer from above (should be the last reference!)
-
-		// the event (spilled)
-		assertNextEvent(reader, eventSize, CancelCheckpointMarker.class, true, 1, false, true);
-		assertEquals(BUFFER_DATA_SIZE * 3 + eventSize + 4, partition.getTotalNumberOfBytes()); // already updated during spilling
-		assertEquals(1, partition.getBuffersInBacklog());
-
-		// last buffer (spilled)
-		assertNextBuffer(reader, BUFFER_DATA_SIZE, true, 0, true, true);
-		assertEquals(BUFFER_DATA_SIZE * 3 + eventSize + 4, partition.getTotalNumberOfBytes()); // already updated during spilling
-		assertEquals(0, partition.getBuffersInBacklog());
-
-		buffer.recycleBuffer();
-		assertTrue(buffer.isRecycled());
-
-		// End of partition
-		assertNextEvent(reader, 4, EndOfPartitionEvent.class, false, 0, false, true);
-		assertEquals(BUFFER_DATA_SIZE * 3 + eventSize + 4, partition.getTotalNumberOfBytes()); // already updated during spilling
-		assertEquals(0, partition.getBuffersInBacklog());
-
-		// finally check that the bufferConsumer has been freed after a successful (or failed) write
-		final long deadline = System.currentTimeMillis() + 30_000L; // 30 secs
-		while (!bufferConsumer.isRecycled() && System.currentTimeMillis() < deadline) {
-			Thread.sleep(1);
-		}
-		assertTrue(bufferConsumer.isRecycled());
-	}
-
-	/**
-	 * Tests {@link SpillableSubpartition#add(BufferConsumer)} with a spillable finished partition.
-	 */
-	@Test
-	public void testAddOnFinishedSpillablePartition() throws Exception {
-		testAddOnFinishedPartition(false);
-	}
-
-	/**
-	 * Tests {@link SpillableSubpartition#add(BufferConsumer)} with a spilled finished partition.
-	 */
-	@Test
-	public void testAddOnFinishedSpilledPartition() throws Exception {
-		testAddOnFinishedPartition(true);
-	}
-
-	/**
-	 * Tests {@link SpillableSubpartition#add(BufferConsumer)} with a finished partition.
-	 *
-	 * @param spilled
-	 * 		whether the partition should be spilled to disk (<tt>true</tt>) or not (<tt>false</tt>,
-	 * 		spillable).
-	 */
-	private void testAddOnFinishedPartition(boolean spilled) throws Exception {
-		SpillableSubpartition partition = createSubpartition();
-		if (spilled) {
-			assertEquals(0, partition.releaseMemory());
-		}
-		partition.finish();
-		// finish adds an EndOfPartitionEvent
-		assertEquals(1, partition.getTotalNumberOfBuffers());
-		// if not spilled, statistics are only updated when consuming the buffers
-		assertEquals(spilled ? 4 : 0, partition.getTotalNumberOfBytes());
-
-		BufferConsumer buffer = createFilledBufferConsumer(BUFFER_DATA_SIZE, BUFFER_DATA_SIZE);
-		try {
-			partition.add(buffer);
-		} finally {
-			if (!buffer.isRecycled()) {
-				buffer.close();
-				Assert.fail("buffer not recycled");
-			}
-		}
-		// still same statistics
-		assertEquals(1, partition.getTotalNumberOfBuffers());
-		// if not spilled, statistics are only updated when consuming the buffers
-		assertEquals(spilled ? 4 : 0, partition.getTotalNumberOfBytes());
-	}
-
-	@Test
-	public void testAddOnReleasedSpillablePartition() throws Exception {
-		testAddOnReleasedPartition(false);
-	}
-
-	@Test
-	public void testAddOnReleasedSpilledPartition() throws Exception {
-		testAddOnReleasedPartition(true);
-	}
-
-	/**
-	 * Tests {@link SpillableSubpartition#add(BufferConsumer)} with a released partition.
-	 *
-	 * @param spilled
-	 * 		whether the partition should be spilled to disk (<tt>true</tt>) or not (<tt>false</tt>,
-	 * 		spillable).
-	 */
-	private void testAddOnReleasedPartition(boolean spilled) throws Exception {
-		SpillableSubpartition partition = createSubpartition();
-		partition.release();
-		if (spilled) {
-			assertEquals(0, partition.releaseMemory());
-		}
-
-		BufferConsumer buffer = createFilledBufferConsumer(BUFFER_DATA_SIZE, BUFFER_DATA_SIZE);
-		boolean bufferRecycled;
-		try {
-			partition.add(buffer);
-		} finally {
-			bufferRecycled = buffer.isRecycled();
-			if (!bufferRecycled) {
-				buffer.close();
-			}
-		}
-		if (!bufferRecycled) {
-			Assert.fail("buffer not recycled");
-		}
-		assertEquals(0, partition.getTotalNumberOfBuffers());
-		assertEquals(0, partition.getTotalNumberOfBytes());
-	}
-
-	/**
-	 * Tests {@link SpillableSubpartition#add(BufferConsumer)} with a spilled partition where adding the
-	 * write request fails with an exception.
-	 */
-	@Test
-	public void testAddOnSpilledPartitionWithSlowWriter() throws Exception {
-		// simulate slow writer by a no-op write operation
-		IOManager ioManager = new IOManagerAsyncWithNoOpBufferFileWriter();
-		SpillableSubpartition partition = createSubpartition(ioManager);
-		assertEquals(0, partition.releaseMemory());
-
-		BufferConsumer buffer = createFilledBufferConsumer(BUFFER_DATA_SIZE, BUFFER_DATA_SIZE);
-		boolean bufferRecycled;
-		try {
-			partition.add(buffer);
-		} finally {
-			ioManager.shutdown();
-			bufferRecycled = buffer.isRecycled();
-			if (!bufferRecycled) {
-				buffer.close();
-			}
-		}
-		if (bufferRecycled) {
-			Assert.fail("buffer recycled before the write operation completed");
-		}
-		assertEquals(1, partition.getTotalNumberOfBuffers());
-		assertEquals(BUFFER_DATA_SIZE, partition.getTotalNumberOfBytes());
-	}
-
-	/**
-	 * Tests {@link SpillableSubpartition#releaseMemory()} with a spillable partition without a view
-	 * but with a writer that does not do any write to check for correct buffer recycling.
-	 */
-	@Test
-	public void testReleaseOnSpillablePartitionWithoutViewWithSlowWriter() throws Exception {
-		testReleaseOnSpillablePartitionWithSlowWriter(false);
-	}
-
-	/**
-	 * Tests {@link SpillableSubpartition#releaseMemory()} with a spillable partition which has a
-	 * view associated with it and a writer that does not do any write to check for correct buffer
-	 * recycling.
-	 */
-	@Test
-	public void testReleaseOnSpillablePartitionWithViewWithSlowWriter() throws Exception {
-		testReleaseOnSpillablePartitionWithSlowWriter(true);
-	}
-
-	/**
-	 * Tests {@link SpillableSubpartition#releaseMemory()} with a spillable partition which has a a
-	 * writer that does not do any write to check for correct buffer recycling.
-	 */
-	private void testReleaseOnSpillablePartitionWithSlowWriter(boolean createView) throws Exception {
-		// simulate slow writer by a no-op write operation
-		IOManager ioManager = new IOManagerAsyncWithNoOpBufferFileWriter();
-		SpillableSubpartition partition = createSubpartition(ioManager);
-
-		BufferConsumer buffer1 = createFilledBufferConsumer(BUFFER_DATA_SIZE, BUFFER_DATA_SIZE);
-		BufferConsumer buffer2 = createFilledBufferConsumer(BUFFER_DATA_SIZE, BUFFER_DATA_SIZE);
-		try {
-			// we need two buffers because the view will use one of them and not release it
-			partition.add(buffer1);
-			partition.add(buffer2);
-			assertFalse("buffer1 should not be recycled (still in the queue)", buffer1.isRecycled());
-			assertFalse("buffer2 should not be recycled (still in the queue)", buffer2.isRecycled());
-			assertEquals(2, partition.getTotalNumberOfBuffers());
-			assertEquals(0, partition.getTotalNumberOfBytes()); // only updated when buffers are consumed or spilled
-
-			if (createView) {
-				// Create a read view
-				partition.finish();
-				partition.createReadView(new NoOpBufferAvailablityListener());
-				assertEquals(0, partition.getTotalNumberOfBytes()); // only updated when buffers are consumed or spilled
-			}
-
-			// one instance of the buffers is placed in the view's nextBuffer and not released
-			// (if there is no view, there will be no additional EndOfPartitionEvent)
-			assertEquals(2, partition.releaseMemory());
-			assertFalse("buffer1 should not be recycled (advertised as nextBuffer)", buffer1.isRecycled());
-			assertFalse("buffer2 should not be recycled (not written yet)", buffer2.isRecycled());
-		} finally {
-			ioManager.shutdown();
-			if (!buffer1.isRecycled()) {
-				buffer1.close();
-			}
-			if (!buffer2.isRecycled()) {
-				buffer2.close();
-			}
-		}
-		// note: a view requires a finished partition which has an additional EndOfPartitionEvent
-		assertEquals(2 + (createView ? 1 : 0), partition.getTotalNumberOfBuffers());
-		// with a view, one buffer remains in nextBuffer and is not counted yet
-		assertEquals(BUFFER_DATA_SIZE + (createView ? 4 : BUFFER_DATA_SIZE), partition.getTotalNumberOfBytes());
-	}
-
-	/**
-	 * Tests {@link SpillableSubpartition#add(BufferConsumer)} with a spilled partition where adding the
-	 * write request fails with an exception.
-	 */
-	@Test
-	public void testAddOnSpilledPartitionWithFailingWriter() throws Exception {
-		IOManager ioManager = new IOManagerAsyncWithClosedBufferFileWriter();
-		SpillableSubpartition partition = createSubpartition(ioManager);
-		assertEquals(0, partition.releaseMemory());
-
-		exception.expect(IOException.class);
-
-		BufferConsumer buffer = createFilledBufferConsumer(BUFFER_DATA_SIZE, BUFFER_DATA_SIZE);
-		boolean bufferRecycled;
-		try {
-			partition.add(buffer);
-		} finally {
-			ioManager.shutdown();
-			bufferRecycled = buffer.isRecycled();
-			if (!bufferRecycled) {
-				buffer.close();
-			}
-		}
-		if (!bufferRecycled) {
-			Assert.fail("buffer not recycled");
-		}
-		assertEquals(0, partition.getTotalNumberOfBuffers());
-		assertEquals(0, partition.getTotalNumberOfBytes());
-	}
-
-	/**
-	 * Tests cleanup of {@link SpillableSubpartition#release()} with a spillable partition and no
-	 * read view attached.
-	 */
-	@Test
-	public void testCleanupReleasedSpillablePartitionNoView() throws Exception {
-		testCleanupReleasedPartition(false, false);
-	}
-
-	/**
-	 * Tests cleanup of {@link SpillableSubpartition#release()} with a spillable partition and a
-	 * read view attached - [FLINK-8371].
-	 */
-	@Test
-	public void testCleanupReleasedSpillablePartitionWithView() throws Exception {
-		testCleanupReleasedPartition(false, true);
-	}
-
-	/**
-	 * Tests cleanup of {@link SpillableSubpartition#release()} with a spilled partition and no
-	 * read view attached.
-	 */
-	@Test
-	public void testCleanupReleasedSpilledPartitionNoView() throws Exception {
-		testCleanupReleasedPartition(true, false);
-	}
-
-	/**
-	 * Tests cleanup of {@link SpillableSubpartition#release()} with a spilled partition and a
-	 * read view attached.
-	 */
-	@Test
-	public void testCleanupReleasedSpilledPartitionWithView() throws Exception {
-		testCleanupReleasedPartition(true, true);
-	}
-
-	/**
-	 * Tests cleanup of {@link SpillableSubpartition#release()}.
-	 *
-	 * @param spilled
-	 * 		whether the partition should be spilled to disk (<tt>true</tt>) or not (<tt>false</tt>,
-	 * 		spillable)
-	 * @param createView
-	 * 		whether the partition should have a view attached to it (<tt>true</tt>) or not (<tt>false</tt>)
-	 */
-	private void testCleanupReleasedPartition(boolean spilled, boolean createView) throws Exception {
-		SpillableSubpartition partition = createSubpartition();
-
-		BufferConsumer buffer1 = createFilledBufferConsumer(BUFFER_DATA_SIZE, BUFFER_DATA_SIZE);
-		BufferConsumer buffer2 = createFilledBufferConsumer(BUFFER_DATA_SIZE, BUFFER_DATA_SIZE);
-		boolean buffer1Recycled;
-		boolean buffer2Recycled;
-		try {
-			partition.add(buffer1);
-			partition.add(buffer2);
-			// create the read view before spilling
-			// (tests both code paths since this view may then contain the spilled view)
-			ResultSubpartitionView view = null;
-			if (createView) {
-				partition.finish();
-				view = partition.createReadView(new NoOpBufferAvailablityListener());
-			}
-			if (spilled) {
-				// note: in case we create a view, one buffer will already reside in the view and
-				//       one EndOfPartitionEvent will be added instead (so overall the number of
-				//       buffers to spill is the same
-				assertEquals(2, partition.releaseMemory());
-			}
-
-			partition.release();
-
-			assertTrue(partition.isReleased());
-			if (createView) {
-				assertTrue(view.isReleased());
-			}
-			assertTrue(buffer1.isRecycled());
-		} finally {
-			buffer1Recycled = buffer1.isRecycled();
-			if (!buffer1Recycled) {
-				buffer1.close();
-			}
-			buffer2Recycled = buffer2.isRecycled();
-			if (!buffer2Recycled) {
-				buffer2.close();
-			}
-		}
-		if (!buffer1Recycled) {
-			Assert.fail("buffer 1 not recycled");
-		}
-		if (!buffer2Recycled) {
-			Assert.fail("buffer 2 not recycled");
-		}
-		// note: in case we create a view, there will be an additional EndOfPartitionEvent
-		assertEquals(createView ? 3 : 2, partition.getTotalNumberOfBuffers());
-		if (spilled) {
-			// with a view, one buffer remains in nextBuffer and is not counted yet
-			assertEquals(BUFFER_DATA_SIZE + (createView ? 4 : BUFFER_DATA_SIZE),
-				partition.getTotalNumberOfBytes());
-		} else {
-			// non-spilled byte statistics are only updated when buffers are consumed
-			assertEquals(0, partition.getTotalNumberOfBytes());
-		}
-	}
-
-	/**
-	 * Tests {@link SpillableSubpartition#spillFinishedBufferConsumers} spilled bytes and
-	 * buffers counting.
-	 */
-	@Test
-	public void testSpillFinishedBufferConsumersFull() throws Exception {
-		SpillableSubpartition partition = createSubpartition();
-		BufferBuilder bufferBuilder = createBufferBuilder(BUFFER_DATA_SIZE);
-
-		partition.add(bufferBuilder.createBufferConsumer());
-		assertEquals(0, partition.releaseMemory());
-		assertEquals(1, partition.getBuffersInBacklog());
-		// finally fill the buffer with some bytes
-		fillBufferBuilder(bufferBuilder, BUFFER_DATA_SIZE).finish();
-		assertEquals(BUFFER_DATA_SIZE, partition.spillFinishedBufferConsumers(false));
-		assertEquals(1, partition.getBuffersInBacklog());
-	}
-
-	/**
-	 * Tests {@link SpillableSubpartition#spillFinishedBufferConsumers} spilled bytes and
-	 * buffers counting with partially filled buffers.
-	 */
-	@Test
-	public void testSpillFinishedBufferConsumersPartial() throws Exception {
-		SpillableSubpartition partition = createSubpartition();
-		BufferBuilder bufferBuilder = createBufferBuilder(BUFFER_DATA_SIZE * 2);
-
-		partition.add(bufferBuilder.createBufferConsumer());
-		fillBufferBuilder(bufferBuilder, BUFFER_DATA_SIZE);
-
-		assertEquals(0, partition.releaseMemory());
-		assertEquals(2, partition.getBuffersInBacklog()); // partial one spilled, buffer consumer still enqueued
-		// finally fill the buffer with some bytes
-		fillBufferBuilder(bufferBuilder, BUFFER_DATA_SIZE).finish();
-		assertEquals(BUFFER_DATA_SIZE, partition.spillFinishedBufferConsumers(false));
-		assertEquals(2, partition.getBuffersInBacklog());
-	}
-
-	/**
-	 * An {@link IOManagerAsync} that creates closed {@link BufferFileWriter} instances in its
-	 * {@link #createBufferFileWriter(FileIOChannel.ID)} method.
-	 *
-	 * <p>These {@link BufferFileWriter} objects will thus throw an exception when trying to add
-	 * write requests, e.g. by calling {@link BufferFileWriter#writeBlock(Object)}.
-	 */
-	private static class IOManagerAsyncWithClosedBufferFileWriter extends IOManagerAsync {
-		@Override
-		public BufferFileWriter createBufferFileWriter(FileIOChannel.ID channelID)
-				throws IOException {
-			BufferFileWriter bufferFileWriter = super.createBufferFileWriter(channelID);
-			bufferFileWriter.close();
-			return bufferFileWriter;
-		}
-	}
-
-}
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java
index 7c083ad..13cab65 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java
@@ -27,10 +27,8 @@ import org.junit.Test;
 import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledBufferConsumer;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-import static org.mockito.Mockito.mock;
 
 /**
  * Basic subpartition behaviour tests.
@@ -40,11 +38,30 @@ public abstract class SubpartitionTestBase extends TestLogger {
 	/**
 	 * Return the subpartition to be tested.
 	 */
-	abstract ResultSubpartition createSubpartition();
+	abstract ResultSubpartition createSubpartition() throws Exception;
+
+	/**
+	 * Return the subpartition to be used for tests where write calls should fail.
+	 */
+	abstract ResultSubpartition createFailingWritesSubpartition() throws Exception;
 
 	// ------------------------------------------------------------------------
 
 	@Test
+	public void createReaderAfterDispose() throws Exception {
+		final ResultSubpartition subpartition = createSubpartition();
+		subpartition.release();
+
+		try {
+			subpartition.createReadView(() -> {});
+			fail("expected an exception");
+		}
+		catch (IllegalStateException e) {
+			// expected
+		}
+	}
+
+	@Test
 	public void testAddAfterFinish() throws Exception {
 		final ResultSubpartition subpartition = createSubpartition();
 
@@ -52,14 +69,14 @@ public abstract class SubpartitionTestBase extends TestLogger {
 			subpartition.finish();
 			assertEquals(1, subpartition.getTotalNumberOfBuffers());
 			assertEquals(0, subpartition.getBuffersInBacklog());
-			assertEquals(0, subpartition.getTotalNumberOfBytes()); // only updated after consuming the buffers
 
 			BufferConsumer bufferConsumer = createFilledBufferConsumer(4096, 4096);
 
 			assertFalse(subpartition.add(bufferConsumer));
+			assertTrue(bufferConsumer.isRecycled());
+
 			assertEquals(1, subpartition.getTotalNumberOfBuffers());
 			assertEquals(0, subpartition.getBuffersInBacklog());
-			assertEquals(0, subpartition.getTotalNumberOfBytes()); // only updated after consuming the buffers
 		} finally {
 			if (subpartition != null) {
 				subpartition.release();
@@ -74,16 +91,11 @@ public abstract class SubpartitionTestBase extends TestLogger {
 		try {
 			subpartition.release();
 
-			assertEquals(0, subpartition.getTotalNumberOfBuffers());
-			assertEquals(0, subpartition.getBuffersInBacklog());
-			assertEquals(0, subpartition.getTotalNumberOfBytes());
-
 			BufferConsumer bufferConsumer = createFilledBufferConsumer(4096, 4096);
 
 			assertFalse(subpartition.add(bufferConsumer));
-			assertEquals(0, subpartition.getTotalNumberOfBuffers());
-			assertEquals(0, subpartition.getBuffersInBacklog());
-			assertEquals(0, subpartition.getTotalNumberOfBytes());
+			assertTrue(bufferConsumer.isRecycled());
+
 		} finally {
 			if (subpartition != null) {
 				subpartition.release();
@@ -92,38 +104,71 @@ public abstract class SubpartitionTestBase extends TestLogger {
 	}
 
 	@Test
-	public void testReleaseParent() throws Exception {
+	public void testReleasingReaderDoesNotReleasePartition() throws Exception {
 		final ResultSubpartition partition = createSubpartition();
-		verifyViewReleasedAfterParentRelease(partition);
+		partition.add(createFilledBufferConsumer(BufferBuilderTestUtils.BUFFER_SIZE));
+		partition.finish();
+
+		final ResultSubpartitionView reader = partition.createReadView(new NoOpBufferAvailablityListener());
+
+		assertFalse(partition.isReleased());
+		assertFalse(reader.isReleased());
+
+		reader.releaseAllResources();
+
+		assertTrue(reader.isReleased());
+		assertFalse(partition.isReleased());
+
+		partition.release();
 	}
 
 	@Test
-	public void testReleaseParentAfterSpilled() throws Exception {
+	public void testReleaseIsIdempotent() throws Exception {
 		final ResultSubpartition partition = createSubpartition();
-		partition.releaseMemory();
+		partition.add(createFilledBufferConsumer(BufferBuilderTestUtils.BUFFER_SIZE));
+		partition.finish();
 
-		verifyViewReleasedAfterParentRelease(partition);
+		partition.release();
+		partition.release();
+		partition.release();
 	}
 
-	private void verifyViewReleasedAfterParentRelease(ResultSubpartition partition) throws Exception {
-		// Add a bufferConsumer
-		BufferConsumer bufferConsumer = createFilledBufferConsumer(BufferBuilderTestUtils.BUFFER_SIZE);
-		partition.add(bufferConsumer);
+	@Test
+	public void testReadAfterDispose() throws Exception {
+		final ResultSubpartition partition = createSubpartition();
+		partition.add(createFilledBufferConsumer(BufferBuilderTestUtils.BUFFER_SIZE));
 		partition.finish();
 
-		// Create the view
-		BufferAvailabilityListener listener = mock(BufferAvailabilityListener.class);
-		ResultSubpartitionView view = partition.createReadView(listener);
+		final ResultSubpartitionView reader = partition.createReadView(new NoOpBufferAvailablityListener());
+		reader.releaseAllResources();
 
-		// The added bufferConsumer and end-of-partition event
-		assertNotNull(view.getNextBuffer());
-		assertNotNull(view.getNextBuffer());
+		// the reader must not throw an exception
+		reader.getNextBuffer();
 
-		// Release the parent
-		assertFalse(view.isReleased());
-		partition.release();
+		// ideally, we want this to be null, but the pipelined partition still serves data
+		// after dispose (which is unintuitive, but does not affect correctness)
+//		assertNull(reader.getNextBuffer());
+	}
+
+	@Test
+	public void testRecycleBufferAndConsumerOnFailure() throws Exception {
+		final ResultSubpartition subpartition = createFailingWritesSubpartition();
+		try {
+			final BufferConsumer consumer = BufferBuilderTestUtils.createFilledBufferConsumer(100);
+
+			try {
+				subpartition.add(consumer);
+				subpartition.flush();
+				fail("should fail with an exception");
+			}
+			catch (Exception ignored) {
+				// expected
+			}
 
-		// Verify that parent release is reflected at partition view
-		assertTrue(view.isReleased());
+			assertTrue(consumer.isRecycled());
+		}
+		finally {
+			subpartition.release();
+		}
 	}
 }


[flink] 04/14: [hotfix][docs] Some fixes to FileSystem Documentation

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

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

commit 4c0bbc488091cf941ec9e8dfcf589bda5e7592cd
Author: Seth Wiesman <sj...@gmail.com>
AuthorDate: Wed May 1 11:45:57 2019 -0500

    [hotfix][docs] Some fixes to FileSystem Documentation
    
    This closes #8326
---
 docs/ops/deployment/aws.md        | 317 -------------------------------
 docs/ops/deployment/aws.zh.md     | 387 --------------------------------------
 docs/ops/filesystems/common.md    |  27 +--
 docs/ops/filesystems/common.zh.md |  27 +--
 docs/ops/filesystems/index.md     |  36 ++--
 docs/ops/filesystems/index.zh.md  | 130 +++----------
 docs/ops/filesystems/oss.md       | 178 ++----------------
 docs/ops/filesystems/oss.zh.md    | 179 ++----------------
 docs/ops/filesystems/s3.md        |  39 ++--
 docs/ops/filesystems/s3.zh.md     |  39 ++--
 10 files changed, 131 insertions(+), 1228 deletions(-)

diff --git a/docs/ops/deployment/aws.md b/docs/ops/deployment/aws.md
index a639e9c..b465340 100644
--- a/docs/ops/deployment/aws.md
+++ b/docs/ops/deployment/aws.md
@@ -63,320 +63,3 @@ HADOOP_CONF_DIR=/etc/hadoop/conf ./bin/flink run -m yarn-cluster -yn 1 examples/
 {% endhighlight %}
 
 {% top %}
-
-### Hadoop-provided S3 file systems
-
-{% panel **Note:** You don't have to configure this manually if you are running [Flink on EMR](#emr-elastic-mapreduce). %}
-
-Apache Flink provides native [S3 FileSystem's](../filesystems/s3.html) out of the box and we recomend using them unless required otherwise, e.g. for using S3 as YARN's resource storage dir
-via the `fs.defaultFS` configuration property in Hadoop's `core-site.xml`.
-
-#### Set S3 FileSystem
-
-Interaction with S3 happens via one of [Hadoop's S3 FileSystem clients](https://wiki.apache.org/hadoop/AmazonS3):
-
-1. `S3AFileSystem` (**recommended** for Hadoop 2.7 and later): file system for reading and writing regular files using Amazon's SDK internally. No maximum file size and works with IAM roles.
-2. `NativeS3FileSystem` (for Hadoop 2.6 and earlier): file system for reading and writing regular files. Maximum object size is 5GB and does not work with IAM roles.
-
-##### `S3AFileSystem` (Recommended)
-
-This is the recommended S3 FileSystem implementation to use. It uses Amazon's SDK internally and works with IAM roles (see [Configure Access Credentials](#configure-access-credentials-1)).
-
-You need to point Flink to a valid Hadoop configuration, which contains the following properties in `core-site.xml`:
-
-{% highlight xml %}
-<configuration>
-
-<property>
-  <name>fs.s3.impl</name>
-  <value>org.apache.hadoop.fs.s3a.S3AFileSystem</value>
-</property>
-
-<!-- Comma separated list of local directories used to buffer
-     large results prior to transmitting them to S3. -->
-<property>
-  <name>fs.s3a.buffer.dir</name>
-  <value>/tmp</value>
-</property>
-
-</configuration>
-{% endhighlight %}
-
-This registers `S3AFileSystem` as the default FileSystem for URIs with the `s3a://` scheme.
-
-##### `NativeS3FileSystem`
-
-This file system is limited to files up to 5GB in size and it does not work with IAM roles (see [Configure Access Credentials](#configure-access-credentials-1)), meaning that you have to manually configure your AWS credentials in the Hadoop config file.
-
-You need to point Flink to a valid Hadoop configuration, which contains the following property in `core-site.xml`:
-
-{% highlight xml %}
-<property>
-  <name>fs.s3.impl</name>
-  <value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value>
-</property>
-{% endhighlight %}
-
-This registers `NativeS3FileSystem` as the default FileSystem for URIs with the `s3://` scheme.
-
-{% top %}
-
-#### Hadoop Configuration
-
-You can specify the [Hadoop configuration](../config.html#hdfs) in various ways pointing Flink to
-the path of the Hadoop configuration directory, for example
-- by setting the environment variable `HADOOP_CONF_DIR`, or
-- by setting the `fs.hdfs.hadoopconf` configuration option in `flink-conf.yaml`:
-{% highlight yaml %}
-fs.hdfs.hadoopconf: /path/to/etc/hadoop
-{% endhighlight %}
-
-This registers `/path/to/etc/hadoop` as Hadoop's configuration directory with Flink. Flink will look for the `core-site.xml` and `hdfs-site.xml` files in the specified directory.
-
-{% top %}
-
-#### Configure Access Credentials
-
-{% panel **Note:** You don't have to configure this manually if you are running [Flink on EMR](#emr-elastic-mapreduce). %}
-
-After setting up the S3 FileSystem, you need to make sure that Flink is allowed to access your S3 buckets.
-
-##### Identity and Access Management (IAM) (Recommended)
-
-When using `S3AFileSystem`, the recommended way of setting up credentials on AWS is via [Identity and Access Management (IAM)](http://docs.aws.amazon.com/IAM/latest/UserGuide/introduction.html). You can use IAM features to securely give Flink instances the credentials that they need in order to access S3 buckets. Details about how to do this are beyond the scope of this documentation. Please refer to the AWS user guide. What you are looking for are [IAM Roles](http://docs.aws.amazon.com/ [...]
-
-If you set this up correctly, you can manage access to S3 within AWS and don't need to distribute any access keys to Flink.
-
-Note that this only works with `S3AFileSystem` and not `NativeS3FileSystem`.
-
-{% top %}
-
-##### Access Keys with `S3AFileSystem` (Discouraged)
-
-Access to S3 can be granted via your **access and secret key pair**. Please note that this is discouraged since the [introduction of IAM roles](https://blogs.aws.amazon.com/security/post/Tx1XG3FX6VMU6O5/A-safer-way-to-distribute-AWS-credentials-to-EC2).
-
-For `S3AFileSystem` you need to configure both `fs.s3a.access.key` and `fs.s3a.secret.key`  in Hadoop's  `core-site.xml`:
-
-{% highlight xml %}
-<property>
-  <name>fs.s3a.access.key</name>
-  <value></value>
-</property>
-
-<property>
-  <name>fs.s3a.secret.key</name>
-  <value></value>
-</property>
-{% endhighlight %}
-
-{% top %}
-
-##### Access Keys with `NativeS3FileSystem` (Discouraged)
-
-Access to S3 can be granted via your **access and secret key pair**. But this is discouraged and you should use `S3AFileSystem` [with the required IAM roles](https://blogs.aws.amazon.com/security/post/Tx1XG3FX6VMU6O5/A-safer-way-to-distribute-AWS-credentials-to-EC2).
-
-For `NativeS3FileSystem` you need to configure both `fs.s3.awsAccessKeyId` and `fs.s3.awsSecretAccessKey`  in Hadoop's  `core-site.xml`:
-
-{% highlight xml %}
-<property>
-  <name>fs.s3.awsAccessKeyId</name>
-  <value></value>
-</property>
-
-<property>
-  <name>fs.s3.awsSecretAccessKey</name>
-  <value></value>
-</property>
-{% endhighlight %}
-
-{% top %}
-
-#### Provide S3 FileSystem Dependency
-
-{% panel **Note:** You don't have to configure this manually if you are running [Flink on EMR](#emr-elastic-mapreduce). %}
-
-Hadoop's S3 FileSystem clients are packaged in the `hadoop-aws` artifact (Hadoop version 2.6 and later). This JAR and all its dependencies need to be added to Flink's classpath, i.e. the class path of both Job and TaskManagers. Depending on which FileSystem implementation and which Flink and Hadoop version you use, you need to provide different dependencies (see below).
-
-There are multiple ways of adding JARs to Flink's class path, the easiest being simply to drop the JARs in Flink's `lib` folder. You need to copy the `hadoop-aws` JAR with all its dependencies. You can also export the directory containing these JARs as part of the `HADOOP_CLASSPATH` environment variable on all machines.
-
-##### Flink for Hadoop 2.7
-
-Depending on which file system you use, please add the following dependencies. You can find these as part of the Hadoop binaries in `hadoop-2.7/share/hadoop/tools/lib`:
-
-- `S3AFileSystem`:
-  - `hadoop-aws-2.7.3.jar`
-  - `aws-java-sdk-s3-1.11.183.jar` and its dependencies:
-    - `aws-java-sdk-core-1.11.183.jar`
-    - `aws-java-sdk-kms-1.11.183.jar`
-    - `jackson-annotations-2.6.7.jar`
-    - `jackson-core-2.6.7.jar`
-    - `jackson-databind-2.6.7.jar`
-    - `joda-time-2.8.1.jar`
-    - `httpcore-4.4.4.jar`
-    - `httpclient-4.5.3.jar`
-
-- `NativeS3FileSystem`:
-  - `hadoop-aws-2.7.3.jar`
-  - `guava-11.0.2.jar`
-
-Note that `hadoop-common` is available as part of Flink, but Guava is shaded by Flink.
-
-##### Flink for Hadoop 2.6
-
-Depending on which file system you use, please add the following dependencies. You can find these as part of the Hadoop binaries in `hadoop-2.6/share/hadoop/tools/lib`:
-
-- `S3AFileSystem`:
-  - `hadoop-aws-2.6.4.jar`
-  - `aws-java-sdk-1.7.4.jar` and its dependencies:
-    - `jackson-annotations-2.1.1.jar`
-    - `jackson-core-2.1.1.jar`
-    - `jackson-databind-2.1.1.jar`
-    - `joda-time-2.2.jar`
-    - `httpcore-4.2.5.jar`
-    - `httpclient-4.2.5.jar`
-
-- `NativeS3FileSystem`:
-  - `hadoop-aws-2.6.4.jar`
-  - `guava-11.0.2.jar`
-
-Note that `hadoop-common` is available as part of Flink, but Guava is shaded by Flink.
-
-##### Flink for Hadoop 2.4 and earlier
-
-These Hadoop versions only have support for `NativeS3FileSystem`. This comes pre-packaged with Flink for Hadoop 2 as part of `hadoop-common`. You don't need to add anything to the classpath.
-
-{% top %}
-
-## Common Issues
-
-The following sections lists common issues when working with Flink on AWS.
-
-### Missing S3 FileSystem Configuration
-
-If your job submission fails with an Exception message noting that `No file system found with scheme s3` this means that no FileSystem has been configured for S3. Please check out the configuration sections for our [shaded Hadoop/Presto](#shaded-hadooppresto-s3-file-systems-recommended) or [generic Hadoop](#set-s3-filesystem) file systems for details on how to configure this properly.
-
-{% highlight plain %}
-org.apache.flink.client.program.ProgramInvocationException: The program execution failed:
-  Failed to submit job cd927567a81b62d7da4c18eaa91c3c39 (WordCount Example) [...]
-Caused by: org.apache.flink.runtime.JobException: Creating the input splits caused an error:
-  No file system found with scheme s3, referenced in file URI 's3://<bucket>/<endpoint>'. [...]
-Caused by: java.io.IOException: No file system found with scheme s3,
-  referenced in file URI 's3://<bucket>/<endpoint>'.
-    at o.a.f.core.fs.FileSystem.get(FileSystem.java:296)
-    at o.a.f.core.fs.Path.getFileSystem(Path.java:311)
-    at o.a.f.api.common.io.FileInputFormat.createInputSplits(FileInputFormat.java:450)
-    at o.a.f.api.common.io.FileInputFormat.createInputSplits(FileInputFormat.java:57)
-    at o.a.f.runtime.executiongraph.ExecutionJobVertex.<init>(ExecutionJobVertex.java:156)
-{% endhighlight %}
-
-{% top %}
-
-### AWS Access Key ID and Secret Access Key Not Specified
-
-If you see your job failing with an Exception noting that the `AWS Access Key ID and Secret Access Key must be specified as the username or password`, your access credentials have not been set up properly. Please refer to the access credential section for our [shaded Hadoop/Presto](#configure-access-credentials) or [generic Hadoop](#configure-access-credentials-1) file systems for details on how to configure this.
-
-{% highlight plain %}
-org.apache.flink.client.program.ProgramInvocationException: The program execution failed:
-  Failed to submit job cd927567a81b62d7da4c18eaa91c3c39 (WordCount Example) [...]
-Caused by: java.io.IOException: The given file URI (s3://<bucket>/<endpoint>) points to the
-  HDFS NameNode at <bucket>, but the File System could not be initialized with that address:
-  AWS Access Key ID and Secret Access Key must be specified as the username or password
-  (respectively) of a s3n URL, or by setting the fs.s3n.awsAccessKeyId
-  or fs.s3n.awsSecretAccessKey properties (respectively) [...]
-Caused by: java.lang.IllegalArgumentException: AWS Access Key ID and Secret Access Key must
-  be specified as the username or password (respectively) of a s3 URL, or by setting
-  the fs.s3n.awsAccessKeyId or fs.s3n.awsSecretAccessKey properties (respectively) [...]
-    at o.a.h.fs.s3.S3Credentials.initialize(S3Credentials.java:70)
-    at o.a.h.fs.s3native.Jets3tNativeFileSystemStore.initialize(Jets3tNativeFileSystemStore.java:80)
-    at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
-    at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
-    at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
-    at java.lang.reflect.Method.invoke(Method.java:606)
-    at o.a.h.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:187)
-    at o.a.h.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
-    at o.a.h.fs.s3native.$Proxy6.initialize(Unknown Source)
-    at o.a.h.fs.s3native.NativeS3FileSystem.initialize(NativeS3FileSystem.java:330)
-    at o.a.f.runtime.fs.hdfs.HadoopFileSystem.initialize(HadoopFileSystem.java:321)
-{% endhighlight %}
-
-{% top %}
-
-### ClassNotFoundException: NativeS3FileSystem/S3AFileSystem Not Found
-
-If you see this Exception, the S3 FileSystem is not part of the class path of Flink. Please refer to [S3 FileSystem dependency section](#provide-s3-filesystem-dependency) for details on how to configure this properly.
-
-{% highlight plain %}
-Caused by: java.lang.RuntimeException: java.lang.RuntimeException: java.lang.ClassNotFoundException: Class org.apache.hadoop.fs.s3native.NativeS3FileSystem not found
-  at org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2186)
-  at org.apache.flink.runtime.fs.hdfs.HadoopFileSystem.getHadoopWrapperClassNameForFileSystem(HadoopFileSystem.java:460)
-  at org.apache.flink.core.fs.FileSystem.getHadoopWrapperClassNameForFileSystem(FileSystem.java:352)
-  at org.apache.flink.core.fs.FileSystem.get(FileSystem.java:280)
-  at org.apache.flink.core.fs.Path.getFileSystem(Path.java:311)
-  at org.apache.flink.api.common.io.FileInputFormat.createInputSplits(FileInputFormat.java:450)
-  at org.apache.flink.api.common.io.FileInputFormat.createInputSplits(FileInputFormat.java:57)
-  at org.apache.flink.runtime.executiongraph.ExecutionJobVertex.<init>(ExecutionJobVertex.java:156)
-  ... 25 more
-Caused by: java.lang.RuntimeException: java.lang.ClassNotFoundException: Class org.apache.hadoop.fs.s3native.NativeS3FileSystem not found
-  at org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2154)
-  at org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2178)
-  ... 32 more
-Caused by: java.lang.ClassNotFoundException: Class org.apache.hadoop.fs.s3native.NativeS3FileSystem not found
-  at org.apache.hadoop.conf.Configuration.getClassByName(Configuration.java:2060)
-  at org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2152)
-  ... 33 more
-{% endhighlight %}
-
-{% top %}
-
-### IOException: `400: Bad Request`
-
-If you have configured everything properly, but get a `Bad Request` Exception **and** your S3 bucket is located in region `eu-central-1`, you might be running an S3 client, which does not support [Amazon's signature version 4](http://docs.aws.amazon.com/AmazonS3/latest/API/sig-v4-authenticating-requests.html).
-
-{% highlight plain %}
-[...]
-Caused by: java.io.IOException: s3://<bucket-in-eu-central-1>/<endpoint> : 400 : Bad Request [...]
-Caused by: org.jets3t.service.impl.rest.HttpException [...]
-{% endhighlight %}
-or
-{% highlight plain %}
-com.amazonaws.services.s3.model.AmazonS3Exception: Status Code: 400, AWS Service: Amazon S3, AWS Request ID: [...], AWS Error Code: null, AWS Error Message: Bad Request, S3 Extended Request ID: [...]
-
-{% endhighlight %}
-
-This should not apply to our shaded Hadoop/Presto S3 file systems but can occur for Hadoop-provided
-S3 file systems. In particular, all Hadoop versions up to 2.7.2 running `NativeS3FileSystem` (which
-depend on `JetS3t 0.9.0` instead of a version [>= 0.9.4](http://www.jets3t.org/RELEASE_NOTES.html))
-are affected but users also reported this happening with the `S3AFileSystem`.
-
-Except for changing the bucket region, you may also be able to solve this by
-[requesting signature version 4 for request authentication](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingAWSSDK.html#specify-signature-version),
-e.g. by adding this to Flink's JVM options in `flink-conf.yaml` (see
-[configuration](../config.html#common-options)):
-{% highlight yaml %}
-env.java.opts: -Dcom.amazonaws.services.s3.enableV4
-{% endhighlight %}
-
-{% top %}
-
-### NullPointerException at org.apache.hadoop.fs.LocalDirAllocator
-
-This Exception is usually caused by skipping the local buffer directory configuration `fs.s3a.buffer.dir` for the `S3AFileSystem`. Please refer to the [S3AFileSystem configuration](#s3afilesystem-recommended) section to see how to configure the `S3AFileSystem` properly.
-
-{% highlight plain %}
-[...]
-Caused by: java.lang.NullPointerException at
-o.a.h.fs.LocalDirAllocator$AllocatorPerContext.confChanged(LocalDirAllocator.java:268) at
-o.a.h.fs.LocalDirAllocator$AllocatorPerContext.getLocalPathForWrite(LocalDirAllocator.java:344) at
-o.a.h.fs.LocalDirAllocator$AllocatorPerContext.createTmpFileForWrite(LocalDirAllocator.java:416) at
-o.a.h.fs.LocalDirAllocator.createTmpFileForWrite(LocalDirAllocator.java:198) at
-o.a.h.fs.s3a.S3AOutputStream.<init>(S3AOutputStream.java:87) at
-o.a.h.fs.s3a.S3AFileSystem.create(S3AFileSystem.java:410) at
-o.a.h.fs.FileSystem.create(FileSystem.java:907) at
-o.a.h.fs.FileSystem.create(FileSystem.java:888) at
-o.a.h.fs.FileSystem.create(FileSystem.java:785) at
-o.a.f.runtime.fs.hdfs.HadoopFileSystem.create(HadoopFileSystem.java:404) at
-o.a.f.runtime.fs.hdfs.HadoopFileSystem.create(HadoopFileSystem.java:48) at
-... 25 more
-{% endhighlight %}
-
-{% top %}
diff --git a/docs/ops/deployment/aws.zh.md b/docs/ops/deployment/aws.zh.md
index ae43c40..b465340 100644
--- a/docs/ops/deployment/aws.zh.md
+++ b/docs/ops/deployment/aws.zh.md
@@ -63,390 +63,3 @@ HADOOP_CONF_DIR=/etc/hadoop/conf ./bin/flink run -m yarn-cluster -yn 1 examples/
 {% endhighlight %}
 
 {% top %}
-
-## S3: Simple Storage Service
-
-[Amazon Simple Storage Service](http://aws.amazon.com/s3/) (Amazon S3) provides cloud object storage for a variety of use cases. You can use S3 with Flink for **reading** and **writing data** as well in conjunction with the [streaming **state backends**]({{ site.baseurl}}/ops/state/state_backends.html) or even as a YARN object storage.
-
-You can use S3 objects like regular files by specifying paths in the following format:
-
-{% highlight plain %}
-s3://<your-bucket>/<endpoint>
-{% endhighlight %}
-
-The endpoint can either be a single file or a directory, for example:
-
-{% highlight java %}
-// Read from S3 bucket
-env.readTextFile("s3://<bucket>/<endpoint>");
-
-// Write to S3 bucket
-stream.writeAsText("s3://<bucket>/<endpoint>");
-
-// Use S3 as FsStatebackend
-env.setStateBackend(new FsStateBackend("s3://<your-bucket>/<endpoint>"));
-{% endhighlight %}
-
-Note that these examples are *not* exhaustive and you can use S3 in other places as well, including your [high availability setup](../jobmanager_high_availability.html) or the [RocksDBStateBackend]({{ site.baseurl }}/ops/state/state_backends.html#the-rocksdbstatebackend); everywhere that Flink expects a FileSystem URI.
-
-For most use cases, you may use one of our shaded `flink-s3-fs-hadoop` and `flink-s3-fs-presto` S3
-filesystem wrappers which are fairly easy to set up. For some cases, however, e.g. for using S3 as
-YARN's resource storage dir, it may be necessary to set up a specific Hadoop S3 FileSystem
-implementation. Both ways are described below.
-
-### Shaded Hadoop/Presto S3 file systems (recommended)
-
-{% panel **Note:** You don't have to configure this manually if you are running [Flink on EMR](#emr-elastic-mapreduce). %}
-
-To use either `flink-s3-fs-hadoop` or `flink-s3-fs-presto`, copy the respective JAR file from the
-`opt` directory to the `lib` directory of your Flink distribution before starting Flink, e.g.
-
-{% highlight bash %}
-cp ./opt/flink-s3-fs-presto-{{ site.version }}.jar ./lib/
-{% endhighlight %}
-
-Both `flink-s3-fs-hadoop` and `flink-s3-fs-presto` register default FileSystem
-wrappers for URIs with the `s3://` scheme, `flink-s3-fs-hadoop` also registers
-for `s3a://` and `flink-s3-fs-presto` also registers for `s3p://`, so you can
-use this to use both at the same time.
-
-#### Configure Access Credentials
-
-After setting up the S3 FileSystem wrapper, you need to make sure that Flink is allowed to access your S3 buckets.
-
-##### Identity and Access Management (IAM) (Recommended)
-
-The recommended way of setting up credentials on AWS is via [Identity and Access Management (IAM)](http://docs.aws.amazon.com/IAM/latest/UserGuide/introduction.html). You can use IAM features to securely give Flink instances the credentials that they need in order to access S3 buckets. Details about how to do this are beyond the scope of this documentation. Please refer to the AWS user guide. What you are looking for are [IAM Roles](http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/iam- [...]
-
-If you set this up correctly, you can manage access to S3 within AWS and don't need to distribute any access keys to Flink.
-
-##### Access Keys (Discouraged)
-
-Access to S3 can be granted via your **access and secret key pair**. Please note that this is discouraged since the [introduction of IAM roles](https://blogs.aws.amazon.com/security/post/Tx1XG3FX6VMU6O5/A-safer-way-to-distribute-AWS-credentials-to-EC2).
-
-You need to configure both `s3.access-key` and `s3.secret-key`  in Flink's  `flink-conf.yaml`:
-
-{% highlight yaml %}
-s3.access-key: your-access-key
-s3.secret-key: your-secret-key
-{% endhighlight %}
-
-{% top %}
-
-### Hadoop-provided S3 file systems - manual setup
-
-{% panel **Note:** You don't have to configure this manually if you are running [Flink on EMR](#emr-elastic-mapreduce). %}
-
-This setup is a bit more complex and we recommend using our shaded Hadoop/Presto file systems
-instead (see above) unless required otherwise, e.g. for using S3 as YARN's resource storage dir
-via the `fs.defaultFS` configuration property in Hadoop's `core-site.xml`.
-
-#### Set S3 FileSystem
-
-Interaction with S3 happens via one of [Hadoop's S3 FileSystem clients](https://wiki.apache.org/hadoop/AmazonS3):
-
-1. `S3AFileSystem` (**recommended** for Hadoop 2.7 and later): file system for reading and writing regular files using Amazon's SDK internally. No maximum file size and works with IAM roles.
-2. `NativeS3FileSystem` (for Hadoop 2.6 and earlier): file system for reading and writing regular files. Maximum object size is 5GB and does not work with IAM roles.
-
-##### `S3AFileSystem` (Recommended)
-
-This is the recommended S3 FileSystem implementation to use. It uses Amazon's SDK internally and works with IAM roles (see [Configure Access Credentials](#configure-access-credentials-1)).
-
-You need to point Flink to a valid Hadoop configuration, which contains the following properties in `core-site.xml`:
-
-{% highlight xml %}
-<configuration>
-
-<property>
-  <name>fs.s3.impl</name>
-  <value>org.apache.hadoop.fs.s3a.S3AFileSystem</value>
-</property>
-
-<!-- Comma separated list of local directories used to buffer
-     large results prior to transmitting them to S3. -->
-<property>
-  <name>fs.s3a.buffer.dir</name>
-  <value>/tmp</value>
-</property>
-
-</configuration>
-{% endhighlight %}
-
-This registers `S3AFileSystem` as the default FileSystem for URIs with the `s3a://` scheme.
-
-##### `NativeS3FileSystem`
-
-This file system is limited to files up to 5GB in size and it does not work with IAM roles (see [Configure Access Credentials](#configure-access-credentials-1)), meaning that you have to manually configure your AWS credentials in the Hadoop config file.
-
-You need to point Flink to a valid Hadoop configuration, which contains the following property in `core-site.xml`:
-
-{% highlight xml %}
-<property>
-  <name>fs.s3.impl</name>
-  <value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value>
-</property>
-{% endhighlight %}
-
-This registers `NativeS3FileSystem` as the default FileSystem for URIs with the `s3://` scheme.
-
-{% top %}
-
-#### Hadoop Configuration
-
-You can specify the [Hadoop configuration](../config.html#hdfs) in various ways pointing Flink to
-the path of the Hadoop configuration directory, for example
-- by setting the environment variable `HADOOP_CONF_DIR`, or
-- by setting the `fs.hdfs.hadoopconf` configuration option in `flink-conf.yaml`:
-{% highlight yaml %}
-fs.hdfs.hadoopconf: /path/to/etc/hadoop
-{% endhighlight %}
-
-This registers `/path/to/etc/hadoop` as Hadoop's configuration directory with Flink. Flink will look for the `core-site.xml` and `hdfs-site.xml` files in the specified directory.
-
-{% top %}
-
-#### Configure Access Credentials
-
-{% panel **Note:** You don't have to configure this manually if you are running [Flink on EMR](#emr-elastic-mapreduce). %}
-
-After setting up the S3 FileSystem, you need to make sure that Flink is allowed to access your S3 buckets.
-
-##### Identity and Access Management (IAM) (Recommended)
-
-When using `S3AFileSystem`, the recommended way of setting up credentials on AWS is via [Identity and Access Management (IAM)](http://docs.aws.amazon.com/IAM/latest/UserGuide/introduction.html). You can use IAM features to securely give Flink instances the credentials that they need in order to access S3 buckets. Details about how to do this are beyond the scope of this documentation. Please refer to the AWS user guide. What you are looking for are [IAM Roles](http://docs.aws.amazon.com/ [...]
-
-If you set this up correctly, you can manage access to S3 within AWS and don't need to distribute any access keys to Flink.
-
-Note that this only works with `S3AFileSystem` and not `NativeS3FileSystem`.
-
-{% top %}
-
-##### Access Keys with `S3AFileSystem` (Discouraged)
-
-Access to S3 can be granted via your **access and secret key pair**. Please note that this is discouraged since the [introduction of IAM roles](https://blogs.aws.amazon.com/security/post/Tx1XG3FX6VMU6O5/A-safer-way-to-distribute-AWS-credentials-to-EC2).
-
-For `S3AFileSystem` you need to configure both `fs.s3a.access.key` and `fs.s3a.secret.key`  in Hadoop's  `core-site.xml`:
-
-{% highlight xml %}
-<property>
-  <name>fs.s3a.access.key</name>
-  <value></value>
-</property>
-
-<property>
-  <name>fs.s3a.secret.key</name>
-  <value></value>
-</property>
-{% endhighlight %}
-
-{% top %}
-
-##### Access Keys with `NativeS3FileSystem` (Discouraged)
-
-Access to S3 can be granted via your **access and secret key pair**. But this is discouraged and you should use `S3AFileSystem` [with the required IAM roles](https://blogs.aws.amazon.com/security/post/Tx1XG3FX6VMU6O5/A-safer-way-to-distribute-AWS-credentials-to-EC2).
-
-For `NativeS3FileSystem` you need to configure both `fs.s3.awsAccessKeyId` and `fs.s3.awsSecretAccessKey`  in Hadoop's  `core-site.xml`:
-
-{% highlight xml %}
-<property>
-  <name>fs.s3.awsAccessKeyId</name>
-  <value></value>
-</property>
-
-<property>
-  <name>fs.s3.awsSecretAccessKey</name>
-  <value></value>
-</property>
-{% endhighlight %}
-
-{% top %}
-
-#### Provide S3 FileSystem Dependency
-
-{% panel **Note:** You don't have to configure this manually if you are running [Flink on EMR](#emr-elastic-mapreduce). %}
-
-Hadoop's S3 FileSystem clients are packaged in the `hadoop-aws` artifact (Hadoop version 2.6 and later). This JAR and all its dependencies need to be added to Flink's classpath, i.e. the class path of both Job and TaskManagers. Depending on which FileSystem implementation and which Flink and Hadoop version you use, you need to provide different dependencies (see below).
-
-There are multiple ways of adding JARs to Flink's class path, the easiest being simply to drop the JARs in Flink's `lib` folder. You need to copy the `hadoop-aws` JAR with all its dependencies. You can also export the directory containing these JARs as part of the `HADOOP_CLASSPATH` environment variable on all machines.
-
-##### Flink for Hadoop 2.7
-
-Depending on which file system you use, please add the following dependencies. You can find these as part of the Hadoop binaries in `hadoop-2.7/share/hadoop/tools/lib`:
-
-- `S3AFileSystem`:
-  - `hadoop-aws-2.7.3.jar`
-  - `aws-java-sdk-s3-1.11.183.jar` and its dependencies:
-    - `aws-java-sdk-core-1.11.183.jar`
-    - `aws-java-sdk-kms-1.11.183.jar`
-    - `jackson-annotations-2.6.7.jar`
-    - `jackson-core-2.6.7.jar`
-    - `jackson-databind-2.6.7.jar`
-    - `joda-time-2.8.1.jar`
-    - `httpcore-4.4.4.jar`
-    - `httpclient-4.5.3.jar`
-
-- `NativeS3FileSystem`:
-  - `hadoop-aws-2.7.3.jar`
-  - `guava-11.0.2.jar`
-
-Note that `hadoop-common` is available as part of Flink, but Guava is shaded by Flink.
-
-##### Flink for Hadoop 2.6
-
-Depending on which file system you use, please add the following dependencies. You can find these as part of the Hadoop binaries in `hadoop-2.6/share/hadoop/tools/lib`:
-
-- `S3AFileSystem`:
-  - `hadoop-aws-2.6.4.jar`
-  - `aws-java-sdk-1.7.4.jar` and its dependencies:
-    - `jackson-annotations-2.1.1.jar`
-    - `jackson-core-2.1.1.jar`
-    - `jackson-databind-2.1.1.jar`
-    - `joda-time-2.2.jar`
-    - `httpcore-4.2.5.jar`
-    - `httpclient-4.2.5.jar`
-
-- `NativeS3FileSystem`:
-  - `hadoop-aws-2.6.4.jar`
-  - `guava-11.0.2.jar`
-
-Note that `hadoop-common` is available as part of Flink, but Guava is shaded by Flink.
-
-##### Flink for Hadoop 2.4 and earlier
-
-These Hadoop versions only have support for `NativeS3FileSystem`. This comes pre-packaged with Flink for Hadoop 2 as part of `hadoop-common`. You don't need to add anything to the classpath.
-
-{% top %}
-
-## Common Issues
-
-The following sections lists common issues when working with Flink on AWS.
-
-### Missing S3 FileSystem Configuration
-
-If your job submission fails with an Exception message noting that `No file system found with scheme s3` this means that no FileSystem has been configured for S3. Please check out the configuration sections for our [shaded Hadoop/Presto](#shaded-hadooppresto-s3-file-systems-recommended) or [generic Hadoop](#set-s3-filesystem) file systems for details on how to configure this properly.
-
-{% highlight plain %}
-org.apache.flink.client.program.ProgramInvocationException: The program execution failed:
-  Failed to submit job cd927567a81b62d7da4c18eaa91c3c39 (WordCount Example) [...]
-Caused by: org.apache.flink.runtime.JobException: Creating the input splits caused an error:
-  No file system found with scheme s3, referenced in file URI 's3://<bucket>/<endpoint>'. [...]
-Caused by: java.io.IOException: No file system found with scheme s3,
-  referenced in file URI 's3://<bucket>/<endpoint>'.
-    at o.a.f.core.fs.FileSystem.get(FileSystem.java:296)
-    at o.a.f.core.fs.Path.getFileSystem(Path.java:311)
-    at o.a.f.api.common.io.FileInputFormat.createInputSplits(FileInputFormat.java:450)
-    at o.a.f.api.common.io.FileInputFormat.createInputSplits(FileInputFormat.java:57)
-    at o.a.f.runtime.executiongraph.ExecutionJobVertex.<init>(ExecutionJobVertex.java:156)
-{% endhighlight %}
-
-{% top %}
-
-### AWS Access Key ID and Secret Access Key Not Specified
-
-If you see your job failing with an Exception noting that the `AWS Access Key ID and Secret Access Key must be specified as the username or password`, your access credentials have not been set up properly. Please refer to the access credential section for our [shaded Hadoop/Presto](#configure-access-credentials) or [generic Hadoop](#configure-access-credentials-1) file systems for details on how to configure this.
-
-{% highlight plain %}
-org.apache.flink.client.program.ProgramInvocationException: The program execution failed:
-  Failed to submit job cd927567a81b62d7da4c18eaa91c3c39 (WordCount Example) [...]
-Caused by: java.io.IOException: The given file URI (s3://<bucket>/<endpoint>) points to the
-  HDFS NameNode at <bucket>, but the File System could not be initialized with that address:
-  AWS Access Key ID and Secret Access Key must be specified as the username or password
-  (respectively) of a s3n URL, or by setting the fs.s3n.awsAccessKeyId
-  or fs.s3n.awsSecretAccessKey properties (respectively) [...]
-Caused by: java.lang.IllegalArgumentException: AWS Access Key ID and Secret Access Key must
-  be specified as the username or password (respectively) of a s3 URL, or by setting
-  the fs.s3n.awsAccessKeyId or fs.s3n.awsSecretAccessKey properties (respectively) [...]
-    at o.a.h.fs.s3.S3Credentials.initialize(S3Credentials.java:70)
-    at o.a.h.fs.s3native.Jets3tNativeFileSystemStore.initialize(Jets3tNativeFileSystemStore.java:80)
-    at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
-    at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
-    at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
-    at java.lang.reflect.Method.invoke(Method.java:606)
-    at o.a.h.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:187)
-    at o.a.h.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
-    at o.a.h.fs.s3native.$Proxy6.initialize(Unknown Source)
-    at o.a.h.fs.s3native.NativeS3FileSystem.initialize(NativeS3FileSystem.java:330)
-    at o.a.f.runtime.fs.hdfs.HadoopFileSystem.initialize(HadoopFileSystem.java:321)
-{% endhighlight %}
-
-{% top %}
-
-### ClassNotFoundException: NativeS3FileSystem/S3AFileSystem Not Found
-
-If you see this Exception, the S3 FileSystem is not part of the class path of Flink. Please refer to [S3 FileSystem dependency section](#provide-s3-filesystem-dependency) for details on how to configure this properly.
-
-{% highlight plain %}
-Caused by: java.lang.RuntimeException: java.lang.RuntimeException: java.lang.ClassNotFoundException: Class org.apache.hadoop.fs.s3native.NativeS3FileSystem not found
-  at org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2186)
-  at org.apache.flink.runtime.fs.hdfs.HadoopFileSystem.getHadoopWrapperClassNameForFileSystem(HadoopFileSystem.java:460)
-  at org.apache.flink.core.fs.FileSystem.getHadoopWrapperClassNameForFileSystem(FileSystem.java:352)
-  at org.apache.flink.core.fs.FileSystem.get(FileSystem.java:280)
-  at org.apache.flink.core.fs.Path.getFileSystem(Path.java:311)
-  at org.apache.flink.api.common.io.FileInputFormat.createInputSplits(FileInputFormat.java:450)
-  at org.apache.flink.api.common.io.FileInputFormat.createInputSplits(FileInputFormat.java:57)
-  at org.apache.flink.runtime.executiongraph.ExecutionJobVertex.<init>(ExecutionJobVertex.java:156)
-  ... 25 more
-Caused by: java.lang.RuntimeException: java.lang.ClassNotFoundException: Class org.apache.hadoop.fs.s3native.NativeS3FileSystem not found
-  at org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2154)
-  at org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2178)
-  ... 32 more
-Caused by: java.lang.ClassNotFoundException: Class org.apache.hadoop.fs.s3native.NativeS3FileSystem not found
-  at org.apache.hadoop.conf.Configuration.getClassByName(Configuration.java:2060)
-  at org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2152)
-  ... 33 more
-{% endhighlight %}
-
-{% top %}
-
-### IOException: `400: Bad Request`
-
-If you have configured everything properly, but get a `Bad Request` Exception **and** your S3 bucket is located in region `eu-central-1`, you might be running an S3 client, which does not support [Amazon's signature version 4](http://docs.aws.amazon.com/AmazonS3/latest/API/sig-v4-authenticating-requests.html).
-
-{% highlight plain %}
-[...]
-Caused by: java.io.IOException: s3://<bucket-in-eu-central-1>/<endpoint> : 400 : Bad Request [...]
-Caused by: org.jets3t.service.impl.rest.HttpException [...]
-{% endhighlight %}
-or
-{% highlight plain %}
-com.amazonaws.services.s3.model.AmazonS3Exception: Status Code: 400, AWS Service: Amazon S3, AWS Request ID: [...], AWS Error Code: null, AWS Error Message: Bad Request, S3 Extended Request ID: [...]
-
-{% endhighlight %}
-
-This should not apply to our shaded Hadoop/Presto S3 file systems but can occur for Hadoop-provided
-S3 file systems. In particular, all Hadoop versions up to 2.7.2 running `NativeS3FileSystem` (which
-depend on `JetS3t 0.9.0` instead of a version [>= 0.9.4](http://www.jets3t.org/RELEASE_NOTES.html))
-are affected but users also reported this happening with the `S3AFileSystem`.
-
-Except for changing the bucket region, you may also be able to solve this by
-[requesting signature version 4 for request authentication](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingAWSSDK.html#specify-signature-version),
-e.g. by adding this to Flink's JVM options in `flink-conf.yaml` (see
-[configuration](../config.html#common-options)):
-{% highlight yaml %}
-env.java.opts: -Dcom.amazonaws.services.s3.enableV4
-{% endhighlight %}
-
-{% top %}
-
-### NullPointerException at org.apache.hadoop.fs.LocalDirAllocator
-
-This Exception is usually caused by skipping the local buffer directory configuration `fs.s3a.buffer.dir` for the `S3AFileSystem`. Please refer to the [S3AFileSystem configuration](#s3afilesystem-recommended) section to see how to configure the `S3AFileSystem` properly.
-
-{% highlight plain %}
-[...]
-Caused by: java.lang.NullPointerException at
-o.a.h.fs.LocalDirAllocator$AllocatorPerContext.confChanged(LocalDirAllocator.java:268) at
-o.a.h.fs.LocalDirAllocator$AllocatorPerContext.getLocalPathForWrite(LocalDirAllocator.java:344) at
-o.a.h.fs.LocalDirAllocator$AllocatorPerContext.createTmpFileForWrite(LocalDirAllocator.java:416) at
-o.a.h.fs.LocalDirAllocator.createTmpFileForWrite(LocalDirAllocator.java:198) at
-o.a.h.fs.s3a.S3AOutputStream.<init>(S3AOutputStream.java:87) at
-o.a.h.fs.s3a.S3AFileSystem.create(S3AFileSystem.java:410) at
-o.a.h.fs.FileSystem.create(FileSystem.java:907) at
-o.a.h.fs.FileSystem.create(FileSystem.java:888) at
-o.a.h.fs.FileSystem.create(FileSystem.java:785) at
-o.a.f.runtime.fs.hdfs.HadoopFileSystem.create(HadoopFileSystem.java:404) at
-o.a.f.runtime.fs.hdfs.HadoopFileSystem.create(HadoopFileSystem.java:48) at
-... 25 more
-{% endhighlight %}
-
-{% top %}
diff --git a/docs/ops/filesystems/common.md b/docs/ops/filesystems/common.md
index edef1dd..a8a371e 100644
--- a/docs/ops/filesystems/common.md
+++ b/docs/ops/filesystems/common.md
@@ -22,14 +22,14 @@ specific language governing permissions and limitations
 under the License.
 -->
 
-Apache Flink provides a number of common configuration settings that work across all file system implementations. 
+Apache Flink provides several standard configuration settings that work across all file system implementations. 
 
 * This will be replaced by the TOC
 {:toc}
 
 ## Default File System
 
-If paths to files do not explicitly specify a file system scheme (and authority), a default scheme (and authority) will be used.
+A default scheme (and authority) is used if paths to files do not explicitly specify a file system scheme (and authority).
 
 {% highlight yaml %}
 fs.default-scheme: <default-fs>
@@ -40,10 +40,10 @@ For example, if the default file system configured as `fs.default-scheme: hdfs:/
 
 ## Connection limiting
 
-You can limit the total number of connections that a file system can concurrently open. This is useful when the file system cannot handle a large number
-of concurrent reads / writes or open connections at the same time.
+You can limit the total number of connections that a file system can concurrently open which is useful when the file system cannot handle a large number
+of concurrent reads/writes or open connections at the same time.
 
-For example, very small HDFS clusters with few RPC handlers can sometimes be overwhelmed by a large Flink job trying to build up many connections during a checkpoint.
+For example, small HDFS clusters with few RPC handlers can sometimes be overwhelmed by a large Flink job trying to build up many connections during a checkpoint.
 
 To limit a specific file system's connections, add the following entries to the Flink configuration. The file system to be limited is identified by
 its scheme.
@@ -56,13 +56,14 @@ fs.<scheme>.limit.timeout: (milliseconds, 0 means infinite)
 fs.<scheme>.limit.stream-timeout: (milliseconds, 0 means infinite)
 {% endhighlight %}
 
-You can limit the number if input/output connections (streams) separately (`fs.<scheme>.limit.input` and `fs.<scheme>.limit.output`), as well as impose a limit on
-the total number of concurrent streams (`fs.<scheme>.limit.total`). If the file system tries to open more streams, the operation will block until some streams are closed.
-If the opening of the stream takes longer than `fs.<scheme>.limit.timeout`, the stream opening will fail.
+You can limit the number of input/output connections (streams) separately (`fs.<scheme>.limit.input` and `fs.<scheme>.limit.output`), as well as impose a limit on
+the total number of concurrent streams (`fs.<scheme>.limit.total`). If the file system tries to open more streams, the operation blocks until some streams close.
+If the opening of the stream takes longer than `fs.<scheme>.limit.timeout`, the stream opening fails.
 
-To prevent inactive streams from taking up the complete pool (preventing new connections to be opened), you can add an inactivity timeout for streams:
-`fs.<scheme>.limit.stream-timeout`. If a stream does not read/write any bytes for at least that amount of time, it is forcibly closed.
+To prevent inactive streams from taking up the full pool (preventing new connections to be opened), you can add an inactivity timeout which forcibly closes them if they do not read/write any bytes for at least that amount of time: `fs.<scheme>.limit.stream-timeout`. 
 
-These limits are enforced per TaskManager, so each TaskManager in a Flink application or cluster will open up to that number of connections.
-In addition, the limits are also only enforced per FileSystem instance. Because File Systems are created per scheme and authority, different
-authorities will have their own connection pool. For example `hdfs://myhdfs:50010/` and `hdfs://anotherhdfs:4399/` will have separate pools.
\ No newline at end of file
+Limit enforcment on a per TaskManager/file system basis.
+Because file systems creation occurs per scheme and authority, different
+authorities have independent connection pools. For example `hdfs://myhdfs:50010/` and `hdfs://anotherhdfs:4399/` will have separate pools.
+
+{% top %}
\ No newline at end of file
diff --git a/docs/ops/filesystems/common.zh.md b/docs/ops/filesystems/common.zh.md
index edef1dd..a8a371e 100644
--- a/docs/ops/filesystems/common.zh.md
+++ b/docs/ops/filesystems/common.zh.md
@@ -22,14 +22,14 @@ specific language governing permissions and limitations
 under the License.
 -->
 
-Apache Flink provides a number of common configuration settings that work across all file system implementations. 
+Apache Flink provides several standard configuration settings that work across all file system implementations. 
 
 * This will be replaced by the TOC
 {:toc}
 
 ## Default File System
 
-If paths to files do not explicitly specify a file system scheme (and authority), a default scheme (and authority) will be used.
+A default scheme (and authority) is used if paths to files do not explicitly specify a file system scheme (and authority).
 
 {% highlight yaml %}
 fs.default-scheme: <default-fs>
@@ -40,10 +40,10 @@ For example, if the default file system configured as `fs.default-scheme: hdfs:/
 
 ## Connection limiting
 
-You can limit the total number of connections that a file system can concurrently open. This is useful when the file system cannot handle a large number
-of concurrent reads / writes or open connections at the same time.
+You can limit the total number of connections that a file system can concurrently open which is useful when the file system cannot handle a large number
+of concurrent reads/writes or open connections at the same time.
 
-For example, very small HDFS clusters with few RPC handlers can sometimes be overwhelmed by a large Flink job trying to build up many connections during a checkpoint.
+For example, small HDFS clusters with few RPC handlers can sometimes be overwhelmed by a large Flink job trying to build up many connections during a checkpoint.
 
 To limit a specific file system's connections, add the following entries to the Flink configuration. The file system to be limited is identified by
 its scheme.
@@ -56,13 +56,14 @@ fs.<scheme>.limit.timeout: (milliseconds, 0 means infinite)
 fs.<scheme>.limit.stream-timeout: (milliseconds, 0 means infinite)
 {% endhighlight %}
 
-You can limit the number if input/output connections (streams) separately (`fs.<scheme>.limit.input` and `fs.<scheme>.limit.output`), as well as impose a limit on
-the total number of concurrent streams (`fs.<scheme>.limit.total`). If the file system tries to open more streams, the operation will block until some streams are closed.
-If the opening of the stream takes longer than `fs.<scheme>.limit.timeout`, the stream opening will fail.
+You can limit the number of input/output connections (streams) separately (`fs.<scheme>.limit.input` and `fs.<scheme>.limit.output`), as well as impose a limit on
+the total number of concurrent streams (`fs.<scheme>.limit.total`). If the file system tries to open more streams, the operation blocks until some streams close.
+If the opening of the stream takes longer than `fs.<scheme>.limit.timeout`, the stream opening fails.
 
-To prevent inactive streams from taking up the complete pool (preventing new connections to be opened), you can add an inactivity timeout for streams:
-`fs.<scheme>.limit.stream-timeout`. If a stream does not read/write any bytes for at least that amount of time, it is forcibly closed.
+To prevent inactive streams from taking up the full pool (preventing new connections to be opened), you can add an inactivity timeout which forcibly closes them if they do not read/write any bytes for at least that amount of time: `fs.<scheme>.limit.stream-timeout`. 
 
-These limits are enforced per TaskManager, so each TaskManager in a Flink application or cluster will open up to that number of connections.
-In addition, the limits are also only enforced per FileSystem instance. Because File Systems are created per scheme and authority, different
-authorities will have their own connection pool. For example `hdfs://myhdfs:50010/` and `hdfs://anotherhdfs:4399/` will have separate pools.
\ No newline at end of file
+Limit enforcment on a per TaskManager/file system basis.
+Because file systems creation occurs per scheme and authority, different
+authorities have independent connection pools. For example `hdfs://myhdfs:50010/` and `hdfs://anotherhdfs:4399/` will have separate pools.
+
+{% top %}
\ No newline at end of file
diff --git a/docs/ops/filesystems/index.md b/docs/ops/filesystems/index.md
index be8bbee..b8548c1 100644
--- a/docs/ops/filesystems/index.md
+++ b/docs/ops/filesystems/index.md
@@ -24,40 +24,37 @@ specific language governing permissions and limitations
 under the License.
 -->
 
-Apache Flink uses file system for both ingest and output of data for streaming and batch applications as well as targets for checkpoint storage. 
-These file systems can be local such as *Unix*, distributed like *HDFS*, or even object stores such as *S3*.
+Apache Flink uses file systems to consume and persistently store data, both for the results of applications and for fault tolerance and recovery.
+These are some of most of the popular file systems, including *local*, *hadoop-compatible*, *S3*, *MapR FS*, *OpenStack Swift FS* and *Aliyum OSS*.
 
 The file system used for a particular file is determined by its URI scheme.
 For example, `file:///home/user/text.txt` refers to a file in the local file system, while `hdfs://namenode:50010/data/user/text.txt` is a file in a specific HDFS cluster.
 
-FileSystem instances are instantiated once per process and then cached / pooled, to
-avoid configuration overhead per stream creation and to enforce certain constraints, such as connection/stream limits.
+File system instances are instantiated once per process and then cached/pooled, to avoid configuration overhead per stream creation and to enforce certain constraints, such as connection/stream limits.
 
 * This will be replaced by the TOC
 {:toc}
 
 ### Built-in File Systems
 
-Flink ships with support for most of the popular file systems, including *local*, *hadoop-compatible*, *S3*, *MapR FS* and *OpenStack Swift FS*.
-Each is identified by the scheme included in the URI of the provide file path. 
-
 Flink ships with implementations for the following file systems:
 
-  - **local**: This file system is used when the scheme is *"file://"*, and it represents the file system of the local machine, including any NFS or SAN that is mounted into that local file system.
+  - **local**: This file system is used when the scheme is *"file://"*, and it represents the file system of the local machine, including any NFS or SAN drives mounted into that local file system.
 
   - **S3**: Flink directly provides file systems to talk to Amazon S3 with two alternative implementations, `flink-s3-fs-presto` and `flink-s3-fs-hadoop`. Both implementations are self-contained with no dependency footprint.
     
   - **MapR FS**: The MapR file system *"maprfs://"* is automatically available when the MapR libraries are in the classpath.
   
   - **OpenStack Swift FS**: Flink directly provides a file system to talk to the OpenStack Swift file system, registered under the scheme *"swift://"*. 
-  The implementation `flink-swift-fs-hadoop` is based on the [Hadoop Project](https://hadoop.apache.org/) but is self-contained with no dependency footprint.
+  The implementation of `flink-swift-fs-hadoop` is based on the [Hadoop Project](https://hadoop.apache.org/) but is self-contained with no dependency footprint.
   To use it when using Flink as a library, add the respective maven dependency (`org.apache.flink:flink-swift-fs-hadoop:{{ site.version }}`
   When starting a Flink application from the Flink binaries, copy or move the respective jar file from the `opt` folder to the `lib` folder.
 
 #### HDFS and Hadoop File System support 
 
-For all schemes where Flink cannot find a directly supported file system, it will fall back to Hadoop.
-All Hadoop file systems are automatically available when `flink-runtime` and the Hadoop libraries are in classpath.
+For all schemes where Flink cannot find a directly supported file system, it falls back to Hadoop.
+All Hadoop file systems are automatically available when `flink-runtime` and the Hadoop libraries are on the classpath.
+
 
 This way, Flink seamlessly supports all of Hadoop file systems, and all Hadoop-compatible file systems (HCFS).
 
@@ -67,12 +64,25 @@ This way, Flink seamlessly supports all of Hadoop file systems, and all Hadoop-c
   - **har**
   - ...
 
+##### Hadoop Configuration
+
+We recommend using Flink's built-in file systems them unless required otherwise, e.g., for using that file system as YARN's resource storage dir via the `fs.defaultFS` configuration property in Hadoop's `core-site.xml`.
+
+If using a Hadoop file system, you can specify the [configuration](../config.html#hdfs) by setting the environment variable `HADOOP_CONF_DIR`, or by setting the `fs.hdfs.hadoopconf` configuration option in `flink-conf.yaml`. 
+
+{% highlight yaml %}
+fs.hdfs.hadoopconf: /path/to/etc/hadoop
+{% endhighlight %}
+
+This registers `/path/to/etc/hadoop` as Hadoop's configuration directory and is where Flink will look for the `core-site.xml` and `hdfs-site.xml` files.
+
+
 ## Adding new File System Implementations
 
 File systems are represented via the `org.apache.flink.core.fs.FileSystem` class, which captures the ways to access and modify files and objects in that file system. 
-Implementations are discovered by Flink through Java's service abstraction, making it easy to add additional file system implementations.
+Implementations are discovered by Flink through Java's service abstraction, making it easy to add new file system implementations.
 
-In order to add a new File System, the following steps are needed:
+To add a new file system:
 
   - Add the File System implementation, which is a subclass of `org.apache.flink.core.fs.FileSystem`.
   - Add a factory that instantiates that file system and declares the scheme under which the FileSystem is registered. This must be a subclass of `org.apache.flink.core.fs.FileSystemFactory`.
diff --git a/docs/ops/filesystems/index.zh.md b/docs/ops/filesystems/index.zh.md
index 9390239..7f2de4e 100644
--- a/docs/ops/filesystems/index.zh.md
+++ b/docs/ops/filesystems/index.zh.md
@@ -24,66 +24,39 @@ specific language governing permissions and limitations
 under the License.
 -->
 
-This page provides details on setting up and configuring different file systems for use with Flink.
-We start by describing how to use and configure the different file systems that are supported by Flink
-out-of-the-box, before describing the necessary steps in order to add support about other/custom file system
-implementations.
+Apache Flink uses file systems to consume and persistently store data, both for the results of applications and for fault tolerance and recovery.
+These are some of most of the popular file systems, including *local*, *hadoop-compatible*, *S3*, *MapR FS*, *OpenStack Swift FS* and *Aliyum OSS*.
 
-## Flink's File System support
+The file system used for a particular file is determined by its URI scheme.
+For example, `file:///home/user/text.txt` refers to a file in the local file system, while `hdfs://namenode:50010/data/user/text.txt` is a file in a specific HDFS cluster.
 
-Flink uses file systems both as *sources* and *sinks* in streaming/batch applications and as a target for *checkpointing*.
-These file systems can for example be *Unix/Windows file systems*, *HDFS*, or even object stores like *S3*.
+File system instances are instantiated once per process and then cached/pooled, to avoid configuration overhead per stream creation and to enforce certain constraints, such as connection/stream limits.
 
-The file system used for a specific file is determined by the file URI's scheme. For example `file:///home/user/text.txt` refers to
-a file in the local file system, while `hdfs://namenode:50010/data/user/text.txt` refers to a file in a specific HDFS cluster.
-
-File systems are represented via the `org.apache.flink.core.fs.FileSystem` class, which captures the ways to access and modify
-files and objects in that file system. FileSystem instances are instantiated once per process and then cached / pooled, to
-avoid configuration overhead per stream creation and to enforce certain constraints, such as connection/stream limits.
+* This will be replaced by the TOC
+{:toc}
 
 ### Built-in File Systems
 
-Flink ships with support for most of the popular file systems, namely *local*, *hadoop-compatible*, *S3*, *MapR FS*
-and *OpenStack Swift FS*. Each of these is identified by the scheme included in the URI of the provide file path. 
-
 Flink ships with implementations for the following file systems:
 
-  - **local**: This file system is used when the scheme is *"file://"*, and it represents the file system of the local machine, 
-including any NFS or SAN that is mounted into that local file system.
-
-  - **S3**: Flink directly provides file systems to talk to Amazon S3. There are two alternative implementations, `flink-s3-fs-presto`
-    and `flink-s3-fs-hadoop`. Both implementations are self-contained with no dependency footprint. There is no need to add Hadoop to
-    the classpath to use them. Both internally use some Hadoop code, but "shade away" all classes to avoid any dependency conflicts.
-
-    - `flink-s3-fs-presto`, registered under the scheme *"s3://"* and *"s3p://"*, is based on code from the [Presto project](https://prestodb.io/).
-      You can configure it the same way you can [configure the Presto file system](https://prestodb.io/docs/0.187/connector/hive.html#amazon-s3-configuration).
-      
-    - `flink-s3-fs-hadoop`, registered under *"s3://"* and *"s3a://"*, based on code from the [Hadoop Project](https://hadoop.apache.org/).
-      The file system can be [configured exactly like Hadoop's s3a](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/index.html#S3A).
+  - **local**: This file system is used when the scheme is *"file://"*, and it represents the file system of the local machine, including any NFS or SAN drives mounted into that local file system.
 
-    To use those file systems when using Flink as a library, add the respective maven dependency (`org.apache.flink:flink-s3-fs-presto:{{ site.version }}`
-    or `org.apache.flink:flink-s3-fs-hadoop:{{ site.version }}`). When starting a Flink application from the Flink binaries, copy or move
-    the respective jar file from the `opt` folder to the `lib` folder. See also [AWS setup](deployment/aws.html) for additional details.
-    
-    <span class="label label-danger">Attention</span>: As described above, both Hadoop and Presto "listen" to paths with scheme set to *"s3://"*. This is 
-    convenient for switching between implementations (Hadoop or Presto), but it can lead to non-determinism when both
-    implementations are required. This can happen when, for example, the job uses the [StreamingFileSink]({{ site.baseurl}}/dev/connectors/streamfile_sink.html) 
-    which only supports Hadoop, but uses Presto for checkpointing. In this case, it is advised to use explicitly *"s3a://"* 
-    as a scheme for the sink (Hadoop) and *"s3p://"* for checkpointing (Presto).
+  - **S3**: Flink directly provides file systems to talk to Amazon S3 with two alternative implementations, `flink-s3-fs-presto` and `flink-s3-fs-hadoop`. Both implementations are self-contained with no dependency footprint.
     
   - **MapR FS**: The MapR file system *"maprfs://"* is automatically available when the MapR libraries are in the classpath.
   
   - **OpenStack Swift FS**: Flink directly provides a file system to talk to the OpenStack Swift file system, registered under the scheme *"swift://"*. 
-  The implementation `flink-swift-fs-hadoop` is based on the [Hadoop Project](https://hadoop.apache.org/) but is self-contained with no dependency footprint.
+  The implementation of `flink-swift-fs-hadoop` is based on the [Hadoop Project](https://hadoop.apache.org/) but is self-contained with no dependency footprint.
   To use it when using Flink as a library, add the respective maven dependency (`org.apache.flink:flink-swift-fs-hadoop:{{ site.version }}`
   When starting a Flink application from the Flink binaries, copy or move the respective jar file from the `opt` folder to the `lib` folder.
 
 #### HDFS and Hadoop File System support 
 
-For all schemes where it cannot find a directly supported file system, Flink will try to use Hadoop to instantiate a file system for the respective scheme.
-All Hadoop file systems are automatically available once `flink-runtime` and the Hadoop libraries are in classpath.
+For all schemes where Flink cannot find a directly supported file system, it falls back to Hadoop.
+All Hadoop file systems are automatically available when `flink-runtime` and the Hadoop libraries are on the classpath.
+
 
-That way, Flink seamlessly supports all of Hadoop file systems, and all Hadoop-compatible file systems (HCFS), for example:
+This way, Flink seamlessly supports all of Hadoop file systems, and all Hadoop-compatible file systems (HCFS).
 
   - **hdfs**
   - **ftp**
@@ -91,84 +64,25 @@ That way, Flink seamlessly supports all of Hadoop file systems, and all Hadoop-c
   - **har**
   - ...
 
+##### Hadoop Configuration
 
-## Common File System configurations
-
-The following configuration settings exist across different file systems.
-
-#### Default File System
-
-If paths to files do not explicitly specify a file system scheme (and authority), a default scheme (and authority) will be used.
-
-{% highlight yaml %}
-fs.default-scheme: <default-fs>
-{% endhighlight %}
-
-For example, if the default file system configured as `fs.default-scheme: hdfs://localhost:9000/`, then a file path of
-`/user/hugo/in.txt` is interpreted as `hdfs://localhost:9000/user/hugo/in.txt`.
-
-#### Connection limiting
-
-You can limit the total number of connections that a file system can concurrently open. This is useful when the file system cannot handle a large number
-of concurrent reads / writes or open connections at the same time.
+We recommend using Flink's built-in file systems them unless required otherwise, e.g., for using that file system as YARN's resource storage dir via the `fs.defaultFS` configuration property in Hadoop's `core-site.xml`.
 
-For example, very small HDFS clusters with few RPC handlers can sometimes be overwhelmed by a large Flink job trying to build up many connections during a checkpoint.
-
-To limit a specific file system's connections, add the following entries to the Flink configuration. The file system to be limited is identified by
-its scheme.
+If using a Hadoop file system, you can specify the [configuration](../config.html#hdfs) by setting the environment variable `HADOOP_CONF_DIR`, or by setting the `fs.hdfs.hadoopconf` configuration option in `flink-conf.yaml`. 
 
 {% highlight yaml %}
-fs.<scheme>.limit.total: (number, 0/-1 mean no limit)
-fs.<scheme>.limit.input: (number, 0/-1 mean no limit)
-fs.<scheme>.limit.output: (number, 0/-1 mean no limit)
-fs.<scheme>.limit.timeout: (milliseconds, 0 means infinite)
-fs.<scheme>.limit.stream-timeout: (milliseconds, 0 means infinite)
+fs.hdfs.hadoopconf: /path/to/etc/hadoop
 {% endhighlight %}
 
-You can limit the number if input/output connections (streams) separately (`fs.<scheme>.limit.input` and `fs.<scheme>.limit.output`), as well as impose a limit on
-the total number of concurrent streams (`fs.<scheme>.limit.total`). If the file system tries to open more streams, the operation will block until some streams are closed.
-If the opening of the stream takes longer than `fs.<scheme>.limit.timeout`, the stream opening will fail.
-
-To prevent inactive streams from taking up the complete pool (preventing new connections to be opened), you can add an inactivity timeout for streams:
-`fs.<scheme>.limit.stream-timeout`. If a stream does not read/write any bytes for at least that amount of time, it is forcibly closed.
-
-These limits are enforced per TaskManager, so each TaskManager in a Flink application or cluster will open up to that number of connections.
-In addition, the limits are also only enforced per FileSystem instance. Because File Systems are created per scheme and authority, different
-authorities will have their own connection pool. For example `hdfs://myhdfs:50010/` and `hdfs://anotherhdfs:4399/` will have separate pools.
-
-## Entropy injection for S3 file systems
-
-The bundled S3 file systems (`flink-s3-fs-presto` and `flink-s3-fs-hadoop`) support entropy injection. Entropy injection is
-a technique to improve scalability of AWS S3 buckets through adding some random characters near the beginning of the key.
-
-If entropy injection is activated, a configured substring in the paths will be replaced by random characters. For example, path
-`s3://my-bucket/checkpoints/_entropy_/dashboard-job/` would be replaced by something like `s3://my-bucket/checkpoints/gf36ikvg/dashboard-job/`.
-
-**Note that this only happens when the file creation passes the option to inject entropy!**, otherwise the file path will
-simply remove the entropy key substring. See
-[FileSystem.create(Path, WriteOption)](https://ci.apache.org/projects/flink/flink-docs-release-1.6/api/java/org/apache/flink/core/fs/FileSystem.html#create-org.apache.flink.core.fs.Path-org.apache.flink.core.fs.FileSystem.WriteOptions-)
-for details.
-
-*Note: The Flink runtime currently passes the option to inject entropy only to checkpoint data files.*
-*All other files, including checkpoint metadata and external URI do not inject entropy, to keep checkpoint URIs predictable.*
-
-To enable entropy injection, configure the *entropy key* and the *entropy length* parameters.
-
-```
-s3.entropy.key: _entropy_
-s3.entropy.length: 4 (default)
-
-```
+This registers `/path/to/etc/hadoop` as Hadoop's configuration directory and is where Flink will look for the `core-site.xml` and `hdfs-site.xml` files.
 
-The `s3.entropy.key` defines the string in paths that is replaced by the random characters. Paths that do not contain the entropy key are left unchanged.
-If a file system operation does not pass the *"inject entropy"* write option, the entropy key substring is simply removed.
-The `s3.entropy.length` defined the number of random alphanumeric characters to replace the entropy key with.
 
 ## Adding new File System Implementations
 
-File system implementations are discovered by Flink through Java's service abstraction, making it easy to add additional file system implementations.
+File systems are represented via the `org.apache.flink.core.fs.FileSystem` class, which captures the ways to access and modify files and objects in that file system. 
+Implementations are discovered by Flink through Java's service abstraction, making it easy to add new file system implementations.
 
-In order to add a new File System, the following steps are needed:
+To add a new file system:
 
   - Add the File System implementation, which is a subclass of `org.apache.flink.core.fs.FileSystem`.
   - Add a factory that instantiates that file system and declares the scheme under which the FileSystem is registered. This must be a subclass of `org.apache.flink.core.fs.FileSystemFactory`.
diff --git a/docs/ops/filesystems/oss.md b/docs/ops/filesystems/oss.md
index 8711e6f..0c98c43 100644
--- a/docs/ops/filesystems/oss.md
+++ b/docs/ops/filesystems/oss.md
@@ -23,16 +23,15 @@ specific language governing permissions and limitations
 under the License.
 -->
 
-* This will be replaced by the TOC
-{:toc}
-
 ## OSS: Object Storage Service
 
-[Aliyun Object Storage Service](https://www.aliyun.com/product/oss) (Aliyun OSS) is widely used especially among China’s cloud users, and it provides cloud object storage for a variety of use cases.
+[Aliyun Object Storage Service](https://www.aliyun.com/product/oss) (Aliyun OSS) is widely used, particularly popular among China’s cloud users, and it provides cloud object storage for a variety of use cases.
+You can use OSS with Flink for **reading** and **writing data** as well in conjunction with the [streaming **state backends**]({{ site.baseurl }}/ops/state/state_backends.html)
 
-[Hadoop file system](http://hadoop.apache.org/docs/current/hadoop-aliyun/tools/hadoop-aliyun/index.html) supports OSS since version 2.9.1. Now, you can also use OSS with Fink for **reading** and **writing data**.
+* This will be replaced by the TOC
+{:toc}
 
-You can access OSS objects like this:
+You can use OSS objects like regular files by specifying paths in the following format:
 
 {% highlight plain %}
 oss://<your-bucket>/<object-name>
@@ -45,15 +44,15 @@ Below shows how to use OSS with Flink:
 env.readTextFile("oss://<your-bucket>/<object-name>");
 
 // Write to OSS bucket
-dataSet.writeAsText("oss://<your-bucket>/<object-name>")
+stream.writeAsText("oss://<your-bucket>/<object-name>")
 
+// Use OSS as FsStatebackend
+env.setStateBackend(new FsStateBackend("oss://<your-bucket>/<object-name>"));
 {% endhighlight %}
 
-There are two ways to use OSS with Flink, our shaded `flink-oss-fs-hadoop` will cover most scenarios. However, you may need to set up a specific Hadoop OSS FileSystem implementation if you want use OSS as YARN's resource storage dir ([This patch](https://issues.apache.org/jira/browse/HADOOP-15919) enables YARN to use OSS). Both ways are described below.
-
-### Shaded Hadoop OSS file system (recommended)
+### Shaded Hadoop OSS file system 
 
-In order to use `flink-oss-fs-hadoop`, copy the respective JAR file from the opt directory to the lib directory of your Flink distribution before starting Flink, e.g.
+To use `flink-oss-fs-hadoop,` copy the respective JAR file from the opt directory to the lib directory of your Flink distribution before starting Flink, e.g.
 
 {% highlight bash %}
 cp ./opt/flink-oss-fs-hadoop-{{ site.version }}.jar ./lib/
@@ -64,7 +63,7 @@ cp ./opt/flink-oss-fs-hadoop-{{ site.version }}.jar ./lib/
 #### Configurations setup
 After setting up the OSS FileSystem wrapper, you need to add some configurations to make sure that Flink is allowed to access your OSS buckets.
 
-In order to use OSS with Flink more easily, you can use the same configuration keys in `flink-conf.yaml` as in Hadoop's `core-site.xml`
+To allow for easy adoption, you can use the same configuration keys in `flink-conf.yaml` as in Hadoop's `core-site.xml`
 
 You can see the configuration keys in the [Hadoop OSS documentation](http://hadoop.apache.org/docs/current/hadoop-aliyun/tools/hadoop-aliyun/index.html).
 
@@ -76,157 +75,4 @@ fs.oss.accessKeyId: Aliyun access key ID
 fs.oss.accessKeySecret: Aliyun access key secret
 {% endhighlight %}
 
-### Hadoop-provided OSS file system - manual setup
-This setup is a bit more complex and we recommend using our shaded Hadoop file systems instead (see above) unless required otherwise, e.g. for using OSS as YARN’s resource storage dir via the fs.defaultFS configuration property in Hadoop’s core-site.xml.
-
-#### Set OSS FileSystem
-You need to point Flink to a valid Hadoop configuration, which contains the following properties in core-site.xml:
-
-{% highlight xml %}
-<configuration>
-
-<property>
-    <name>fs.oss.impl</name>
-    <value>org.apache.hadoop.fs.aliyun.oss.AliyunOSSFileSystem</value>
-  </property>
-
-  <property>
-    <name>fs.oss.endpoint</name>
-    <value>Aliyun OSS endpoint to connect to</value>
-    <description>Aliyun OSS endpoint to connect to. An up-to-date list is provided in the Aliyun OSS Documentation.</description>
-  </property>
-
-  <property>
-    <name>fs.oss.accessKeyId</name>
-    <description>Aliyun access key ID</description>
-  </property>
-
-  <property>
-    <name>fs.oss.accessKeySecret</name>
-    <description>Aliyun access key secret</description>
-  </property>
-
-  <property>
-    <name>fs.oss.buffer.dir</name>
-    <value>/tmp/oss</value>
-  </property>
-
-</property>
-
-</configuration>
-{% endhighlight %}
-
-#### Hadoop Configuration
-
-You can specify the [Hadoop configuration](../config.html#hdfs) in various ways pointing Flink to
-the path of the Hadoop configuration directory, for example
-- by setting the environment variable `HADOOP_CONF_DIR`, or
-- by setting the `fs.hdfs.hadoopconf` configuration option in `flink-conf.yaml`:
-{% highlight yaml %}
-fs.hdfs.hadoopconf: /path/to/etc/hadoop
-{% endhighlight %}
-
-This registers `/path/to/etc/hadoop` as Hadoop's configuration directory with Flink. Flink will look for the `core-site.xml` and `hdfs-site.xml` files in the specified directory.
-
-#### Provide OSS FileSystem Dependency
-
-You can find Hadoop OSS FileSystem are packaged in the hadoop-aliyun artifact. This JAR and all its dependencies need to be added to Flink’s classpath, i.e. the class path of both Job and TaskManagers.
-
-There are multiple ways of adding JARs to Flink’s class path, the easiest being simply to drop the JARs in Flink’s lib folder. You need to copy the hadoop-aliyun JAR with all its dependencies (You can find these as part of the Hadoop binaries in hadoop-3/share/hadoop/tools/lib). You can also export the directory containing these JARs as part of the HADOOP_CLASSPATH environment variable on all machines.
-
-## An Example
-Below is an example shows the result of our setup (data is generated by TPC-DS tool)
-
-{% highlight java %}
-// Read from OSS bucket
-scala> val dataSet = benv.readTextFile("oss://<your-bucket>/50/call_center/data-m-00049")
-dataSet: org.apache.flink.api.scala.DataSet[String] = org.apache.flink.api.scala.DataSet@31940704
-
-scala> dataSet.print()
-1|AAAAAAAABAAAAAAA|1998-01-01|||2450952|NY Metro|large|2935|1670015|8AM-4PM|Bob Belcher|6|More than other authori|Shared others could not count fully dollars. New members ca|Julius Tran|3|pri|6|cally|730|Ash Hill|Boulevard|Suite 0|Oak Grove|Williamson County|TN|38370|United States|-5|0.11|
-2|AAAAAAAACAAAAAAA|1998-01-01|2000-12-31||2450806|Mid Atlantic|medium|1574|594972|8AM-8AM|Felipe Perkins|2|A bit narrow forms matter animals. Consist|Largely blank years put substantially deaf, new others. Question|Julius Durham|5|anti|1|ought|984|Center Hill|Way|Suite 70|Midway|Williamson County|TN|31904|United States|-5|0.12|
-3|AAAAAAAACAAAAAAA|2001-01-01|||2450806|Mid Atlantic|medium|1574|1084486|8AM-4PM|Mark Hightower|2|Wrong troops shall work sometimes in a opti|Largely blank years put substantially deaf, new others. Question|Julius Durham|1|ought|2|able|984|Center Hill|Way|Suite 70|Midway|Williamson County|TN|31904|United States|-5|0.01|
-4|AAAAAAAAEAAAAAAA|1998-01-01|2000-01-01||2451063|North Midwest|medium|10137|6578913|8AM-4PM|Larry Mccray|2|Dealers make most historical, direct students|Rich groups catch longer other fears; future,|Matthew Clifton|4|ese|3|pri|463|Pine Ridge|RD|Suite U|Five Points|Ziebach County|SD|56098|United States|-6|0.05|
-5|AAAAAAAAEAAAAAAA|2000-01-02|2001-12-31||2451063|North Midwest|small|17398|4610470|8AM-8AM|Larry Mccray|2|Dealers make most historical, direct students|Blue, due beds come. Politicians would not make far thoughts. Specifically new horses partic|Gary Colburn|4|ese|3|pri|463|Pine Ridge|RD|Suite U|Five Points|Ziebach County|SD|56098|United States|-6|0.12|
-6|AAAAAAAAEAAAAAAA|2002-01-01|||2451063|North Midwest|medium|13118|6585236|8AM-4PM|Larry Mccray|5|Silly particles could pro|Blue, due beds come. Politicians would not make far thoughts. Specifically new horses partic|Gary Colburn|5|anti|3|pri|463|Pine Ridge|RD|Suite U|Five Points|Ziebach County|SD|56098|United States|-6|0.11|
-7|AAAAAAAAHAAAAAAA|1998-01-01|||2451024|Pacific Northwest|small|6280|1739560|8AM-4PM|Alden Snyder|6|Major, formal states can suppor|Reduced, subsequent bases could not lik|Frederick Weaver|5|anti|4|ese|415|Jefferson Tenth|Court|Suite 180|Riverside|Walker County|AL|39231|United States|-6|0.00|
-8|AAAAAAAAIAAAAAAA|1998-01-01|2000-12-31||2450808|California|small|4766|2459256|8AM-12AM|Wayne Ray|6|Here possible notions arrive only. Ar|Common, free creditors should exper|Daniel Weller|5|anti|2|able|550|Cedar Elm|Ct.|Suite I|Fairview|Williamson County|TN|35709|United States|-5|0.06|
-
-scala> dataSet.count()
-res0: Long = 8
-
-// Write to OSS bucket
-scala> dataSet.writeAsText("oss://<your-bucket>/50/call_center/data-m-00049.1")
-
-scala> benv.execute("My batch program")
-res1: org.apache.flink.api.common.JobExecutionResult = org.apache.flink.api.common.JobExecutionResult@77476fcf
-
-scala> val newDataSet = benv.readTextFile("oss://<your-bucket>/50/call_center/data-m-00049.1")
-newDataSet: org.apache.flink.api.scala.DataSet[String] = org.apache.flink.api.scala.DataSet@40b70f31
-
-scala> newDataSet.count()
-res2: Long = 8
-
-{% endhighlight %}
-
-## Common Issues
-### Could not find OSS file system
-If your job submission fails with an Exception message like below, please check if our shaded jar (flink-oss-fs-hadoop-{{ site.version }}.jar) is in the lib dir.
-
-{% highlight plain %}
-Caused by: org.apache.flink.runtime.client.JobExecutionException: Could not set up JobManager
-	at org.apache.flink.runtime.jobmaster.JobManagerRunner.<init>(JobManagerRunner.java:176)
-	at org.apache.flink.runtime.dispatcher.Dispatcher$DefaultJobManagerRunnerFactory.createJobManagerRunner(Dispatcher.java:1058)
-	at org.apache.flink.runtime.dispatcher.Dispatcher.lambda$createJobManagerRunner$5(Dispatcher.java:308)
-	at org.apache.flink.util.function.CheckedSupplier.lambda$unchecked$0(CheckedSupplier.java:34)
-	... 7 more
-Caused by: org.apache.flink.runtime.JobException: Creating the input splits caused an error: Could not find a file system implementation for scheme 'oss'. The scheme is not directly supported by Flink and no Hadoop file system to support this scheme could be loaded.
-	at org.apache.flink.runtime.executiongraph.ExecutionJobVertex.<init>(ExecutionJobVertex.java:273)
-	at org.apache.flink.runtime.executiongraph.ExecutionGraph.attachJobGraph(ExecutionGraph.java:827)
-	at org.apache.flink.runtime.executiongraph.ExecutionGraphBuilder.buildGraph(ExecutionGraphBuilder.java:232)
-	at org.apache.flink.runtime.executiongraph.ExecutionGraphBuilder.buildGraph(ExecutionGraphBuilder.java:100)
-	at org.apache.flink.runtime.jobmaster.JobMaster.createExecutionGraph(JobMaster.java:1151)
-	at org.apache.flink.runtime.jobmaster.JobMaster.createAndRestoreExecutionGraph(JobMaster.java:1131)
-	at org.apache.flink.runtime.jobmaster.JobMaster.<init>(JobMaster.java:294)
-	at org.apache.flink.runtime.jobmaster.JobManagerRunner.<init>(JobManagerRunner.java:157)
-	... 10 more
-Caused by: org.apache.flink.core.fs.UnsupportedFileSystemSchemeException: Could not find a file system implementation for scheme 'oss'. The scheme is not directly supported by Flink and no Hadoop file system to support this scheme could be loaded.
-	at org.apache.flink.core.fs.FileSystem.getUnguardedFileSystem(FileSystem.java:403)
-	at org.apache.flink.core.fs.FileSystem.get(FileSystem.java:318)
-	at org.apache.flink.core.fs.Path.getFileSystem(Path.java:298)
-	at org.apache.flink.api.common.io.FileInputFormat.createInputSplits(FileInputFormat.java:587)
-	at org.apache.flink.api.common.io.FileInputFormat.createInputSplits(FileInputFormat.java:62)
-	at org.apache.flink.runtime.executiongraph.ExecutionJobVertex.<init>(ExecutionJobVertex.java:259)
-	... 17 more
-Caused by: org.apache.flink.core.fs.UnsupportedFileSystemSchemeException: Hadoop is not in the classpath/dependencies.
-	at org.apache.flink.core.fs.UnsupportedSchemeFactory.create(UnsupportedSchemeFactory.java:64)
-	at org.apache.flink.core.fs.FileSystem.getUnguardedFileSystem(FileSystem.java:399)
-	... 22 more
-{% endhighlight %}
-
-### Missing configuration(s)
-If your job submission fails with an Exception message like below, please check if the corresponding configurations exits in `flink-conf.yaml`
-
-{% highlight plain %}
-Caused by: org.apache.flink.runtime.JobException: Creating the input splits caused an error: Aliyun OSS endpoint should not be null or empty. Please set proper endpoint with 'fs.oss.endpoint'.
-	at org.apache.flink.runtime.executiongraph.ExecutionJobVertex.<init>(ExecutionJobVertex.java:273)
-	at org.apache.flink.runtime.executiongraph.ExecutionGraph.attachJobGraph(ExecutionGraph.java:827)
-	at org.apache.flink.runtime.executiongraph.ExecutionGraphBuilder.buildGraph(ExecutionGraphBuilder.java:232)
-	at org.apache.flink.runtime.executiongraph.ExecutionGraphBuilder.buildGraph(ExecutionGraphBuilder.java:100)
-	at org.apache.flink.runtime.jobmaster.JobMaster.createExecutionGraph(JobMaster.java:1151)
-	at org.apache.flink.runtime.jobmaster.JobMaster.createAndRestoreExecutionGraph(JobMaster.java:1131)
-	at org.apache.flink.runtime.jobmaster.JobMaster.<init>(JobMaster.java:294)
-	at org.apache.flink.runtime.jobmaster.JobManagerRunner.<init>(JobManagerRunner.java:157)
-	... 10 more
-Caused by: java.lang.IllegalArgumentException: Aliyun OSS endpoint should not be null or empty. Please set proper endpoint with 'fs.oss.endpoint'.
-	at org.apache.flink.fs.shaded.hadoop3.org.apache.hadoop.fs.aliyun.oss.AliyunOSSFileSystemStore.initialize(AliyunOSSFileSystemStore.java:145)
-	at org.apache.flink.fs.shaded.hadoop3.org.apache.hadoop.fs.aliyun.oss.AliyunOSSFileSystem.initialize(AliyunOSSFileSystem.java:323)
-	at org.apache.flink.fs.osshadoop.OSSFileSystemFactory.create(OSSFileSystemFactory.java:87)
-	at org.apache.flink.core.fs.FileSystem.getUnguardedFileSystem(FileSystem.java:395)
-	at org.apache.flink.core.fs.FileSystem.get(FileSystem.java:318)
-	at org.apache.flink.core.fs.Path.getFileSystem(Path.java:298)
-	at org.apache.flink.api.common.io.FileInputFormat.createInputSplits(FileInputFormat.java:587)
-	at org.apache.flink.api.common.io.FileInputFormat.createInputSplits(FileInputFormat.java:62)
-	at org.apache.flink.runtime.executiongraph.ExecutionJobVertex.<init>(ExecutionJobVertex.java:259)
-	... 17 more
-{% endhighlight %}
+{% top %}
diff --git a/docs/ops/filesystems/oss.zh.md b/docs/ops/filesystems/oss.zh.md
index fa2dfb1..d6834d1 100644
--- a/docs/ops/filesystems/oss.zh.md
+++ b/docs/ops/filesystems/oss.zh.md
@@ -23,17 +23,15 @@ specific language governing permissions and limitations
 under the License.
 -->
 
-* ToC
-{:toc}
-
-
 ## OSS: Object Storage Service
 
-[Aliyun Object Storage Service](https://www.aliyun.com/product/oss) (Aliyun OSS) is widely used especially among China’s cloud users, and it provides cloud object storage for a variety of use cases.
+[Aliyun Object Storage Service](https://www.aliyun.com/product/oss) (Aliyun OSS) is widely used, particularly popular among China’s cloud users, and it provides cloud object storage for a variety of use cases.
+You can use OSS with Flink for **reading** and **writing data** as well in conjunction with the [streaming **state backends**]({{ site.baseurl }}/ops/state/state_backends.html)
 
-[Hadoop file system](http://hadoop.apache.org/docs/current/hadoop-aliyun/tools/hadoop-aliyun/index.html) supports OSS since version 2.9.1. Now, you can also use OSS with Fink for **reading** and **writing data**.
+* This will be replaced by the TOC
+{:toc}
 
-You can access OSS objects like this:
+You can use OSS objects like regular files by specifying paths in the following format:
 
 {% highlight plain %}
 oss://<your-bucket>/<object-name>
@@ -46,15 +44,15 @@ Below shows how to use OSS with Flink:
 env.readTextFile("oss://<your-bucket>/<object-name>");
 
 // Write to OSS bucket
-dataSet.writeAsText("oss://<your-bucket>/<object-name>")
+stream.writeAsText("oss://<your-bucket>/<object-name>")
 
+// Use OSS as FsStatebackend
+env.setStateBackend(new FsStateBackend("oss://<your-bucket>/<object-name>"));
 {% endhighlight %}
 
-There are two ways to use OSS with Flink, our shaded `flink-oss-fs-hadoop` will cover most scenarios. However, you may need to set up a specific Hadoop OSS FileSystem implementation if you want use OSS as YARN's resource storage dir ([This patch](https://issues.apache.org/jira/browse/HADOOP-15919) enables YARN to use OSS). Both ways are described below.
-
-### Shaded Hadoop OSS file system (recommended)
+### Shaded Hadoop OSS file system 
 
-In order to use `flink-oss-fs-hadoop`, copy the respective JAR file from the opt directory to the lib directory of your Flink distribution before starting Flink, e.g.
+To use `flink-oss-fs-hadoop,` copy the respective JAR file from the opt directory to the lib directory of your Flink distribution before starting Flink, e.g.
 
 {% highlight bash %}
 cp ./opt/flink-oss-fs-hadoop-{{ site.version }}.jar ./lib/
@@ -65,7 +63,7 @@ cp ./opt/flink-oss-fs-hadoop-{{ site.version }}.jar ./lib/
 #### Configurations setup
 After setting up the OSS FileSystem wrapper, you need to add some configurations to make sure that Flink is allowed to access your OSS buckets.
 
-In order to use OSS with Flink more easily, you can use the same configuration keys in `flink-conf.yaml` as in Hadoop's `core-site.xml`
+To allow for easy adoption, you can use the same configuration keys in `flink-conf.yaml` as in Hadoop's `core-site.xml`
 
 You can see the configuration keys in the [Hadoop OSS documentation](http://hadoop.apache.org/docs/current/hadoop-aliyun/tools/hadoop-aliyun/index.html).
 
@@ -77,157 +75,4 @@ fs.oss.accessKeyId: Aliyun access key ID
 fs.oss.accessKeySecret: Aliyun access key secret
 {% endhighlight %}
 
-### Hadoop-provided OSS file system - manual setup
-This setup is a bit more complex and we recommend using our shaded Hadoop file systems instead (see above) unless required otherwise, e.g. for using OSS as YARN’s resource storage dir via the fs.defaultFS configuration property in Hadoop’s core-site.xml.
-
-#### Set OSS FileSystem
-You need to point Flink to a valid Hadoop configuration, which contains the following properties in core-site.xml:
-
-{% highlight xml %}
-<configuration>
-
-<property>
-    <name>fs.oss.impl</name>
-    <value>org.apache.hadoop.fs.aliyun.oss.AliyunOSSFileSystem</value>
-  </property>
-
-  <property>
-    <name>fs.oss.endpoint</name>
-    <value>Aliyun OSS endpoint to connect to</value>
-    <description>Aliyun OSS endpoint to connect to. An up-to-date list is provided in the Aliyun OSS Documentation.</description>
-  </property>
-
-  <property>
-    <name>fs.oss.accessKeyId</name>
-    <description>Aliyun access key ID</description>
-  </property>
-
-  <property>
-    <name>fs.oss.accessKeySecret</name>
-    <description>Aliyun access key secret</description>
-  </property>
-
-  <property>
-    <name>fs.oss.buffer.dir</name>
-    <value>/tmp/oss</value>
-  </property>
-
-</property>
-
-</configuration>
-{% endhighlight %}
-
-#### Hadoop Configuration
-
-You can specify the [Hadoop configuration](../config.html#hdfs) in various ways pointing Flink to
-the path of the Hadoop configuration directory, for example
-- by setting the environment variable `HADOOP_CONF_DIR`, or
-- by setting the `fs.hdfs.hadoopconf` configuration option in `flink-conf.yaml`:
-{% highlight yaml %}
-fs.hdfs.hadoopconf: /path/to/etc/hadoop
-{% endhighlight %}
-
-This registers `/path/to/etc/hadoop` as Hadoop's configuration directory with Flink. Flink will look for the `core-site.xml` and `hdfs-site.xml` files in the specified directory.
-
-#### Provide OSS FileSystem Dependency
-
-You can find Hadoop OSS FileSystem are packaged in the hadoop-aliyun artifact. This JAR and all its dependencies need to be added to Flink’s classpath, i.e. the class path of both Job and TaskManagers.
-
-There are multiple ways of adding JARs to Flink’s class path, the easiest being simply to drop the JARs in Flink’s lib folder. You need to copy the hadoop-aliyun JAR with all its dependencies (You can find these as part of the Hadoop binaries in hadoop-3/share/hadoop/tools/lib). You can also export the directory containing these JARs as part of the HADOOP_CLASSPATH environment variable on all machines.
-
-## An Example
-Below is an example shows the result of our setup (data is generated by TPC-DS tool)
-
-{% highlight java %}
-// Read from OSS bucket
-scala> val dataSet = benv.readTextFile("oss://<your-bucket>/50/call_center/data-m-00049")
-dataSet: org.apache.flink.api.scala.DataSet[String] = org.apache.flink.api.scala.DataSet@31940704
-
-scala> dataSet.print()
-1|AAAAAAAABAAAAAAA|1998-01-01|||2450952|NY Metro|large|2935|1670015|8AM-4PM|Bob Belcher|6|More than other authori|Shared others could not count fully dollars. New members ca|Julius Tran|3|pri|6|cally|730|Ash Hill|Boulevard|Suite 0|Oak Grove|Williamson County|TN|38370|United States|-5|0.11|
-2|AAAAAAAACAAAAAAA|1998-01-01|2000-12-31||2450806|Mid Atlantic|medium|1574|594972|8AM-8AM|Felipe Perkins|2|A bit narrow forms matter animals. Consist|Largely blank years put substantially deaf, new others. Question|Julius Durham|5|anti|1|ought|984|Center Hill|Way|Suite 70|Midway|Williamson County|TN|31904|United States|-5|0.12|
-3|AAAAAAAACAAAAAAA|2001-01-01|||2450806|Mid Atlantic|medium|1574|1084486|8AM-4PM|Mark Hightower|2|Wrong troops shall work sometimes in a opti|Largely blank years put substantially deaf, new others. Question|Julius Durham|1|ought|2|able|984|Center Hill|Way|Suite 70|Midway|Williamson County|TN|31904|United States|-5|0.01|
-4|AAAAAAAAEAAAAAAA|1998-01-01|2000-01-01||2451063|North Midwest|medium|10137|6578913|8AM-4PM|Larry Mccray|2|Dealers make most historical, direct students|Rich groups catch longer other fears; future,|Matthew Clifton|4|ese|3|pri|463|Pine Ridge|RD|Suite U|Five Points|Ziebach County|SD|56098|United States|-6|0.05|
-5|AAAAAAAAEAAAAAAA|2000-01-02|2001-12-31||2451063|North Midwest|small|17398|4610470|8AM-8AM|Larry Mccray|2|Dealers make most historical, direct students|Blue, due beds come. Politicians would not make far thoughts. Specifically new horses partic|Gary Colburn|4|ese|3|pri|463|Pine Ridge|RD|Suite U|Five Points|Ziebach County|SD|56098|United States|-6|0.12|
-6|AAAAAAAAEAAAAAAA|2002-01-01|||2451063|North Midwest|medium|13118|6585236|8AM-4PM|Larry Mccray|5|Silly particles could pro|Blue, due beds come. Politicians would not make far thoughts. Specifically new horses partic|Gary Colburn|5|anti|3|pri|463|Pine Ridge|RD|Suite U|Five Points|Ziebach County|SD|56098|United States|-6|0.11|
-7|AAAAAAAAHAAAAAAA|1998-01-01|||2451024|Pacific Northwest|small|6280|1739560|8AM-4PM|Alden Snyder|6|Major, formal states can suppor|Reduced, subsequent bases could not lik|Frederick Weaver|5|anti|4|ese|415|Jefferson Tenth|Court|Suite 180|Riverside|Walker County|AL|39231|United States|-6|0.00|
-8|AAAAAAAAIAAAAAAA|1998-01-01|2000-12-31||2450808|California|small|4766|2459256|8AM-12AM|Wayne Ray|6|Here possible notions arrive only. Ar|Common, free creditors should exper|Daniel Weller|5|anti|2|able|550|Cedar Elm|Ct.|Suite I|Fairview|Williamson County|TN|35709|United States|-5|0.06|
-
-scala> dataSet.count()
-res0: Long = 8
-
-// Write to OSS bucket
-scala> dataSet.writeAsText("oss://<your-bucket>/50/call_center/data-m-00049.1")
-
-scala> benv.execute("My batch program")
-res1: org.apache.flink.api.common.JobExecutionResult = org.apache.flink.api.common.JobExecutionResult@77476fcf
-
-scala> val newDataSet = benv.readTextFile("oss://<your-bucket>/50/call_center/data-m-00049.1")
-newDataSet: org.apache.flink.api.scala.DataSet[String] = org.apache.flink.api.scala.DataSet@40b70f31
-
-scala> newDataSet.count()
-res2: Long = 8
-
-{% endhighlight %}
-
-## Common Issues
-### Could not find OSS file system
-If your job submission fails with an Exception message like below, please check if our shaded jar (flink-oss-fs-hadoop-{{ site.version }}.jar) is in the lib dir.
-
-{% highlight plain %}
-Caused by: org.apache.flink.runtime.client.JobExecutionException: Could not set up JobManager
-	at org.apache.flink.runtime.jobmaster.JobManagerRunner.<init>(JobManagerRunner.java:176)
-	at org.apache.flink.runtime.dispatcher.Dispatcher$DefaultJobManagerRunnerFactory.createJobManagerRunner(Dispatcher.java:1058)
-	at org.apache.flink.runtime.dispatcher.Dispatcher.lambda$createJobManagerRunner$5(Dispatcher.java:308)
-	at org.apache.flink.util.function.CheckedSupplier.lambda$unchecked$0(CheckedSupplier.java:34)
-	... 7 more
-Caused by: org.apache.flink.runtime.JobException: Creating the input splits caused an error: Could not find a file system implementation for scheme 'oss'. The scheme is not directly supported by Flink and no Hadoop file system to support this scheme could be loaded.
-	at org.apache.flink.runtime.executiongraph.ExecutionJobVertex.<init>(ExecutionJobVertex.java:273)
-	at org.apache.flink.runtime.executiongraph.ExecutionGraph.attachJobGraph(ExecutionGraph.java:827)
-	at org.apache.flink.runtime.executiongraph.ExecutionGraphBuilder.buildGraph(ExecutionGraphBuilder.java:232)
-	at org.apache.flink.runtime.executiongraph.ExecutionGraphBuilder.buildGraph(ExecutionGraphBuilder.java:100)
-	at org.apache.flink.runtime.jobmaster.JobMaster.createExecutionGraph(JobMaster.java:1151)
-	at org.apache.flink.runtime.jobmaster.JobMaster.createAndRestoreExecutionGraph(JobMaster.java:1131)
-	at org.apache.flink.runtime.jobmaster.JobMaster.<init>(JobMaster.java:294)
-	at org.apache.flink.runtime.jobmaster.JobManagerRunner.<init>(JobManagerRunner.java:157)
-	... 10 more
-Caused by: org.apache.flink.core.fs.UnsupportedFileSystemSchemeException: Could not find a file system implementation for scheme 'oss'. The scheme is not directly supported by Flink and no Hadoop file system to support this scheme could be loaded.
-	at org.apache.flink.core.fs.FileSystem.getUnguardedFileSystem(FileSystem.java:403)
-	at org.apache.flink.core.fs.FileSystem.get(FileSystem.java:318)
-	at org.apache.flink.core.fs.Path.getFileSystem(Path.java:298)
-	at org.apache.flink.api.common.io.FileInputFormat.createInputSplits(FileInputFormat.java:587)
-	at org.apache.flink.api.common.io.FileInputFormat.createInputSplits(FileInputFormat.java:62)
-	at org.apache.flink.runtime.executiongraph.ExecutionJobVertex.<init>(ExecutionJobVertex.java:259)
-	... 17 more
-Caused by: org.apache.flink.core.fs.UnsupportedFileSystemSchemeException: Hadoop is not in the classpath/dependencies.
-	at org.apache.flink.core.fs.UnsupportedSchemeFactory.create(UnsupportedSchemeFactory.java:64)
-	at org.apache.flink.core.fs.FileSystem.getUnguardedFileSystem(FileSystem.java:399)
-	... 22 more
-{% endhighlight %}
-
-### Missing configuration(s)
-If your job submission fails with an Exception message like below, please check if the corresponding configurations exits in `flink-conf.yaml`
-
-{% highlight plain %}
-Caused by: org.apache.flink.runtime.JobException: Creating the input splits caused an error: Aliyun OSS endpoint should not be null or empty. Please set proper endpoint with 'fs.oss.endpoint'.
-	at org.apache.flink.runtime.executiongraph.ExecutionJobVertex.<init>(ExecutionJobVertex.java:273)
-	at org.apache.flink.runtime.executiongraph.ExecutionGraph.attachJobGraph(ExecutionGraph.java:827)
-	at org.apache.flink.runtime.executiongraph.ExecutionGraphBuilder.buildGraph(ExecutionGraphBuilder.java:232)
-	at org.apache.flink.runtime.executiongraph.ExecutionGraphBuilder.buildGraph(ExecutionGraphBuilder.java:100)
-	at org.apache.flink.runtime.jobmaster.JobMaster.createExecutionGraph(JobMaster.java:1151)
-	at org.apache.flink.runtime.jobmaster.JobMaster.createAndRestoreExecutionGraph(JobMaster.java:1131)
-	at org.apache.flink.runtime.jobmaster.JobMaster.<init>(JobMaster.java:294)
-	at org.apache.flink.runtime.jobmaster.JobManagerRunner.<init>(JobManagerRunner.java:157)
-	... 10 more
-Caused by: java.lang.IllegalArgumentException: Aliyun OSS endpoint should not be null or empty. Please set proper endpoint with 'fs.oss.endpoint'.
-	at org.apache.flink.fs.shaded.hadoop3.org.apache.hadoop.fs.aliyun.oss.AliyunOSSFileSystemStore.initialize(AliyunOSSFileSystemStore.java:145)
-	at org.apache.flink.fs.shaded.hadoop3.org.apache.hadoop.fs.aliyun.oss.AliyunOSSFileSystem.initialize(AliyunOSSFileSystem.java:323)
-	at org.apache.flink.fs.osshadoop.OSSFileSystemFactory.create(OSSFileSystemFactory.java:87)
-	at org.apache.flink.core.fs.FileSystem.getUnguardedFileSystem(FileSystem.java:395)
-	at org.apache.flink.core.fs.FileSystem.get(FileSystem.java:318)
-	at org.apache.flink.core.fs.Path.getFileSystem(Path.java:298)
-	at org.apache.flink.api.common.io.FileInputFormat.createInputSplits(FileInputFormat.java:587)
-	at org.apache.flink.api.common.io.FileInputFormat.createInputSplits(FileInputFormat.java:62)
-	at org.apache.flink.runtime.executiongraph.ExecutionJobVertex.<init>(ExecutionJobVertex.java:259)
-	... 17 more
-{% endhighlight %}
+{% top %}
diff --git a/docs/ops/filesystems/s3.md b/docs/ops/filesystems/s3.md
index b668ac9..f25e266 100644
--- a/docs/ops/filesystems/s3.md
+++ b/docs/ops/filesystems/s3.md
@@ -49,31 +49,27 @@ env.setStateBackend(new FsStateBackend("s3://<your-bucket>/<endpoint>"));
 
 Note that these examples are *not* exhaustive and you can use S3 in other places as well, including your [high availability setup](../jobmanager_high_availability.html) or the [RocksDBStateBackend]({{ site.baseurl }}/ops/state/state_backends.html#the-rocksdbstatebackend); everywhere that Flink expects a FileSystem URI.
 
-For most use cases, you may use one of our shaded `flink-s3-fs-hadoop` and `flink-s3-fs-presto` S3
-filesystem wrappers which are fairly easy to set up. For some cases, however, e.g. for using S3 as
-YARN's resource storage dir, it may be necessary to set up a specific Hadoop S3 FileSystem
-implementation. Both ways are described below.
+For most use cases, you may use one of our shaded `flink-s3-fs-hadoop` and `flink-s3-fs-presto` S3 filesystem wrappers which are self-contained and easy to set up.
+For some cases, however, e.g., for using S3 as YARN's resource storage dir, it may be necessary to set up a specific [Hadoop S3 FileSystem](../deployment/aws.html#hadoop-provided-s3-file-systems) implementation.
 
-### Shaded Hadoop/Presto S3 file systems
+### Shaded Hadoop/Presto S3 File Systems
 
 {% panel **Note:** You don't have to configure this manually if you are running [Flink on EMR](../deployment/aws.html). %}
 
 Flink provides two file systems to talk to Amazon S3, `flink-s3-fs-presto` and `flink-s3-fs-hadoop`.
-Both implementations are self-contained with no dependency footprint.
-There is no need to add Hadoop to the classpath to use them.
-Both internally use some Hadoop code, but "shade away" all classes to avoid any dependency conflicts.
+Both implementations are self-contained with no dependency footprint, so there is no need to add Hadoop to the classpath to use them.
 
   - `flink-s3-fs-presto`, registered under the scheme *"s3://"* and *"s3p://"*, is based on code from the [Presto project](https://prestodb.io/).
-  You can configure it the same way you can [configure the Presto file system](https://prestodb.io/docs/0.187/connector/hive.html#amazon-s3-configuration) by placing adding the configurations to your `flink-conf.yaml`.
+  You can configure it the same way you can [configure the Presto file system](https://prestodb.io/docs/0.187/connector/hive.html#amazon-s3-configuration) by placing adding the configurations to your `flink-conf.yaml`. Presto is the recommended file system for checkpointing to S3.
       
   - `flink-s3-fs-hadoop`, registered under *"s3://"* and *"s3a://"*, based on code from the [Hadoop Project](https://hadoop.apache.org/).
-  The file system can be [configured exactly like Hadoop's s3a](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/index.html#S3A) by placing adding the configurations to your `flink-conf.yaml`.
+  The file system can be [configured exactly like Hadoop's s3a](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/index.html#S3A) by placing adding the configurations to your `flink-conf.yaml`. Shaded Hadoop is the only S3 file system with support for the [StreamingFileSink]({{ site.baseurl}}/dev/connectors/streamfile_sink.html).
     
 Both `flink-s3-fs-hadoop` and `flink-s3-fs-presto` register default FileSystem
 wrappers for URIs with the `s3://` scheme, `flink-s3-fs-hadoop` also registers
 for `s3a://` and `flink-s3-fs-presto` also registers for `s3p://`, so you can
 use this to use both at the same time.
-This can happen when, for example, the job uses the [StreamingFileSink]({{ site.baseurl}}/dev/connectors/streamfile_sink.html) which only supports Hadoop, but uses Presto for checkpointing.
+For example, the job uses the [StreamingFileSink]({{ site.baseurl}}/dev/connectors/streamfile_sink.html) which only supports Hadoop, but uses Presto for checkpointing.
 In this case, it is advised to use explicitly *"s3a://"* as a scheme for the sink (Hadoop) and *"s3p://"* for checkpointing (Presto).
     
 To use either `flink-s3-fs-hadoop` or `flink-s3-fs-presto`, copy the respective JAR file from the `opt` directory to the `lib` directory of your Flink distribution before starting Flink, e.g.
@@ -88,7 +84,7 @@ After setting up the S3 FileSystem wrapper, you need to make sure that Flink is
 
 ##### Identity and Access Management (IAM) (Recommended)
 
-The recommended way of setting up credentials on AWS is via [Identity and Access Management (IAM)](http://docs.aws.amazon.com/IAM/latest/UserGuide/introduction.html). You can use IAM features to securely give Flink instances the credentials that they need in order to access S3 buckets. Details about how to do this are beyond the scope of this documentation. Please refer to the AWS user guide. What you are looking for are [IAM Roles](http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/iam- [...]
+The recommended way of setting up credentials on AWS is via [Identity and Access Management (IAM)](http://docs.aws.amazon.com/IAM/latest/UserGuide/introduction.html). You can use IAM features to securely give Flink instances the credentials that they need to access S3 buckets. Details about how to do this are beyond the scope of this documentation. Please refer to the AWS user guide. What you are looking for are [IAM Roles](http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/iam-roles-for [...]
 
 If you set this up correctly, you can manage access to S3 within AWS and don't need to distribute any access keys to Flink.
 
@@ -106,7 +102,7 @@ s3.secret-key: your-secret-key
 ## Configure Non-S3 Endpoint
 
 The S3 Filesystems also support using S3 compliant object stores such as [IBM's Cloud Object Storage](https://www.ibm.com/cloud/object-storage) and [Minio](https://min.io/).
-To do so, simply configure your endpoint in `flink-conf.yaml`. 
+To do so, configure your endpoint in `flink-conf.yaml`. 
 
 {% highlight yaml %}
 s3.endpoint: your-endpoint-hostname
@@ -115,18 +111,15 @@ s3.endpoint: your-endpoint-hostname
 ## Entropy injection for S3 file systems
 
 The bundled S3 file systems (`flink-s3-fs-presto` and `flink-s3-fs-hadoop`) support entropy injection. Entropy injection is
-a technique to improve scalability of AWS S3 buckets through adding some random characters near the beginning of the key.
+a technique to improve the scalability of AWS S3 buckets through adding some random characters near the beginning of the key.
 
-If entropy injection is activated, a configured substring in the paths will be replaced by random characters. For example, path
+If entropy injection is activated, a configured substring in the path is replaced with random characters. For example, path
 `s3://my-bucket/checkpoints/_entropy_/dashboard-job/` would be replaced by something like `s3://my-bucket/checkpoints/gf36ikvg/dashboard-job/`.
-
-**Note that this only happens when the file creation passes the option to inject entropy!**, otherwise the file path will
-simply remove the entropy key substring. See
-[FileSystem.create(Path, WriteOption)](https://ci.apache.org/projects/flink/flink-docs-release-1.6/api/java/org/apache/flink/core/fs/FileSystem.html#create-org.apache.flink.core.fs.Path-org.apache.flink.core.fs.FileSystem.WriteOptions-)
+**This only happens when the file creation passes the option to inject entropy!**
+Otherwise, the file path removes the entropy key substring entirely. See [FileSystem.create(Path, WriteOption)](https://ci.apache.org/projects/flink/flink-docs-release-1.6/api/java/org/apache/flink/core/fs/FileSystem.html#create-org.apache.flink.core.fs.Path-org.apache.flink.core.fs.FileSystem.WriteOptions-)
 for details.
 
-*Note: The Flink runtime currently passes the option to inject entropy only to checkpoint data files.*
-*All other files, including checkpoint metadata and external URI do not inject entropy, to keep checkpoint URIs predictable.*
+{% panel **Note:** The Flink runtime currently passes the option to inject entropy only to checkpoint data files. All other files, including checkpoint metadata and external URI, do not inject entropy to keep checkpoint URIs predictable. %}
 
 To enable entropy injection, configure the *entropy key* and the *entropy length* parameters.
 
@@ -138,4 +131,6 @@ s3.entropy.length: 4 (default)
 
 The `s3.entropy.key` defines the string in paths that is replaced by the random characters. Paths that do not contain the entropy key are left unchanged.
 If a file system operation does not pass the *"inject entropy"* write option, the entropy key substring is simply removed.
-The `s3.entropy.length` defined the number of random alphanumeric characters to replace the entropy key with.
\ No newline at end of file
+The `s3.entropy.length` defines the number of random alphanumeric characters used for entropy.
+
+{% top %}
\ No newline at end of file
diff --git a/docs/ops/filesystems/s3.zh.md b/docs/ops/filesystems/s3.zh.md
index b668ac9..f25e266 100644
--- a/docs/ops/filesystems/s3.zh.md
+++ b/docs/ops/filesystems/s3.zh.md
@@ -49,31 +49,27 @@ env.setStateBackend(new FsStateBackend("s3://<your-bucket>/<endpoint>"));
 
 Note that these examples are *not* exhaustive and you can use S3 in other places as well, including your [high availability setup](../jobmanager_high_availability.html) or the [RocksDBStateBackend]({{ site.baseurl }}/ops/state/state_backends.html#the-rocksdbstatebackend); everywhere that Flink expects a FileSystem URI.
 
-For most use cases, you may use one of our shaded `flink-s3-fs-hadoop` and `flink-s3-fs-presto` S3
-filesystem wrappers which are fairly easy to set up. For some cases, however, e.g. for using S3 as
-YARN's resource storage dir, it may be necessary to set up a specific Hadoop S3 FileSystem
-implementation. Both ways are described below.
+For most use cases, you may use one of our shaded `flink-s3-fs-hadoop` and `flink-s3-fs-presto` S3 filesystem wrappers which are self-contained and easy to set up.
+For some cases, however, e.g., for using S3 as YARN's resource storage dir, it may be necessary to set up a specific [Hadoop S3 FileSystem](../deployment/aws.html#hadoop-provided-s3-file-systems) implementation.
 
-### Shaded Hadoop/Presto S3 file systems
+### Shaded Hadoop/Presto S3 File Systems
 
 {% panel **Note:** You don't have to configure this manually if you are running [Flink on EMR](../deployment/aws.html). %}
 
 Flink provides two file systems to talk to Amazon S3, `flink-s3-fs-presto` and `flink-s3-fs-hadoop`.
-Both implementations are self-contained with no dependency footprint.
-There is no need to add Hadoop to the classpath to use them.
-Both internally use some Hadoop code, but "shade away" all classes to avoid any dependency conflicts.
+Both implementations are self-contained with no dependency footprint, so there is no need to add Hadoop to the classpath to use them.
 
   - `flink-s3-fs-presto`, registered under the scheme *"s3://"* and *"s3p://"*, is based on code from the [Presto project](https://prestodb.io/).
-  You can configure it the same way you can [configure the Presto file system](https://prestodb.io/docs/0.187/connector/hive.html#amazon-s3-configuration) by placing adding the configurations to your `flink-conf.yaml`.
+  You can configure it the same way you can [configure the Presto file system](https://prestodb.io/docs/0.187/connector/hive.html#amazon-s3-configuration) by placing adding the configurations to your `flink-conf.yaml`. Presto is the recommended file system for checkpointing to S3.
       
   - `flink-s3-fs-hadoop`, registered under *"s3://"* and *"s3a://"*, based on code from the [Hadoop Project](https://hadoop.apache.org/).
-  The file system can be [configured exactly like Hadoop's s3a](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/index.html#S3A) by placing adding the configurations to your `flink-conf.yaml`.
+  The file system can be [configured exactly like Hadoop's s3a](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/index.html#S3A) by placing adding the configurations to your `flink-conf.yaml`. Shaded Hadoop is the only S3 file system with support for the [StreamingFileSink]({{ site.baseurl}}/dev/connectors/streamfile_sink.html).
     
 Both `flink-s3-fs-hadoop` and `flink-s3-fs-presto` register default FileSystem
 wrappers for URIs with the `s3://` scheme, `flink-s3-fs-hadoop` also registers
 for `s3a://` and `flink-s3-fs-presto` also registers for `s3p://`, so you can
 use this to use both at the same time.
-This can happen when, for example, the job uses the [StreamingFileSink]({{ site.baseurl}}/dev/connectors/streamfile_sink.html) which only supports Hadoop, but uses Presto for checkpointing.
+For example, the job uses the [StreamingFileSink]({{ site.baseurl}}/dev/connectors/streamfile_sink.html) which only supports Hadoop, but uses Presto for checkpointing.
 In this case, it is advised to use explicitly *"s3a://"* as a scheme for the sink (Hadoop) and *"s3p://"* for checkpointing (Presto).
     
 To use either `flink-s3-fs-hadoop` or `flink-s3-fs-presto`, copy the respective JAR file from the `opt` directory to the `lib` directory of your Flink distribution before starting Flink, e.g.
@@ -88,7 +84,7 @@ After setting up the S3 FileSystem wrapper, you need to make sure that Flink is
 
 ##### Identity and Access Management (IAM) (Recommended)
 
-The recommended way of setting up credentials on AWS is via [Identity and Access Management (IAM)](http://docs.aws.amazon.com/IAM/latest/UserGuide/introduction.html). You can use IAM features to securely give Flink instances the credentials that they need in order to access S3 buckets. Details about how to do this are beyond the scope of this documentation. Please refer to the AWS user guide. What you are looking for are [IAM Roles](http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/iam- [...]
+The recommended way of setting up credentials on AWS is via [Identity and Access Management (IAM)](http://docs.aws.amazon.com/IAM/latest/UserGuide/introduction.html). You can use IAM features to securely give Flink instances the credentials that they need to access S3 buckets. Details about how to do this are beyond the scope of this documentation. Please refer to the AWS user guide. What you are looking for are [IAM Roles](http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/iam-roles-for [...]
 
 If you set this up correctly, you can manage access to S3 within AWS and don't need to distribute any access keys to Flink.
 
@@ -106,7 +102,7 @@ s3.secret-key: your-secret-key
 ## Configure Non-S3 Endpoint
 
 The S3 Filesystems also support using S3 compliant object stores such as [IBM's Cloud Object Storage](https://www.ibm.com/cloud/object-storage) and [Minio](https://min.io/).
-To do so, simply configure your endpoint in `flink-conf.yaml`. 
+To do so, configure your endpoint in `flink-conf.yaml`. 
 
 {% highlight yaml %}
 s3.endpoint: your-endpoint-hostname
@@ -115,18 +111,15 @@ s3.endpoint: your-endpoint-hostname
 ## Entropy injection for S3 file systems
 
 The bundled S3 file systems (`flink-s3-fs-presto` and `flink-s3-fs-hadoop`) support entropy injection. Entropy injection is
-a technique to improve scalability of AWS S3 buckets through adding some random characters near the beginning of the key.
+a technique to improve the scalability of AWS S3 buckets through adding some random characters near the beginning of the key.
 
-If entropy injection is activated, a configured substring in the paths will be replaced by random characters. For example, path
+If entropy injection is activated, a configured substring in the path is replaced with random characters. For example, path
 `s3://my-bucket/checkpoints/_entropy_/dashboard-job/` would be replaced by something like `s3://my-bucket/checkpoints/gf36ikvg/dashboard-job/`.
-
-**Note that this only happens when the file creation passes the option to inject entropy!**, otherwise the file path will
-simply remove the entropy key substring. See
-[FileSystem.create(Path, WriteOption)](https://ci.apache.org/projects/flink/flink-docs-release-1.6/api/java/org/apache/flink/core/fs/FileSystem.html#create-org.apache.flink.core.fs.Path-org.apache.flink.core.fs.FileSystem.WriteOptions-)
+**This only happens when the file creation passes the option to inject entropy!**
+Otherwise, the file path removes the entropy key substring entirely. See [FileSystem.create(Path, WriteOption)](https://ci.apache.org/projects/flink/flink-docs-release-1.6/api/java/org/apache/flink/core/fs/FileSystem.html#create-org.apache.flink.core.fs.Path-org.apache.flink.core.fs.FileSystem.WriteOptions-)
 for details.
 
-*Note: The Flink runtime currently passes the option to inject entropy only to checkpoint data files.*
-*All other files, including checkpoint metadata and external URI do not inject entropy, to keep checkpoint URIs predictable.*
+{% panel **Note:** The Flink runtime currently passes the option to inject entropy only to checkpoint data files. All other files, including checkpoint metadata and external URI, do not inject entropy to keep checkpoint URIs predictable. %}
 
 To enable entropy injection, configure the *entropy key* and the *entropy length* parameters.
 
@@ -138,4 +131,6 @@ s3.entropy.length: 4 (default)
 
 The `s3.entropy.key` defines the string in paths that is replaced by the random characters. Paths that do not contain the entropy key are left unchanged.
 If a file system operation does not pass the *"inject entropy"* write option, the entropy key substring is simply removed.
-The `s3.entropy.length` defined the number of random alphanumeric characters to replace the entropy key with.
\ No newline at end of file
+The `s3.entropy.length` defines the number of random alphanumeric characters used for entropy.
+
+{% top %}
\ No newline at end of file


[flink] 01/14: [FLINK-12378][docs] Consolidate FileSystem Documentation

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

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

commit b50896a55288b363fd369368c0823abbe99aa36c
Author: Seth Wiesman <sj...@gmail.com>
AuthorDate: Tue Apr 30 16:45:49 2019 -0500

    [FLINK-12378][docs] Consolidate FileSystem Documentation
---
 docs/ops/deployment/aws.md                         |  74 +--------
 docs/ops/filesystems.md                            | 177 ---------------------
 docs/ops/filesystems/common.md                     |  68 ++++++++
 docs/ops/filesystems/common.zh.md                  |  68 ++++++++
 docs/ops/filesystems/index.md                      |  83 ++++++++++
 .../{filesystems.zh.md => filesystems/index.zh.md} |   2 +
 docs/ops/{deployment => filesystems}/oss.md        |   7 +-
 docs/ops/{deployment => filesystems}/oss.zh.md     |   4 +-
 docs/ops/filesystems/s3.md                         | 134 ++++++++++++++++
 docs/ops/filesystems/s3.zh.md                      | 134 ++++++++++++++++
 docs/redirects/filesystems.md                      |  24 +++
 docs/redirects/oss.md                              |  24 +++
 12 files changed, 544 insertions(+), 255 deletions(-)

diff --git a/docs/ops/deployment/aws.md b/docs/ops/deployment/aws.md
index ae43c40..a639e9c 100644
--- a/docs/ops/deployment/aws.md
+++ b/docs/ops/deployment/aws.md
@@ -64,81 +64,11 @@ HADOOP_CONF_DIR=/etc/hadoop/conf ./bin/flink run -m yarn-cluster -yn 1 examples/
 
 {% top %}
 
-## S3: Simple Storage Service
-
-[Amazon Simple Storage Service](http://aws.amazon.com/s3/) (Amazon S3) provides cloud object storage for a variety of use cases. You can use S3 with Flink for **reading** and **writing data** as well in conjunction with the [streaming **state backends**]({{ site.baseurl}}/ops/state/state_backends.html) or even as a YARN object storage.
-
-You can use S3 objects like regular files by specifying paths in the following format:
-
-{% highlight plain %}
-s3://<your-bucket>/<endpoint>
-{% endhighlight %}
-
-The endpoint can either be a single file or a directory, for example:
-
-{% highlight java %}
-// Read from S3 bucket
-env.readTextFile("s3://<bucket>/<endpoint>");
-
-// Write to S3 bucket
-stream.writeAsText("s3://<bucket>/<endpoint>");
-
-// Use S3 as FsStatebackend
-env.setStateBackend(new FsStateBackend("s3://<your-bucket>/<endpoint>"));
-{% endhighlight %}
-
-Note that these examples are *not* exhaustive and you can use S3 in other places as well, including your [high availability setup](../jobmanager_high_availability.html) or the [RocksDBStateBackend]({{ site.baseurl }}/ops/state/state_backends.html#the-rocksdbstatebackend); everywhere that Flink expects a FileSystem URI.
-
-For most use cases, you may use one of our shaded `flink-s3-fs-hadoop` and `flink-s3-fs-presto` S3
-filesystem wrappers which are fairly easy to set up. For some cases, however, e.g. for using S3 as
-YARN's resource storage dir, it may be necessary to set up a specific Hadoop S3 FileSystem
-implementation. Both ways are described below.
-
-### Shaded Hadoop/Presto S3 file systems (recommended)
-
-{% panel **Note:** You don't have to configure this manually if you are running [Flink on EMR](#emr-elastic-mapreduce). %}
-
-To use either `flink-s3-fs-hadoop` or `flink-s3-fs-presto`, copy the respective JAR file from the
-`opt` directory to the `lib` directory of your Flink distribution before starting Flink, e.g.
-
-{% highlight bash %}
-cp ./opt/flink-s3-fs-presto-{{ site.version }}.jar ./lib/
-{% endhighlight %}
-
-Both `flink-s3-fs-hadoop` and `flink-s3-fs-presto` register default FileSystem
-wrappers for URIs with the `s3://` scheme, `flink-s3-fs-hadoop` also registers
-for `s3a://` and `flink-s3-fs-presto` also registers for `s3p://`, so you can
-use this to use both at the same time.
-
-#### Configure Access Credentials
-
-After setting up the S3 FileSystem wrapper, you need to make sure that Flink is allowed to access your S3 buckets.
-
-##### Identity and Access Management (IAM) (Recommended)
-
-The recommended way of setting up credentials on AWS is via [Identity and Access Management (IAM)](http://docs.aws.amazon.com/IAM/latest/UserGuide/introduction.html). You can use IAM features to securely give Flink instances the credentials that they need in order to access S3 buckets. Details about how to do this are beyond the scope of this documentation. Please refer to the AWS user guide. What you are looking for are [IAM Roles](http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/iam- [...]
-
-If you set this up correctly, you can manage access to S3 within AWS and don't need to distribute any access keys to Flink.
-
-##### Access Keys (Discouraged)
-
-Access to S3 can be granted via your **access and secret key pair**. Please note that this is discouraged since the [introduction of IAM roles](https://blogs.aws.amazon.com/security/post/Tx1XG3FX6VMU6O5/A-safer-way-to-distribute-AWS-credentials-to-EC2).
-
-You need to configure both `s3.access-key` and `s3.secret-key`  in Flink's  `flink-conf.yaml`:
-
-{% highlight yaml %}
-s3.access-key: your-access-key
-s3.secret-key: your-secret-key
-{% endhighlight %}
-
-{% top %}
-
-### Hadoop-provided S3 file systems - manual setup
+### Hadoop-provided S3 file systems
 
 {% panel **Note:** You don't have to configure this manually if you are running [Flink on EMR](#emr-elastic-mapreduce). %}
 
-This setup is a bit more complex and we recommend using our shaded Hadoop/Presto file systems
-instead (see above) unless required otherwise, e.g. for using S3 as YARN's resource storage dir
+Apache Flink provides native [S3 FileSystem's](../filesystems/s3.html) out of the box and we recomend using them unless required otherwise, e.g. for using S3 as YARN's resource storage dir
 via the `fs.defaultFS` configuration property in Hadoop's `core-site.xml`.
 
 #### Set S3 FileSystem
diff --git a/docs/ops/filesystems.md b/docs/ops/filesystems.md
deleted file mode 100644
index 025255b..0000000
--- a/docs/ops/filesystems.md
+++ /dev/null
@@ -1,177 +0,0 @@
----
-title: "File Systems"
-nav-parent_id: ops
-nav-pos: 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
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
--->
-
-This page provides details on setting up and configuring different file systems for use with Flink.
-We start by describing how to use and configure the different file systems that are supported by Flink
-out-of-the-box, before describing the necessary steps in order to add support about other/custom file system
-implementations.
-
-## Flink's File System support
-
-Flink uses file systems both as *sources* and *sinks* in streaming/batch applications and as a target for *checkpointing*.
-These file systems can for example be *Unix/Windows file systems*, *HDFS*, or even object stores like *S3*.
-
-The file system used for a specific file is determined by the file URI's scheme. For example `file:///home/user/text.txt` refers to
-a file in the local file system, while `hdfs://namenode:50010/data/user/text.txt` refers to a file in a specific HDFS cluster.
-
-File systems are represented via the `org.apache.flink.core.fs.FileSystem` class, which captures the ways to access and modify
-files and objects in that file system. FileSystem instances are instantiated once per process and then cached / pooled, to
-avoid configuration overhead per stream creation and to enforce certain constraints, such as connection/stream limits.
-
-### Built-in File Systems
-
-Flink ships with support for most of the popular file systems, namely *local*, *hadoop-compatible*, *S3*, *MapR FS*
-and *OpenStack Swift FS*. Each of these is identified by the scheme included in the URI of the provide file path. 
-
-Flink ships with implementations for the following file systems:
-
-  - **local**: This file system is used when the scheme is *"file://"*, and it represents the file system of the local machine, 
-including any NFS or SAN that is mounted into that local file system.
-
-  - **S3**: Flink directly provides file systems to talk to Amazon S3. There are two alternative implementations, `flink-s3-fs-presto`
-    and `flink-s3-fs-hadoop`. Both implementations are self-contained with no dependency footprint. There is no need to add Hadoop to
-    the classpath to use them. Both internally use some Hadoop code, but "shade away" all classes to avoid any dependency conflicts.
-
-    - `flink-s3-fs-presto`, registered under the scheme *"s3://"* and *"s3p://"*, is based on code from the [Presto project](https://prestodb.io/).
-      You can configure it the same way you can [configure the Presto file system](https://prestodb.io/docs/0.187/connector/hive.html#amazon-s3-configuration).
-      
-    - `flink-s3-fs-hadoop`, registered under *"s3://"* and *"s3a://"*, based on code from the [Hadoop Project](https://hadoop.apache.org/).
-      The file system can be [configured exactly like Hadoop's s3a](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/index.html#S3A).
-
-    To use those file systems when using Flink as a library, add the respective maven dependency (`org.apache.flink:flink-s3-fs-presto:{{ site.version }}`
-    or `org.apache.flink:flink-s3-fs-hadoop:{{ site.version }}`). When starting a Flink application from the Flink binaries, copy or move
-    the respective jar file from the `opt` folder to the `lib` folder. See also [AWS setup](deployment/aws.html) for additional details.
-    
-    <span class="label label-danger">Attention</span>: As described above, both Hadoop and Presto "listen" to paths with scheme set to *"s3://"*. This is 
-    convenient for switching between implementations (Hadoop or Presto), but it can lead to non-determinism when both
-    implementations are required. This can happen when, for example, the job uses the [StreamingFileSink]({{ site.baseurl}}/dev/connectors/streamfile_sink.html) 
-    which only supports Hadoop, but uses Presto for checkpointing. In this case, it is advised to use explicitly *"s3a://"* 
-    as a scheme for the sink (Hadoop) and *"s3p://"* for checkpointing (Presto).
-    
-  - **MapR FS**: The MapR file system *"maprfs://"* is automatically available when the MapR libraries are in the classpath.
-  
-  - **OpenStack Swift FS**: Flink directly provides a file system to talk to the OpenStack Swift file system, registered under the scheme *"swift://"*. 
-  The implementation `flink-swift-fs-hadoop` is based on the [Hadoop Project](https://hadoop.apache.org/) but is self-contained with no dependency footprint.
-  To use it when using Flink as a library, add the respective maven dependency (`org.apache.flink:flink-swift-fs-hadoop:{{ site.version }}`
-  When starting a Flink application from the Flink binaries, copy or move the respective jar file from the `opt` folder to the `lib` folder.
-
-#### HDFS and Hadoop File System support 
-
-For all schemes where it cannot find a directly supported file system, Flink will try to use Hadoop to instantiate a file system for the respective scheme.
-All Hadoop file systems are automatically available once `flink-runtime` and the Hadoop libraries are in classpath.
-
-That way, Flink seamlessly supports all of Hadoop file systems, and all Hadoop-compatible file systems (HCFS), for example:
-
-  - **hdfs**
-  - **ftp**
-  - **s3n** and **s3a**
-  - **har**
-  - ...
-
-
-## Common File System configurations
-
-The following configuration settings exist across different file systems.
-
-#### Default File System
-
-If paths to files do not explicitly specify a file system scheme (and authority), a default scheme (and authority) will be used.
-
-{% highlight yaml %}
-fs.default-scheme: <default-fs>
-{% endhighlight %}
-
-For example, if the default file system configured as `fs.default-scheme: hdfs://localhost:9000/`, then a file path of
-`/user/hugo/in.txt` is interpreted as `hdfs://localhost:9000/user/hugo/in.txt`.
-
-#### Connection limiting
-
-You can limit the total number of connections that a file system can concurrently open. This is useful when the file system cannot handle a large number
-of concurrent reads / writes or open connections at the same time.
-
-For example, very small HDFS clusters with few RPC handlers can sometimes be overwhelmed by a large Flink job trying to build up many connections during a checkpoint.
-
-To limit a specific file system's connections, add the following entries to the Flink configuration. The file system to be limited is identified by
-its scheme.
-
-{% highlight yaml %}
-fs.<scheme>.limit.total: (number, 0/-1 mean no limit)
-fs.<scheme>.limit.input: (number, 0/-1 mean no limit)
-fs.<scheme>.limit.output: (number, 0/-1 mean no limit)
-fs.<scheme>.limit.timeout: (milliseconds, 0 means infinite)
-fs.<scheme>.limit.stream-timeout: (milliseconds, 0 means infinite)
-{% endhighlight %}
-
-You can limit the number if input/output connections (streams) separately (`fs.<scheme>.limit.input` and `fs.<scheme>.limit.output`), as well as impose a limit on
-the total number of concurrent streams (`fs.<scheme>.limit.total`). If the file system tries to open more streams, the operation will block until some streams are closed.
-If the opening of the stream takes longer than `fs.<scheme>.limit.timeout`, the stream opening will fail.
-
-To prevent inactive streams from taking up the complete pool (preventing new connections to be opened), you can add an inactivity timeout for streams:
-`fs.<scheme>.limit.stream-timeout`. If a stream does not read/write any bytes for at least that amount of time, it is forcibly closed.
-
-These limits are enforced per TaskManager, so each TaskManager in a Flink application or cluster will open up to that number of connections.
-In addition, the limits are also only enforced per FileSystem instance. Because File Systems are created per scheme and authority, different
-authorities will have their own connection pool. For example `hdfs://myhdfs:50010/` and `hdfs://anotherhdfs:4399/` will have separate pools.
-
-## Entropy injection for S3 file systems
-
-The bundled S3 file systems (`flink-s3-fs-presto` and `flink-s3-fs-hadoop`) support entropy injection. Entropy injection is
-a technique to improve scalability of AWS S3 buckets through adding some random characters near the beginning of the key.
-
-If entropy injection is activated, a configured substring in the paths will be replaced by random characters. For example, path
-`s3://my-bucket/checkpoints/_entropy_/dashboard-job/` would be replaced by something like `s3://my-bucket/checkpoints/gf36ikvg/dashboard-job/`.
-
-**Note that this only happens when the file creation passes the option to inject entropy!**, otherwise the file path will
-simply remove the entropy key substring. See
-[FileSystem.create(Path, WriteOption)](https://ci.apache.org/projects/flink/flink-docs-release-1.6/api/java/org/apache/flink/core/fs/FileSystem.html#create-org.apache.flink.core.fs.Path-org.apache.flink.core.fs.FileSystem.WriteOptions-)
-for details.
-
-*Note: The Flink runtime currently passes the option to inject entropy only to checkpoint data files.*
-*All other files, including checkpoint metadata and external URI do not inject entropy, to keep checkpoint URIs predictable.*
-
-To enable entropy injection, configure the *entropy key* and the *entropy length* parameters.
-
-```
-s3.entropy.key: _entropy_
-s3.entropy.length: 4 (default)
-
-```
-
-The `s3.entropy.key` defines the string in paths that is replaced by the random characters. Paths that do not contain the entropy key are left unchanged.
-If a file system operation does not pass the *"inject entropy"* write option, the entropy key substring is simply removed.
-The `s3.entropy.length` defined the number of random alphanumeric characters to replace the entropy key with.
-
-## Adding new File System Implementations
-
-File system implementations are discovered by Flink through Java's service abstraction, making it easy to add additional file system implementations.
-
-In order to add a new File System, the following steps are needed:
-
-  - Add the File System implementation, which is a subclass of `org.apache.flink.core.fs.FileSystem`.
-  - Add a factory that instantiates that file system and declares the scheme under which the FileSystem is registered. This must be a subclass of `org.apache.flink.core.fs.FileSystemFactory`.
-  - Add a service entry. Create a file `META-INF/services/org.apache.flink.core.fs.FileSystemFactory` which contains the class name of your file system factory class.
-
-See the [Java Service Loader docs](https://docs.oracle.com/javase/8/docs/api/java/util/ServiceLoader.html) for more details on how service loaders work.
-
-{% top %}
diff --git a/docs/ops/filesystems/common.md b/docs/ops/filesystems/common.md
new file mode 100644
index 0000000..edef1dd
--- /dev/null
+++ b/docs/ops/filesystems/common.md
@@ -0,0 +1,68 @@
+---
+title: "Common Configurations"
+nav-parent_id: filesystems
+nav-pos: 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.
+-->
+
+Apache Flink provides a number of common configuration settings that work across all file system implementations. 
+
+* This will be replaced by the TOC
+{:toc}
+
+## Default File System
+
+If paths to files do not explicitly specify a file system scheme (and authority), a default scheme (and authority) will be used.
+
+{% highlight yaml %}
+fs.default-scheme: <default-fs>
+{% endhighlight %}
+
+For example, if the default file system configured as `fs.default-scheme: hdfs://localhost:9000/`, then a file path of
+`/user/hugo/in.txt` is interpreted as `hdfs://localhost:9000/user/hugo/in.txt`.
+
+## Connection limiting
+
+You can limit the total number of connections that a file system can concurrently open. This is useful when the file system cannot handle a large number
+of concurrent reads / writes or open connections at the same time.
+
+For example, very small HDFS clusters with few RPC handlers can sometimes be overwhelmed by a large Flink job trying to build up many connections during a checkpoint.
+
+To limit a specific file system's connections, add the following entries to the Flink configuration. The file system to be limited is identified by
+its scheme.
+
+{% highlight yaml %}
+fs.<scheme>.limit.total: (number, 0/-1 mean no limit)
+fs.<scheme>.limit.input: (number, 0/-1 mean no limit)
+fs.<scheme>.limit.output: (number, 0/-1 mean no limit)
+fs.<scheme>.limit.timeout: (milliseconds, 0 means infinite)
+fs.<scheme>.limit.stream-timeout: (milliseconds, 0 means infinite)
+{% endhighlight %}
+
+You can limit the number if input/output connections (streams) separately (`fs.<scheme>.limit.input` and `fs.<scheme>.limit.output`), as well as impose a limit on
+the total number of concurrent streams (`fs.<scheme>.limit.total`). If the file system tries to open more streams, the operation will block until some streams are closed.
+If the opening of the stream takes longer than `fs.<scheme>.limit.timeout`, the stream opening will fail.
+
+To prevent inactive streams from taking up the complete pool (preventing new connections to be opened), you can add an inactivity timeout for streams:
+`fs.<scheme>.limit.stream-timeout`. If a stream does not read/write any bytes for at least that amount of time, it is forcibly closed.
+
+These limits are enforced per TaskManager, so each TaskManager in a Flink application or cluster will open up to that number of connections.
+In addition, the limits are also only enforced per FileSystem instance. Because File Systems are created per scheme and authority, different
+authorities will have their own connection pool. For example `hdfs://myhdfs:50010/` and `hdfs://anotherhdfs:4399/` will have separate pools.
\ No newline at end of file
diff --git a/docs/ops/filesystems/common.zh.md b/docs/ops/filesystems/common.zh.md
new file mode 100644
index 0000000..edef1dd
--- /dev/null
+++ b/docs/ops/filesystems/common.zh.md
@@ -0,0 +1,68 @@
+---
+title: "Common Configurations"
+nav-parent_id: filesystems
+nav-pos: 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.
+-->
+
+Apache Flink provides a number of common configuration settings that work across all file system implementations. 
+
+* This will be replaced by the TOC
+{:toc}
+
+## Default File System
+
+If paths to files do not explicitly specify a file system scheme (and authority), a default scheme (and authority) will be used.
+
+{% highlight yaml %}
+fs.default-scheme: <default-fs>
+{% endhighlight %}
+
+For example, if the default file system configured as `fs.default-scheme: hdfs://localhost:9000/`, then a file path of
+`/user/hugo/in.txt` is interpreted as `hdfs://localhost:9000/user/hugo/in.txt`.
+
+## Connection limiting
+
+You can limit the total number of connections that a file system can concurrently open. This is useful when the file system cannot handle a large number
+of concurrent reads / writes or open connections at the same time.
+
+For example, very small HDFS clusters with few RPC handlers can sometimes be overwhelmed by a large Flink job trying to build up many connections during a checkpoint.
+
+To limit a specific file system's connections, add the following entries to the Flink configuration. The file system to be limited is identified by
+its scheme.
+
+{% highlight yaml %}
+fs.<scheme>.limit.total: (number, 0/-1 mean no limit)
+fs.<scheme>.limit.input: (number, 0/-1 mean no limit)
+fs.<scheme>.limit.output: (number, 0/-1 mean no limit)
+fs.<scheme>.limit.timeout: (milliseconds, 0 means infinite)
+fs.<scheme>.limit.stream-timeout: (milliseconds, 0 means infinite)
+{% endhighlight %}
+
+You can limit the number if input/output connections (streams) separately (`fs.<scheme>.limit.input` and `fs.<scheme>.limit.output`), as well as impose a limit on
+the total number of concurrent streams (`fs.<scheme>.limit.total`). If the file system tries to open more streams, the operation will block until some streams are closed.
+If the opening of the stream takes longer than `fs.<scheme>.limit.timeout`, the stream opening will fail.
+
+To prevent inactive streams from taking up the complete pool (preventing new connections to be opened), you can add an inactivity timeout for streams:
+`fs.<scheme>.limit.stream-timeout`. If a stream does not read/write any bytes for at least that amount of time, it is forcibly closed.
+
+These limits are enforced per TaskManager, so each TaskManager in a Flink application or cluster will open up to that number of connections.
+In addition, the limits are also only enforced per FileSystem instance. Because File Systems are created per scheme and authority, different
+authorities will have their own connection pool. For example `hdfs://myhdfs:50010/` and `hdfs://anotherhdfs:4399/` will have separate pools.
\ No newline at end of file
diff --git a/docs/ops/filesystems/index.md b/docs/ops/filesystems/index.md
new file mode 100644
index 0000000..be8bbee
--- /dev/null
+++ b/docs/ops/filesystems/index.md
@@ -0,0 +1,83 @@
+---
+title: "File Systems"
+nav-id: filesystems
+nav-parent_id: ops
+nav-show_overview: true
+nav-pos: 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
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+Apache Flink uses file system for both ingest and output of data for streaming and batch applications as well as targets for checkpoint storage. 
+These file systems can be local such as *Unix*, distributed like *HDFS*, or even object stores such as *S3*.
+
+The file system used for a particular file is determined by its URI scheme.
+For example, `file:///home/user/text.txt` refers to a file in the local file system, while `hdfs://namenode:50010/data/user/text.txt` is a file in a specific HDFS cluster.
+
+FileSystem instances are instantiated once per process and then cached / pooled, to
+avoid configuration overhead per stream creation and to enforce certain constraints, such as connection/stream limits.
+
+* This will be replaced by the TOC
+{:toc}
+
+### Built-in File Systems
+
+Flink ships with support for most of the popular file systems, including *local*, *hadoop-compatible*, *S3*, *MapR FS* and *OpenStack Swift FS*.
+Each is identified by the scheme included in the URI of the provide file path. 
+
+Flink ships with implementations for the following file systems:
+
+  - **local**: This file system is used when the scheme is *"file://"*, and it represents the file system of the local machine, including any NFS or SAN that is mounted into that local file system.
+
+  - **S3**: Flink directly provides file systems to talk to Amazon S3 with two alternative implementations, `flink-s3-fs-presto` and `flink-s3-fs-hadoop`. Both implementations are self-contained with no dependency footprint.
+    
+  - **MapR FS**: The MapR file system *"maprfs://"* is automatically available when the MapR libraries are in the classpath.
+  
+  - **OpenStack Swift FS**: Flink directly provides a file system to talk to the OpenStack Swift file system, registered under the scheme *"swift://"*. 
+  The implementation `flink-swift-fs-hadoop` is based on the [Hadoop Project](https://hadoop.apache.org/) but is self-contained with no dependency footprint.
+  To use it when using Flink as a library, add the respective maven dependency (`org.apache.flink:flink-swift-fs-hadoop:{{ site.version }}`
+  When starting a Flink application from the Flink binaries, copy or move the respective jar file from the `opt` folder to the `lib` folder.
+
+#### HDFS and Hadoop File System support 
+
+For all schemes where Flink cannot find a directly supported file system, it will fall back to Hadoop.
+All Hadoop file systems are automatically available when `flink-runtime` and the Hadoop libraries are in classpath.
+
+This way, Flink seamlessly supports all of Hadoop file systems, and all Hadoop-compatible file systems (HCFS).
+
+  - **hdfs**
+  - **ftp**
+  - **s3n** and **s3a**
+  - **har**
+  - ...
+
+## Adding new File System Implementations
+
+File systems are represented via the `org.apache.flink.core.fs.FileSystem` class, which captures the ways to access and modify files and objects in that file system. 
+Implementations are discovered by Flink through Java's service abstraction, making it easy to add additional file system implementations.
+
+In order to add a new File System, the following steps are needed:
+
+  - Add the File System implementation, which is a subclass of `org.apache.flink.core.fs.FileSystem`.
+  - Add a factory that instantiates that file system and declares the scheme under which the FileSystem is registered. This must be a subclass of `org.apache.flink.core.fs.FileSystemFactory`.
+  - Add a service entry. Create a file `META-INF/services/org.apache.flink.core.fs.FileSystemFactory` which contains the class name of your file system factory class.
+
+See the [Java Service Loader docs](https://docs.oracle.com/javase/8/docs/api/java/util/ServiceLoader.html) for more details on how service loaders work.
+
+{% top %}
diff --git a/docs/ops/filesystems.zh.md b/docs/ops/filesystems/index.zh.md
similarity index 99%
rename from docs/ops/filesystems.zh.md
rename to docs/ops/filesystems/index.zh.md
index 4df02cc..9390239 100644
--- a/docs/ops/filesystems.zh.md
+++ b/docs/ops/filesystems/index.zh.md
@@ -1,6 +1,8 @@
 ---
 title: "文件系统"
+nav-id: filesystems
 nav-parent_id: ops
+nav-show_overview: true
 nav-pos: 12
 ---
 <!--
diff --git a/docs/ops/deployment/oss.md b/docs/ops/filesystems/oss.md
similarity index 99%
rename from docs/ops/deployment/oss.md
rename to docs/ops/filesystems/oss.md
index b706767..8711e6f 100644
--- a/docs/ops/deployment/oss.md
+++ b/docs/ops/filesystems/oss.md
@@ -1,8 +1,8 @@
 ---
 title: "Aliyun Object Storage Service (OSS)"
 nav-title: Aliyun OSS
-nav-parent_id: deployment
-nav-pos: 9
+nav-parent_id: filesystems
+nav-pos: 2
 ---
 <!--
 Licensed to the Apache Software Foundation (ASF) under one
@@ -23,10 +23,9 @@ specific language governing permissions and limitations
 under the License.
 -->
 
-* ToC
+* This will be replaced by the TOC
 {:toc}
 
-
 ## OSS: Object Storage Service
 
 [Aliyun Object Storage Service](https://www.aliyun.com/product/oss) (Aliyun OSS) is widely used especially among China’s cloud users, and it provides cloud object storage for a variety of use cases.
diff --git a/docs/ops/deployment/oss.zh.md b/docs/ops/filesystems/oss.zh.md
similarity index 99%
rename from docs/ops/deployment/oss.zh.md
rename to docs/ops/filesystems/oss.zh.md
index 356510e..fa2dfb1 100644
--- a/docs/ops/deployment/oss.zh.md
+++ b/docs/ops/filesystems/oss.zh.md
@@ -1,8 +1,8 @@
 ---
 title: "Aliyun 对象存储服务 (OSS)"
 nav-title: Aliyun OSS
-nav-parent_id: deployment
-nav-pos: 9
+nav-parent_id: filesystems
+nav-pos: 2
 ---
 <!--
 Licensed to the Apache Software Foundation (ASF) under one
diff --git a/docs/ops/filesystems/s3.md b/docs/ops/filesystems/s3.md
new file mode 100644
index 0000000..9ae1476
--- /dev/null
+++ b/docs/ops/filesystems/s3.md
@@ -0,0 +1,134 @@
+---
+title: "Amazon S3"
+nav-title: Amazon S3
+nav-parent_id: filesystems
+nav-pos: 1
+---
+<!--
+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.
+-->
+
+[Amazon Simple Storage Service](http://aws.amazon.com/s3/) (Amazon S3) provides cloud object storage for a variety of use cases. You can use S3 with Flink for **reading** and **writing data** as well in conjunction with the [streaming **state backends**]({{ site.baseurl}}/ops/state/state_backends.html).
+
+* This will be replaced by the TOC
+{:toc}
+
+You can use S3 objects like regular files by specifying paths in the following format:
+
+{% highlight plain %}
+s3://<your-bucket>/<endpoint>
+{% endhighlight %}
+
+The endpoint can either be a single file or a directory, for example:
+
+{% highlight java %}
+// Read from S3 bucket
+env.readTextFile("s3://<bucket>/<endpoint>");
+
+// Write to S3 bucket
+stream.writeAsText("s3://<bucket>/<endpoint>");
+
+// Use S3 as FsStatebackend
+env.setStateBackend(new FsStateBackend("s3://<your-bucket>/<endpoint>"));
+{% endhighlight %}
+
+Note that these examples are *not* exhaustive and you can use S3 in other places as well, including your [high availability setup](../jobmanager_high_availability.html) or the [RocksDBStateBackend]({{ site.baseurl }}/ops/state/state_backends.html#the-rocksdbstatebackend); everywhere that Flink expects a FileSystem URI.
+
+For most use cases, you may use one of our shaded `flink-s3-fs-hadoop` and `flink-s3-fs-presto` S3
+filesystem wrappers which are fairly easy to set up. For some cases, however, e.g. for using S3 as
+YARN's resource storage dir, it may be necessary to set up a specific Hadoop S3 FileSystem
+implementation. Both ways are described below.
+
+### Shaded Hadoop/Presto S3 file systems
+
+{% panel **Note:** You don't have to configure this manually if you are running [Flink on EMR](../deployment/aws.html). %}
+
+Flink provides two file systems to talk to Amazon S3, `flink-s3-fs-presto` and `flink-s3-fs-hadoop`.
+Both implementations are self-contained with no dependency footprint.
+There is no need to add Hadoop to the classpath to use them.
+Both internally use some Hadoop code, but "shade away" all classes to avoid any dependency conflicts.
+
+  - `flink-s3-fs-presto`, registered under the scheme *"s3://"* and *"s3p://"*, is based on code from the [Presto project](https://prestodb.io/).
+  You can configure it the same way you can [configure the Presto file system](https://prestodb.io/docs/0.187/connector/hive.html#amazon-s3-configuration).
+      
+  - `flink-s3-fs-hadoop`, registered under *"s3://"* and *"s3a://"*, based on code from the [Hadoop Project](https://hadoop.apache.org/).
+  The file system can be [configured exactly like Hadoop's s3a](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/index.html#S3A).
+    
+Both `flink-s3-fs-hadoop` and `flink-s3-fs-presto` register default FileSystem
+wrappers for URIs with the `s3://` scheme, `flink-s3-fs-hadoop` also registers
+for `s3a://` and `flink-s3-fs-presto` also registers for `s3p://`, so you can
+use this to use both at the same time.
+This can happen when, for example, the job uses the [StreamingFileSink]({{ site.baseurl}}/dev/connectors/streamfile_sink.html) which only supports Hadoop, but uses Presto for checkpointing.
+In this case, it is advised to use explicitly *"s3a://"* as a scheme for the sink (Hadoop) and *"s3p://"* for checkpointing (Presto).
+    
+To use either `flink-s3-fs-hadoop` or `flink-s3-fs-presto`, copy the respective JAR file from the `opt` directory to the `lib` directory of your Flink distribution before starting Flink, e.g.
+
+{% highlight bash %}
+cp ./opt/flink-s3-fs-presto-{{ site.version }}.jar ./lib/
+{% endhighlight %}
+
+#### Configure Access Credentials
+
+After setting up the S3 FileSystem wrapper, you need to make sure that Flink is allowed to access your S3 buckets.
+
+##### Identity and Access Management (IAM) (Recommended)
+
+The recommended way of setting up credentials on AWS is via [Identity and Access Management (IAM)](http://docs.aws.amazon.com/IAM/latest/UserGuide/introduction.html). You can use IAM features to securely give Flink instances the credentials that they need in order to access S3 buckets. Details about how to do this are beyond the scope of this documentation. Please refer to the AWS user guide. What you are looking for are [IAM Roles](http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/iam- [...]
+
+If you set this up correctly, you can manage access to S3 within AWS and don't need to distribute any access keys to Flink.
+
+##### Access Keys (Discouraged)
+
+Access to S3 can be granted via your **access and secret key pair**. Please note that this is discouraged since the [introduction of IAM roles](https://blogs.aws.amazon.com/security/post/Tx1XG3FX6VMU6O5/A-safer-way-to-distribute-AWS-credentials-to-EC2).
+
+You need to configure both `s3.access-key` and `s3.secret-key`  in Flink's  `flink-conf.yaml`:
+
+{% highlight yaml %}
+s3.access-key: your-access-key
+s3.secret-key: your-secret-key
+{% endhighlight %}
+
+{% top %}
+
+## Entropy injection for S3 file systems
+
+The bundled S3 file systems (`flink-s3-fs-presto` and `flink-s3-fs-hadoop`) support entropy injection. Entropy injection is
+a technique to improve scalability of AWS S3 buckets through adding some random characters near the beginning of the key.
+
+If entropy injection is activated, a configured substring in the paths will be replaced by random characters. For example, path
+`s3://my-bucket/checkpoints/_entropy_/dashboard-job/` would be replaced by something like `s3://my-bucket/checkpoints/gf36ikvg/dashboard-job/`.
+
+**Note that this only happens when the file creation passes the option to inject entropy!**, otherwise the file path will
+simply remove the entropy key substring. See
+[FileSystem.create(Path, WriteOption)](https://ci.apache.org/projects/flink/flink-docs-release-1.6/api/java/org/apache/flink/core/fs/FileSystem.html#create-org.apache.flink.core.fs.Path-org.apache.flink.core.fs.FileSystem.WriteOptions-)
+for details.
+
+*Note: The Flink runtime currently passes the option to inject entropy only to checkpoint data files.*
+*All other files, including checkpoint metadata and external URI do not inject entropy, to keep checkpoint URIs predictable.*
+
+To enable entropy injection, configure the *entropy key* and the *entropy length* parameters.
+
+```
+s3.entropy.key: _entropy_
+s3.entropy.length: 4 (default)
+
+```
+
+The `s3.entropy.key` defines the string in paths that is replaced by the random characters. Paths that do not contain the entropy key are left unchanged.
+If a file system operation does not pass the *"inject entropy"* write option, the entropy key substring is simply removed.
+The `s3.entropy.length` defined the number of random alphanumeric characters to replace the entropy key with.
\ No newline at end of file
diff --git a/docs/ops/filesystems/s3.zh.md b/docs/ops/filesystems/s3.zh.md
new file mode 100644
index 0000000..9ae1476
--- /dev/null
+++ b/docs/ops/filesystems/s3.zh.md
@@ -0,0 +1,134 @@
+---
+title: "Amazon S3"
+nav-title: Amazon S3
+nav-parent_id: filesystems
+nav-pos: 1
+---
+<!--
+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.
+-->
+
+[Amazon Simple Storage Service](http://aws.amazon.com/s3/) (Amazon S3) provides cloud object storage for a variety of use cases. You can use S3 with Flink for **reading** and **writing data** as well in conjunction with the [streaming **state backends**]({{ site.baseurl}}/ops/state/state_backends.html).
+
+* This will be replaced by the TOC
+{:toc}
+
+You can use S3 objects like regular files by specifying paths in the following format:
+
+{% highlight plain %}
+s3://<your-bucket>/<endpoint>
+{% endhighlight %}
+
+The endpoint can either be a single file or a directory, for example:
+
+{% highlight java %}
+// Read from S3 bucket
+env.readTextFile("s3://<bucket>/<endpoint>");
+
+// Write to S3 bucket
+stream.writeAsText("s3://<bucket>/<endpoint>");
+
+// Use S3 as FsStatebackend
+env.setStateBackend(new FsStateBackend("s3://<your-bucket>/<endpoint>"));
+{% endhighlight %}
+
+Note that these examples are *not* exhaustive and you can use S3 in other places as well, including your [high availability setup](../jobmanager_high_availability.html) or the [RocksDBStateBackend]({{ site.baseurl }}/ops/state/state_backends.html#the-rocksdbstatebackend); everywhere that Flink expects a FileSystem URI.
+
+For most use cases, you may use one of our shaded `flink-s3-fs-hadoop` and `flink-s3-fs-presto` S3
+filesystem wrappers which are fairly easy to set up. For some cases, however, e.g. for using S3 as
+YARN's resource storage dir, it may be necessary to set up a specific Hadoop S3 FileSystem
+implementation. Both ways are described below.
+
+### Shaded Hadoop/Presto S3 file systems
+
+{% panel **Note:** You don't have to configure this manually if you are running [Flink on EMR](../deployment/aws.html). %}
+
+Flink provides two file systems to talk to Amazon S3, `flink-s3-fs-presto` and `flink-s3-fs-hadoop`.
+Both implementations are self-contained with no dependency footprint.
+There is no need to add Hadoop to the classpath to use them.
+Both internally use some Hadoop code, but "shade away" all classes to avoid any dependency conflicts.
+
+  - `flink-s3-fs-presto`, registered under the scheme *"s3://"* and *"s3p://"*, is based on code from the [Presto project](https://prestodb.io/).
+  You can configure it the same way you can [configure the Presto file system](https://prestodb.io/docs/0.187/connector/hive.html#amazon-s3-configuration).
+      
+  - `flink-s3-fs-hadoop`, registered under *"s3://"* and *"s3a://"*, based on code from the [Hadoop Project](https://hadoop.apache.org/).
+  The file system can be [configured exactly like Hadoop's s3a](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/index.html#S3A).
+    
+Both `flink-s3-fs-hadoop` and `flink-s3-fs-presto` register default FileSystem
+wrappers for URIs with the `s3://` scheme, `flink-s3-fs-hadoop` also registers
+for `s3a://` and `flink-s3-fs-presto` also registers for `s3p://`, so you can
+use this to use both at the same time.
+This can happen when, for example, the job uses the [StreamingFileSink]({{ site.baseurl}}/dev/connectors/streamfile_sink.html) which only supports Hadoop, but uses Presto for checkpointing.
+In this case, it is advised to use explicitly *"s3a://"* as a scheme for the sink (Hadoop) and *"s3p://"* for checkpointing (Presto).
+    
+To use either `flink-s3-fs-hadoop` or `flink-s3-fs-presto`, copy the respective JAR file from the `opt` directory to the `lib` directory of your Flink distribution before starting Flink, e.g.
+
+{% highlight bash %}
+cp ./opt/flink-s3-fs-presto-{{ site.version }}.jar ./lib/
+{% endhighlight %}
+
+#### Configure Access Credentials
+
+After setting up the S3 FileSystem wrapper, you need to make sure that Flink is allowed to access your S3 buckets.
+
+##### Identity and Access Management (IAM) (Recommended)
+
+The recommended way of setting up credentials on AWS is via [Identity and Access Management (IAM)](http://docs.aws.amazon.com/IAM/latest/UserGuide/introduction.html). You can use IAM features to securely give Flink instances the credentials that they need in order to access S3 buckets. Details about how to do this are beyond the scope of this documentation. Please refer to the AWS user guide. What you are looking for are [IAM Roles](http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/iam- [...]
+
+If you set this up correctly, you can manage access to S3 within AWS and don't need to distribute any access keys to Flink.
+
+##### Access Keys (Discouraged)
+
+Access to S3 can be granted via your **access and secret key pair**. Please note that this is discouraged since the [introduction of IAM roles](https://blogs.aws.amazon.com/security/post/Tx1XG3FX6VMU6O5/A-safer-way-to-distribute-AWS-credentials-to-EC2).
+
+You need to configure both `s3.access-key` and `s3.secret-key`  in Flink's  `flink-conf.yaml`:
+
+{% highlight yaml %}
+s3.access-key: your-access-key
+s3.secret-key: your-secret-key
+{% endhighlight %}
+
+{% top %}
+
+## Entropy injection for S3 file systems
+
+The bundled S3 file systems (`flink-s3-fs-presto` and `flink-s3-fs-hadoop`) support entropy injection. Entropy injection is
+a technique to improve scalability of AWS S3 buckets through adding some random characters near the beginning of the key.
+
+If entropy injection is activated, a configured substring in the paths will be replaced by random characters. For example, path
+`s3://my-bucket/checkpoints/_entropy_/dashboard-job/` would be replaced by something like `s3://my-bucket/checkpoints/gf36ikvg/dashboard-job/`.
+
+**Note that this only happens when the file creation passes the option to inject entropy!**, otherwise the file path will
+simply remove the entropy key substring. See
+[FileSystem.create(Path, WriteOption)](https://ci.apache.org/projects/flink/flink-docs-release-1.6/api/java/org/apache/flink/core/fs/FileSystem.html#create-org.apache.flink.core.fs.Path-org.apache.flink.core.fs.FileSystem.WriteOptions-)
+for details.
+
+*Note: The Flink runtime currently passes the option to inject entropy only to checkpoint data files.*
+*All other files, including checkpoint metadata and external URI do not inject entropy, to keep checkpoint URIs predictable.*
+
+To enable entropy injection, configure the *entropy key* and the *entropy length* parameters.
+
+```
+s3.entropy.key: _entropy_
+s3.entropy.length: 4 (default)
+
+```
+
+The `s3.entropy.key` defines the string in paths that is replaced by the random characters. Paths that do not contain the entropy key are left unchanged.
+If a file system operation does not pass the *"inject entropy"* write option, the entropy key substring is simply removed.
+The `s3.entropy.length` defined the number of random alphanumeric characters to replace the entropy key with.
\ No newline at end of file
diff --git a/docs/redirects/filesystems.md b/docs/redirects/filesystems.md
new file mode 100644
index 0000000..4aad652
--- /dev/null
+++ b/docs/redirects/filesystems.md
@@ -0,0 +1,24 @@
+---
+title: "File Systems"
+layout: redirect
+redirect: /ops/filesystems.html
+permalink: /ops/filesystems/index.html
+---
+<!--
+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.
+-->
\ No newline at end of file
diff --git a/docs/redirects/oss.md b/docs/redirects/oss.md
new file mode 100644
index 0000000..3b34502
--- /dev/null
+++ b/docs/redirects/oss.md
@@ -0,0 +1,24 @@
+---
+title: "Aliyun Object Storage Service (OSS)"
+layout: redirect
+redirect: /ops/deployment/oss.html
+permalink: /ops/filesystems/oss.html
+---
+<!--
+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.
+-->
\ No newline at end of file


[flink] 09/14: [hotfix] [tests] Remove unnecessary timeouts from SingleInputGate Tests

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

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

commit e94ec4feafd15ee0562ae1f711f2f19b3d3d85f6
Author: Stephan Ewen <se...@apache.org>
AuthorDate: Fri May 10 13:50:26 2019 +0200

    [hotfix] [tests] Remove unnecessary timeouts from SingleInputGate Tests
    
    These tests very lightweight and no longer run with any concurrency or external interaction
    and hence do not require any timeouts.
---
 .../runtime/io/network/partition/consumer/SingleInputGateTest.java    | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java
index 5dda8db..07fcd26 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java
@@ -80,7 +80,7 @@ public class SingleInputGateTest extends InputGateTestBase {
 	 * Tests basic correctness of buffer-or-event interleaving and correct <code>null</code> return
 	 * value after receiving all end-of-partition events.
 	 */
-	@Test(timeout = 120 * 1000)
+	@Test
 	public void testBasicGetNextLogic() throws Exception {
 		// Setup
 		final SingleInputGate inputGate = createInputGate();
@@ -129,7 +129,7 @@ public class SingleInputGateTest extends InputGateTestBase {
 		testIsAvailable(inputGate, inputGate, inputChannel);
 	}
 
-	@Test(timeout = 120 * 1000)
+	@Test
 	public void testIsMoreAvailableReadingFromSingleInputChannel() throws Exception {
 		// Setup
 		final SingleInputGate inputGate = createInputGate();


[flink] 11/14: [hotfix] [tests] Consolidate mocking of ResultPartition in one utility class

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

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

commit 8a76952dab6e9533c2815878ffde96806658161e
Author: Stephan Ewen <se...@apache.org>
AuthorDate: Wed Apr 17 17:31:42 2019 +0200

    [hotfix] [tests] Consolidate mocking of ResultPartition in one utility class
    
    This also removes the use of Mockito from these classes.
---
 .../runtime/io/network/NetworkEnvironmentTest.java | 20 +-----
 .../io/network/partition/PartitionTestUtils.java   | 74 ++++++++++++++++++++++
 .../partition/PipelinedSubpartitionTest.java       |  2 +-
 .../io/network/partition/ResultPartitionTest.java  | 23 +------
 4 files changed, 78 insertions(+), 41 deletions(-)

diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java
index 3a2014a..f20feb7 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java
@@ -18,18 +18,13 @@
 
 package org.apache.flink.runtime.io.network;
 
-import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.TaskManagerOptions;
-import org.apache.flink.runtime.io.disk.iomanager.NoOpIOManager;
 import org.apache.flink.runtime.io.network.partition.InputChannelTestUtils;
-import org.apache.flink.runtime.io.network.partition.NoOpResultPartitionConsumableNotifier;
+import org.apache.flink.runtime.io.network.partition.PartitionTestUtils;
 import org.apache.flink.runtime.io.network.partition.ResultPartition;
-import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
-import org.apache.flink.runtime.io.network.partition.ResultPartitionManager;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
 import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel;
 import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
-import org.apache.flink.runtime.taskmanager.NoOpTaskActions;
 import org.apache.flink.runtime.taskmanager.Task;
 
 import org.junit.Rule;
@@ -277,18 +272,7 @@ public class NetworkEnvironmentTest {
 	private static ResultPartition createResultPartition(
 			final ResultPartitionType partitionType, final int channels) {
 
-		return new ResultPartition(
-			"TestTask-" + partitionType + ":" + channels,
-			new NoOpTaskActions(),
-			new JobID(),
-			new ResultPartitionID(),
-			partitionType,
-			channels,
-			channels,
-			mock(ResultPartitionManager.class),
-			new NoOpResultPartitionConsumableNotifier(),
-			new NoOpIOManager(),
-			false);
+		return PartitionTestUtils.createPartition(partitionType, channels);
 	}
 
 	/**
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionTestUtils.java
new file mode 100644
index 0000000..3391030
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionTestUtils.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.io.disk.iomanager.NoOpIOManager;
+import org.apache.flink.runtime.taskmanager.NoOpTaskActions;
+
+/**
+ * This class should consolidate all mocking logic for ResultPartitions.
+ * While using Mockito internally (for now), the use of Mockito should not
+ * leak out of this class.
+ */
+public class PartitionTestUtils {
+
+	public static ResultPartition createPartition() {
+		return createPartition(ResultPartitionType.PIPELINED_BOUNDED);
+	}
+
+	public static ResultPartition createPartition(ResultPartitionType type) {
+		return createPartition(
+				new NoOpResultPartitionConsumableNotifier(),
+				type,
+				false);
+	}
+
+	public static ResultPartition createPartition(ResultPartitionType type, int numChannels) {
+		return createPartition(new NoOpResultPartitionConsumableNotifier(), type, numChannels, false);
+	}
+
+	public static ResultPartition createPartition(
+			ResultPartitionConsumableNotifier notifier,
+			ResultPartitionType type,
+			boolean sendScheduleOrUpdateConsumersMessage) {
+
+		return createPartition(notifier, type, 1, sendScheduleOrUpdateConsumersMessage);
+	}
+
+	public static ResultPartition createPartition(
+			ResultPartitionConsumableNotifier notifier,
+			ResultPartitionType type,
+			int numChannels,
+			boolean sendScheduleOrUpdateConsumersMessage) {
+
+		return new ResultPartition(
+				"TestTask",
+				new NoOpTaskActions(),
+				new JobID(),
+				new ResultPartitionID(),
+				type,
+				numChannels,
+				numChannels,
+				new ResultPartitionManager(),
+				notifier,
+				new NoOpIOManager(),
+				sendScheduleOrUpdateConsumersMessage);
+	}
+}
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java
index 8535973..36cd353 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java
@@ -73,7 +73,7 @@ public class PipelinedSubpartitionTest extends SubpartitionTestBase {
 
 	@Override
 	PipelinedSubpartition createSubpartition() {
-		final ResultPartition parent = mock(ResultPartition.class);
+		final ResultPartition parent = PartitionTestUtils.createPartition();
 
 		return new PipelinedSubpartition(0, parent);
 	}
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java
index 9e3c117..c911df7 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java
@@ -19,19 +19,18 @@
 package org.apache.flink.runtime.io.network.partition;
 
 import org.apache.flink.api.common.JobID;
-import org.apache.flink.runtime.io.disk.iomanager.NoOpIOManager;
 import org.apache.flink.runtime.io.network.NetworkEnvironment;
 import org.apache.flink.runtime.io.network.NetworkEnvironmentBuilder;
 import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
 import org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils;
 import org.apache.flink.runtime.io.network.buffer.BufferConsumer;
-import org.apache.flink.runtime.taskmanager.NoOpTaskActions;
 import org.apache.flink.runtime.taskmanager.TaskActions;
 
 import org.junit.Assert;
 import org.junit.Test;
 
 import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledBufferConsumer;
+import static org.apache.flink.runtime.io.network.partition.PartitionTestUtils.createPartition;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -250,24 +249,4 @@ public class ResultPartitionTest {
 			network.shutdown();
 		}
 	}
-
-	// ------------------------------------------------------------------------
-
-	private static ResultPartition createPartition(
-			ResultPartitionConsumableNotifier notifier,
-			ResultPartitionType type,
-			boolean sendScheduleOrUpdateConsumersMessage) {
-		return new ResultPartition(
-			"TestTask",
-			new NoOpTaskActions(),
-			new JobID(),
-			new ResultPartitionID(),
-			type,
-			1,
-			1,
-			mock(ResultPartitionManager.class),
-			notifier,
-			new NoOpIOManager(),
-			sendScheduleOrUpdateConsumersMessage);
-	}
 }


[flink] 10/14: [hotfix] [tests] Network and Partition Tests pass in Testing IOManager rather than dysfunctional mock

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

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

commit dfd08ab29ce2aa1c7f1f6699557f4b9515473ebe
Author: Stephan Ewen <se...@apache.org>
AuthorDate: Mon Apr 15 22:22:36 2019 +0200

    [hotfix] [tests] Network and Partition Tests pass in Testing IOManager rather than dysfunctional mock
---
 .../runtime/io/disk/iomanager/NoOpIOManager.java   | 73 ++++++++++++++++++++++
 .../runtime/io/network/NetworkEnvironmentTest.java |  5 +-
 .../io/network/partition/ResultPartitionTest.java  | 14 +----
 3 files changed, 78 insertions(+), 14 deletions(-)

diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/NoOpIOManager.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/NoOpIOManager.java
new file mode 100644
index 0000000..f98c46f
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/NoOpIOManager.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.disk.iomanager;
+
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel.ID;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.LinkedBlockingQueue;
+
+/**
+ * An {@link IOManager} that cannot do I/O but serves as a mock for tests.
+ */
+public class NoOpIOManager extends IOManager {
+
+	public NoOpIOManager() {
+		super(new String[] {EnvironmentInformation.getTemporaryFileDirectory()});
+	}
+
+	@Override
+	public BlockChannelWriter<MemorySegment> createBlockChannelWriter(ID channelID, LinkedBlockingQueue<MemorySegment> returnQueue) throws IOException {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public BlockChannelWriterWithCallback<MemorySegment> createBlockChannelWriter(ID channelID, RequestDoneCallback<MemorySegment> callback) throws IOException {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public BlockChannelReader<MemorySegment> createBlockChannelReader(ID channelID, LinkedBlockingQueue<MemorySegment> returnQueue) throws IOException {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public BufferFileWriter createBufferFileWriter(ID channelID) throws IOException {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public BufferFileReader createBufferFileReader(ID channelID, RequestDoneCallback<Buffer> callback) throws IOException {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public BufferFileSegmentReader createBufferFileSegmentReader(ID channelID, RequestDoneCallback<FileSegment> callback) throws IOException {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public BulkBlockChannelReader createBulkBlockChannelReader(ID channelID, List<MemorySegment> targetSegments, int numBlocks) throws IOException {
+		throw new UnsupportedOperationException();
+	}
+}
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java
index bedd090..3a2014a 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java
@@ -20,7 +20,7 @@ package org.apache.flink.runtime.io.network;
 
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.TaskManagerOptions;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.io.disk.iomanager.NoOpIOManager;
 import org.apache.flink.runtime.io.network.partition.InputChannelTestUtils;
 import org.apache.flink.runtime.io.network.partition.NoOpResultPartitionConsumableNotifier;
 import org.apache.flink.runtime.io.network.partition.ResultPartition;
@@ -276,6 +276,7 @@ public class NetworkEnvironmentTest {
 	 */
 	private static ResultPartition createResultPartition(
 			final ResultPartitionType partitionType, final int channels) {
+
 		return new ResultPartition(
 			"TestTask-" + partitionType + ":" + channels,
 			new NoOpTaskActions(),
@@ -286,7 +287,7 @@ public class NetworkEnvironmentTest {
 			channels,
 			mock(ResultPartitionManager.class),
 			new NoOpResultPartitionConsumableNotifier(),
-			mock(IOManager.class),
+			new NoOpIOManager(),
 			false);
 	}
 
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java
index 8456e9c..9e3c117 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java
@@ -19,8 +19,7 @@
 package org.apache.flink.runtime.io.network.partition;
 
 import org.apache.flink.api.common.JobID;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
+import org.apache.flink.runtime.io.disk.iomanager.NoOpIOManager;
 import org.apache.flink.runtime.io.network.NetworkEnvironment;
 import org.apache.flink.runtime.io.network.NetworkEnvironmentBuilder;
 import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
@@ -29,7 +28,6 @@ import org.apache.flink.runtime.io.network.buffer.BufferConsumer;
 import org.apache.flink.runtime.taskmanager.NoOpTaskActions;
 import org.apache.flink.runtime.taskmanager.TaskActions;
 
-import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -50,14 +48,6 @@ import static org.mockito.Mockito.verify;
  */
 public class ResultPartitionTest {
 
-	/** Asynchronous I/O manager. */
-	private static final IOManager ioManager = new IOManagerAsync();
-
-	@AfterClass
-	public static void shutdown() {
-		ioManager.shutdown();
-	}
-
 	/**
 	 * Tests the schedule or update consumers message sending behaviour depending on the relevant flags.
 	 */
@@ -277,7 +267,7 @@ public class ResultPartitionTest {
 			1,
 			mock(ResultPartitionManager.class),
 			notifier,
-			ioManager,
+			new NoOpIOManager(),
 			sendScheduleOrUpdateConsumersMessage);
 	}
 }


[flink] 05/14: [hotfix] [docs] Minor cleanup in filesystem docs

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

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

commit d14fc9e49b6519927e12e2272ba7fa86be78d086
Author: Stephan Ewen <se...@apache.org>
AuthorDate: Fri May 10 10:32:56 2019 +0200

    [hotfix] [docs] Minor cleanup in filesystem docs
---
 docs/ops/filesystems/index.md    | 13 +++++++------
 docs/ops/filesystems/index.zh.md | 15 ++++++++-------
 2 files changed, 15 insertions(+), 13 deletions(-)

diff --git a/docs/ops/filesystems/index.md b/docs/ops/filesystems/index.md
index b8548c1..0d4a1be 100644
--- a/docs/ops/filesystems/index.md
+++ b/docs/ops/filesystems/index.md
@@ -25,7 +25,7 @@ under the License.
 -->
 
 Apache Flink uses file systems to consume and persistently store data, both for the results of applications and for fault tolerance and recovery.
-These are some of most of the popular file systems, including *local*, *hadoop-compatible*, *S3*, *MapR FS*, *OpenStack Swift FS* and *Aliyum OSS*.
+These are some of most of the popular file systems, including *local*, *hadoop-compatible*, *S3*, *MapR FS*, *OpenStack Swift FS* and *Aliyun OSS*.
 
 The file system used for a particular file is determined by its URI scheme.
 For example, `file:///home/user/text.txt` refers to a file in the local file system, while `hdfs://namenode:50010/data/user/text.txt` is a file in a specific HDFS cluster.
@@ -35,7 +35,7 @@ File system instances are instantiated once per process and then cached/pooled,
 * This will be replaced by the TOC
 {:toc}
 
-### Built-in File Systems
+## Built-in File Systems
 
 Flink ships with implementations for the following file systems:
 
@@ -50,7 +50,7 @@ Flink ships with implementations for the following file systems:
   To use it when using Flink as a library, add the respective maven dependency (`org.apache.flink:flink-swift-fs-hadoop:{{ site.version }}`
   When starting a Flink application from the Flink binaries, copy or move the respective jar file from the `opt` folder to the `lib` folder.
 
-#### HDFS and Hadoop File System support 
+## HDFS and Hadoop File System support 
 
 For all schemes where Flink cannot find a directly supported file system, it falls back to Hadoop.
 All Hadoop file systems are automatically available when `flink-runtime` and the Hadoop libraries are on the classpath.
@@ -64,11 +64,12 @@ This way, Flink seamlessly supports all of Hadoop file systems, and all Hadoop-c
   - **har**
   - ...
 
-##### Hadoop Configuration
+### Hadoop Configuration
 
-We recommend using Flink's built-in file systems them unless required otherwise, e.g., for using that file system as YARN's resource storage dir via the `fs.defaultFS` configuration property in Hadoop's `core-site.xml`.
+We recommend using Flink's built-in file systems unless required otherwise. Using a Hadoop File System directly may be required, for example, when using that file system for YARN's resource storage, via the `fs.defaultFS` configuration property in Hadoop's `core-site.xml`.
 
-If using a Hadoop file system, you can specify the [configuration](../config.html#hdfs) by setting the environment variable `HADOOP_CONF_DIR`, or by setting the `fs.hdfs.hadoopconf` configuration option in `flink-conf.yaml`. 
+Putting the Hadoop configuration in the same class path as the Hadoop libraries makes the Hadoop File Systems pick up that configuration.
+You can reference another Hadoop configuration by setting the environment variable `HADOOP_CONF_DIR`, or by referencing it via the [Flink configuration](../config.html#hdfs).
 
 {% highlight yaml %}
 fs.hdfs.hadoopconf: /path/to/etc/hadoop
diff --git a/docs/ops/filesystems/index.zh.md b/docs/ops/filesystems/index.zh.md
index 7f2de4e..0d4a1be 100644
--- a/docs/ops/filesystems/index.zh.md
+++ b/docs/ops/filesystems/index.zh.md
@@ -1,5 +1,5 @@
 ---
-title: "文件系统"
+title: "File Systems"
 nav-id: filesystems
 nav-parent_id: ops
 nav-show_overview: true
@@ -25,7 +25,7 @@ under the License.
 -->
 
 Apache Flink uses file systems to consume and persistently store data, both for the results of applications and for fault tolerance and recovery.
-These are some of most of the popular file systems, including *local*, *hadoop-compatible*, *S3*, *MapR FS*, *OpenStack Swift FS* and *Aliyum OSS*.
+These are some of most of the popular file systems, including *local*, *hadoop-compatible*, *S3*, *MapR FS*, *OpenStack Swift FS* and *Aliyun OSS*.
 
 The file system used for a particular file is determined by its URI scheme.
 For example, `file:///home/user/text.txt` refers to a file in the local file system, while `hdfs://namenode:50010/data/user/text.txt` is a file in a specific HDFS cluster.
@@ -35,7 +35,7 @@ File system instances are instantiated once per process and then cached/pooled,
 * This will be replaced by the TOC
 {:toc}
 
-### Built-in File Systems
+## Built-in File Systems
 
 Flink ships with implementations for the following file systems:
 
@@ -50,7 +50,7 @@ Flink ships with implementations for the following file systems:
   To use it when using Flink as a library, add the respective maven dependency (`org.apache.flink:flink-swift-fs-hadoop:{{ site.version }}`
   When starting a Flink application from the Flink binaries, copy or move the respective jar file from the `opt` folder to the `lib` folder.
 
-#### HDFS and Hadoop File System support 
+## HDFS and Hadoop File System support 
 
 For all schemes where Flink cannot find a directly supported file system, it falls back to Hadoop.
 All Hadoop file systems are automatically available when `flink-runtime` and the Hadoop libraries are on the classpath.
@@ -64,11 +64,12 @@ This way, Flink seamlessly supports all of Hadoop file systems, and all Hadoop-c
   - **har**
   - ...
 
-##### Hadoop Configuration
+### Hadoop Configuration
 
-We recommend using Flink's built-in file systems them unless required otherwise, e.g., for using that file system as YARN's resource storage dir via the `fs.defaultFS` configuration property in Hadoop's `core-site.xml`.
+We recommend using Flink's built-in file systems unless required otherwise. Using a Hadoop File System directly may be required, for example, when using that file system for YARN's resource storage, via the `fs.defaultFS` configuration property in Hadoop's `core-site.xml`.
 
-If using a Hadoop file system, you can specify the [configuration](../config.html#hdfs) by setting the environment variable `HADOOP_CONF_DIR`, or by setting the `fs.hdfs.hadoopconf` configuration option in `flink-conf.yaml`. 
+Putting the Hadoop configuration in the same class path as the Hadoop libraries makes the Hadoop File Systems pick up that configuration.
+You can reference another Hadoop configuration by setting the environment variable `HADOOP_CONF_DIR`, or by referencing it via the [Flink configuration](../config.html#hdfs).
 
 {% highlight yaml %}
 fs.hdfs.hadoopconf: /path/to/etc/hadoop


[flink] 06/14: [FLINK-12388][docs] Update the production readiness checklist

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

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

commit 754cd71dd92dfcfff3e1ef23083790422188ce9e
Author: Seth Wiesman <sj...@gmail.com>
AuthorDate: Wed May 1 21:30:27 2019 -0500

    [FLINK-12388][docs] Update the production readiness checklist
    
    This closes #8330
---
 docs/ops/production_ready.md    | 87 +++++++++++++++-------------------------
 docs/ops/production_ready.zh.md | 88 +++++++++++++++--------------------------
 2 files changed, 63 insertions(+), 112 deletions(-)

diff --git a/docs/ops/production_ready.md b/docs/ops/production_ready.md
index 08fef2b..ef97173 100644
--- a/docs/ops/production_ready.md
+++ b/docs/ops/production_ready.md
@@ -22,79 +22,54 @@ specific language governing permissions and limitations
 under the License.
 -->
 
+The production readiness checklist provides an overview of configuration options that should be carefully considered before bringing an Apache Flink job into production. 
+While the Flink community has attempted to provide sensible defaults for each configuration, it is important to review this list and ensure the options chosen are sufficient for your needs. 
+
 * ToC
 {:toc}
 
-## Production Readiness Checklist
-
-Purpose of this production readiness checklist is to provide a condensed overview of configuration options that are
-important and need **careful considerations** if you plan to bring your Flink job into **production**. For most of these options
-Flink provides out-of-the-box defaults to make usage and adoption of Flink easier. For many users and scenarios, those
-defaults are good starting points for development and completely sufficient for "one-shot" jobs. 
-
-However, once you are planning to bring a Flink application to production the requirements typically increase. For example,
-you want your job to be (re-)scalable and to have a good upgrade story for your job and new Flink versions.
-
-In the following, we present a collection of configuration options that you should check before your job goes into production.
-
-### Set maximum parallelism for operators explicitly
-
-Maximum parallelism is a configuration parameter that is newly introduced in Flink 1.2 and has important implications
-for the (re-)scalability of your Flink job. This parameter, which can be set on a per-job and/or per-operator granularity,
-determines the maximum parallelism to which you can scale operators. It is important to understand that (as of now) there
-is **no way to change** this parameter after your job has been started, except for restarting your job completely 
-from scratch (i.e. with a new state, and not from a previous checkpoint/savepoint). Even if Flink would provide some way
-to change maximum parallelism for existing savepoints in the future, you can already assume that for large states this is 
-likely a long running operation that you want to avoid. At this point, you might wonder why not just to use a very high
-value as default for this parameter. The reason behind this is that high maximum parallelism can have some impact on your
-application's performance and even state sizes, because Flink has to maintain certain metadata for its ability to rescale which
-can increase with the maximum parallelism. In general, you should choose a max parallelism that is high enough to fit your
-future needs in scalability, but keeping it as low as possible can give slightly better performance. In particular,
-a maximum parallelism higher that 128 will typically result in slightly bigger state snapshots from the keyed backends.
+### Set An Explicit Max Parallelism
 
-Notice that maximum parallelism must fulfill the following conditions:
+The max parallelism, set on a per-job and per-operator granularity, determines the maximum parallelism to which a stateful operator can scale.
+There is currently **no way to change** the maximum parallelism of an operator after a job has started without discarding that operators state. 
+The reason maximum parallelism exists, versus allowing stateful operators to be infinitely scalable, is that it has some impact on your application's performance and state size.
+Flink has to maintain specific metadata for its ability to rescale state which grows linearly with max parallelism.
+In general, you should choose max parallelism that is high enough to fit your future needs in scalability, while keeping it low enough to maintain reasonable performance.
 
-`0 < parallelism  <= max parallelism <= 2^15`
+{% panel **Note:** Maximum parallelism must fulfill the following conditions: `0 < parallelism  <= max parallelism <= 2^15` %}
 
-You can set the maximum parallelism by `setMaxParallelism(int maxparallelism)`. By default, Flink will choose the maximum
-parallelism as a function of the parallelism when the job is first started:
+You can explicitly set maximum parallelism by using `setMaxParallelism(int maxparallelism)`. 
+If no max parallelism is set Flink will decide using a function of the operators parallelism when the job is first started:
 
 - `128` : for all parallelism <= 128.
 - `MIN(nextPowerOfTwo(parallelism + (parallelism / 2)), 2^15)` : for all parallelism > 128.
 
-### Set UUIDs for operators
+### Set UUIDs For All Operators
 
-As mentioned in the documentation for [savepoints]({{ site.baseurl }}/ops/state/savepoints.html), users should set uids for
-operators. Those operator uids are important for Flink's mapping of operator states to operators which, in turn, is 
-essential for savepoints. By default operator uids are generated by traversing the JobGraph and hashing certain operator 
-properties. While this is comfortable from a user perspective, it is also very fragile, as changes to the JobGraph (e.g.
-exchanging an operator) will result in new UUIDs. To establish a stable mapping, we need stable operator uids provided 
-by the user through `setUid(String uid)`.
+As mentioned in the documentation for [savepoints]({{ site.baseurl }}/ops/state/savepoints.html), users should set uids for each operator in their `DataStream`.
+Uids are necessary for Flink's mapping of operator states to operators which, in turn, is essential for savepoints.
+By default, operator uids are generated by traversing the JobGraph and hashing specific operator properties.
+While this is comfortable from a user perspective, it is also very fragile, as changes to the JobGraph (e.g., exchanging an operator) results in new UUIDs.
+To establish a stable mapping, we need stable operator uids provided by the user through `setUid(String uid)`.
 
-### Choice of state backend
+### Choose The Right State Backend
 
-Currently, Flink has the limitation that it can only restore the state from a savepoint for the same state backend that
-took the savepoint. For example, this means that we can not take a savepoint with a memory state backend, then change
-the job to use a RocksDB state backend and restore. While we are planning to make backends interoperable in the near
-future, they are not yet. This means you should carefully consider which backend you use for your job before going to
-production.
+Currently, Flink's savepoint binary format is state backend specific.
+A savepoint taken with one state backend cannot be restored using another, and you should carefully consider which backend you use before going to production.
 
-In general, we recommend using RocksDB because this is currently the only state backend that supports large states (i.e.
-state that exceeds the available main memory) and asynchronous snapshots. From our experience, asynchronous snapshots are
-very important for large states because they do not block the operators and Flink can write the snapshots without stopping 
-stream processing. However, RocksDB can have worse performance than, for example, the memory-based state backends. If
-you are sure that your state will never exceed main memory and blocking the stream processing to write it is not an issue,
-you **could consider** to not use the RocksDB backends. However, at this point, we **strongly recommend** using RocksDB
-for production.
+In general, we recommend avoiding `MemoryStateBackend` in production because it stores its snapshots inside the JobManager as opposed to persistent disk.
+When deciding between `FsStateBackend` and `RocksDB`, it is a choice between performance and scalability.
+`FsStateBackend` is very fast as each state access and update operates on objects on the Java heap; however, state size is limited by available memory within the cluster.
+On the other hand, `RocksDB` can scale based on available disk space and is the only state backend to support incremental snapshots.
+However, each state access and update requires (de-)serialization and potentially reading from disk which leads to average performance that is an order of magnitude slower than the memory state backends.
+Carefully read through the [state backend documentation]({{ site.baseurl }}/ops/state/state_backends.html) to fully understand the pros and cons of each option.
 
-### Config JobManager High Availability(HA)
+### Configure JobManager High Availability
 
-The JobManager coordinates every Flink deployment. It is responsible for both *scheduling* and *resource management*.
+The JobManager serves as a central coordinator for each Flink deployment, being responsible for both scheduling and resource management of the cluster.
+It is a single point of failure within the cluster, and if it crashes, no new jobs can be submitted, and running applications will fail. 
 
-By default, there is a single JobManager instance per Flink cluster. This creates a *single point of failure* (SPOF): 
-if the JobManager crashes, no new programs can be submitted and running programs fail.
+Configuring [High Availability]({{ site.baseurl }}/ops/jobmanager_high_availability.html), in conjunction with Apache Zookeeper, allows for a swift recovery and is highly recommended for production setups. 
 
-With JobManager High Availability, you can recover from JobManager failures and thereby eliminate the *SPOF*. 
-We **strongly recommend** you configure [high availability]({{ site.baseurl }}/ops/jobmanager_high_availability.html) for production.
 
 {% top %}
diff --git a/docs/ops/production_ready.zh.md b/docs/ops/production_ready.zh.md
index d5f7ee1..e5cff46 100644
--- a/docs/ops/production_ready.zh.md
+++ b/docs/ops/production_ready.zh.md
@@ -22,79 +22,55 @@ specific language governing permissions and limitations
 under the License.
 -->
 
+The production readiness checklist provides an overview of configuration options that should be carefully considered before bringing an Apache Flink job into production. 
+While the Flink community has attempted to provide sensible defaults for each configuration, it is important to review this list and ensure the options chosen are sufficient for your needs. 
+
 * ToC
 {:toc}
 
-## Production Readiness Checklist
-
-Purpose of this production readiness checklist is to provide a condensed overview of configuration options that are
-important and need **careful considerations** if you plan to bring your Flink job into **production**. For most of these options
-Flink provides out-of-the-box defaults to make usage and adoption of Flink easier. For many users and scenarios, those
-defaults are good starting points for development and completely sufficient for "one-shot" jobs. 
-
-However, once you are planning to bring a Flink application to production the requirements typically increase. For example,
-you want your job to be (re-)scalable and to have a good upgrade story for your job and new Flink versions.
-
-In the following, we present a collection of configuration options that you should check before your job goes into production.
+### Set An Explicit Max Parallelism
 
-### Set maximum parallelism for operators explicitly
+The max parallelism, set on a per-job and per-operator granularity, determines the maximum parallelism to which a stateful operator can scale.
+There is currently **no way to change** the maximum parallelism of an operator after a job has started without discarding that operators state. 
+The reason maximum parallelism exists, versus allowing stateful operators to be infinitely scalable, is that it has some impact on your application's performance and state size.
+Flink has to maintain specific metadata for its ability to rescale state which grows linearly with max parallelism.
+In general, you should choose max parallelism that is high enough to fit your future needs in scalability, while keeping it low enough to maintain reasonable performance.
 
-Maximum parallelism is a configuration parameter that is newly introduced in Flink 1.2 and has important implications
-for the (re-)scalability of your Flink job. This parameter, which can be set on a per-job and/or per-operator granularity,
-determines the maximum parallelism to which you can scale operators. It is important to understand that (as of now) there
-is **no way to change** this parameter after your job has been started, except for restarting your job completely 
-from scratch (i.e. with a new state, and not from a previous checkpoint/savepoint). Even if Flink would provide some way
-to change maximum parallelism for existing savepoints in the future, you can already assume that for large states this is 
-likely a long running operation that you want to avoid. At this point, you might wonder why not just to use a very high
-value as default for this parameter. The reason behind this is that high maximum parallelism can have some impact on your
-application's performance and even state sizes, because Flink has to maintain certain metadata for its ability to rescale which
-can increase with the maximum parallelism. In general, you should choose a max parallelism that is high enough to fit your
-future needs in scalability, but keeping it as low as possible can give slightly better performance. In particular,
-a maximum parallelism higher that 128 will typically result in slightly bigger state snapshots from the keyed backends.
+{% panel **Note:** Maximum parallelism must fulfill the following conditions: `0 < parallelism  <= max parallelism <= 2^15` %}
 
-Notice that maximum parallelism must fulfill the following conditions:
-
-`0 < parallelism  <= max parallelism <= 2^15`
-
-You can set the maximum parallelism by `setMaxParallelism(int maxparallelism)`. By default, Flink will choose the maximum
-parallelism as a function of the parallelism when the job is first started:
+You can explicitly set maximum parallelism by using `setMaxParallelism(int maxparallelism)`. 
+If no max parallelism is set Flink will decide using a function of the operators parallelism when the job is first started:
 
 - `128` : for all parallelism <= 128.
 - `MIN(nextPowerOfTwo(parallelism + (parallelism / 2)), 2^15)` : for all parallelism > 128.
 
-### Set UUIDs for operators
+### Set UUIDs For All Operators
 
-As mentioned in the documentation for [savepoints]({{ site.baseurl }}/ops/state/savepoints.html), users should set uids for
-operators. Those operator uids are important for Flink's mapping of operator states to operators which, in turn, is 
-essential for savepoints. By default operator uids are generated by traversing the JobGraph and hashing certain operator 
-properties. While this is comfortable from a user perspective, it is also very fragile, as changes to the JobGraph (e.g.
-exchanging an operator) will result in new UUIDs. To establish a stable mapping, we need stable operator uids provided 
-by the user through `setUid(String uid)`.
+As mentioned in the documentation for [savepoints]({{ site.baseurl }}/ops/state/savepoints.html), users should set uids for each operator in their `DataStream`.
+Uids are necessary for Flink's mapping of operator states to operators which, in turn, is essential for savepoints.
+By default, operator uids are generated by traversing the JobGraph and hashing specific operator properties.
+While this is comfortable from a user perspective, it is also very fragile, as changes to the JobGraph (e.g., exchanging an operator) results in new UUIDs.
+To establish a stable mapping, we need stable operator uids provided by the user through `setUid(String uid)`.
 
-### Choice of state backend
+### Choose The Right State Backend
 
-Currently, Flink has the limitation that it can only restore the state from a savepoint for the same state backend that
-took the savepoint. For example, this means that we can not take a savepoint with a memory state backend, then change
-the job to use a RocksDB state backend and restore. While we are planning to make backends interoperable in the near
-future, they are not yet. This means you should carefully consider which backend you use for your job before going to
-production.
+Currently, Flink's savepoint binary format is state backend specific.
+A savepoint taken with one state backend cannot be restored using another, and you should carefully consider which backend you use before going to production.
 
-In general, we recommend using RocksDB because this is currently the only state backend that supports large states (i.e.
-state that exceeds the available main memory) and asynchronous snapshots. From our experience, asynchronous snapshots are
-very important for large states because they do not block the operators and Flink can write the snapshots without stopping 
-stream processing. However, RocksDB can have worse performance than, for example, the memory-based state backends. If
-you are sure that your state will never exceed main memory and blocking the stream processing to write it is not an issue,
-you **could consider** to not use the RocksDB backends. However, at this point, we **strongly recommend** using RocksDB
-for production.
+In general, we recommend avoiding `MemoryStateBackend` in production because it stores its snapshots inside the JobManager as opposed to persistent disk.
+When deciding between `FsStateBackend` and `RocksDB`, it is a choice between performance and scalability.
+`FsStateBackend` is very fast as each state access and update operates on objects on the Java heap; however, state size is limited by available memory within the cluster.
+On the other hand, `RocksDB` can scale based on available disk space and is the only state backend to support incremental snapshots.
+However, each state access and update requires (de-)serialization and potentially reading from disk which leads to average performance that is an order of magnitude slower than the memory state backends.
+Carefully read through the [state backend documentation]({{ site.baseurl }}/ops/state/state_backends.html) to fully understand the pros and cons of each option.
 
-### Config JobManager High Availability(HA)
+### Configure JobManager High Availability
 
-The JobManager coordinates every Flink deployment. It is responsible for both *scheduling* and *resource management*.
+The JobManager serves as a central coordinator for each Flink deployment, being responsible for both scheduling and resource management of the cluster.
+It is a single point of failure within the cluster, and if it crashes, no new jobs can be submitted, and running applications will fail. 
 
-By default, there is a single JobManager instance per Flink cluster. This creates a *single point of failure* (SPOF): 
-if the JobManager crashes, no new programs can be submitted and running programs fail.
+Configuring [High Availability]({{ site.baseurl }}/ops/jobmanager_high_availability.html), in conjunction with Apache Zookeeper, allows for a swift recovery and is highly recommended for production setups. 
 
-With JobManager High Availability, you can recover from JobManager failures and thereby eliminate the *SPOF*. 
-We **strongly recommend** you configure [high availability]({{ site.baseurl }}/ops/jobmanager_high_availability.html) for production.
 
 {% top %}
+


[flink] 13/14: [hotfix] [tests] Move utility methods into correct test class.

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

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

commit 422f7b54f0bf66f51e792d24942cb7d951219da0
Author: Stephan Ewen <se...@apache.org>
AuthorDate: Thu Apr 25 17:53:59 2019 +0200

    [hotfix] [tests] Move utility methods into correct test class.
    
    The methods were intended to be generic for tests across partitions and hence placed in the
    SubpartitionTestBase.
    
    However, the SubpartitionTestBase can only really test common contract behavior, like behavior on
    disposal, finishing, and buffer reycling contracts in those cases. Producer / consumer behavior
    is sufficiently different between both implementations that it does not make sense at this point
    to try and share the tests.
---
 .../PipelinedSubpartitionWithReadViewTest.java     | 93 +++++++++++++++++++++-
 .../io/network/partition/SubpartitionTestBase.java | 91 +--------------------
 2 files changed, 91 insertions(+), 93 deletions(-)

diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionWithReadViewTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionWithReadViewTest.java
index 6f9920e..af4eb05 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionWithReadViewTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionWithReadViewTest.java
@@ -18,12 +18,17 @@
 
 package org.apache.flink.runtime.io.network.partition;
 
+import org.apache.flink.runtime.event.AbstractEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
 import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
 
 import org.junit.After;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+import javax.annotation.Nullable;
+
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
@@ -31,14 +36,14 @@ import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.
 import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createEventBufferConsumer;
 import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledBufferBuilder;
 import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledBufferConsumer;
-import static org.apache.flink.runtime.io.network.partition.SubpartitionTestBase.assertNextBuffer;
-import static org.apache.flink.runtime.io.network.partition.SubpartitionTestBase.assertNextEvent;
-import static org.apache.flink.runtime.io.network.partition.SubpartitionTestBase.assertNoNextBuffer;
 import static org.apache.flink.runtime.io.network.util.TestBufferFactory.BUFFER_SIZE;
+import static org.apache.flink.util.Preconditions.checkArgument;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.greaterThan;
+import static org.hamcrest.core.IsInstanceOf.instanceOf;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.mockito.Mockito.mock;
 
@@ -273,4 +278,86 @@ public class PipelinedSubpartitionWithReadViewTest {
 		assertEquals(5 * BUFFER_SIZE, subpartition.getTotalNumberOfBytes());
 		assertEquals(4, availablityListener.getNumNotifications());
 	}
+
+	// ------------------------------------------------------------------------
+
+	static void assertNextBuffer(
+			ResultSubpartitionView readView,
+			int expectedReadableBufferSize,
+			boolean expectedIsMoreAvailable,
+			int expectedBuffersInBacklog,
+			boolean expectedNextBufferIsEvent,
+			boolean expectedRecycledAfterRecycle) throws IOException, InterruptedException {
+		assertNextBufferOrEvent(
+				readView,
+				expectedReadableBufferSize,
+				true,
+				null,
+				expectedIsMoreAvailable,
+				expectedBuffersInBacklog,
+				expectedNextBufferIsEvent,
+				expectedRecycledAfterRecycle);
+	}
+
+	static void assertNextEvent(
+			ResultSubpartitionView readView,
+			int expectedReadableBufferSize,
+			Class<? extends AbstractEvent> expectedEventClass,
+			boolean expectedIsMoreAvailable,
+			int expectedBuffersInBacklog,
+			boolean expectedNextBufferIsEvent,
+			boolean expectedRecycledAfterRecycle) throws IOException, InterruptedException {
+		assertNextBufferOrEvent(
+				readView,
+				expectedReadableBufferSize,
+				false,
+				expectedEventClass,
+				expectedIsMoreAvailable,
+				expectedBuffersInBacklog,
+				expectedNextBufferIsEvent,
+				expectedRecycledAfterRecycle);
+	}
+
+	private static void assertNextBufferOrEvent(
+			ResultSubpartitionView readView,
+			int expectedReadableBufferSize,
+			boolean expectedIsBuffer,
+			@Nullable Class<? extends AbstractEvent> expectedEventClass,
+			boolean expectedIsMoreAvailable,
+			int expectedBuffersInBacklog,
+			boolean expectedNextBufferIsEvent,
+			boolean expectedRecycledAfterRecycle) throws IOException, InterruptedException {
+		checkArgument(expectedEventClass == null || !expectedIsBuffer);
+
+		ResultSubpartition.BufferAndBacklog bufferAndBacklog = readView.getNextBuffer();
+		assertNotNull(bufferAndBacklog);
+		try {
+			assertEquals("buffer size", expectedReadableBufferSize,
+					bufferAndBacklog.buffer().readableBytes());
+			assertEquals("buffer or event", expectedIsBuffer,
+					bufferAndBacklog.buffer().isBuffer());
+			if (expectedEventClass != null) {
+				Assert.assertThat(EventSerializer
+								.fromBuffer(bufferAndBacklog.buffer(), ClassLoader.getSystemClassLoader()),
+						instanceOf(expectedEventClass));
+			}
+			assertEquals("more available", expectedIsMoreAvailable,
+					bufferAndBacklog.isMoreAvailable());
+			assertEquals("more available", expectedIsMoreAvailable, readView.isAvailable());
+			assertEquals("backlog", expectedBuffersInBacklog, bufferAndBacklog.buffersInBacklog());
+			assertEquals("next is event", expectedNextBufferIsEvent,
+					bufferAndBacklog.nextBufferIsEvent());
+			assertEquals("next is event", expectedNextBufferIsEvent,
+					readView.nextBufferIsEvent());
+
+			assertFalse("not recycled", bufferAndBacklog.buffer().isRecycled());
+		} finally {
+			bufferAndBacklog.buffer().recycleBuffer();
+		}
+		assertEquals("recycled", expectedRecycledAfterRecycle, bufferAndBacklog.buffer().isRecycled());
+	}
+
+	static void assertNoNextBuffer(ResultSubpartitionView readView) throws IOException, InterruptedException {
+		assertNull(readView.getNextBuffer());
+	}
 }
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java
index e4006be..7c083ad 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java
@@ -18,27 +18,18 @@
 
 package org.apache.flink.runtime.io.network.partition;
 
-import org.apache.flink.runtime.event.AbstractEvent;
-import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
 import org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils;
 import org.apache.flink.runtime.io.network.buffer.BufferConsumer;
 import org.apache.flink.util.TestLogger;
 
 import org.junit.Test;
 
-import javax.annotation.Nullable;
-
-import java.io.IOException;
-
 import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledBufferConsumer;
-import static org.apache.flink.util.Preconditions.checkArgument;
-import static org.hamcrest.core.IsInstanceOf.instanceOf;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 import static org.mockito.Mockito.mock;
 
 /**
@@ -135,84 +126,4 @@ public abstract class SubpartitionTestBase extends TestLogger {
 		// Verify that parent release is reflected at partition view
 		assertTrue(view.isReleased());
 	}
-
-	static void assertNextBuffer(
-			ResultSubpartitionView readView,
-			int expectedReadableBufferSize,
-			boolean expectedIsMoreAvailable,
-			int expectedBuffersInBacklog,
-			boolean expectedNextBufferIsEvent,
-			boolean expectedRecycledAfterRecycle) throws IOException, InterruptedException {
-		assertNextBufferOrEvent(
-			readView,
-			expectedReadableBufferSize,
-			true,
-			null,
-			expectedIsMoreAvailable,
-			expectedBuffersInBacklog,
-			expectedNextBufferIsEvent,
-			expectedRecycledAfterRecycle);
-	}
-
-	static void assertNextEvent(
-			ResultSubpartitionView readView,
-			int expectedReadableBufferSize,
-			Class<? extends AbstractEvent> expectedEventClass,
-			boolean expectedIsMoreAvailable,
-			int expectedBuffersInBacklog,
-			boolean expectedNextBufferIsEvent,
-			boolean expectedRecycledAfterRecycle) throws IOException, InterruptedException {
-		assertNextBufferOrEvent(
-			readView,
-			expectedReadableBufferSize,
-			false,
-			expectedEventClass,
-			expectedIsMoreAvailable,
-			expectedBuffersInBacklog,
-			expectedNextBufferIsEvent,
-			expectedRecycledAfterRecycle);
-	}
-
-	private static void assertNextBufferOrEvent(
-			ResultSubpartitionView readView,
-			int expectedReadableBufferSize,
-			boolean expectedIsBuffer,
-			@Nullable Class<? extends AbstractEvent> expectedEventClass,
-			boolean expectedIsMoreAvailable,
-			int expectedBuffersInBacklog,
-			boolean expectedNextBufferIsEvent,
-			boolean expectedRecycledAfterRecycle) throws IOException, InterruptedException {
-		checkArgument(expectedEventClass == null || !expectedIsBuffer);
-
-		ResultSubpartition.BufferAndBacklog bufferAndBacklog = readView.getNextBuffer();
-		assertNotNull(bufferAndBacklog);
-		try {
-			assertEquals("buffer size", expectedReadableBufferSize,
-				bufferAndBacklog.buffer().readableBytes());
-			assertEquals("buffer or event", expectedIsBuffer,
-				bufferAndBacklog.buffer().isBuffer());
-			if (expectedEventClass != null) {
-				assertThat(EventSerializer
-						.fromBuffer(bufferAndBacklog.buffer(), ClassLoader.getSystemClassLoader()),
-					instanceOf(expectedEventClass));
-			}
-			assertEquals("more available", expectedIsMoreAvailable,
-				bufferAndBacklog.isMoreAvailable());
-			assertEquals("more available", expectedIsMoreAvailable, readView.isAvailable());
-			assertEquals("backlog", expectedBuffersInBacklog, bufferAndBacklog.buffersInBacklog());
-			assertEquals("next is event", expectedNextBufferIsEvent,
-				bufferAndBacklog.nextBufferIsEvent());
-			assertEquals("next is event", expectedNextBufferIsEvent,
-				readView.nextBufferIsEvent());
-
-			assertFalse("not recycled", bufferAndBacklog.buffer().isRecycled());
-		} finally {
-			bufferAndBacklog.buffer().recycleBuffer();
-		}
-		assertEquals("recycled", expectedRecycledAfterRecycle, bufferAndBacklog.buffer().isRecycled());
-	}
-
-	static void assertNoNextBuffer(ResultSubpartitionView readView) throws IOException, InterruptedException {
-		assertNull(readView.getNextBuffer());
-	}
 }


[flink] 02/14: [FLINK-8513][docs] Add documentation for connecting to non-AWS S3 endpoints

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

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

commit 4dae94d952958858af722c6285398bc6402708e4
Author: Seth Wiesman <sj...@gmail.com>
AuthorDate: Tue Apr 30 16:52:25 2019 -0500

    [FLINK-8513][docs] Add documentation for connecting to non-AWS S3 endpoints
---
 docs/ops/filesystems/s3.md    | 9 ++++++++-
 docs/ops/filesystems/s3.zh.md | 9 ++++++++-
 2 files changed, 16 insertions(+), 2 deletions(-)

diff --git a/docs/ops/filesystems/s3.md b/docs/ops/filesystems/s3.md
index 9ae1476..4efcca2 100644
--- a/docs/ops/filesystems/s3.md
+++ b/docs/ops/filesystems/s3.md
@@ -103,7 +103,14 @@ s3.access-key: your-access-key
 s3.secret-key: your-secret-key
 {% endhighlight %}
 
-{% top %}
+## Configure Non-S3 Endpoint
+
+The S3 Filesystems also support using S3 compliant object stores such as [IBM's Cloud Object Storage](https://www.ibm.com/cloud/object-storage) and [Minio](https://min.io/).
+To do so, simply configure your endpoint in `flink-conf.yaml`. 
+
+{% highlight yaml %}
+s3.endpoint: your-endpoint-hostname
+{% endhighlight %}
 
 ## Entropy injection for S3 file systems
 
diff --git a/docs/ops/filesystems/s3.zh.md b/docs/ops/filesystems/s3.zh.md
index 9ae1476..4efcca2 100644
--- a/docs/ops/filesystems/s3.zh.md
+++ b/docs/ops/filesystems/s3.zh.md
@@ -103,7 +103,14 @@ s3.access-key: your-access-key
 s3.secret-key: your-secret-key
 {% endhighlight %}
 
-{% top %}
+## Configure Non-S3 Endpoint
+
+The S3 Filesystems also support using S3 compliant object stores such as [IBM's Cloud Object Storage](https://www.ibm.com/cloud/object-storage) and [Minio](https://min.io/).
+To do so, simply configure your endpoint in `flink-conf.yaml`. 
+
+{% highlight yaml %}
+s3.endpoint: your-endpoint-hostname
+{% endhighlight %}
 
 ## Entropy injection for S3 file systems
 


[flink] 07/14: [hotfix] Expand JavaDoc of MemorySegment.wrap()

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

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

commit 541e413c9053ab3be030e6079333dca7779e2305
Author: Stephan Ewen <se...@apache.org>
AuthorDate: Sun Apr 14 14:52:05 2019 +0200

    [hotfix] Expand JavaDoc of MemorySegment.wrap()
---
 .../src/main/java/org/apache/flink/core/memory/MemorySegment.java      | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)

diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java
index 614696c..0331736 100644
--- a/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java
+++ b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java
@@ -271,7 +271,8 @@ public abstract class MemorySegment {
 
 	/**
 	 * Wraps the chunk of the underlying memory located between <tt>offset</tt> and
-	 * <tt>length</tt> in a NIO ByteBuffer.
+	 * <tt>length</tt> in a NIO ByteBuffer. The ByteBuffer has the full segment as capacity
+	 * and the offset and length parameters set the buffers position and limit.
 	 *
 	 * @param offset The offset in the memory segment.
 	 * @param length The number of bytes to be wrapped as a buffer.