kezhuw commented on a change in pull request #15273: URL: https://github.com/apache/flink/pull/15273#discussion_r597375675
########## File path: flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java ########## @@ -217,10 +222,14 @@ public int size() { /** * Checks whether the memory segment was freed. * + * <p>This method internally involves cross-thread synchronization. Do not use for performance + * sensitive code paths. + * * @return <tt>true</tt>, if the memory segment has been freed, <tt>false</tt> otherwise. */ public boolean isFreed() { - return address > addressLimit; + // in performance sensitive cases, use 'address > addressLimit' instead + return isFreedAtomic.get(); Review comment: I think we could promote this atomic handling to `MemoryUtils.createMemoryCleaner` or even `MemoryUtils.allocateUnsafe`(with enriched struct). Then we could remove this caveat. I think we are only guarding unsafe part ? If yes, I recommend to move this concurrent guarding to unsafe`cleaner`. ########## File path: flink-core/src/test/java/org/apache/flink/core/memory/OffHeapUnsafeMemorySegmentTest.java ########## @@ -58,4 +61,31 @@ public void testCallCleanerOnFree() { .free(); assertTrue(cleanerFuture.isDone()); } + + @Test + public void testCallCleanerOnceOnConcurrentFree() throws InterruptedException { + final AtomicInteger counter = new AtomicInteger(0); Review comment: I think we should also test throwing path, in that path the final `counter` should also be one. -- 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