forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-11877][runtime] Abstract AbstractTwoInputStreamTask from TwoIn…
…putStreamTask
- Loading branch information
1 parent
68062f5
commit b6299b0
Showing
2 changed files
with
124 additions
and
68 deletions.
There are no files selected for viewing
102 changes: 102 additions & 0 deletions
102
...va/src/main/java/org/apache/flink/streaming/runtime/tasks/AbstractTwoInputStreamTask.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,102 @@ | ||
/* | ||
* 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:https://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.tasks; | ||
|
||
import org.apache.flink.annotation.Internal; | ||
import org.apache.flink.api.common.typeutils.TypeSerializer; | ||
import org.apache.flink.runtime.execution.Environment; | ||
import org.apache.flink.runtime.io.network.partition.consumer.InputGate; | ||
import org.apache.flink.runtime.metrics.MetricNames; | ||
import org.apache.flink.streaming.api.graph.StreamConfig; | ||
import org.apache.flink.streaming.api.graph.StreamEdge; | ||
import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; | ||
import org.apache.flink.streaming.runtime.metrics.MinWatermarkGauge; | ||
import org.apache.flink.streaming.runtime.metrics.WatermarkGauge; | ||
|
||
import java.util.ArrayList; | ||
import java.util.Collection; | ||
import java.util.List; | ||
|
||
/** | ||
* Abstract class for executing a {@link TwoInputStreamOperator}. | ||
*/ | ||
@Internal | ||
public abstract class AbstractTwoInputStreamTask<IN1, IN2, OUT> extends StreamTask<OUT, TwoInputStreamOperator<IN1, IN2, OUT>> { | ||
|
||
protected final WatermarkGauge input1WatermarkGauge; | ||
protected final WatermarkGauge input2WatermarkGauge; | ||
protected final MinWatermarkGauge minInputWatermarkGauge; | ||
|
||
/** | ||
* Constructor for initialization, possibly with initial state (recovery / savepoint / etc). | ||
* | ||
* @param env The task environment for this task. | ||
*/ | ||
public AbstractTwoInputStreamTask(Environment env) { | ||
super(env); | ||
|
||
input1WatermarkGauge = new WatermarkGauge(); | ||
input2WatermarkGauge = new WatermarkGauge(); | ||
minInputWatermarkGauge = new MinWatermarkGauge(input1WatermarkGauge, input2WatermarkGauge); | ||
} | ||
|
||
@Override | ||
public void init() throws Exception { | ||
StreamConfig configuration = getConfiguration(); | ||
ClassLoader userClassLoader = getUserCodeClassLoader(); | ||
|
||
TypeSerializer<IN1> inputDeserializer1 = configuration.getTypeSerializerIn1(userClassLoader); | ||
TypeSerializer<IN2> inputDeserializer2 = configuration.getTypeSerializerIn2(userClassLoader); | ||
|
||
int numberOfInputs = configuration.getNumberOfInputs(); | ||
|
||
ArrayList<InputGate> inputList1 = new ArrayList<InputGate>(); | ||
ArrayList<InputGate> inputList2 = new ArrayList<InputGate>(); | ||
|
||
List<StreamEdge> inEdges = configuration.getInPhysicalEdges(userClassLoader); | ||
|
||
for (int i = 0; i < numberOfInputs; i++) { | ||
int inputType = inEdges.get(i).getTypeNumber(); | ||
InputGate reader = getEnvironment().getInputGate(i); | ||
switch (inputType) { | ||
case 1: | ||
inputList1.add(reader); | ||
break; | ||
case 2: | ||
inputList2.add(reader); | ||
break; | ||
default: | ||
throw new RuntimeException("Invalid input type number: " + inputType); | ||
} | ||
} | ||
|
||
createInputProcessor(inputList1, inputList2, inputDeserializer1, inputDeserializer2); | ||
|
||
headOperator.getMetricGroup().gauge(MetricNames.IO_CURRENT_INPUT_WATERMARK, minInputWatermarkGauge); | ||
headOperator.getMetricGroup().gauge(MetricNames.IO_CURRENT_INPUT_1_WATERMARK, input1WatermarkGauge); | ||
headOperator.getMetricGroup().gauge(MetricNames.IO_CURRENT_INPUT_2_WATERMARK, input2WatermarkGauge); | ||
// wrap watermark gauge since registered metrics must be unique | ||
getEnvironment().getMetricGroup().gauge(MetricNames.IO_CURRENT_INPUT_WATERMARK, minInputWatermarkGauge::getValue); | ||
} | ||
|
||
protected abstract void createInputProcessor( | ||
Collection<InputGate> inputGates1, | ||
Collection<InputGate> inputGates2, | ||
TypeSerializer<IN1> inputDeserializer1, | ||
TypeSerializer<IN2> inputDeserializer2) throws Exception; | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters