jiangxin369 commented on code in PR #23851:
URL: https://github.com/apache/flink/pull/23851#discussion_r1505305577


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/BufferManager.java:
##########
@@ -132,15 +149,73 @@ private boolean shouldContinueRequest(BufferPool 
bufferPool) {
         }
     }
 
-    /** Requests exclusive buffers from the provider. */
-    void requestExclusiveBuffers(int numExclusiveBuffers) throws IOException {
-        checkArgument(numExclusiveBuffers >= 0, "Num exclusive buffers must be 
non-negative.");
+    private void resizeBufferQueue() {
+        if (shouldRequestExclusiveBufferFromGlobal()) {
+            return;
+        }
+
+        SingleInputGate inputGate = inputChannel.inputGate;
+        int currentSize = inputGate.getBufferPool().getNumBuffers();
+        int numRemoteChannels =
+                inputGate.getNumberOfInputChannels() - 
inputGate.getNumberOfLocalInputChannels();

Review Comment:
   Yes, it is. The `NumberOfLocalInputChannels` is only updated during setting 
up a `KnownInputChannel` or transferring a `UnknowInputChannel` to a known one. 
The netty thread in `BufferManager` is working after these phases, so it is 
impossible for two threads to visit the `NumberOfLocalInputChannels` 
simultaneously.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java:
##########
@@ -272,16 +273,22 @@ public void reserveSegments(int 
numberOfSegmentsToReserve) throws IOException {
                 "Can not reserve more segments than number of minimum 
segments.");
 
         CompletableFuture<?> toNotify = null;
+
+        int numSegmentsNeeded;
         synchronized (availableMemorySegments) {
             checkDestroyed();
+            numSegmentsNeeded = numberOfSegmentsToReserve - 
numberOfRequestedMemorySegments;
+        }
 
-            if (numberOfRequestedMemorySegments < numberOfSegmentsToReserve) {
-                availableMemorySegments.addAll(
-                        networkBufferPool.requestPooledMemorySegmentsBlocking(
-                                numberOfSegmentsToReserve - 
numberOfRequestedMemorySegments));
+        if (numSegmentsNeeded > 0) {

Review Comment:
   It happens when all buffers in the global buffer pool are used up even if 
all LBPs are only allocated their `min` buffers. At this time if an InputGate 
is reserving segments, it needs to wait for another LBP destroyed. Destroying 
an LBP will cause a redistribution over all LBPs, which requires the same lock 
during reserving segments, so the deadlock happens.



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

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to