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_r341068032
##########
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) {
Review comment:
Can not we define the following class:
```
/** wrapper class encapsulating multiple RecordWriters */
class RecordWriters<OUT> {
RecordWriter<SerializationDelegate<StreamRecord<OUT>>>[] recordWriters;
CompletableFuture<?>[] outputAvailability;
public RecordWriter<...> getRecordWriter(int outputIndex) { return
recordWriters[outpuIndex];}
public CompletableFuture<?> isAvailable() { ... } ;
public void close() { // loop and close... }
public void broadcastEvent(AbstractEvent event) { // loop and broadcast } ;
}
```
?
It seems like we could use it directly in all of the places, both in
`StreamTask` and pass it to the `OperatorChain`.
----------------------------------------------------------------
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