You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2019/05/06 07:34:10 UTC

[GitHub] [flink] zhijiangW commented on a change in pull request #8290: [FLINK-12070] [network] Implement new bounded blocking subpartitions

zhijiangW commented on a change in pull request #8290: [FLINK-12070] [network] Implement new bounded blocking subpartitions
URL: https://github.com/apache/flink/pull/8290#discussion_r281084471
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartition.java
 ##########
 @@ -0,0 +1,416 @@
+/*
+ * 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.runtime.io.network.partition.BoundedBlockingSubpartitionMemory.Writer;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.IOUtils;
+
+import org.apache.flink.shaded.netty4.io.netty.util.internal.PlatformDependent;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+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.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+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.
+ */
+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, filePath, Integer.MAX_VALUE);
+	}
+
+	/**
+	 * Constructor for testing. By default regions are rolled over at 2GB (max size of direct buffers
+	 * in Java). This constructor allows tests to pass in a smaller threshold to test rolling over
+	 * without having to actually produce more than 2GB during testing.
+	 */
+	@VisibleForTesting
+	BoundedBlockingSubpartition(
+			int index,
+			ResultPartition parent,
+			Path filePath,
+			int maxMMapRegionSize) throws IOException {
+
+		super(index, parent);
+
+		final FileChannel fc = FileChannel.open(filePath,
+				StandardOpenOption.READ, StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW);
+
+		this.memory = new MemoryMappedBuffers(filePath, fc, maxMMapRegionSize);
+		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 {
 
 Review comment:
   `flushCurrentBuffer` might be called by both task thread and flusher thread, so it exists race condition here. The variable `currentBuffer` should be visible for both threads.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services