flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From GitBox <...@apache.org>
Subject [GitHub] [flink] zhijiangW commented on a change in pull request #10029: [FLINK-14553][runtime] Respect non-blocking output in StreamTask#processInput
Date Thu, 31 Oct 2019 06:47:40 GMT
zhijiangW 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_r340980280
 
 

 ##########
 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:
   > Do you mean that isBlocking = false happens only on the inputs?
   
   Exactly, the local pool provides two ways of blocking/non-blocking request atm. For the
output it uses the blocking way before, and for the input it uses the non-blocking way.
   
   > Is there any downside of doing availabilityHelper.resetUnavailable(); always - including
for non blocking cases? Performance overhead?
   
   I am not quite sure of it, so my previous thought was to avoid impacting the unrelevant
path in best-effort way, otherwise we have to verify the performance concern.
   
   > I think we could do that to unify/simplify things a bit.....
   
   Agree, if we can convince the trivial impact, the unification work has very low priority
and can be focused in future if have time.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

Mime
View raw message