You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2019/06/26 14:17:38 UTC

[GitHub] [flink] pnowojski commented on a change in pull request #8826: [FLINK-12479][operators] Integrate StreamInputProcessor(s) with mailbox

pnowojski commented on a change in pull request #8826: [FLINK-12479][operators] Integrate StreamInputProcessor(s) with mailbox
URL: https://github.com/apache/flink/pull/8826#discussion_r297688045
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputSelectableProcessor.java
 ##########
 @@ -286,64 +316,33 @@ private void checkAndSetAvailable(int inputIndex) {
 		}
 	}
 
-	/**
-	 * @return false if both of the inputs are finished, true otherwise.
-	 */
-	private boolean waitForAvailableInput(InputSelection inputSelection)
-		throws ExecutionException, InterruptedException, IOException {
-
+	private CompletableFuture<?> waitForAvailableInput(InputSelection inputSelection) {
 		if (inputSelection.isALLMaskOf2()) {
 			return waitForAvailableEitherInput();
 		} else {
-			waitForOneInput(
+			return waitForOneInput(
 				(inputSelection.getInputMask() == InputSelection.FIRST.getInputMask()) ? input1 : input2);
-			return true;
 		}
 	}
 
-	private boolean waitForAvailableEitherInput()
-		throws ExecutionException, InterruptedException {
-
-		CompletableFuture<?> future1 = input1.isFinished() ? UNAVAILABLE : input1.isAvailable();
-		CompletableFuture<?> future2 = input2.isFinished() ? UNAVAILABLE : input2.isAvailable();
-
-		if (future1 == UNAVAILABLE && future2 == UNAVAILABLE) {
-			return false;
+	private CompletableFuture<?> waitForAvailableEitherInput() {
+		if (input1.isFinished() && input2.isFinished()) {
+			return AVAILABLE;
 		}
-
-		// block to wait for a available input
-		CompletableFuture.anyOf(future1, future2).get();
-
-		if (future1.isDone()) {
-			setAvailableInput(input1.getInputIndex());
+		if (input1.isFinished()) {
+			return input2.isAvailable();
 		}
-		if (future2.isDone()) {
-			setAvailableInput(input2.getInputIndex());
+		if (input2.isFinished()) {
+			return input1.isAvailable();
 		}
-
-		return true;
+		return CompletableFuture.anyOf(input1.isAvailable(), input2.isAvailable());
 	}
 
-	private void waitForOneInput(StreamTaskInput input)
-		throws IOException, ExecutionException, InterruptedException {
-
+	private CompletableFuture<?> waitForOneInput(StreamTaskInput input) {
 		if (input.isFinished()) {
-			throw new IOException("Could not read the finished input: input" + (input.getInputIndex() + 1) +  ".");
+			throw new IllegalStateException("Could not read the finished input: input" + (input.getInputIndex() + 1) +  ".");
 
 Review comment:
   I have also realised, that this shouldn't be a `IllegalStateException`, since this is a user error - I have refactored the code accordingly.

----------------------------------------------------------------
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