pnowojski commented on a change in pull request #8124: [FLINK-11877] Implement 
the runtime handling of the InputSelectable interface
URL: https://github.com/apache/flink/pull/8124#discussion_r275409843
 
 

 ##########
 File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputSelectableProcessor.java
 ##########
 @@ -0,0 +1,433 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.SimpleCounter;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
+import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup;
+import org.apache.flink.streaming.api.operators.InputSelectable;
+import org.apache.flink.streaming.api.operators.InputSelection;
+import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.metrics.WatermarkGauge;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Input reader for {@link 
org.apache.flink.streaming.runtime.tasks.TwoInputSelectableStreamTask}
+ * in the case that the operator is InputSelectable.
+ *
+ * @param <IN1> The type of the records that arrive on the first input
+ * @param <IN2> The type of the records that arrive on the second input
+ */
+@Internal
+public class StreamTwoInputSelectableProcessor<IN1, IN2> {
+
+       private static final Logger LOG = 
LoggerFactory.getLogger(StreamTwoInputSelectableProcessor.class);
+
+       private volatile boolean continuousProcessing = true;
+
+       private final NetworkInput input1;
+       private final NetworkInput input2;
+
+       private final Object lock;
+
+       private final TwoInputStreamOperator<IN1, IN2, ?> streamOperator;
+
+       private final InputSelectable inputSelector;
+
+       private final AuxiliaryHandler auxiliaryHandler;
+
+       private final CompletableFuture<Integer>[] listenFutures;
+
+       private final boolean[] isFinished;
+
+       private InputSelection inputSelection;
+
+       private AtomicInteger availableInputsMask = new AtomicInteger();
+
+       private int lastReadingInputMask;
+
+       private static final int TWO_INPUT_ANY_MASK = (int) new 
InputSelection.Builder()
+               .select(1)
+               .select(2)
+               .build()
+               .getInputMask();
+
+       private static final int INPUT1_ID = 1;
+       private static final int INPUT2_ID = 2;
+
+       // ---------------- Metrics ------------------
+
+       private final WatermarkGauge input1WatermarkGauge;
+       private final WatermarkGauge input2WatermarkGauge;
+
+       private Counter numRecordsIn;
+
+       public StreamTwoInputSelectableProcessor(
+               Collection<InputGate> inputGates1,
+               Collection<InputGate> inputGates2,
+               TypeSerializer<IN1> inputSerializer1,
+               TypeSerializer<IN2> inputSerializer2,
+               Object lock,
+               IOManager ioManager,
+               StreamStatusMaintainer streamStatusMaintainer,
+               TwoInputStreamOperator<IN1, IN2, ?> streamOperator,
+               WatermarkGauge input1WatermarkGauge,
+               WatermarkGauge input2WatermarkGauge) {
+
+               checkState(streamOperator instanceof InputSelectable);
+
+               // create a NetworkInput instance for each input
+               StreamTwoInputStreamStatusHandler streamStatusHandler = new 
StreamTwoInputStreamStatusHandler(
+                       streamStatusMaintainer, lock);
+               this.input1 = new NetworkInput(INPUT1_ID, inputGates1, 
inputSerializer1, streamStatusHandler, ioManager);
+               this.input2 = new NetworkInput(INPUT2_ID, inputGates2, 
inputSerializer2, streamStatusHandler, ioManager);
+
+               this.lock = checkNotNull(lock);
+
+               this.streamOperator = checkNotNull(streamOperator);
+               this.inputSelector = (InputSelectable) streamOperator;
+
+               this.input1WatermarkGauge = input1WatermarkGauge;
+               this.input2WatermarkGauge = input2WatermarkGauge;
+
+               this.auxiliaryHandler = new AuxiliaryHandler();
+
+               this.listenFutures = new CompletableFuture[]{null, null};
+               this.isFinished = new boolean[]{false, false};
+       }
+
+       /**
+        * Notes that it must be called after calling all operator's open(). 
This ensures that
+        * the first input selection determined by the operator at and before 
opening is effective.
+        */
+       public void init() {
+               inputSelection = inputSelector.nextSelection();
+
+               availableInputsMask.set(TWO_INPUT_ANY_MASK);
+
+               lastReadingInputMask = (int) 
InputSelection.SECOND.getInputMask();
+
+               if (numRecordsIn == null) {
+                       try {
+                               numRecordsIn = ((OperatorMetricGroup) 
streamOperator
+                                       
.getMetricGroup()).getIOMetricGroup().getNumRecordsInCounter();
+                       } catch (Exception e) {
+                               LOG.warn("An exception occurred during the 
metrics setup.", e);
+                               numRecordsIn = new SimpleCounter();
+                       }
+               }
+       }
+
+       public boolean processInput() throws Exception {
+               // cache inputs reference on the stack, to make the code more 
JIT friendly
+               final NetworkInput input1 = this.input1;
+               final NetworkInput input2 = this.input2;
+
+               while (continuousProcessing) {
+                       final int selectionMask = (int) 
inputSelection.getInputMask();
+                       final int availableMask = availableInputsMask.get();
+
+                       int readingInputMask = selectionMask & availableMask;
+                       if (readingInputMask == TWO_INPUT_ANY_MASK) {
+                               // the input selection is `ALL` and both inputs 
are available, we read two inputs in turn
+                               readingInputMask -= lastReadingInputMask;
 
 Review comment:
   Yes, it would be more general. Whether it would be less efficient, I don't 
know but I would bet that not necessarily. In the end, this logic should be 
encapsulated and unit tested one way or another, either:
   
   1. propper `fairSelectNextIndex` implementation
   2. or implement currently specialised version `fairSelectNextIndexOutOf2` 
which would throw `UnsupportedOperationException` for input selections > 2

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


With regards,
Apache Git Services

Reply via email to