pnowojski commented on a change in pull request #11687: URL: https://github.com/apache/flink/pull/11687#discussion_r414711050
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java ########## @@ -97,6 +100,18 @@ public LocalInputChannel( // Consume // ------------------------------------------------------------------------ + @Override + public void readRecoveredState(ChannelStateReader reader) throws IOException, InterruptedException { + synchronized (bufferQueue) { + // In most of cases we only need one buffer for reading recovered state except in very large record case. + // Then only one floating buffer is required to avoid receive more floating buffers after recovery. Even + // though we need more buffers for recovery in large record case, it only increases some interactions with pool. + numRequiredBuffers = 1; + } + + super.readRecoveredState(reader); Review comment: ? ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ########## @@ -371,7 +373,14 @@ boolean isWaitingForFloatingBuffers() { @VisibleForTesting public Buffer getNextReceivedBuffer() { - return receivedBuffers.poll(); + synchronized (receivedBuffers) { + return receivedBuffers.poll(); + } Review comment: Why add `synchornized` section? ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ########## @@ -477,11 +362,11 @@ public int unsynchronizedGetNumberOfQueuedBuffers() { } public int unsynchronizedGetExclusiveBuffersUsed() { - return Math.max(0, initialCredit - bufferQueue.exclusiveBuffers.size()); + return Math.max(0, initialCredit - bufferManager.getNumberOfAvailableExclusiveBuffers()); Review comment: I would keep the `unsynchronized` prefix for this method and `getNumberOfAvailableFloatingBuffers` as well. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ########## @@ -321,111 +274,43 @@ private void notifyCreditAvailable() { partitionRequestClient.notifyCreditAvailable(this); } - /** - * Exclusive buffer is recycled to this input channel directly and it may trigger return extra - * floating buffer and notify increased credit to the producer. - * - * @param segment The exclusive segment of this channel. - */ - @Override - public void recycle(MemorySegment segment) { - int numAddedBuffers; - - synchronized (bufferQueue) { - // Similar to notifyBufferAvailable(), make sure that we never add a buffer - // after releaseAllResources() released all buffers (see below for details). - if (isReleased.get()) { - try { - memorySegmentProvider.recycleMemorySegments(Collections.singletonList(segment)); - return; - } catch (Throwable t) { - ExceptionUtils.rethrow(t); - } - } - numAddedBuffers = bufferQueue.addExclusiveBuffer(new NetworkBuffer(segment, this), numRequiredBuffers); - } - - if (numAddedBuffers > 0 && unannouncedCredit.getAndAdd(numAddedBuffers) == 0) { - notifyCreditAvailable(); - } - } - public int getNumberOfAvailableBuffers() { - synchronized (bufferQueue) { - return bufferQueue.getAvailableBufferSize(); - } + return bufferManager.getNumberOfAvailableBuffers(); } public int getNumberOfRequiredBuffers() { - return numRequiredBuffers; + return bufferManager.getNumberOfRequiredBuffers(); } Review comment: This method is not synchronized. However it could be marked as `@VisibleForTesting`, so we could ignore this issue for now + adding `unsynchronized` prefix? ########## 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(); + Buffer buffer = bufferPool.requestBuffer(); + if (buffer != null) { + bufferQueue.addFloatingBuffer(buffer); + numRequestedBuffers++; + } else if (bufferPool.addBufferListener(this)) { + isWaitingForFloatingBuffers = true; + break; + } + } + } + return numRequestedBuffers; + } + + // ------------------------------------------------------------------------ + // Buffer recycle + // ------------------------------------------------------------------------ + + /** + * Exclusive buffer is recycled to this channel manager directly and it may trigger return extra + * floating buffer based on <tt>numRequiredBuffers</tt>. + * + * @param segment The exclusive segment of this channel. + */ + @Override + public void recycle(MemorySegment segment) { + int numAddedBuffers = 0; + synchronized (bufferQueue) { + try { + // Similar to notifyBufferAvailable(), make sure that we never add a buffer + // after channel released all buffers via releaseAllResources(). + if (inputChannel.isReleased()) { + globalPool.recycleMemorySegments(Collections.singletonList(segment)); + } else { + numAddedBuffers = bufferQueue.addExclusiveBuffer(new NetworkBuffer(segment, this), numRequiredBuffers); + } + } catch (Throwable t) { + ExceptionUtils.rethrow(t); + } + } + inputChannel.notifyBufferAvailable(numAddedBuffers); + } + + /** + * Recycles all the exclusive and floating buffers from the given buffer queue. + */ + void releaseAllBuffers(ArrayDeque<Buffer> buffers) throws IOException { + // Gather all exclusive buffers and recycle them to global pool in batch, because + // we do not want to trigger redistribution of buffers after each recycle. + final List<MemorySegment> exclusiveRecyclingSegments = new ArrayList<>(); + + Buffer buffer; + while ((buffer = buffers.poll()) != null) { + if (buffer.getRecycler() == this) { + exclusiveRecyclingSegments.add(buffer.getMemorySegment()); + } else { + buffer.recycleBuffer(); + } + } + synchronized (bufferQueue) { + bufferQueue.releaseAll(exclusiveRecyclingSegments); + } + + if (exclusiveRecyclingSegments.size() > 0) { + globalPool.recycleMemorySegments(exclusiveRecyclingSegments); + } + } + + // ------------------------------------------------------------------------ + // Buffer listener notification + // ------------------------------------------------------------------------ + + /** + * The buffer pool notifies this listener of an available floating buffer. If the listener is released or + * currently does not need extra buffers, the buffer should be returned to the buffer pool. Otherwise, + * the buffer will be added into the <tt>bufferQueue</tt>. + * + * @param buffer Buffer that becomes available in buffer pool. + * @return NotificationResult indicates whether this channel accepts the buffer and is waiting for + * more floating buffers. + */ + @Override + public BufferListener.NotificationResult notifyBufferAvailable(Buffer buffer) { + BufferListener.NotificationResult notificationResult = BufferListener.NotificationResult.BUFFER_NOT_USED; + Throwable throwable = null; + synchronized (bufferQueue) { + try { + checkState(isWaitingForFloatingBuffers, "This channel should be waiting for floating buffers."); + + // Important: make sure that we never add a buffer after releaseAllResources() + // released all buffers. Following scenarios exist: + // 1) releaseAllBuffers() already released buffers inside bufferQueue + // -> while isReleased is set correctly in InputChannel + // 2) releaseAllBuffers() did not yet release buffers from bufferQueue + // -> we may or may not have set isReleased yet but will always wait for the + // lock on bufferQueue to release buffers + if (inputChannel.isReleased() || bufferQueue.getAvailableBufferSize() >= numRequiredBuffers) { + isWaitingForFloatingBuffers = false; + return notificationResult; + } + + bufferQueue.addFloatingBuffer(buffer); + + if (bufferQueue.getAvailableBufferSize() == numRequiredBuffers) { + isWaitingForFloatingBuffers = false; + notificationResult = BufferListener.NotificationResult.BUFFER_USED_NO_NEED_MORE; + } else { + notificationResult = BufferListener.NotificationResult.BUFFER_USED_NEED_MORE; + } + } catch (Throwable t) { + throwable = t; + } + } + + if (throwable != null) { + inputChannel.setError(throwable); + } else if (notificationResult != NotificationResult.BUFFER_NOT_USED) { + inputChannel.notifyBufferAvailable(1); + } + return notificationResult; + } + + @Override + public void notifyBufferDestroyed() { + // Nothing to do actually. + } + + // ------------------------------------------------------------------------ + // Getter properties + // ------------------------------------------------------------------------ + + int getNumberOfRequiredBuffers() { Review comment: `unsychronized` prefix + `@VisibleForTesting`? ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ########## @@ -321,111 +274,43 @@ private void notifyCreditAvailable() { partitionRequestClient.notifyCreditAvailable(this); } - /** - * Exclusive buffer is recycled to this input channel directly and it may trigger return extra - * floating buffer and notify increased credit to the producer. - * - * @param segment The exclusive segment of this channel. - */ - @Override - public void recycle(MemorySegment segment) { - int numAddedBuffers; - - synchronized (bufferQueue) { - // Similar to notifyBufferAvailable(), make sure that we never add a buffer - // after releaseAllResources() released all buffers (see below for details). - if (isReleased.get()) { - try { - memorySegmentProvider.recycleMemorySegments(Collections.singletonList(segment)); - return; - } catch (Throwable t) { - ExceptionUtils.rethrow(t); - } - } - numAddedBuffers = bufferQueue.addExclusiveBuffer(new NetworkBuffer(segment, this), numRequiredBuffers); - } - - if (numAddedBuffers > 0 && unannouncedCredit.getAndAdd(numAddedBuffers) == 0) { - notifyCreditAvailable(); - } - } - public int getNumberOfAvailableBuffers() { - synchronized (bufferQueue) { - return bufferQueue.getAvailableBufferSize(); - } + return bufferManager.getNumberOfAvailableBuffers(); } public int getNumberOfRequiredBuffers() { - return numRequiredBuffers; + return bufferManager.getNumberOfRequiredBuffers(); } public int getSenderBacklog() { Review comment: I think this method is bugged (unsychronized and used in the multi threaded context), however it seems like it's result is never used in the production code - `InputChannel#getNextBuffer` doesn't need to return `buffersInBacklog`, am I right? So we could drop it? If so, we could do it as a follow up ticket, as this is already a pre-existing issue. ########## 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, Review comment: This cyclic dependency can be an issue for refactoring/reusing the code. Can not we cut it? As it is, you are trying to decouple buffer manager from input channel, but after all they are still very strongly coupled and both have a shared state. ########## 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(); + Buffer buffer = bufferPool.requestBuffer(); + if (buffer != null) { + bufferQueue.addFloatingBuffer(buffer); + numRequestedBuffers++; + } else if (bufferPool.addBufferListener(this)) { + isWaitingForFloatingBuffers = true; + break; + } + } + } + return numRequestedBuffers; + } + + // ------------------------------------------------------------------------ + // Buffer recycle + // ------------------------------------------------------------------------ + + /** + * Exclusive buffer is recycled to this channel manager directly and it may trigger return extra + * floating buffer based on <tt>numRequiredBuffers</tt>. + * + * @param segment The exclusive segment of this channel. + */ + @Override + public void recycle(MemorySegment segment) { + int numAddedBuffers = 0; + synchronized (bufferQueue) { + try { + // Similar to notifyBufferAvailable(), make sure that we never add a buffer + // after channel released all buffers via releaseAllResources(). + if (inputChannel.isReleased()) { + globalPool.recycleMemorySegments(Collections.singletonList(segment)); + } else { + numAddedBuffers = bufferQueue.addExclusiveBuffer(new NetworkBuffer(segment, this), numRequiredBuffers); + } + } catch (Throwable t) { + ExceptionUtils.rethrow(t); + } + } + inputChannel.notifyBufferAvailable(numAddedBuffers); + } + + /** + * Recycles all the exclusive and floating buffers from the given buffer queue. + */ + void releaseAllBuffers(ArrayDeque<Buffer> buffers) throws IOException { + // Gather all exclusive buffers and recycle them to global pool in batch, because + // we do not want to trigger redistribution of buffers after each recycle. + final List<MemorySegment> exclusiveRecyclingSegments = new ArrayList<>(); + + Buffer buffer; + while ((buffer = buffers.poll()) != null) { + if (buffer.getRecycler() == this) { + exclusiveRecyclingSegments.add(buffer.getMemorySegment()); + } else { + buffer.recycleBuffer(); + } + } + synchronized (bufferQueue) { + bufferQueue.releaseAll(exclusiveRecyclingSegments); + } + + if (exclusiveRecyclingSegments.size() > 0) { + globalPool.recycleMemorySegments(exclusiveRecyclingSegments); + } + } + + // ------------------------------------------------------------------------ + // Buffer listener notification + // ------------------------------------------------------------------------ + + /** + * The buffer pool notifies this listener of an available floating buffer. If the listener is released or + * currently does not need extra buffers, the buffer should be returned to the buffer pool. Otherwise, + * the buffer will be added into the <tt>bufferQueue</tt>. + * + * @param buffer Buffer that becomes available in buffer pool. + * @return NotificationResult indicates whether this channel accepts the buffer and is waiting for + * more floating buffers. + */ + @Override + public BufferListener.NotificationResult notifyBufferAvailable(Buffer buffer) { + BufferListener.NotificationResult notificationResult = BufferListener.NotificationResult.BUFFER_NOT_USED; + Throwable throwable = null; + synchronized (bufferQueue) { + try { + checkState(isWaitingForFloatingBuffers, "This channel should be waiting for floating buffers."); + + // Important: make sure that we never add a buffer after releaseAllResources() + // released all buffers. Following scenarios exist: + // 1) releaseAllBuffers() already released buffers inside bufferQueue + // -> while isReleased is set correctly in InputChannel + // 2) releaseAllBuffers() did not yet release buffers from bufferQueue + // -> we may or may not have set isReleased yet but will always wait for the + // lock on bufferQueue to release buffers + if (inputChannel.isReleased() || bufferQueue.getAvailableBufferSize() >= numRequiredBuffers) { + isWaitingForFloatingBuffers = false; + return notificationResult; + } + + bufferQueue.addFloatingBuffer(buffer); + + if (bufferQueue.getAvailableBufferSize() == numRequiredBuffers) { + isWaitingForFloatingBuffers = false; + notificationResult = BufferListener.NotificationResult.BUFFER_USED_NO_NEED_MORE; + } else { + notificationResult = BufferListener.NotificationResult.BUFFER_USED_NEED_MORE; + } + } catch (Throwable t) { + throwable = t; + } + } + + if (throwable != null) { + inputChannel.setError(throwable); + } else if (notificationResult != NotificationResult.BUFFER_NOT_USED) { + inputChannel.notifyBufferAvailable(1); + } + return notificationResult; + } + + @Override + public void notifyBufferDestroyed() { + // Nothing to do actually. + } + + // ------------------------------------------------------------------------ + // Getter properties + // ------------------------------------------------------------------------ + + int getNumberOfRequiredBuffers() { + return numRequiredBuffers; + } + + @VisibleForTesting + boolean isWaitingForFloatingBuffers() { Review comment: `unsychronized` prefix? ########## File path: flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/BackPressureITCase.java ########## @@ -94,7 +94,7 @@ private static Configuration createNetworkBufferConfiguration() { final Configuration configuration = new Configuration(); final int memorySegmentSizeKb = 32; - final MemorySize networkBuffersMemory = MemorySize.parse(memorySegmentSizeKb * (NUM_TASKS + 2) + "kb"); + final MemorySize networkBuffersMemory = MemorySize.parse(memorySegmentSizeKb * 6 + "kb"); Review comment: Why `6` instead of `(NUM_TASKS + 3)`? And also why has it increased? ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/AbstractRecordReader.java ########## @@ -63,6 +63,8 @@ protected AbstractRecordReader(InputGate inputGate, String[] tmpDirectories) { } protected boolean getNextRecord(T target) throws IOException, InterruptedException { + inputGate.requestPartitions(); Review comment: > I remembered in the early version, the requestPartitions was also placed inside SingleInputGate#getNext method. Because of the mailbox requirement, it was migrated into #setup afterwards. But that wasn't a good design, and it's still not now :( ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ########## @@ -105,12 +98,15 @@ public RemoteInputChannel( int maxBackoff, InputChannelMetrics metrics) { - super(inputGate, channelIndex, partitionId, initialBackOff, maxBackoff, - metrics.getNumBytesInRemoteCounter(), metrics.getNumBuffersInRemoteCounter()); + super(inputGate, channelIndex, partitionId, initialBackOff, maxBackoff, metrics); this.connectionId = checkNotNull(connectionId); this.connectionManager = checkNotNull(connectionManager); - this.bufferManager = new BufferManager(this, 0); + // In theory it should get the total number of states to indicate the numRequiredBuffers. + // Since we can not get this information in advance, and considering only one input channel + // will read state at the same time by design, then we give a maximum value here to reduce + // unnecessary interactions with buffer pool during recovery. + this.bufferManager = new BufferManager(this, Integer.MAX_VALUE); Review comment: `Integer.MAX_VALUE`? Doesn't it mean that all floating buffers will be stuck permanently in on `RemoteInputChannel` (that happened to ask for them first?) ########## 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(); + Buffer buffer = bufferPool.requestBuffer(); + if (buffer != null) { + bufferQueue.addFloatingBuffer(buffer); + numRequestedBuffers++; + } else if (bufferPool.addBufferListener(this)) { + isWaitingForFloatingBuffers = true; + break; + } + } + } + return numRequestedBuffers; + } + + // ------------------------------------------------------------------------ + // Buffer recycle + // ------------------------------------------------------------------------ + + /** + * Exclusive buffer is recycled to this channel manager directly and it may trigger return extra + * floating buffer based on <tt>numRequiredBuffers</tt>. + * + * @param segment The exclusive segment of this channel. + */ + @Override + public void recycle(MemorySegment segment) { + int numAddedBuffers = 0; + synchronized (bufferQueue) { + try { + // Similar to notifyBufferAvailable(), make sure that we never add a buffer + // after channel released all buffers via releaseAllResources(). + if (inputChannel.isReleased()) { + globalPool.recycleMemorySegments(Collections.singletonList(segment)); + } else { + numAddedBuffers = bufferQueue.addExclusiveBuffer(new NetworkBuffer(segment, this), numRequiredBuffers); + } + } catch (Throwable t) { + ExceptionUtils.rethrow(t); + } + } + inputChannel.notifyBufferAvailable(numAddedBuffers); + } + + /** + * Recycles all the exclusive and floating buffers from the given buffer queue. + */ + void releaseAllBuffers(ArrayDeque<Buffer> buffers) throws IOException { + // Gather all exclusive buffers and recycle them to global pool in batch, because + // we do not want to trigger redistribution of buffers after each recycle. + final List<MemorySegment> exclusiveRecyclingSegments = new ArrayList<>(); + + Buffer buffer; + while ((buffer = buffers.poll()) != null) { + if (buffer.getRecycler() == this) { + exclusiveRecyclingSegments.add(buffer.getMemorySegment()); + } else { + buffer.recycleBuffer(); + } + } + synchronized (bufferQueue) { + bufferQueue.releaseAll(exclusiveRecyclingSegments); + } + + if (exclusiveRecyclingSegments.size() > 0) { + globalPool.recycleMemorySegments(exclusiveRecyclingSegments); + } + } + + // ------------------------------------------------------------------------ + // Buffer listener notification + // ------------------------------------------------------------------------ + + /** + * The buffer pool notifies this listener of an available floating buffer. If the listener is released or + * currently does not need extra buffers, the buffer should be returned to the buffer pool. Otherwise, + * the buffer will be added into the <tt>bufferQueue</tt>. + * + * @param buffer Buffer that becomes available in buffer pool. + * @return NotificationResult indicates whether this channel accepts the buffer and is waiting for + * more floating buffers. + */ + @Override + public BufferListener.NotificationResult notifyBufferAvailable(Buffer buffer) { + BufferListener.NotificationResult notificationResult = BufferListener.NotificationResult.BUFFER_NOT_USED; + Throwable throwable = null; + synchronized (bufferQueue) { + try { + checkState(isWaitingForFloatingBuffers, "This channel should be waiting for floating buffers."); + + // Important: make sure that we never add a buffer after releaseAllResources() + // released all buffers. Following scenarios exist: + // 1) releaseAllBuffers() already released buffers inside bufferQueue + // -> while isReleased is set correctly in InputChannel + // 2) releaseAllBuffers() did not yet release buffers from bufferQueue + // -> we may or may not have set isReleased yet but will always wait for the + // lock on bufferQueue to release buffers + if (inputChannel.isReleased() || bufferQueue.getAvailableBufferSize() >= numRequiredBuffers) { + isWaitingForFloatingBuffers = false; + return notificationResult; + } + + bufferQueue.addFloatingBuffer(buffer); + + if (bufferQueue.getAvailableBufferSize() == numRequiredBuffers) { + isWaitingForFloatingBuffers = false; + notificationResult = BufferListener.NotificationResult.BUFFER_USED_NO_NEED_MORE; + } else { + notificationResult = BufferListener.NotificationResult.BUFFER_USED_NEED_MORE; + } + } catch (Throwable t) { + throwable = t; + } + } + + if (throwable != null) { + inputChannel.setError(throwable); + } else if (notificationResult != NotificationResult.BUFFER_NOT_USED) { + inputChannel.notifyBufferAvailable(1); + } + return notificationResult; + } + + @Override + public void notifyBufferDestroyed() { + // Nothing to do actually. + } + + // ------------------------------------------------------------------------ + // Getter properties + // ------------------------------------------------------------------------ + + int getNumberOfRequiredBuffers() { + return numRequiredBuffers; + } + + @VisibleForTesting + boolean isWaitingForFloatingBuffers() { + return isWaitingForFloatingBuffers; + } + + int getNumberOfAvailableBuffers() { + synchronized (bufferQueue) { + return bufferQueue.getAvailableBufferSize(); + } + } + + int getNumberOfAvailableExclusiveBuffers() { + return bufferQueue.exclusiveBuffers.size(); + } + + int getNumberOfAvailableFloatingBuffers() { + return bufferQueue.floatingBuffers.size(); + } Review comment: `unsychronized` prefix? ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/BufferManager.java ########## @@ -65,14 +65,10 @@ @GuardedBy("bufferQueue") private int numRequiredBuffers; - public BufferManager( - MemorySegmentProvider globalPool, - InputChannel inputChannel, - int numRequiredBuffers) { - - this.globalPool = checkNotNull(globalPool); - this.inputChannel = checkNotNull(inputChannel); + public BufferManager(InputChannel inputChannel, int numRequiredBuffers) { checkArgument(numRequiredBuffers >= 0); + this.inputChannel = checkNotNull(inputChannel); + this.globalPool = inputChannel.inputGate.getMemorySegmentProvider(); Review comment: It would be better to inject this field instead of relaying on even tighter coupling with `inputChannel`. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java ########## @@ -168,6 +175,12 @@ public void run() { Optional<BufferAndAvailability> getNextBuffer() throws IOException, InterruptedException { checkError(); + BufferAndAvailability bufferAndAvailability = getNextRecoveredStateBuffer(); + if (bufferAndAvailability != null) { Review comment: If this is `null`, but we are still recovering, we will actually go through the the `subpartitionView.getNextBuffer()`? ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java ########## @@ -168,6 +175,12 @@ public void run() { Optional<BufferAndAvailability> getNextBuffer() throws IOException, InterruptedException { checkError(); + BufferAndAvailability bufferAndAvailability = getNextRecoveredStateBuffer(); Review comment: Having to go everything through `getRecoveredStateBuffer()` every time is not very clean. ########## 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: This should probably be injected in the constructor ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java ########## @@ -87,10 +90,14 @@ public LocalInputChannel( int maxBackoff, InputChannelMetrics metrics) { - super(inputGate, channelIndex, partitionId, initialBackoff, maxBackoff, metrics.getNumBytesInLocalCounter(), metrics.getNumBuffersInLocalCounter()); + super(inputGate, channelIndex, partitionId, initialBackoff, maxBackoff, metrics); this.partitionManager = checkNotNull(partitionManager); this.taskEventPublisher = checkNotNull(taskEventPublisher); + // In most cases we only need one buffer for reading recovered state except for very large record. + // Then only one floating buffer is required. Even though we need more buffers for recovery for + // large record, it only increases some interactions with pool. + this.bufferManager = new BufferManager(this, 1); Review comment: Does it mean that for every `LocalInputChannel` after recovery we are wasting a single buffer? edit: Ok, I see, that it's being released after end of reading from recovered state, but this is very mangled and hard to understand :( ---------------------------------------------------------------- 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