zhijiangW commented on a change in pull request #11687:
URL: https://github.com/apache/flink/pull/11687#discussion_r419489175



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/BufferManager.java
##########
@@ -0,0 +1,356 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition.consumer;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentProvider;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferListener;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.util.ExceptionUtils;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * The general buffer manager used by {@link InputChannel} to request/recycle
+ * exclusive or floating buffers.
+ */
+public class BufferManager implements BufferListener, BufferRecycler {
+
+       /** The available buffer queue wraps both exclusive and requested 
floating buffers. */
+       private final AvailableBufferQueue bufferQueue = new 
AvailableBufferQueue();
+
+       /** The buffer provider for requesting exclusive buffers. */
+       private final MemorySegmentProvider globalPool;
+
+       /** The input channel to own this buffer manager. */
+       private final InputChannel inputChannel;
+
+       /** The tag indicates whether it is waiting for additional floating 
buffers from the buffer pool. */
+       @GuardedBy("bufferQueue")
+       private boolean isWaitingForFloatingBuffers;
+
+       /** The total number of required buffers for the respective input 
channel. */
+       @GuardedBy("bufferQueue")
+       private int numRequiredBuffers;
+
+       public BufferManager(
+               MemorySegmentProvider globalPool,
+               InputChannel inputChannel,
+               int numRequiredBuffers) {
+
+               this.globalPool = checkNotNull(globalPool);
+               this.inputChannel = checkNotNull(inputChannel);
+               checkArgument(numRequiredBuffers >= 0);
+               this.numRequiredBuffers = numRequiredBuffers;
+       }
+
+       // 
------------------------------------------------------------------------
+       // Buffer request
+       // 
------------------------------------------------------------------------
+
+       @Nullable
+       Buffer requestBuffer() {
+               synchronized (bufferQueue) {
+                       return bufferQueue.takeBuffer();
+               }
+       }
+
+       /**
+        * Requests exclusive buffers from the provider and returns the number 
of requested amount.
+        */
+       int requestExclusiveBuffers() throws IOException {
+               Collection<MemorySegment> segments = 
globalPool.requestMemorySegments();
+               checkArgument(!segments.isEmpty(), "The number of exclusive 
buffers per channel should be larger than 0.");
+
+               synchronized (bufferQueue) {
+                       for (MemorySegment segment : segments) {
+                               bufferQueue.addExclusiveBuffer(new 
NetworkBuffer(segment, this), numRequiredBuffers);
+                       }
+               }
+               return segments.size();
+       }
+
+       /**
+        * Requests floating buffers from the buffer pool based on the given 
required amount, and returns the actual
+        * requested amount. If the required amount is not fully satisfied, it 
will register as a listener.
+        */
+       int requestFloatingBuffers(int numRequired) throws IOException {
+               int numRequestedBuffers = 0;
+               synchronized (bufferQueue) {
+                       // Similar to notifyBufferAvailable(), make sure that 
we never add a buffer after channel
+                       // released all buffers via releaseAllResources().
+                       if (inputChannel.isReleased()) {
+                               return numRequestedBuffers;
+                       }
+
+                       numRequiredBuffers = numRequired;
+
+                       while (bufferQueue.getAvailableBufferSize() < 
numRequiredBuffers && !isWaitingForFloatingBuffers) {
+                               BufferPool bufferPool = 
inputChannel.inputGate.getBufferPool();

Review comment:
       I also hope so, but the truth is not, because we can not get the proper 
`BufferPool` while constructing the `BufferManager`. 
   
   The `BufferPool` is created in `SingleInputGate#setup`, and it is also the 
main reason of relying on `InputChannel` component in `BufferManager` to bypass 
this issue.




----------------------------------------------------------------
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


Reply via email to