pnowojski commented on a change in pull request #8361: [FLINK-12434][network]
Replace listeners with CompletableFuture in InputGates
URL: https://github.com/apache/flink/pull/8361#discussion_r282110599
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
##########
@@ -197,32 +197,49 @@ public void requestPartitions() throws IOException,
InterruptedException {
return Optional.of(bufferOrEvent);
}
- @Override
- public Optional<BufferOrEvent> pollNextBufferOrEvent() throws
UnsupportedOperationException {
- throw new UnsupportedOperationException();
- }
-
- private InputGateWithData waitAndGetNextInputGate() throws IOException,
InterruptedException {
+ private Optional<InputGateWithData> waitAndGetNextInputGate(boolean
blocking) throws IOException, InterruptedException {
while (true) {
- InputGate inputGate;
- boolean moreInputGatesAvailable;
synchronized (inputGatesWithData) {
while (inputGatesWithData.size() == 0) {
- inputGatesWithData.wait();
+ if (blocking) {
+ inputGatesWithData.wait();
+ } else {
+ resetIsAvailable();
+ return Optional.empty();
+ }
+ }
+ final InputGate inputGate =
inputGatesWithData.remove();
+
+ // In case of inputGatesWithData being
inaccurate do not block on an empty inputGate, but just poll the data.
+ Optional<BufferOrEvent> bufferOrEvent =
inputGate.pollNextBufferOrEvent();
+
+ if (bufferOrEvent.isPresent() &&
bufferOrEvent.get().moreAvailable()) {
+ // enqueue the inputGate at the end to
avoid starvation
+ inputGatesWithData.add(inputGate);
+ } else {
+ inputGate.isAvailable().thenRun(() ->
queueInputGate(inputGate));
Review comment:
Yes, but this is not an issue here, since if `inputGate.isAvailable() ==
AVAILABLE` it would mean that `bufferOrEvent.get().moreAvailable()` returns
true - so when hot looping we do not enter this code path.
----------------------------------------------------------------
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