pnowojski commented on a change in pull request #16589:
URL: https://github.com/apache/flink/pull/16589#discussion_r678603336



##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/MultipleInputSelectionHandler.java
##########
@@ -101,20 +114,26 @@ public DataInputStatus calculateOverallStatus() throws 
IOException {
     }
 
     void nextSelection() {
-        if (inputSelectable == null) {
-            inputSelection = InputSelection.ALL;
+        if (inputSelectable == null || areAllDataInputsFinished()) {
+            selectedInputsMask = InputSelection.ALL.getInputMask();
+        } else if (dataFinishedButNotPartition != 0) {
+            selectedInputsMask =
+                    (inputSelectable.nextSelection().getInputMask() | 
dataFinishedButNotPartition)
+                            & allSelectedMask;
         } else {
-            inputSelection = inputSelectable.nextSelection();
+            selectedInputsMask = 
inputSelectable.nextSelection().getInputMask();

Review comment:
       Why don't you go one step further and squash `allDataInputsFinished` 
with `dataFinishedButNotPartition != 0`  and `inputSelectable == null` into a 
single enum check? Something like 
   ```
   private OperatingMode mode = PROCESSING_NO_INPUT_SELECTABLE | PROCESSING | 
NOT_ALL_DATA_INPUTS_FINISHED | ALL_DATA_INPUTS_FINISHED
   ```
   ? It would be one comparison faster and I think cleaner?




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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to