Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -337,10 +337,9 @@ private void waitForOneInput(StreamTaskInput input)

private boolean checkFinished() throws Exception {
if (getInput(lastReadInputIndex).isFinished()) {
inputSelection = (lastReadInputIndex == 0) ? InputSelection.SECOND : InputSelection.FIRST;

synchronized (lock) {
operatorChain.endInput(getInputId(lastReadInputIndex));
inputSelection = inputSelector.nextSelection();
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import org.apache.flink.runtime.jobgraph.OperatorID;
import org.apache.flink.streaming.api.graph.StreamConfig;
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
import org.apache.flink.streaming.api.operators.BoundedMultiInput;
import org.apache.flink.streaming.api.operators.InputSelectable;
import org.apache.flink.streaming.api.operators.InputSelection;
import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
Expand Down Expand Up @@ -239,7 +240,7 @@ public void processElement2(StreamRecord<Integer> element) {
* Test operator for sequential reading.
*/
public static class SequentialReadingStreamOperator extends AbstractStreamOperator<String>
implements TwoInputStreamOperator<String, Integer, String>, InputSelectable {
implements TwoInputStreamOperator<String, Integer, String>, InputSelectable, BoundedMultiInput {

private final String name;

Expand All @@ -265,13 +266,18 @@ public void processElement1(StreamRecord<String> element) {
@Override
public void processElement2(StreamRecord<Integer> element) {
output.collect(element.replace("[" + name + "-2]: " + element.getValue()));
}

this.inputSelection = InputSelection.SECOND;
@Override
public void endInput(int inputId) {
if (inputId == 1) {
inputSelection = InputSelection.SECOND;
}
}
}

private static class SpecialRuleReadingStreamOperator extends AbstractStreamOperator<String>
implements TwoInputStreamOperator<String, Integer, String>, InputSelectable {
implements TwoInputStreamOperator<String, Integer, String>, InputSelectable, BoundedMultiInput {

private final String name;

Expand Down Expand Up @@ -338,6 +344,11 @@ public void processElement2(StreamRecord<Integer> element) {

inputSelection = InputSelection.SECOND;
}

@Override
public void endInput(int inputId) {
inputSelection = (inputId == 1) ? InputSelection.SECOND : InputSelection.FIRST;
}
}

private static class TestReadFinishedInputStreamOperator extends AbstractStreamOperator<String>
Expand Down