reswqa commented on code in PR #23957:
URL: https://github.com/apache/flink/pull/23957#discussion_r1448184760


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/shuffle/TieredResultPartition.java:
##########
@@ -187,6 +187,7 @@ public void finish() throws IOException {
 
     @Override
     public void close() {
+        storageMemoryManager.release();

Review Comment:
   I wonder how do we guarantee that `numRequestedBuffers.get() > 
bufferPool.getNumBuffers()` at this time, Otherwise, the buffer will be 
recycled to `bufferQueue` instead of `LocalBufferPool`.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/shuffle/TieredResultPartitionTest.java:
##########
@@ -192,15 +190,38 @@ void testMetricsUpdateForBroadcastOnlyResultPartition() 
throws Exception {
         try (TieredResultPartition partition =
                 createTieredStoreResultPartition(2, bufferPool, true)) {
             partition.broadcastRecord(ByteBuffer.allocate(bufferSize));
-            IOMetrics ioMetrics = taskIOMetricGroup.createSnapshot();
-            assertThat(ioMetrics.getResultPartitionBytes()).hasSize(1);
-            ResultPartitionBytes partitionBytes =
-                    
ioMetrics.getResultPartitionBytes().values().iterator().next();
-            assertThat(partitionBytes.getSubpartitionBytes())
-                    .containsExactly(bufferSize, bufferSize);
+            verifySubpartitionBytes(bufferSize, bufferSize);
         }
     }
 
+    @Test
+    @Timeout(60)

Review Comment:
   Local timeout should be avoid according by 
https://flink.apache.org/how-to-contribute/code-style-and-quality-common/#avoid-timeouts-in-junit-tests.



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