Github user NicoK commented on a diff in the pull request:
https://github.com/apache/flink/pull/4509#discussion_r143233556
--- Diff:
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
---
@@ -331,37 +353,41 @@ public int getNumberOfAvailableBuffers() {
*/
@Override
public boolean notifyBufferAvailable(Buffer buffer) {
- checkState(isWaitingForFloatingBuffers.get(), "This channel
should be waiting for floating buffers.");
+ checkState(isWaitingForFloatingBuffers, "This channel should be
waiting for floating buffers.");
+ boolean needMoreBuffers;
synchronized (availableBuffers) {
// Important: the isReleased check should be inside the
synchronized block.
- if (isReleased.get() || availableBuffers.size() >=
senderBacklog.get()) {
- isWaitingForFloatingBuffers.set(false);
+ if (isReleased.get() || availableBuffers.size() >=
senderBacklog.get() + initialCredit) {
--- End diff --
1) The `senderBacklog` is only ever accessed under the
`synchronized(availableBuffers)`. I can't really think of a reason to drag it
out of there and keep it an `AtomicInteger` at the moment - what do you think
about making it a simple `int` instead?
2) Since we release extraneous floating buffers in `recycle()`, I guess,
`availableBuffers.size() > senderBacklog.get() + initialCredit` is not possible
here, is it? Should we `checkState()` this and otherwise ignore this case in
here?
---