zhijiangW commented on a change in pull request #9993: 
[FLINK-14498][runtime]Introduce NetworkBufferPool#isAvailable() for interacting 
with LocalBufferPool.
URL: https://github.com/apache/flink/pull/9993#discussion_r339025236
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java
 ##########
 @@ -231,18 +231,43 @@ private BufferBuilder toBufferBuilder(MemorySegment 
memorySegment) {
 
        @Nullable
        private MemorySegment requestMemorySegment(boolean isBlocking) throws 
InterruptedException, IOException {
+               // a flag to indicate whether the blocking request thread is
+               // waiting and needs to be notified
+               final AtomicBoolean waitingFlag = new AtomicBoolean(false);
                MemorySegment segment = null;
                synchronized (availableMemorySegments) {
                        returnExcessMemorySegments();
 
                        // fill availableMemorySegments with at least one 
element, wait if required
                        while (availableMemorySegments.isEmpty()) {
                                segment = requestMemorySegmentFromGlobal();
-                               if (segment != null || !isBlocking) {
+                               if (segment != null || !isBlocking || 
!availableMemorySegments.isEmpty()) {
                                        break;
                                }
 
-                               availableMemorySegments.wait(2000);
+                               if (numberOfRequestedMemorySegments < 
currentPoolSize) {
 
 Review comment:
   This new added whole logic seems a bit complex and heavy-weight to maintain, 
then I suggest improving it in a shortcut way in the below 
`requestMemorySegmentFromGlobal` method. 
        ```
    if (numberOfRequestedMemorySegments < currentPoolSize) {
                        final MemorySegment segment = 
networkBufferPool.requestMemorySegment();
                        if (segment != null) {
                                numberOfRequestedMemorySegments++;
                                return segment;
                        } else {
                                networkBufferPool.isAvailable().thenRun(()-> {
                                        if 
(!Thread.holdsLock(availableMemorySegments)) {
                                                synchronized 
(availableMemorySegments) {
                                                        
availableMemorySegments.notify();
                                                }
                                        }
                                });
                        }
                }
   ```
   And here we still refer to previous `availableMemorySegments.wait(2000)` 
directly.  That means if the below `networkBufferPool.isAvailable().thenRun()` 
is executed by the current thread, then we might wait for unnecessary 2 seconds 
to trigger next request. But we do not make it worse than before and in most 
cases I guess `networkBufferPool.isAvailable().thenRun()` should be executed by 
another thread which would wakeup the wait immediately.

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

Reply via email to