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_r275299014
########## 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; Review comment: For input processing , using the internal loop to continuously process records is about 1000 ops/ms faster than using the external loop (the internal and external loop structures are shown later). `volatile boolean continuousProcessing` is introduced to interrupt the input processing when the task is canceled or some cases are need to interrupt in the future. For production code, `continuousProcessing` is assigned `false` only in `StreamTwoInputSelectableProcessor#stop()` now. _(If the task cancel is considered only, `continuousProcessing` can be replaced by a public method `AbstractTwoInputStreamTask#isRunning()` which is added to expose `AbstractTwoInputStreamTask#running`)._ The code structures for the internal and external loops are as follows (if `StreamTwoInputProcessor` uses the internal loop to continuously process records, it will be faster). 1. Uses a internal loop in the input processor to continuously process records ``` === TwoInputXStreamTask.java === @Override protected void run() throws Exception { // cache processor reference on the stack, to make the code more JIT friendly final StreamTwoInputXProcessor<IN1, IN2> inputProcessor = this.inputProcessor; ... while (running && inputProcessor.processInput()) { // all the work happens in the "processInput" method } ... } @Override protected void cancelTask() { running = false; if (inputProcessor != null) { inputProcessor.stop(); } } === StreamTwoInputXProcessor.java === public boolean processInput() throws Exception { while (continuousProcessing) { ... if (element.isRecord()) { continue; } ... } return true; } public void stop() { continuousProcessing = false; ... } ``` 2. Uses a external loop in the stream task to continuously process records ``` === TwoInputXStreamTask.java === @Override protected void run() throws Exception { // cache processor reference on the stack, to make the code more JIT friendly final StreamTwoInputXProcessor<IN1, IN2> inputProcessor = this.inputProcessor; ... while (running && inputProcessor.processInput()) { // all the work happens in the "processInput" method } ... } @Override protected void cancelTask() { running = false; } === StreamTwoInputXProcessor.java === public boolean processInput() throws Exception { while (true) { ... if (element.isRecord()) { return true; } ... } } ``` ---------------------------------------------------------------- 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