[GitHub] [flink] zhijiangW commented on a change in pull request #8880: [FLINK-12986] [network] Fix instability of BoundedBlockingSubpartition under memory pressure.

2019-06-27 Thread GitBox
zhijiangW commented on a change in pull request #8880: [FLINK-12986] [network] 
Fix instability of BoundedBlockingSubpartition under memory pressure.
URL: https://github.com/apache/flink/pull/8880#discussion_r298213709
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/FileChannelMemoryMappedBoundedData.java
 ##
 @@ -0,0 +1,190 @@
+/*
+ * 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.util.IOUtils;
+
+import org.apache.flink.shaded.netty4.io.netty.util.internal.PlatformDependent;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+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.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+final class FileChannelMemoryMappedBoundedData implements BoundedData {
+
+   /** The file channel backing the memory mapped file. */
+   private final FileChannel fileChannel;
+
+   /** The reusable array with header buffer and data buffer, to use 
gathering writes on the
+* file channel ({@link 
java.nio.channels.GatheringByteChannel#write(ByteBuffer[])}). */
+   private final ByteBuffer[] headerAndBufferArray;
+
+   /** All memory mapped regions. */
+   private final ArrayList memoryMappedRegions;
+
+   /** The path of the memory mapped file. */
+   private final Path filePath;
+
+   /** The position in the file channel. Cached for efficiency, because an 
actual position
+* lookup in the channel involves various locks and checks. */
+   private long pos;
+
+   /** The position where the current memory mapped region must end. */
+   private long endOfCurrentRegion;
+
+   /** The position where the current memory mapped started. */
+   private long startOfCurrentRegion;
+
+   /** The maximum size of each mapped region. */
+   private final long maxRegionSize;
+
+   FileChannelMemoryMappedBoundedData(
 
 Review comment:
   I also thought so before, so it might add @VisiableForTesting


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #8880: [FLINK-12986] [network] Fix instability of BoundedBlockingSubpartition under memory pressure.

2019-06-27 Thread GitBox
zhijiangW commented on a change in pull request #8880: [FLINK-12986] [network] 
Fix instability of BoundedBlockingSubpartition under memory pressure.
URL: https://github.com/apache/flink/pull/8880#discussion_r298093109
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/MemoryMappedBoundedData.java
 ##
 @@ -220,28 +225,29 @@ private static int alignSize(int maxRegionSize) {
 * The "reader" for the memory region. It slices a sequence of buffers 
from the
 * sequence of mapped ByteBuffers.
 */
-   static final class BufferSlicer {
+   static final class BufferSlicer implements BoundedData.Reader {
 
-   /** The reader/decoder to the memory mapped region with the 
data we currently read from.
+   /** The memory mapped region we currently read from.
 * Max 2GB large. Further regions may be in the {@link 
#furtherData} field. */
-   private BufferToByteBuffer.Reader data;
+   private ByteBuffer data;
 
 Review comment:
   it might be better to name `currentData` comparing with `furtherData`


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #8880: [FLINK-12986] [network] Fix instability of BoundedBlockingSubpartition under memory pressure.

2019-06-27 Thread GitBox
zhijiangW commented on a change in pull request #8880: [FLINK-12986] [network] 
Fix instability of BoundedBlockingSubpartition under memory pressure.
URL: https://github.com/apache/flink/pull/8880#discussion_r298033167
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/FileChannelMemoryMappedBoundedData.java
 ##
 @@ -0,0 +1,190 @@
+/*
+ * 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.util.IOUtils;
+
+import org.apache.flink.shaded.netty4.io.netty.util.internal.PlatformDependent;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+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.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+final class FileChannelMemoryMappedBoundedData implements BoundedData {
+
+   /** The file channel backing the memory mapped file. */
+   private final FileChannel fileChannel;
+
+   /** The reusable array with header buffer and data buffer, to use 
gathering writes on the
+* file channel ({@link 
java.nio.channels.GatheringByteChannel#write(ByteBuffer[])}). */
+   private final ByteBuffer[] headerAndBufferArray;
+
+   /** All memory mapped regions. */
+   private final ArrayList memoryMappedRegions;
+
+   /** The path of the memory mapped file. */
+   private final Path filePath;
+
+   /** The position in the file channel. Cached for efficiency, because an 
actual position
+* lookup in the channel involves various locks and checks. */
+   private long pos;
+
+   /** The position where the current memory mapped region must end. */
+   private long endOfCurrentRegion;
+
+   /** The position where the current memory mapped started. */
+   private long startOfCurrentRegion;
+
+   /** The maximum size of each mapped region. */
+   private final long maxRegionSize;
+
+   FileChannelMemoryMappedBoundedData(
+   Path filePath,
+   FileChannel fileChannel,
+   int maxSizePerMappedRegion) throws IOException {
+
+   this.filePath = filePath;
+   this.fileChannel = fileChannel;
+   this.headerAndBufferArray = 
BufferReaderWriterUtil.allocatedWriteBufferArray();
+   this.memoryMappedRegions = new ArrayList<>(4);
+   this.maxRegionSize = maxSizePerMappedRegion;
+   this.endOfCurrentRegion = maxSizePerMappedRegion;
+   }
+
+   @Override
+   public void writeBuffer(Buffer buffer) throws IOException {
+   if (tryWriteBuffer(buffer)) {
+   return;
+   }
+
+   mapRegionAndStartNext();
+
+   if (!tryWriteBuffer(buffer)) {
+   throwTooLargeBuffer(buffer);
+   }
+   }
+
+   private boolean tryWriteBuffer(Buffer buffer) throws IOException {
+   final long spaceLeft = endOfCurrentRegion - pos;
+   final long bytesWritten = 
BufferReaderWriterUtil.writeToByteChannelIfBelowSize(
+   fileChannel, buffer, headerAndBufferArray, 
spaceLeft);
+
+   if (bytesWritten >= 0) {
+   pos += bytesWritten;
+   return true;
+   }
+   else {
+   return false;
+   }
+   }
+
+   @Override
+   public BoundedData.Reader createReader() {
+   checkState(!fileChannel.isOpen());
+
+   final List buffers = memoryMappedRegions.stream()
+   .map((bb) -> 
bb.duplicate().order(ByteOrder.nativeOrder()))
+   .collect(Collectors.toList());
+
+   return new MemoryMappedBoundedData.BufferSlicer(buffers);
+   }
+
+  

[GitHub] [flink] zhijiangW commented on a change in pull request #8880: [FLINK-12986] [network] Fix instability of BoundedBlockingSubpartition under memory pressure.

2019-06-27 Thread GitBox
zhijiangW commented on a change in pull request #8880: [FLINK-12986] [network] 
Fix instability of BoundedBlockingSubpartition under memory pressure.
URL: https://github.com/apache/flink/pull/8880#discussion_r298030956
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/FileChannelMemoryMappedBoundedData.java
 ##
 @@ -0,0 +1,190 @@
+/*
+ * 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.util.IOUtils;
+
+import org.apache.flink.shaded.netty4.io.netty.util.internal.PlatformDependent;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+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.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+final class FileChannelMemoryMappedBoundedData implements BoundedData {
+
+   /** The file channel backing the memory mapped file. */
+   private final FileChannel fileChannel;
+
+   /** The reusable array with header buffer and data buffer, to use 
gathering writes on the
+* file channel ({@link 
java.nio.channels.GatheringByteChannel#write(ByteBuffer[])}). */
+   private final ByteBuffer[] headerAndBufferArray;
+
+   /** All memory mapped regions. */
+   private final ArrayList memoryMappedRegions;
+
+   /** The path of the memory mapped file. */
+   private final Path filePath;
+
+   /** The position in the file channel. Cached for efficiency, because an 
actual position
+* lookup in the channel involves various locks and checks. */
+   private long pos;
+
+   /** The position where the current memory mapped region must end. */
+   private long endOfCurrentRegion;
+
+   /** The position where the current memory mapped started. */
+   private long startOfCurrentRegion;
+
+   /** The maximum size of each mapped region. */
+   private final long maxRegionSize;
+
+   FileChannelMemoryMappedBoundedData(
 
 Review comment:
   could be private if it is constructed only from factory atm


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #8880: [FLINK-12986] [network] Fix instability of BoundedBlockingSubpartition under memory pressure.

2019-06-27 Thread GitBox
zhijiangW commented on a change in pull request #8880: [FLINK-12986] [network] 
Fix instability of BoundedBlockingSubpartition under memory pressure.
URL: https://github.com/apache/flink/pull/8880#discussion_r298030365
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/FileChannelMemoryMappedBoundedData.java
 ##
 @@ -0,0 +1,190 @@
+/*
+ * 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.util.IOUtils;
+
+import org.apache.flink.shaded.netty4.io.netty.util.internal.PlatformDependent;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+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.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+final class FileChannelMemoryMappedBoundedData implements BoundedData {
+
+   /** The file channel backing the memory mapped file. */
+   private final FileChannel fileChannel;
+
+   /** The reusable array with header buffer and data buffer, to use 
gathering writes on the
+* file channel ({@link 
java.nio.channels.GatheringByteChannel#write(ByteBuffer[])}). */
+   private final ByteBuffer[] headerAndBufferArray;
+
+   /** All memory mapped regions. */
+   private final ArrayList memoryMappedRegions;
+
+   /** The path of the memory mapped file. */
+   private final Path filePath;
+
+   /** The position in the file channel. Cached for efficiency, because an 
actual position
+* lookup in the channel involves various locks and checks. */
+   private long pos;
+
+   /** The position where the current memory mapped region must end. */
+   private long endOfCurrentRegion;
+
+   /** The position where the current memory mapped started. */
+   private long startOfCurrentRegion;
+
+   /** The maximum size of each mapped region. */
+   private final long maxRegionSize;
+
+   FileChannelMemoryMappedBoundedData(
+   Path filePath,
+   FileChannel fileChannel,
+   int maxSizePerMappedRegion) throws IOException {
 
 Review comment:
   nit: `IOException` would never be thrown atm


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #8880: [FLINK-12986] [network] Fix instability of BoundedBlockingSubpartition under memory pressure.

2019-06-27 Thread GitBox
zhijiangW commented on a change in pull request #8880: [FLINK-12986] [network] 
Fix instability of BoundedBlockingSubpartition under memory pressure.
URL: https://github.com/apache/flink/pull/8880#discussion_r298030022
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/FileChannelMemoryMappedBoundedData.java
 ##
 @@ -0,0 +1,190 @@
+/*
+ * 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.util.IOUtils;
+
+import org.apache.flink.shaded.netty4.io.netty.util.internal.PlatformDependent;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+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.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+final class FileChannelMemoryMappedBoundedData implements BoundedData {
+
+   /** The file channel backing the memory mapped file. */
+   private final FileChannel fileChannel;
+
+   /** The reusable array with header buffer and data buffer, to use 
gathering writes on the
+* file channel ({@link 
java.nio.channels.GatheringByteChannel#write(ByteBuffer[])}). */
+   private final ByteBuffer[] headerAndBufferArray;
+
+   /** All memory mapped regions. */
+   private final ArrayList memoryMappedRegions;
+
+   /** The path of the memory mapped file. */
+   private final Path filePath;
+
+   /** The position in the file channel. Cached for efficiency, because an 
actual position
+* lookup in the channel involves various locks and checks. */
+   private long pos;
+
+   /** The position where the current memory mapped region must end. */
+   private long endOfCurrentRegion;
+
+   /** The position where the current memory mapped started. */
+   private long startOfCurrentRegion;
+
+   /** The maximum size of each mapped region. */
+   private final long maxRegionSize;
+
+   FileChannelMemoryMappedBoundedData(
+   Path filePath,
+   FileChannel fileChannel,
+   int maxSizePerMappedRegion) throws IOException {
+
+   this.filePath = filePath;
+   this.fileChannel = fileChannel;
+   this.headerAndBufferArray = 
BufferReaderWriterUtil.allocatedWriteBufferArray();
+   this.memoryMappedRegions = new ArrayList<>(4);
+   this.maxRegionSize = maxSizePerMappedRegion;
+   this.endOfCurrentRegion = maxSizePerMappedRegion;
+   }
+
+   @Override
+   public void writeBuffer(Buffer buffer) throws IOException {
+   if (tryWriteBuffer(buffer)) {
+   return;
+   }
+
+   mapRegionAndStartNext();
+
+   if (!tryWriteBuffer(buffer)) {
+   throwTooLargeBuffer(buffer);
+   }
+   }
+
+   private boolean tryWriteBuffer(Buffer buffer) throws IOException {
+   final long spaceLeft = endOfCurrentRegion - pos;
+   final long bytesWritten = 
BufferReaderWriterUtil.writeToByteChannelIfBelowSize(
+   fileChannel, buffer, headerAndBufferArray, 
spaceLeft);
+
+   if (bytesWritten >= 0) {
+   pos += bytesWritten;
+   return true;
+   }
+   else {
+   return false;
+   }
+   }
+
+   @Override
+   public BoundedData.Reader createReader() {
+   checkState(!fileChannel.isOpen());
+
+   final List buffers = memoryMappedRegions.stream()
+   .map((bb) -> 
bb.duplicate().order(ByteOrder.nativeOrder()))
+   .collect(Collectors.toList());
+
+   return new MemoryMappedBoundedData.BufferSlicer(buffers);
+   }
+
+  

[GitHub] [flink] zhijiangW commented on a change in pull request #8880: [FLINK-12986] [network] Fix instability of BoundedBlockingSubpartition under memory pressure.

2019-06-27 Thread GitBox
zhijiangW commented on a change in pull request #8880: [FLINK-12986] [network] 
Fix instability of BoundedBlockingSubpartition under memory pressure.
URL: https://github.com/apache/flink/pull/8880#discussion_r298029284
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/FileChannelMemoryMappedBoundedData.java
 ##
 @@ -0,0 +1,190 @@
+/*
+ * 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.util.IOUtils;
+
+import org.apache.flink.shaded.netty4.io.netty.util.internal.PlatformDependent;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+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.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+final class FileChannelMemoryMappedBoundedData implements BoundedData {
+
+   /** The file channel backing the memory mapped file. */
+   private final FileChannel fileChannel;
+
+   /** The reusable array with header buffer and data buffer, to use 
gathering writes on the
+* file channel ({@link 
java.nio.channels.GatheringByteChannel#write(ByteBuffer[])}). */
+   private final ByteBuffer[] headerAndBufferArray;
+
+   /** All memory mapped regions. */
+   private final ArrayList memoryMappedRegions;
+
+   /** The path of the memory mapped file. */
+   private final Path filePath;
+
+   /** The position in the file channel. Cached for efficiency, because an 
actual position
+* lookup in the channel involves various locks and checks. */
+   private long pos;
+
+   /** The position where the current memory mapped region must end. */
+   private long endOfCurrentRegion;
+
+   /** The position where the current memory mapped started. */
+   private long startOfCurrentRegion;
+
+   /** The maximum size of each mapped region. */
+   private final long maxRegionSize;
+
+   FileChannelMemoryMappedBoundedData(
+   Path filePath,
+   FileChannel fileChannel,
+   int maxSizePerMappedRegion) throws IOException {
+
+   this.filePath = filePath;
+   this.fileChannel = fileChannel;
+   this.headerAndBufferArray = 
BufferReaderWriterUtil.allocatedWriteBufferArray();
+   this.memoryMappedRegions = new ArrayList<>(4);
+   this.maxRegionSize = maxSizePerMappedRegion;
+   this.endOfCurrentRegion = maxSizePerMappedRegion;
+   }
+
+   @Override
+   public void writeBuffer(Buffer buffer) throws IOException {
+   if (tryWriteBuffer(buffer)) {
+   return;
+   }
+
+   mapRegionAndStartNext();
+
+   if (!tryWriteBuffer(buffer)) {
+   throwTooLargeBuffer(buffer);
+   }
+   }
+
+   private boolean tryWriteBuffer(Buffer buffer) throws IOException {
+   final long spaceLeft = endOfCurrentRegion - pos;
+   final long bytesWritten = 
BufferReaderWriterUtil.writeToByteChannelIfBelowSize(
+   fileChannel, buffer, headerAndBufferArray, 
spaceLeft);
+
+   if (bytesWritten >= 0) {
+   pos += bytesWritten;
+   return true;
+   }
+   else {
+   return false;
+   }
+   }
+
+   @Override
+   public BoundedData.Reader createReader() {
+   checkState(!fileChannel.isOpen());
+
+   final List buffers = memoryMappedRegions.stream()
+   .map((bb) -> 
bb.duplicate().order(ByteOrder.nativeOrder()))
+   .collect(Collectors.toList());
+
+   return new MemoryMappedBoundedData.BufferSlicer(buffers);
+   }
+
+  

[GitHub] [flink] zhijiangW commented on a change in pull request #8880: [FLINK-12986] [network] Fix instability of BoundedBlockingSubpartition under memory pressure.

2019-06-27 Thread GitBox
zhijiangW commented on a change in pull request #8880: [FLINK-12986] [network] 
Fix instability of BoundedBlockingSubpartition under memory pressure.
URL: https://github.com/apache/flink/pull/8880#discussion_r298029284
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/FileChannelMemoryMappedBoundedData.java
 ##
 @@ -0,0 +1,190 @@
+/*
+ * 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.util.IOUtils;
+
+import org.apache.flink.shaded.netty4.io.netty.util.internal.PlatformDependent;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+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.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+final class FileChannelMemoryMappedBoundedData implements BoundedData {
+
+   /** The file channel backing the memory mapped file. */
+   private final FileChannel fileChannel;
+
+   /** The reusable array with header buffer and data buffer, to use 
gathering writes on the
+* file channel ({@link 
java.nio.channels.GatheringByteChannel#write(ByteBuffer[])}). */
+   private final ByteBuffer[] headerAndBufferArray;
+
+   /** All memory mapped regions. */
+   private final ArrayList memoryMappedRegions;
+
+   /** The path of the memory mapped file. */
+   private final Path filePath;
+
+   /** The position in the file channel. Cached for efficiency, because an 
actual position
+* lookup in the channel involves various locks and checks. */
+   private long pos;
+
+   /** The position where the current memory mapped region must end. */
+   private long endOfCurrentRegion;
+
+   /** The position where the current memory mapped started. */
+   private long startOfCurrentRegion;
+
+   /** The maximum size of each mapped region. */
+   private final long maxRegionSize;
+
+   FileChannelMemoryMappedBoundedData(
+   Path filePath,
+   FileChannel fileChannel,
+   int maxSizePerMappedRegion) throws IOException {
+
+   this.filePath = filePath;
+   this.fileChannel = fileChannel;
+   this.headerAndBufferArray = 
BufferReaderWriterUtil.allocatedWriteBufferArray();
+   this.memoryMappedRegions = new ArrayList<>(4);
+   this.maxRegionSize = maxSizePerMappedRegion;
+   this.endOfCurrentRegion = maxSizePerMappedRegion;
+   }
+
+   @Override
+   public void writeBuffer(Buffer buffer) throws IOException {
+   if (tryWriteBuffer(buffer)) {
+   return;
+   }
+
+   mapRegionAndStartNext();
+
+   if (!tryWriteBuffer(buffer)) {
+   throwTooLargeBuffer(buffer);
+   }
+   }
+
+   private boolean tryWriteBuffer(Buffer buffer) throws IOException {
+   final long spaceLeft = endOfCurrentRegion - pos;
+   final long bytesWritten = 
BufferReaderWriterUtil.writeToByteChannelIfBelowSize(
+   fileChannel, buffer, headerAndBufferArray, 
spaceLeft);
+
+   if (bytesWritten >= 0) {
+   pos += bytesWritten;
+   return true;
+   }
+   else {
+   return false;
+   }
+   }
+
+   @Override
+   public BoundedData.Reader createReader() {
+   checkState(!fileChannel.isOpen());
+
+   final List buffers = memoryMappedRegions.stream()
+   .map((bb) -> 
bb.duplicate().order(ByteOrder.nativeOrder()))
+   .collect(Collectors.toList());
+
+   return new MemoryMappedBoundedData.BufferSlicer(buffers);
+   }
+
+