pnowojski commented on a change in pull request #10029: [FLINK-14553][runtime]
Respect non-blocking output in StreamTask#processInput
URL: https://github.com/apache/flink/pull/10029#discussion_r340038462
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
##########
@@ -281,10 +289,41 @@ protected void processInput(DefaultActionContext
context) throws Exception {
if (status == InputStatus.END_OF_INPUT) {
context.allActionsCompleted();
}
- else if (status == InputStatus.NOTHING_AVAILABLE) {
+
+ CompletableFuture<?> jointFuture =
getInputOutputJointFuture(status);
+ if (jointFuture != null) {
SuspendedMailboxDefaultAction suspendedDefaultAction =
context.suspendDefaultAction();
-
inputProcessor.getAvailableFuture().thenRun(suspendedDefaultAction::resume);
+ jointFuture.thenRun(suspendedDefaultAction::resume);
+ }
+ }
+
+ /**
+ * @return a combination of input and output futures if at-least one
future of them is not
+ * completed, otherwise return null if all input and outputs are
available.
+ */
+ private CompletableFuture<?> getInputOutputJointFuture(InputStatus
status) {
+ if (status == InputStatus.MORE_AVAILABLE &&
isOutputAvailable()) {
+ return null;
+ }
+
+ int length = recordWriters.size();
+ for (int i = 0; i < length; i++) {
+ inputOutputFutures[i] =
recordWriters.get(i).getAvailableFuture();
+ }
+ inputOutputFutures[length] =
inputProcessor.getAvailableFuture();
+ return CompletableFuture.allOf(inputOutputFutures);
Review comment:
Shouldn't we distinguish three separate cases:
1. just input unavailable
2. just output unavailable
3. both unavailable
and return only a relevant future for each of the case? It might be faster
slightly faster and a bit cleaner (assuming that the logic of doing so won't be
too complicated, but it shouldn't)
```
boolean outputUnavailable = !recordWriters.isAvailable();
if (inputStatus == NOTHING_AVAILABLE && outputUnavailable)
return CompletableFuture.allOf(inputProcessor.getAvailableFuture(),
recordWriters.getAvailableFuture());
else if (inputStatus == NOTHING_AVAILABLE)
return inputProcessor.getAvailableFuture();
else if (outputUnavailable)
return recordWriters.getAvailableFuture();
```
both inputs and outputs being unavailable will be a veeerryy rare temporary
case (having both nothing to process and a back-pressure).
----------------------------------------------------------------
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