Side Outputs
In addition to the main stream that results from DataStream
operations, you can also produce anynumber of additional side output result streams. The type of data in the result streams does nothave to match the type of data in the main stream and the types of the different side outputs canalso differ. This operation can be useful when you want to split a stream of data where you wouldnormally have to replicate the stream and then filter out from each stream the data that you don’twant to have.
When using side outputs, you first need to define an OutputTag
that will be used to identify aside output stream:
// this needs to be an anonymous inner class, so that we can analyze the type
OutputTag<String> outputTag = new OutputTag<String>("side-output") {};
val outputTag = OutputTag[String]("side-output")
Notice how the OutputTag
is typed according to the type of elements that the side output streamcontains.
Emitting data to a side output is possible from the following functions:
- ProcessFunction
- KeyedProcessFunction
- CoProcessFunction
- KeyedCoProcessFunction
- ProcessWindowFunction
- ProcessAllWindowFunction
You can use the Context
parameter, which is exposed to users in the above functions, to emitdata to a side output identified by an OutputTag
. Here is an example of emitting side outputdata from a ProcessFunction
:
DataStream<Integer> input = ...;
final OutputTag<String> outputTag = new OutputTag<String>("side-output"){};
SingleOutputStreamOperator<Integer> mainDataStream = input
.process(new ProcessFunction<Integer, Integer>() {
@Override
public void processElement(
Integer value,
Context ctx,
Collector<Integer> out) throws Exception {
// emit data to regular output
out.collect(value);
// emit data to side output
ctx.output(outputTag, "sideout-" + String.valueOf(value));
}
});
val input: DataStream[Int] = ...
val outputTag = OutputTag[String]("side-output")
val mainDataStream = input
.process(new ProcessFunction[Int, Int] {
override def processElement(
value: Int,
ctx: ProcessFunction[Int, Int]#Context,
out: Collector[Int]): Unit = {
// emit data to regular output
out.collect(value)
// emit data to side output
ctx.output(outputTag, "sideout-" + String.valueOf(value))
}
})
For retrieving the side output stream you use getSideOutput(OutputTag)
on the result of the DataStream
operation. This will give you a DataStream
that is typedto the result of the side output stream:
final OutputTag<String> outputTag = new OutputTag<String>("side-output"){};
SingleOutputStreamOperator<Integer> mainDataStream = ...;
DataStream<String> sideOutputStream = mainDataStream.getSideOutput(outputTag);
val outputTag = OutputTag[String]("side-output")
val mainDataStream = ...
val sideOutputStream: DataStream[String] = mainDataStream.getSideOutput(outputTag)