Skip to content

Commit

Permalink
[FLINK-11877][runtime] Abstract AbstractTwoInputStreamTask from TwoIn…
Browse files Browse the repository at this point in the history
…putStreamTask
  • Loading branch information
sunhaibotb authored and pnowojski committed Jun 12, 2019
1 parent 68062f5 commit b6299b0
Show file tree
Hide file tree
Showing 2 changed files with 124 additions and 68 deletions.
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;
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,91 +21,45 @@
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.io.StreamTwoInputProcessor;
import org.apache.flink.streaming.runtime.metrics.MinWatermarkGauge;
import org.apache.flink.streaming.runtime.metrics.WatermarkGauge;

import java.util.ArrayList;
import java.util.List;
import java.util.Collection;

/**
* A {@link StreamTask} for executing a {@link TwoInputStreamOperator}.
* A {@link StreamTask} that executes a {@link TwoInputStreamOperator} but does not support
* the {@link TwoInputStreamOperator} to select input for reading.
*/
@Internal
public class TwoInputStreamTask<IN1, IN2, OUT> extends StreamTask<OUT, TwoInputStreamOperator<IN1, IN2, OUT>> {
public class TwoInputStreamTask<IN1, IN2, OUT> extends AbstractTwoInputStreamTask<IN1, IN2, OUT> {

private StreamTwoInputProcessor<IN1, IN2> inputProcessor;

private final WatermarkGauge input1WatermarkGauge;
private final WatermarkGauge input2WatermarkGauge;
private final MinWatermarkGauge minInputWatermarkGauge;

/**
* Constructor for initialization, possibly with initial state (recovery / savepoint / etc).
*
* @param env The task environment for this task.
*/
public TwoInputStreamTask(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);
}
}
protected void createInputProcessor(
Collection<InputGate> inputGates1,
Collection<InputGate> inputGates2,
TypeSerializer<IN1> inputDeserializer1,
TypeSerializer<IN2> inputDeserializer2) throws Exception {

this.inputProcessor = new StreamTwoInputProcessor<>(
inputList1, inputList2,
inputDeserializer1, inputDeserializer2,
this,
configuration.getCheckpointMode(),
getCheckpointLock(),
getEnvironment().getIOManager(),
getEnvironment().getTaskManagerInfo().getConfiguration(),
getStreamStatusMaintainer(),
this.headOperator,
getEnvironment().getMetricGroup().getIOMetricGroup(),
input1WatermarkGauge,
input2WatermarkGauge,
getTaskNameWithSubtaskAndId());

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);
inputGates1, inputGates2,
inputDeserializer1, inputDeserializer2,
this,
getConfiguration().getCheckpointMode(),
getCheckpointLock(),
getEnvironment().getIOManager(),
getEnvironment().getTaskManagerInfo().getConfiguration(),
getStreamStatusMaintainer(),
this.headOperator,
getEnvironment().getMetricGroup().getIOMetricGroup(),
input1WatermarkGauge,
input2WatermarkGauge,
getTaskNameWithSubtaskAndId());
}

@Override
Expand Down

0 comments on commit b6299b0

Please sign in to comment.