xintongsong commented on a change in pull request #15273:
URL: https://github.com/apache/flink/pull/15273#discussion_r597451030



##########
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 guess that's a trade-off. The more operations we guard against 
concurrency, the more affected by cross-thread synchronization. In this case, 
I'm in favor of guarding all the segment freeings, rather than only the unsafe 
cleaners. The performance cost on calling `free()` should be acceptable given 
that ideally each segment should only be freed once.
   
   As for `isFree()`, it's currently only used for checking before releasing 
the segment, or in test cases. I'm adding this warning message trying to 
prevent it from being used in performance sensitive cases in future, like in 
`put()` and `get()`. Alternatively, I'm also ok with not changing this method, 
living with the fact that freeing of the segment may not always be visible to 
this method immediately. In that case, we may want to add a warning on that as 
well.




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


Reply via email to