TanYuxin-tyx commented on a change in pull request #17936:
URL: https://github.com/apache/flink/pull/17936#discussion_r782806918
##########
File path:
flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/BatchShuffleReadBufferPoolTest.java
##########
@@ -98,6 +98,34 @@ public void testRecycle() throws Exception {
assertEquals(bufferPool.getNumTotalBuffers(),
bufferPool.getAvailableBuffers());
}
+ @Test
+ public void testBufferOperationTimestampUpdated() throws Exception {
+ BatchShuffleReadBufferPool bufferPool = new
BatchShuffleReadBufferPool(1024, 1024);
+ long oldTimestamp = bufferPool.getLastBufferOperationTimestamp();
+ List<MemorySegment> buffers = bufferPool.requestBuffers();
+ assertEquals(1, buffers.size());
+ long nowTimestamp = bufferPool.getLastBufferOperationTimestamp();
+ // The timestamp is updated when requesting buffers successfully
+ assertTrue(nowTimestamp > oldTimestamp);
+ assertEquals(nowTimestamp,
bufferPool.getLastBufferOperationTimestamp());
+
+ oldTimestamp = nowTimestamp;
+ bufferPool.recycle(buffers);
+ // The timestamp is updated when recycling buffers
+ assertTrue(bufferPool.getLastBufferOperationTimestamp() >
oldTimestamp);
+
+ bufferPool.requestBuffers();
+
+ oldTimestamp = bufferPool.getLastBufferOperationTimestamp();
+ buffers = bufferPool.requestBuffers();
+ assertEquals(0, buffers.size());
Review comment:
Fixed it. And replaced `bufferPool.requestBuffers();` with `buffers =
bufferPool.requestBuffers();`.
##########
File path:
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionReadSchedulerTest.java
##########
@@ -192,6 +199,131 @@ public void testOnReadBufferRequestError() throws
Exception {
assertAllResourcesReleased();
}
+ @Test
+ public void testRequestTimeoutIsRefreshedButFailed() throws Exception {
+ Duration bufferRequestTimeout = Duration.ofSeconds(3);
+ FakedBatchShuffleReadBufferPool bufferPool =
+ new FakedBatchShuffleReadBufferPool(bufferSize * 2,
bufferSize, true);
+ FakedSortMergeResultPartitionReadScheduler readScheduler =
+ new FakedSortMergeResultPartitionReadScheduler(
+ bufferPool, executor, this, bufferRequestTimeout);
+
+ SortMergeSubpartitionReader subpartitionReader0 =
+ readScheduler.crateSubpartitionReader(
+ new NoOpBufferAvailablityListener(), 0,
partitionedFile);
+ SortMergeSubpartitionReader subpartitionReader1 =
+ readScheduler.crateSubpartitionReader(
+ new NoOpBufferAvailablityListener(), 1,
partitionedFile);
+
+ PriorityQueue<SortMergeSubpartitionReader> allReaders = new
PriorityQueue<>();
+ allReaders.add(subpartitionReader0);
+ allReaders.add(subpartitionReader1);
+
+ long startTimestamp = System.nanoTime();
+ Queue<MemorySegment> allocatedBuffers =
readScheduler.allocateBuffers(allReaders);
+ long requestDuration = System.nanoTime() - startTimestamp;
+
+ assertEquals(0, allocatedBuffers.size());
+ assertTrue(requestDuration > bufferRequestTimeout.toNanos() * 2);
+ assertExpectedTimeoutException(subpartitionReader0.getFailureCause());
+ assertExpectedTimeoutException(subpartitionReader1.getFailureCause());
+
+ bufferPool.destroy();
+ readScheduler.release();
+ }
+
+ @Test
+ public void testRequestTimeoutIsRefreshedAndSuccess() throws Exception {
+ Duration bufferRequestTimeout = Duration.ofSeconds(3);
+ FakedBatchShuffleReadBufferPool bufferPool =
+ new FakedBatchShuffleReadBufferPool(bufferSize * 2,
bufferSize, false);
+ FakedSortMergeResultPartitionReadScheduler readScheduler =
+ new FakedSortMergeResultPartitionReadScheduler(
+ bufferPool, executor, this, bufferRequestTimeout);
+
+ SortMergeSubpartitionReader subpartitionReader0 =
+ readScheduler.crateSubpartitionReader(
+ new NoOpBufferAvailablityListener(), 0,
partitionedFile);
+ SortMergeSubpartitionReader subpartitionReader1 =
+ readScheduler.crateSubpartitionReader(
+ new NoOpBufferAvailablityListener(), 1,
partitionedFile);
+
+ PriorityQueue<SortMergeSubpartitionReader> allReaders = new
PriorityQueue<>();
+ allReaders.add(subpartitionReader0);
+ allReaders.add(subpartitionReader1);
+
+ long startTimestamp = System.nanoTime();
+ Queue<MemorySegment> allocatedBuffers =
readScheduler.allocateBuffers(allReaders);
+ long requestDuration = System.nanoTime() - startTimestamp;
+
+ assertEquals(2, allocatedBuffers.size());
+ assertTrue(requestDuration > bufferRequestTimeout.toNanos() * 2);
+ assertNull(subpartitionReader0.getFailureCause());
+ assertNull(subpartitionReader1.getFailureCause());
+
+ bufferPool.destroy();
+ readScheduler.release();
+ }
+
+ private static void assertExpectedTimeoutException(Throwable throwable) {
+ assertNotNull(throwable);
+ assertTrue(
+ ExceptionUtils.findThrowableWithMessage(throwable, "Buffer
request timeout")
+ .isPresent());
+ }
+
+ private static class FakedBatchShuffleReadBufferPool extends
BatchShuffleReadBufferPool {
+ private List<MemorySegment> requestedBuffers;
+ private final boolean isRecycleOnlyOneBuffer;
+ private boolean isFirstlyCallGetTimestamp;
+ private int recycleBufferIndex;
+
+ FakedBatchShuffleReadBufferPool(
+ long totalBytes, int bufferSize, boolean
isRecycleOnlyOneBuffer) throws Exception {
+ super(totalBytes, bufferSize);
+
+ this.isRecycleOnlyOneBuffer = isRecycleOnlyOneBuffer;
+ this.requestedBuffers = requestBuffers();
+ this.isFirstlyCallGetTimestamp = true;
+ this.recycleBufferIndex = 0;
+ }
+
+ @Override
+ public long getLastBufferOperationTimestamp() {
+ if (isFirstlyCallGetTimestamp) {
+ isFirstlyCallGetTimestamp = false;
+ return super.getLastBufferOperationTimestamp();
+ }
+
+ if (recycleBufferIndex == 0 || (recycleBufferIndex == 1 &&
!isRecycleOnlyOneBuffer)) {
+ recycle(requestedBuffers.get(recycleBufferIndex++));
+ }
+ return super.getLastBufferOperationTimestamp();
+ }
+
+ @Override
+ public void destroy() {
+ requestedBuffers.clear();
+ super.destroy();
+ }
+ }
+
+ private static class FakedSortMergeResultPartitionReadScheduler
Review comment:
Fixed.
##########
File path:
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionReadSchedulerTest.java
##########
@@ -192,6 +199,131 @@ public void testOnReadBufferRequestError() throws
Exception {
assertAllResourcesReleased();
}
+ @Test
+ public void testRequestTimeoutIsRefreshedButFailed() throws Exception {
+ Duration bufferRequestTimeout = Duration.ofSeconds(3);
+ FakedBatchShuffleReadBufferPool bufferPool =
+ new FakedBatchShuffleReadBufferPool(bufferSize * 2,
bufferSize, true);
+ FakedSortMergeResultPartitionReadScheduler readScheduler =
+ new FakedSortMergeResultPartitionReadScheduler(
+ bufferPool, executor, this, bufferRequestTimeout);
+
+ SortMergeSubpartitionReader subpartitionReader0 =
+ readScheduler.crateSubpartitionReader(
+ new NoOpBufferAvailablityListener(), 0,
partitionedFile);
+ SortMergeSubpartitionReader subpartitionReader1 =
+ readScheduler.crateSubpartitionReader(
+ new NoOpBufferAvailablityListener(), 1,
partitionedFile);
+
+ PriorityQueue<SortMergeSubpartitionReader> allReaders = new
PriorityQueue<>();
+ allReaders.add(subpartitionReader0);
+ allReaders.add(subpartitionReader1);
+
+ long startTimestamp = System.nanoTime();
+ Queue<MemorySegment> allocatedBuffers =
readScheduler.allocateBuffers(allReaders);
+ long requestDuration = System.nanoTime() - startTimestamp;
+
+ assertEquals(0, allocatedBuffers.size());
+ assertTrue(requestDuration > bufferRequestTimeout.toNanos() * 2);
+ assertExpectedTimeoutException(subpartitionReader0.getFailureCause());
+ assertExpectedTimeoutException(subpartitionReader1.getFailureCause());
+
+ bufferPool.destroy();
+ readScheduler.release();
+ }
+
+ @Test
+ public void testRequestTimeoutIsRefreshedAndSuccess() throws Exception {
+ Duration bufferRequestTimeout = Duration.ofSeconds(3);
+ FakedBatchShuffleReadBufferPool bufferPool =
+ new FakedBatchShuffleReadBufferPool(bufferSize * 2,
bufferSize, false);
+ FakedSortMergeResultPartitionReadScheduler readScheduler =
+ new FakedSortMergeResultPartitionReadScheduler(
+ bufferPool, executor, this, bufferRequestTimeout);
+
+ SortMergeSubpartitionReader subpartitionReader0 =
+ readScheduler.crateSubpartitionReader(
+ new NoOpBufferAvailablityListener(), 0,
partitionedFile);
+ SortMergeSubpartitionReader subpartitionReader1 =
+ readScheduler.crateSubpartitionReader(
+ new NoOpBufferAvailablityListener(), 1,
partitionedFile);
+
+ PriorityQueue<SortMergeSubpartitionReader> allReaders = new
PriorityQueue<>();
+ allReaders.add(subpartitionReader0);
+ allReaders.add(subpartitionReader1);
+
+ long startTimestamp = System.nanoTime();
+ Queue<MemorySegment> allocatedBuffers =
readScheduler.allocateBuffers(allReaders);
+ long requestDuration = System.nanoTime() - startTimestamp;
+
+ assertEquals(2, allocatedBuffers.size());
+ assertTrue(requestDuration > bufferRequestTimeout.toNanos() * 2);
+ assertNull(subpartitionReader0.getFailureCause());
+ assertNull(subpartitionReader1.getFailureCause());
+
+ bufferPool.destroy();
+ readScheduler.release();
+ }
+
+ private static void assertExpectedTimeoutException(Throwable throwable) {
+ assertNotNull(throwable);
+ assertTrue(
+ ExceptionUtils.findThrowableWithMessage(throwable, "Buffer
request timeout")
+ .isPresent());
+ }
+
+ private static class FakedBatchShuffleReadBufferPool extends
BatchShuffleReadBufferPool {
Review comment:
Fixed.
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]