...
Regarding the second point, process/ connectAndProcess and other operations performed on DataStreams will return an instance of corresponding interfaces. We will change the return type of one-input/two-output/two-input process and toSink methods as shown in the table below.
stream type | function | old return type | new return type |
NonKeyedPartitionStream | process(OneInputStreamProcessFunction) | NonKeyedPartitionStream | ProcessConfigurableAndNonKeyedPartitionStream |
process(TwoOutputStreamProcessFunction) | TwoNonKeyedPartitionStreams | ProcessConfigurableAndTwoNonKeyedPartitionStreams | |
connectAndProcess(NonKeyedPartitionStream, TwoInputStreamProcessFunction) | NonKeyedPartitionStream | ProcessConfigurableAndNonKeyedPartitionStream | |
connectAndProcess(BroadcastStream, | |||
TwoInputBroadcastStreamProcessFunction | NonKeyedPartitionStream | ProcessConfigurableAndNonKeyedPartitionStream | |
toSink(Sink) | void | ProcessConfigurable | |
KeyedPartitionStream | process(OneInputStreamProcessFunction, KeySelector) | KeyedPartitionStream | ProcessConfigurableAndKeyedPartitionStream |
process(OneInputStreamProcessFunction) | NonKeyedPartitionStream | ProcessConfigurableAndNonKeyedPartitionStream | |
process(TwoOutputStreamProcessFunction, KeySelector, KeySelector) | TwoKeyedPartitionStreams | ProcessConfigurableAndTwoKeyedPartitionStreams | |
process(TwoOutputStreamProcessFunction) | TwoNonKeyedPartitionStreams | ProcessConfigurableAndTwoNonKeyedPartitionStreams | |
connectAndProcess(KeyedPartitionStream, | |||
TwoInputNonBroadcastStreamProcessFunction); | NonKeyedPartitionStream | ProcessConfigurableAndNonKeyedPartitionStream | |
connectAndProcess(KeyedPartitionStream, | |||
TwoInputNonBroadcastStreamProcessFunction, KeySelector) | KeyedPartitionStream | ProcessConfigurableAndKeyedPartitionStream | |
connectAndProcess(BroadcastStream, | |||
TwoInputBroadcastStreamProcessFunction) | NonKeyedPartitionStream | ProcessConfigurableAndNonKeyedPartitionStream | |
connectAndProcess(BroadcastStream, | |||
TwoInputBroadcastStreamProcessFunction, KeySelector) | KeyedPartitionStream | ProcessConfigurableAndKeyedPartitionStream | |
toSink(Sink) | void | ProcessConfigurable | |
GlobalStream | process(OneInputStreamProcessFunction) | GlobalStream | ProcessConfigurableAndGlobalStream |
process(TwoOutputStreamProcessFunction) | TwoGlobalStreams | ProcessConfigurableAndTwoGlobalStreams | |
connectAndProcess(GlobalStream, | |||
TwoInputNonBroadcastStreamProcessFunction) | GlobalStream | ProcessConfigurableAndGlobalStream | |
toSink(Sink) | void | ProcessConfigurable | |
BroadcastStream | connectAndProcess(KeyedPartitionStream, | ||
TwoInputBroadcastStreamProcessFunction) | NonKeyedPartitionStream | ProcessConfigurableAndNonKeyedPartitionStream | |
connectAndProcess(NonKeyedPartitionStream, | |||
TwoInputBroadcastStreamProcessFunction) | NonKeyedPartitionStream | ProcessConfigurableAndNonKeyedPartitionStream | |
connectAndProcess(KeyedPartitionStream, | |||
TwoInputBroadcastStreamProcessFunction, KeySelector) | KeyedPartitionStream | ProcessConfigurableAndKeyedPartitionStream |
Move SlotSharingGroup to flink-core-api module
...
Code Block | ||||
---|---|---|---|---|
| ||||
/** * A RuntimeContext contains information about the context in which process functions are executed. * Each parallel instance of the function will have a context through which it can access contextual * information, such as the current key and execution mode. Through this context, we can also * interact with the execution layer, such as getting state, emitting watermark, registering timer, etc. */ public interface RuntimeContext { /** Get the {@link JobInfo} of this process function. */ JobInfo getJobInfo(); /** Get the {@link TaskInfo} of this process function. */ TaskInfo getTaskInfo(); /** Get the {@link ProcessingTimeManager} of this process function. */ ProcessingTimeManager getProcessingTimeManager(); /** Get the {@link StateManager} of this process function. */ StateManager getStateManager(); /** Get the {@link WatermarkManager} of this process function. */ WatermarkManager getWatermarkManager(); /** Get the metric group of this process function. */ OperatorMetricGroup getMetricGroup(); /** Get the {@link TimestampManager} of this process function. */ TimestampManager getTimestampManager(); } |
Only partition independent methods are supported for its subclass NonPartitionedContext, they include getJobInfo, getTaskInfo, getWatermarkManager, getMetricGroup, getTimestampManager. But partition-related operations such as registering and deleting processing timers and getting state are not supported since the current partition cannot be decided.
ProcessFunction
At the same time, we should expand the implementation of Process Function. In more detail, we will introduce a new method called onProcessingTimer to all type of process functions. It is the callback for the timer registered through RuntimeContext#ProcessingTimeManager.
Note: The logic for processing watermark is omitted, but we will elaborate on it later in the future sub-FLIPs.
...
Code Block | ||||
---|---|---|---|---|
| ||||
/** This class contains all logical related to process records from single input. */ public interface OneInputStreamProcessFunction<IN, OUT> extends ProcessFunction { // Omit logic of processing data and life-cycle methods. They can be found in FLIP-1409. /** * Callback for processing timer. * * @param timestamp when this callback is triggered. * @param output to emit record. * @param ctx, runtime context in which this function is executed. */ default void onProcessingTimer(long timestamp, Collector<OUT> output, RuntimeContext ctx) {} } |
...
TwoInputNonBroadcastStreamProcessFunction
Code Block | ||||
---|---|---|---|---|
| ||||
/** This class contains all logical related to process records from two non-broadcast input. */ public interface TwoInputStreamProcessFunction<IN1TwoInputNonBroadcastStreamProcessFunction<IN1, IN2, OUT> extends ProcessFunction { // Omit logic of processing data and life-cycle methods. They can be found in FLIP-1409. /** * Callback for processing timer. * * @param timestamp when this callback is triggered. * @param output to emit record. * @param ctx, runtime context in which this function is executed. */ default void onProcessingTimer(long timestamp, Collector<OUT> output, RuntimeContext ctx) {} } |
...
TwoInputBroadcastStreamProcessFunction
Code Block | ||||
---|---|---|---|---|
| ||||
/** * This class contains all logical related to process records from two input. a broadcast stream and a * non-broadcast stream. */ public interface TwoInputStreamProcessFunction<IN1TwoInputBroadcastStreamProcessFunction<IN1, IN2, OUT> extends ProcessFunction { // Omit logic of processing data and life-cycle methods. They can be found in FLIP-1409. /** * Callback for processing timer. * * @param timestamp when this callback is triggered. * @param output to emit record. * @param ctx, runtime context in which this function is executed. */ default void onProcessingTimer(long timestamp, Collector<OUT> output, RuntimeContext ctx) {} } |
...
Code Block | ||||
---|---|---|---|---|
| ||||
/** This class contains all logical related to process and emit records to two outputs. */ public interface TwoOutputStreamProcessFunction<IN, OUT1, OUT2> extends ProcessFunction { // Omit logic of processing data and life-cycle methods. They can be found in FLIP-1409. /** * Callback for processing timer. * * @param timestamp when this callback is triggered. * @param output1 to emit record. * @param output2 to emit record. * @param ctx, runtime context in which this function is executed. */ default void onProcessingTimer( long timestamp, Collector<OUT1> output1, Collector<OUT2> output2, RuntimeContext ctx) {} } |
...