pnowojski commented on a change in pull request #11098: [FLINK-16060][task] 
Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383185280
 
 

 ##########
 File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/MultipleInputSelectionHandler.java
 ##########
 @@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.runtime.io;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.api.operators.InputSelectable;
+import org.apache.flink.streaming.api.operators.InputSelection;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This handler is mainly used for selecting the next available input index
+ * in {@link StreamMultipleInputProcessor}.
+ */
+@Internal
+public class MultipleInputSelectionHandler {
+
+       @Nullable
+       private final InputSelectable inputSelector;
+
+       private InputSelection inputSelection = InputSelection.ALL;
+
+       private final long allSelectedMask;
+
+       private long availableInputsMask;
+
+       private long endOfInputMask;
+
+       public MultipleInputSelectionHandler(@Nullable InputSelectable 
inputSelectable, int inputCount) {
+               this.inputSelector = inputSelectable;
+               this.allSelectedMask = (1 << inputCount) - 1;
+               this.availableInputsMask = allSelectedMask;
+       }
+
+       public InputStatus reportInputStatus(InputStatus inputStatus, int 
inputIndex) throws IOException {
+               switch (inputStatus) {
+                       case MORE_AVAILABLE:
+                               checkState(checkBitMask(availableInputsMask, 
inputIndex));
+                               return InputStatus.MORE_AVAILABLE;
+                       case NOTHING_AVAILABLE:
+                               availableInputsMask = 
unsetBitMask(availableInputsMask, inputIndex);
+                               break;
+                       case END_OF_INPUT:
+                               endOfInputMask = setBitMask(endOfInputMask, 
inputIndex);
+                               break;
+                       default:
+                               throw new 
UnsupportedOperationException("Unsupported inputStatus = " + inputStatus);
+               }
+
+               return calculateInputStatus();
+       }
+
+       public InputStatus calculateInputStatus() throws IOException {
+               if (endOfInputMask == allSelectedMask) {
+                       return InputStatus.END_OF_INPUT;
+               }
+
+               if (isAnyInputAvailable()) {
+                       return InputStatus.MORE_AVAILABLE;
+               }
+               else {
+                       long selectedNotFinishedInputMask = 
inputSelection.getInputMask() & ~(endOfInputMask);
+                       if (selectedNotFinishedInputMask == 0) {
+                               throw new IOException("Can not make a progress: 
all selected inputs are already finished");
+                       }
+                       return InputStatus.NOTHING_AVAILABLE;
+               }
+       }
+
+       void nextSelection() {
+               if (inputSelector == null) {
+                       inputSelection = InputSelection.ALL;
 
 Review comment:
   It would be indeed nicer. However this brings extra overhead of 
virtualisation, and might mess up with the optimiser for the common path (no  
`InputSelector`), preventing inlining etc. I'm not sure how much does it matter 
in here though. 
   
   However if we wanted to do it, it should also be changed in the two input 
processor as well, and would require more cautious regression benchmarks. So I 
would suggest to do it independently.

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

Reply via email to