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_r340581946
##########
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);
+ }
+
+ /**
+ * @return true if all the record writers are available.
+ */
+ private boolean isOutputAvailable() {
+ for (RecordWriter recordWriter : recordWriters) {
+ if (!recordWriter.isAvailable()) {
Review comment:
> considering that LocalBufferPool#isAvailable is actually not used in input
side
Do you mean that `isBlocking = false` happens only on the inputs? In that
case I would say this is a still a buggy code path, that just happens to be not
used... at the moment. I would still prefer to have it working in case of some
refactorings in the future.
Is there any downside of doing `availabilityHelper.resetUnavailable();`
always - including for non blocking cases? Performance overhead?
> We can remove this condition here and also adjust the interaction between
RemoteInputChannel and LocalBufferPool to make use of isAvailable way instead
of BufferListener interface.
I think we could do that to unify/simplify things a bit. However I would try
to not do this in this PR as I don't think it has that high priority right now.
We could create a ticket and log it for the future for someone to grab it.
----------------------------------------------------------------
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