pnowojski commented on a change in pull request #9993:
[FLINK-14498][runtime]Introduce NetworkBufferPool#isAvailable() for interacting
with LocalBufferPool.
URL: https://github.com/apache/flink/pull/9993#discussion_r341081009
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java
##########
@@ -269,6 +267,17 @@ private MemorySegment requestMemorySegmentFromGlobal()
throws IOException {
if (segment != null) {
numberOfRequestedMemorySegments++;
return segment;
+ } else if (isBlocking) {
+ // if the future is completed before the
callback is registered,
+ // the request thread will wait 2s before
polling an available
+ // segment from the global pool, which is not a
big problem.
Review comment:
Would that work?
```
@Nullable
private MemorySegment requestMemorySegment(boolean isBlocking) throws
InterruptedException, IOException, ExecutionException {
if (!isBlocking) {
return requestMemorySegment();
}
else {
MemorySegment segment = null;
while (segment == null) {
segment = requestMemorySegment();
if (segment == null) {
isAvailable().get();
}
}
return segment;
}
}
@Nullable
private MemorySegment requestMemorySegment() throws IOException {
MemorySegment segment = null;
synchronized (availableMemorySegments) {
returnExcessMemorySegments();
if (availableMemorySegments.isEmpty()) {
segment = requestMemorySegmentFromGlobal();
}
if (segment == null) {
segment = availableMemorySegments.poll();
}
if (isUnavailable()) {
availabilityHelper.resetUnavailable();
}
}
return segment;
}
```
One downside of this is that with the above version, we are calling
`returnExcessMemorySegments()` and `availabilityHelper.resetUnavailable();`
multiple times per one blocking call - but only if the buffer is not presetn,
so probably performance overhead of the extra checks do not matter.
----------------------------------------------------------------
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]
With regards,
Apache Git Services