Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

...

We want to introduce outputTag and support operator collect arbitrary types of records with defined output Tags. In this prototype, it demonstrated how things works in raw/hacky form.

User may declare multiple output tags and use get output stream with one previously defined outputtag.

final OutputTag<S> sideOutput1 = new OutputTag<S>(SValue) {}; 
 

Update userFunctions using RichCollector(Or MultiCollector) the rationale behind is Collector has been used in lot of places other than stream and transformation, adding direct to Collector interface will introduce many empty methods in those classes.

public interface RichCollector<T> extends Collector<T>{

...

   <S> void collect(OutputTag<S> tag, S value);
}

FlatMapFunction as example

flatMap(String value, Collector<Tuple2<String, Integer>> out){
           CollectorWrapper wrapper = new CollectorWrapper<>(out);
    //out.collect(new Tuple2<String, Integer>(token, 1));

                 wrapper.collect(new Tuple2<String, Integer>(token, 1));

        wrapper.collect(sideOutput1, "sideout");
}

User may pass outputtag defined eearlier and get a corresponding outputstream.

There can be more than one outputtag share same type, however, getSideOutput only returns collected record with exams same outputtag type and value.

       flatMap(..).getSideOutput(sideOutput1) 
    StreamGraph add virtualOutputNodes each map to single outputtag from upstream node
    Stream Record and StreamEdge both add outputTag, StreamConfig stores all type of side output serializers 

Record writer can utilize this information and compare with outputType( impl in prototype) or OutputTag (better implementation) or each Output<OUT> and just write matched stream record to channel

It allows good compatibility without drastic change of current “single typed” output model.

    public class SideOutputTransformation<T> extends StreamTransformation<T> {
       publicpublic SideOutputTransformation(StreamTransformation input, OutputTag<T> tag) {
          super super("SideOutput", tag.getTypeInformation(), input.getParallelism());
           thisthis.input = input;
      }

        }

  • OperatorChain Outputs needs to filter stream record with outputtag

Instead of assuming output stream record values are same type, it will need to check output type and stream record type, only output with same outputtag typeinfo

...

To some extend, getOutput works like split stream and select by outputtag of a specific tag. It might be great to embed "outputselector" and reuse selectvirtualnode. Problem is split and select only takes list of strings. Flink might consider allow split assign tags, select choose tags. No need to add another type of virtual node which almost do exactly same as select

In current form, many functions assume single OUT type.

@Public
public interface JoinFunction<IN1, IN2, OUT> extends Function, Serializable {
  OUT join(IN1 first, IN2 second) throws Exception;
}
...
public void join(IN1 left, IN2 right, Collector<OUT> out) throws Exception {
  out.collect(this.wrappedFunction.join(left, right));
}

...

.


Compatibility, Deprecation, and Migration Plan

...