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_r338958514
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java
 ##########
 @@ -170,9 +194,16 @@ public void recycle(MemorySegment segment) {
                                        throw new IllegalStateException("Buffer 
pool is destroyed.");
                                }
 
-                               final MemorySegment segment = 
availableMemorySegments.poll(2, TimeUnit.SECONDS);
-                               if (segment != null) {
-                                       segments.add(segment);
+                               synchronized (availabilityHelper) {
+                                       final MemorySegment segment = 
availableMemorySegments.poll();
 
 Review comment:
   We could deduplicate some common codes with above `#requestMemorySegment()`.
   
   `@Nullable
   private MemorySegment internalRequestMemorySegment() {
                assert Thread.holdsLock(availableMemorySegments);
   
                final MemorySegment segment = availableMemorySegments.poll();
                if (availableMemorySegments.isEmpty() && segment != null) {
                        availabilityHelper.resetUnavailable();
                }
                return segment;
   }`
   
   Then we could refactor here as 
   
   `
   final MemorySegment segment = internalRequestMemorySegment();
   if (segment == null) {
        availabilityHelper.wait(2000);
   }
   `

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