sunhaibotb 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_r275196822
########## 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: That's a good structure. But for performance reasons, the code here optimizes the fair selection algorithm in the case of two inputs. That is, the algorithm implemented here satisfies only two inputs. If the encapsulated `fairSelectNextIndex()` is added to the `InputSelection` class, I understand that the algorithm should be universal and adapt to any number of inputs, then the algorithm is relatively complex and poor efficiency for two inputs. ---------------------------------------------------------------- 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