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_r339413614
 
 

 ##########
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java
 ##########
 @@ -518,4 +528,176 @@ public void go() throws Exception {
                        globalPool.destroy();
                }
        }
+
+       /**
+        * Tests {@link NetworkBufferPool#isAvailable()}, verifying that the 
buffer availability is
+        * correctly maintained and the future callback is correctly processed.
+        */
+       @Test
+       public void testBufferAvailabilityAndFutureCallback() throws Exception {
+               final int numBuffers = 10;
+               final int numberOfSegmentsToRequest = 5;
+               final Duration requestSegmentsTimeout = Duration.ofSeconds(10L);
+
+               final NetworkBufferPool globalPool = new NetworkBufferPool(
+                               numBuffers,
+                               128,
+                               numberOfSegmentsToRequest,
+                               requestSegmentsTimeout);
+
+               try {
+                       assertTrue(globalPool.isAvailable().isDone());
+
+                       List<MemorySegment> segments = new 
ArrayList<>(numberOfSegmentsToRequest);
+                       for (int i = 0; i < numberOfSegmentsToRequest; ++i) {
+                               MemorySegment segment = 
globalPool.requestMemorySegment();
+                               assertNotNull(segment);
+                               segments.add(segment);
+                               assertTrue(globalPool.isAvailable().isDone());
+                       }
+
+                       final List<MemorySegment> exclusiveSegments = 
globalPool.requestMemorySegments();
+                       assertEquals(numberOfSegmentsToRequest, 
exclusiveSegments.size());
+
+                       assertFalse(globalPool.isAvailable().isDone());
+                       assertEquals(0, 
globalPool.getNumberOfAvailableMemorySegments());
+
+                       final AtomicBoolean completeFlag = new 
AtomicBoolean(false);
+                       CompletableFuture<?> availableFuture = 
globalPool.isAvailable();
+                       availableFuture.whenComplete((ignored, throwable) -> 
completeFlag.set(true));
+
+                       // recycle one buffer
+                       globalPool.recycle(segments.get(0));
+                       assertTrue(completeFlag.get());
+                       assertTrue(availableFuture.isDone());
+                       assertTrue(globalPool.isAvailable().isDone());
+                       assertEquals(1, 
globalPool.getNumberOfAvailableMemorySegments());
+
+                       CheckedThread asyncRequest = new CheckedThread() {
+                               @Override
+                               public void go() throws Exception {
+                                       
exclusiveSegments.addAll(globalPool.requestMemorySegments());
+                               }
+                       };
+                       asyncRequest.start();
+
+                       // wait until no buffer is available
+                       final Deadline deadline = 
Deadline.fromNow(Duration.ofSeconds(10L));
+                       while (globalPool.getNumberOfAvailableMemorySegments() 
> 0) {
+                               Thread.sleep(50);
+                               if (!deadline.hasTimeLeft()) {
+                                       fail("Waiting timeout.");
+                               }
+                       }
+                       assertFalse(globalPool.isAvailable().isDone());
+                       assertEquals(numberOfSegmentsToRequest, 
exclusiveSegments.size());
+
+                       for (int i = 1; i < numberOfSegmentsToRequest; ++i) {
+                               globalPool.recycle(segments.get(i));
+                       }
+                       segments.clear();
+
+                       asyncRequest.sync();
+                       assertFalse(globalPool.isAvailable().isDone());
+                       assertEquals(numBuffers, exclusiveSegments.size());
+                       assertFalse(globalPool.isAvailable().isDone());
+
+                       globalPool.recycleMemorySegments(exclusiveSegments);
+                       exclusiveSegments.clear();
+                       assertTrue(globalPool.isAvailable().isDone());
+                       assertEquals(numBuffers, 
globalPool.getNumberOfAvailableMemorySegments());
+
+               } finally {
+                       globalPool.destroy();
+               }
+       }
+
+       /**
+        * Tests that blocking request of multi local buffer pools can be 
fulfilled by recycled segments
+        * to the global network buffer pool.
+        */
+       @Test
+       public void testBlockingRequestFromMultiLocalBufferPool() throws 
Exception {
+               final int localBufferPoolSize = 10;
+               final int numLocalBufferPool = 10;
+               final int numBuffers = numLocalBufferPool * localBufferPoolSize;
+
+               final ExecutorService executorService = 
Executors.newFixedThreadPool(numLocalBufferPool);
+               final NetworkBufferPool globalPool = new 
NetworkBufferPool(numBuffers, 128, 5);
+
+               try {
+                       // create local buffer pools
+                       final List<BufferPool> localBufferPools = new 
ArrayList<>(numLocalBufferPool);
+                       for (int i  = 0; i < numLocalBufferPool; ++i) {
+                               localBufferPools.add(
+                                       
globalPool.createBufferPool(localBufferPoolSize, Integer.MAX_VALUE));
+                       }
+
+                       // request some buffers from the global pool
+                       final List<MemorySegment> segments = new 
ArrayList<>(numBuffers / 2);
+                       for (int i = 0; i < numBuffers / 2; ++i) {
+                               segments.add(globalPool.requestMemorySegment());
+                       }
+                       assertEquals(numBuffers - segments.size(), 
globalPool.getNumberOfAvailableMemorySegments());
+                       assertTrue(globalPool.isAvailable().isDone());
+
+                       // blocking request buffers form local buffer pools
+                       final CountDownLatch latch = new 
CountDownLatch(numLocalBufferPool);
+                       final AtomicReference<Throwable> cause = new 
AtomicReference<>();
+                       final BlockingQueue<BufferBuilder> segmentsRequested = 
new ArrayBlockingQueue<>(numBuffers);
+                       for (final BufferPool bufferPool: localBufferPools) {
+                               executorService.submit(() -> {
+                                       for (int num = localBufferPoolSize; num 
> 0; --num) {
+                                               try {
+                                                       
segmentsRequested.add(bufferPool.requestBufferBuilderBlocking());
+                                               } catch (Exception e) {
+                                                       cause.set(e);
+                                               }
+                                       }
+                                       latch.countDown();
+                               });
+                       }
+
+                       // wait until all available buffers are requested
+                       final Deadline deadline = 
Deadline.fromNow(Duration.ofSeconds(10L));
+                       while (segmentsRequested.size() < numBuffers - 
segments.size()) {
+                               Thread.sleep(50);
+                               if (!deadline.hasTimeLeft()) {
+                                       fail("Waiting timeout.");
+                               }
+                       }
+                       assertEquals(0, 
globalPool.getNumberOfAvailableMemorySegments());
+                       assertEquals(numBuffers - segments.size(), 
segmentsRequested.size());
+                       assertFalse(globalPool.isAvailable().isDone());
+
+                       // recycle the previously requested segments
+                       final int numToRecycle = segments.size() / 2;
+                       for (int i = 0; i < numToRecycle; ++i) {
+                               
globalPool.recycle(segments.remove(segments.size() - 1));
+                       }
+                       globalPool.recycleMemorySegments(segments);
+                       segments.clear();
+
+                       // wait until all blocking buffer requests succeed
+                       assertTrue(latch.await(10L, TimeUnit.SECONDS));
+                       assertEquals(0, 
globalPool.getNumberOfAvailableMemorySegments());
+                       assertFalse(globalPool.isAvailable().isDone());
+
+                       // recycle all the requested buffers
 
 Review comment:
   The following parts are all for the resource cleanup, then it is better to 
make it in the finally region and remove unnecessary `assert`.

----------------------------------------------------------------
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:
[email protected]


With regards,
Apache Git Services

Reply via email to