Skip to content

Commit 158e447

Browse files
committed
[FLINK-16317][operators] Provide support for watermarks in MultipleInputStreamOperator
1 parent 5b3d4eb commit 158e447

File tree

12 files changed

+371
-31
lines changed

12 files changed

+371
-31
lines changed
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,48 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
package org.apache.flink.streaming.api.operators;
20+
21+
import org.apache.flink.annotation.Experimental;
22+
import org.apache.flink.streaming.api.watermark.Watermark;
23+
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
24+
25+
import static org.apache.flink.util.Preconditions.checkArgument;
26+
27+
/**
28+
* Base abstract implementation of {@link Input} interface intended to be used when extending
29+
* {@link AbstractStreamOperatorV2}.
30+
*/
31+
@Experimental
32+
public abstract class AbstractInput<IN, OUT> implements Input<IN> {
33+
protected final AbstractStreamOperatorV2<OUT> owner;
34+
protected final int inputId;
35+
protected final Output<StreamRecord<OUT>> output;
36+
37+
public AbstractInput(AbstractStreamOperatorV2<OUT> owner, int inputId) {
38+
checkArgument(inputId > 0, "Inputs are index from 1");
39+
this.owner = owner;
40+
this.inputId = inputId;
41+
this.output = owner.output;
42+
}
43+
44+
@Override
45+
public void processWatermark(Watermark mark) throws Exception {
46+
owner.reportWatermark(mark, inputId);
47+
}
48+
}

flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorV2.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -465,7 +465,7 @@ public void processWatermark(Watermark mark) throws Exception {
465465
}
466466

467467
protected void reportWatermark(Watermark mark, int inputId) throws Exception {
468-
inputWatermarks[inputId] = mark.getTimestamp();
468+
inputWatermarks[inputId - 1] = mark.getTimestamp();
469469
long newMin = mark.getTimestamp();
470470
for (long inputWatermark : inputWatermarks) {
471471
newMin = Math.min(inputWatermark, newMin);

flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/Input.java

+9
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@
1919
package org.apache.flink.streaming.api.operators;
2020

2121
import org.apache.flink.annotation.PublicEvolving;
22+
import org.apache.flink.streaming.api.watermark.Watermark;
2223
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
2324

2425
/**
@@ -31,4 +32,12 @@ public interface Input<IN> {
3132
* This method is guaranteed to not be called concurrently with other methods of the operator.
3233
*/
3334
void processElement(StreamRecord<IN> element) throws Exception;
35+
36+
/**
37+
* Processes a {@link Watermark} that arrived on the first input of this two-input operator.
38+
* This method is guaranteed to not be called concurrently with other methods of the operator.
39+
*
40+
* @see org.apache.flink.streaming.api.watermark.Watermark
41+
*/
42+
void processWatermark(Watermark mark) throws Exception;
3443
}

flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java

+7-5
Original file line numberDiff line numberDiff line change
@@ -81,13 +81,14 @@ public StreamMultipleInputProcessor(
8181
this.inputSelectionHandler = checkNotNull(inputSelectionHandler);
8282

8383
List<Input> inputs = streamOperator.getInputs();
84-
int operatorsCount = inputs.size();
84+
int inputsCount = inputs.size();
8585

86-
this.inputProcessors = new InputProcessor[operatorsCount];
87-
this.streamStatuses = new StreamStatus[operatorsCount];
86+
this.inputProcessors = new InputProcessor[inputsCount];
87+
this.streamStatuses = new StreamStatus[inputsCount];
8888
this.numRecordsIn = numRecordsIn;
8989

90-
for (int i = 0; i < operatorsCount; i++) {
90+
for (int i = 0; i < inputsCount; i++) {
91+
streamStatuses[i] = StreamStatus.ACTIVE;
9192
StreamTaskNetworkOutput dataOutput = new StreamTaskNetworkOutput<>(
9293
inputs.get(i),
9394
streamStatusMaintainer,
@@ -282,7 +283,8 @@ public void emitRecord(StreamRecord<T> record) throws Exception {
282283

283284
@Override
284285
public void emitWatermark(Watermark watermark) throws Exception {
285-
throw new UnsupportedOperationException();
286+
inputWatermarkGauge.setCurrentWatermark(watermark.getTimestamp());
287+
input.processWatermark(watermark);
286288
}
287289

288290
@Override

flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTask.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -63,7 +63,7 @@ public void init() throws Exception {
6363
for (int i = 0; i < inputDeserializers.length; i++) {
6464
inputLists[i] = new ArrayList<>();
6565
watermarkGauges[i] = new WatermarkGauge();
66-
headOperator.getMetricGroup().gauge(MetricNames.currentInputWatermarkName(i), watermarkGauges[i]);
66+
headOperator.getMetricGroup().gauge(MetricNames.currentInputWatermarkName(i + 1), watermarkGauges[i]);
6767
}
6868

6969
MinWatermarkGauge minInputWatermarkGauge = new MinWatermarkGauge(watermarkGauges);

0 commit comments

Comments
 (0)