reswqa commented on code in PR #22084: URL: https://github.com/apache/flink/pull/22084#discussion_r1128937275
########## flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java: ########## @@ -503,12 +508,11 @@ private void onGlobalPoolAvailable() { mayNotifyAvailable(toNotify); } + @GuardedBy("availableMemorySegments") private boolean shouldBeAvailable() { assert Thread.holdsLock(availableMemorySegments); - return !availableMemorySegments.isEmpty() - && unavailableSubpartitionsCount == 0 - && numberOfRequestedOverdraftMemorySegments == 0; + return !availableMemorySegments.isEmpty() && unavailableSubpartitionsCount == 0; Review Comment: Before we allows convert `numberOfRequestedOverdraftMemorySegments` to `numberOfRequestedMemorySegments `, there still be a situation where both `availableMemorySegment` and `overdraft buffer` are all not zero, and this state should obviously be defined as available. But after that, I'm not sure whether this situation still exists. I'm a little worried about the potential bug of multithreading if we add `numberOfRequestedOverdraftMemorySegments == 0` back to `shouldBeAvailable`. For me, the key point is that if we think "`The availableMemorySegments` is always empty when `numberOfRequestedOverdraftMemorySegments` != 0." is tenable, does `!availableMemorySegments.isEmpty()` already include `numberOfRequestedOverdraftMemorySegments == 0`? Even if this will not introduce bug, why should we impose useless constraints? Actually, before overdraft buffer was introduced, the definition of `available` was very clear: There is at least one `availableMemorySegment` and no subpartitions has reached `maxBuffersPerChannel`. IMO, Introducing the overdraft mechanism should not break this protocol, overdraft buffer should not affect the judgment of availability. -- 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