pnowojski commented on a change in pull request #13529: URL: https://github.com/apache/flink/pull/13529#discussion_r501739663
########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sort/MultiInputSortingDataInputs.java ########## @@ -0,0 +1,341 @@ +/* + * 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.api.operators.sort; + +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.AlgorithmOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.io.InputStatus; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.memory.MemoryAllocationException; +import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.operators.sort.ExternalSorter; +import org.apache.flink.runtime.operators.sort.PushSorter; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.io.PushingAsyncDataInput; +import org.apache.flink.streaming.runtime.io.StreamTaskInput; +import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.streamstatus.StreamStatus; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.MutableObjectIterator; + +import javax.annotation.Nonnull; + +import java.io.IOException; +import java.util.Arrays; +import java.util.PriorityQueue; +import java.util.concurrent.CompletableFuture; + +/** + * An entry class for creating coupled, sorting inputs. The inputs are sorted independently and afterwards + * the inputs are being merged in order. It is done, by reporting availability only for the input which is + * current head for the sorted inputs. + */ +public class MultiInputSortingDataInputs<K> { Review comment: Ok, usually I would suggest to squash such commits (to avoid having a dead code), but that's a minor issue, while indeed otherwise those two squashed commits would be quite big, so I'm fine with leaving it as it is. ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java ########## @@ -23,16 +23,12 @@ import org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter; Review comment: Ok, I see, and I agree that your change would increase usability of this code, protecting against those kind of bugs. However as you proposed it would currently add an extra `volatile` check. I think this could be addressed, by combining this check with anti-starvation check. However on the other hand... It looks like I have implemented it differently in the `StreamMultipleInputProcessor`, where I do not see such check at all, and the processor completely relies on the `InputStatus`, which all in all I think is easier way of handling this. `isAvailable()` is checked only after the first `processInput` call after being woken up: ``` private int selectNextReadingInputIndex() { if (!inputSelectionHandler.isAnyInputAvailable()) { fullCheckAndSetAvailable(); } ... } ``` Regarding the documentation. `AvailabilityProvider` already has quite a bit of java docs, but maybe you could help updating it to avoid such confusion in the future? ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/TwoInputSelectionHandler.java ########## @@ -70,6 +70,10 @@ boolean areAllInputsSelected() { return inputSelection.areAllInputsSelected(); } + boolean isInputUnavailable(int inputIndex) { Review comment: +0.1 for `isInputAvailable` from my side. `isInputUnavailable` sounds a bit strange :) ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamOneInputProcessor.java ########## @@ -62,6 +62,16 @@ public StreamOneInputProcessor( return input.getAvailableFuture(); } + @Override + public boolean isAvailable() { + return input.isAvailable(); + } + + @Override + public boolean isApproximatelyAvailable() { + return input.isApproximatelyAvailable(); + } + Review comment: Why do we need this commit? The default implementations should be doing the same thing. ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamOneInputProcessor.java ########## @@ -45,16 +45,16 @@ private final StreamTaskInput<IN> input; private final DataOutput<IN> output; - private final OperatorChain<?, ?> operatorChain; + private final BoundedMultiInput endOfInputAware; public StreamOneInputProcessor( StreamTaskInput<IN> input, DataOutput<IN> output, - OperatorChain<?, ?> operatorChain) { + BoundedMultiInput endOfInputAware) { Review comment: nit: it would be nice to keep the remaining processors in sync with this change? ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamOneInputProcessor.java ########## @@ -45,16 +45,16 @@ private final StreamTaskInput<IN> input; private final DataOutput<IN> output; - private final OperatorChain<?, ?> operatorChain; + private final BoundedMultiInput endOfInputAware; public StreamOneInputProcessor( StreamTaskInput<IN> input, DataOutput<IN> output, - OperatorChain<?, ?> operatorChain) { + BoundedMultiInput endOfInputAware) { Review comment: nit: it would be nice to keep the remaining processors in sync with this change? Due to the fact you are touching the processors in a couple of later commits, to avoid conflicts I would be also fine with using `BoundedMultiInput` in `StreamTwoInputProcessor` and `StreamMultipleInputProcessor` in another hotfix/refactor commit as the last commit. ---------------------------------------------------------------- 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]
