Flink源码分析系列文档目录
请点击:Flink 源码分析系列文档目录
问题背景
Flink的side output为我们提供了数据分类输出的方式,根据条件将一个流分为多个数据流。如果getSideOutput
调用紧跟着产生side output的算子,side output可以正常输出,但如果中间间隔了其他算子,side output的数据会全部丢失。
例子1:
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
DataStream source = env.socketTextStream("localhost", 9999);
OutputTag outputTag = new OutputTag("test") {};
SingleOutputStreamOperator process = source.process(new ProcessFunction() {
@Override
public void processElement(String value, Context ctx, Collector out) throws Exception {
String[] s = value.split(" ");
String word = s[0];
String ts = s[1];
if (word.startsWith("a")) {
out.collect(value);
} else {
ctx.output(outputTag, value);
}
}
})
process.print("Main: ");
process.getSideOutput(outputTag).print("Side output: ");
env.execute();
这个例子process
处产生side output,同时我们调用了process
的getSideOutput
方法,这种用法可以正常输出side output数据。
例子2:
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
DataStream source = env.socketTextStream("localhost", 9999);
OutputTag outputTag = new OutputTag("test") {};
SingleOutputStreamOperator process = source.process(new ProcessFunction() {
@Override
public void processElement(String value, Context ctx, Collector out) throws Exception {
String[] s = value.split(" ");
String word = s[0];
String ts = s[1];
if (word.startsWith("a")) {
out.collect(value);
} else {
ctx.output(outputTag, value);
}
}
})
.assignTimestampsAndWatermarks(WatermarkStrategy
.forBoundedOutOfOrderness(Duration.ofSeconds(4))
.withTimestampAssigner((data, ts) -> Long.parseLong(data.split(" ")[1])));
process.print("Main: ");
process.getSideOutput(outputTag).print("Side output: ");
env.execute();
和例子1不同,例子2在process
调用之后紧接着为assignTimestampsAndWatermarks
,并不是getSideOutput
。这种情况我们是无法获取到side output输出的。
问题分析
Flink仅有processFunction可使用SideOutput。包含如下4种:
- ProcessFunction
- CoProcessFunction
- ProcessWindowFunction
- ProcessAllWindowFunction
如果SideOutput下游紧跟着getSideOutput
,类似例子1,通过debug processFunction
发现,其中Collector和Context最底层调用的为BroadcastingOutputCollector
的两个重载collect
方法,分别为:
@Override
public void collect(StreamRecord record) {
for (Output> output : outputs) {
output.collect(record);
}
}
@Override
public void collect(OutputTag outputTag, StreamRecord record) {
for (Output> output : outputs) {
output.collect(outputTag, record);
}
}
调查它的outputs
变量,它包含2个RecordWriterOutput
。其中一个RecordWriterOutput
中的OutputTag
为null,另一个为例子中创建的outputTag。为什么会创建出两个RecordWriterOutput
,这两个output的作用是什么?我们需要从getSideOutput
方法开始从头捋一捋。
SingleOutputStreamOperator
的getSideOutput
方法代码如下:
public DataStream getSideOutput(OutputTag sideOutputTag) {
sideOutputTag = clean(requireNonNull(sideOutputTag));
// make a defensive copy
sideOutputTag = new OutputTag(sideOutputTag.getId(), sideOutputTag.getTypeInfo());
TypeInformation> type = requestedSideOutputs.get(sideOutputTag);
if (type != null && !type.equals(sideOutputTag.getTypeInfo())) {
throw new UnsupportedOperationException(
"A side output with a matching id was "
+ "already requested with a different type. This is not allowed, side output "
+ "ids need to be unique.");
}
requestedSideOutputs.put(sideOutputTag, sideOutputTag.getTypeInfo());
// 生成了一个SideOutputTransformation
SideOutputTransformation sideOutputTransformation =
new SideOutputTransformation<>(this.getTransformation(), sideOutputTag);
return new DataStream<>(this.getExecutionEnvironment(), sideOutputTransformation);
}
getSideOutput
方法的主要作用为DataStream
加入一个SideOutputTransformation
。
再查看翻译SideOutputTransformation
的过程,代码位于SideOutputTransformationTranslator
:
private Collection translateInternal(
final SideOutputTransformation transformation, final Context context) {
checkNotNull(transformation);
checkNotNull(context);
final StreamGraph streamGraph = context.getStreamGraph();
// 获取StreamGraph上游的transformation,这里应该只有一个
final List> parentTransformations = transformation.getInputs();
checkState(
parentTransformations.size() == 1,
"Expected exactly one input transformation but found "
+ parentTransformations.size());
final List virtualResultIds = new ArrayList<>();
final Transformation> parentTransformation = parentTransformations.get(0);
for (int inputId : context.getStreamNodeIds(parentTransformation)) {
final int virtualId = Transformation.getNewNodeId();
// 为StreamGraph加入了一个虚拟node,虚拟node并不会引入额外计算
streamGraph.addVirtualSideOutputNode(inputId, virtualId, transformation.getOutputTag());
virtualResultIds.add(virtualId);
}
return virtualResultIds;
}
通过这段代码可以得知SideOutputTransformation对应的为virtual node。和正常的Stream node不同,virtual node并不会真的去创建一个stream node。那么addVirtualSideOutputNode
究竟做了什么?我们继续向下分析。
StreamGraph
的addVirtualSideOutputNode
方法:
public void addVirtualSideOutputNode(
Integer originalId, Integer virtualId, OutputTag outputTag) {
if (virtualSideOutputNodes.containsKey(virtualId)) {
throw new IllegalStateException("Already has virtual output node with id " + virtualId);
}
// verify that we don't already have a virtual node for the given originalId/outputTag
// combination with a different TypeInformation. This would indicate that someone is trying
// to read a side output from an operation with a different type for the same side output
// id.
for (Tuple2 tag : virtualSideOutputNodes.values()) {
if (!tag.f0.equals(originalId)) {
// different source operator
continue;
}
if (tag.f1.getId().equals(outputTag.getId())
&& !tag.f1.getTypeInfo().equals(outputTag.getTypeInfo())) {
throw new IllegalArgumentException(
"Trying to add a side output for the same "
+ "side-output id with a different type. This is not allowed. Side-output ID: "
+ tag.f1.getId());
}
}
// 加入虚拟node到virtualSideOutputNodes集合
virtualSideOutputNodes.put(virtualId, new Tuple2<>(originalId, outputTag));
}
这个方法仅仅是将虚拟节点id,原始Transformation节点id和outputTag三者的对应关系保存了起来。
virtualSideOutputNodes
变量什么时候会被用到?我们追查到生成Edge的逻辑,位于addEdgeInternal
方法,内容较多。我们只关注相关代码片段,如下所示:
// ...
// 在上面例子中。创建出的Edge上游vertex包含process函数,下游vertex包含print
// 同时附带outputTag为test
if (virtualSideOutputNodes.containsKey(upStreamVertexID)) {
int virtualId = upStreamVertexID;
upStreamVertexID = virtualSideOutputNodes.get(virtualId).f0;
if (outputTag == null) {
outputTag = virtualSideOutputNodes.get(virtualId).f1;
}
addEdgeInternal(
upStreamVertexID,
downStreamVertexID,
typeNumber,
partitioner,
null,
outputTag,
exchangeMode);
}
// ...
到此不难发现我们例子中有两个print算子,分别对应标准流print和SideOutput的print。他们又分别正好对应了两个Edge,其中side output对应Edge的OutputTag为例子中的test。
分析完sideoutput如何影响到StreamGraph edge的创建之后,我们继续分析BroadcastingOutputCollector
是如何被创建出来的。
跟踪它构造函数的调用位置,发现它是在OperatorChain
中被创建出来。我们查看它的createOutputCollector
方法,内容如下:
private WatermarkGaugeExposingOutput> createOutputCollector(
StreamTask, ?> containingTask,
StreamConfig operatorConfig,
Map chainedConfigs,
ClassLoader userCodeClassloader,
Map> streamOutputs,
List> allOperatorWrappers,
MailboxExecutorFactory mailboxExecutorFactory) {
List>, StreamEdge>> allOutputs =
new ArrayList<>(4);
// create collectors for the network outputs
// 获取所有非chained的出边,即NetworkOutput
for (StreamEdge outputEdge : operatorConfig.getNonChainedOutputs(userCodeClassloader)) {
@SuppressWarnings("unchecked")
RecordWriterOutput output = (RecordWriterOutput) streamOutputs.get(outputEdge);
allOutputs.add(new Tuple2<>(output, outputEdge));
}
// Create collectors for the chained outputs
// 这里会遍历它的所有chained的出边
for (StreamEdge outputEdge : operatorConfig.getChainedOutputs(userCodeClassloader)) {
int outputId = outputEdge.getTargetId();
StreamConfig chainedOpConfig = chainedConfigs.get(outputId);
// 例子中创建出的实际类型为RecordWriterOutput
WatermarkGaugeExposingOutput> output =
createOperatorChain(
containingTask,
chainedOpConfig,
chainedConfigs,
userCodeClassloader,
streamOutputs,
allOperatorWrappers,
// 创建出的output会保存自己的OutputTag
outputEdge.getOutputTag(),
mailboxExecutorFactory);
// 存入allOutputs集合
allOutputs.add(new Tuple2<>(output, outputEdge));
}
if (allOutputs.size() == 1) {
return allOutputs.get(0).f0;
} else {
// send to N outputs. Note that this includes the special case
// of sending to zero outputs
// 将allOutputs集合转换为数组
@SuppressWarnings({"unchecked"})
Output>[] asArray = new Output[allOutputs.size()];
for (int i = 0; i < allOutputs.size(); i++) {
asArray[i] = allOutputs.get(i).f0;
}
// This is the inverse of creating the normal ChainingOutput.
// If the chaining output does not copy we need to copy in the broadcast output,
// otherwise multi-chaining would not work correctly.
// 将allOutputs传递给CopyingBroadcastingOutputCollector或者BroadcastingOutputCollector
if (containingTask.getExecutionConfig().isObjectReuseEnabled()) {
return closer.register(new CopyingBroadcastingOutputCollector<>(asArray));
} else {
return closer.register(new BroadcastingOutputCollector<>(asArray));
}
}
}
可以得知BroadcastingOutputCollector
包含了所有的下游output
。
这时候,我们回到前面所述的BroadcastingOutputCollector
的collect
方法:
@Override
public void collect(StreamRecord record) {
for (Output> output : outputs) {
output.collect(record);
}
}
@Override
public void collect(OutputTag outputTag, StreamRecord record) {
for (Output> output : outputs) {
output.collect(outputTag, record);
}
}
不难看出,这个方法将数据广播到所有的output中,根据情况确定是否发送outputTag。
我们再去查看RecordWriterOutput
的collect
方法:
public void collect(OutputTag outputTag, StreamRecord record) {
if (OutputTag.isResponsibleFor(this.outputTag, outputTag)) {
this.pushToRecordWriter(record);
}
}
到这里一切就就清晰了,只有当collector
的outputTag
和collect
方法中传入的outputTag
相同(调用OutputTag
的equals
方法)的时候,数据才会写入recordWriter中,否则数据会被忽略掉。例子1的两个RecordWriterOutput
分别对应了标准流输出和side output输出。
反过来我们看看例子2。例子2中process
函数的直接下游为assignTimestampsAndWatermarks
,没有getSideOutput
调用,因此在process
和assignTimestampsAndWatermarks
之间不会产生SideOutputTransformation
。也就是说连接这两个算子的edge不会被标注OutputTag。即数据发往assignTimestampsAndWatermarks
算子的RecordWriterOutput
持有的outputTag
变量为null,所有发往side output的元素会被忽略。
本博客为作者原创,欢迎大家参与讨论和批评指正。如需转载请注明出处。