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_r275400858
########## 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 800~1000 ops/ms faster than using the external loop Hmmm, it would be strange. I can see only two possibilities for this outcome: 1. You accidentally measured the impact of adding second `volatile` and this was the actual performance regression 2. JIT was not able to inline `StreamTwoInputSelectableProcessor#processInput` since it's too big. Regarding point 2. - I wasn't commenting on it yet, since I was doing only a high level review, but before merging this code, a lot of those methods should be broken down to smaller ones. So if you will be sure that 1. is not the case, please refactor the code of `StreamTwoInputSelectableProcessor#processInput` or `NetworkInput#pollNextElement` by extracting some of the code to a smaller methods. For example both branches of the `if (readingInputIndex == 0) {` could be extracted. My point is that keeping those two loops working both of them on `while (continuousProcessing)` condition doesn't make sense. One of them should be removed. It would be easier to keep the things how they were, since this pattern of loop inside `XInputStreamTask` and `XInputStreamProcessor` is also in `OneInput` case. If for whatever reasons we decide to change it for `TwoInput`, we should make sure that `OneInput` is kept up to date and in sync. ---------------------------------------------------------------- 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