Class KeyedBroadcastProcessFunction<KS,IN1,IN2,OUT>
- Type Parameters:
KS- The key type of the input keyed stream.IN1- The input type of the keyed (non-broadcast) side.IN2- The input type of the broadcast side.OUT- The output type of the operator.
- All Implemented Interfaces:
Serializable,org.apache.flink.api.common.functions.Function,org.apache.flink.api.common.functions.RichFunction
BroadcastConnectedStream that
connects BroadcastStream, i.e.
a stream with broadcast state, with a KeyedStream.
The stream with the broadcast state can be created using the DataStream.broadcast(MapStateDescriptor[])
keyedStream.broadcast(MapStateDescriptor)} method.
The user has to implement two methods:
- the
#processBroadcastElement(Object, Context, Collector)which will be applied to each element in the broadcast side - and the
#processElement(Object, ReadOnlyContext, Collector)which will be applied to the non-broadcasted/keyed side.
The processElementOnBroadcastSide() takes as an argument (among others) a context that
allows it to read/write to the broadcast state and also apply a transformation to all (local)
keyed states, while the processElement() has read-only access to the broadcast state, but
can read/write to the keyed state and register timers.
- See Also:
-
Nested Class Summary
Nested ClassesModifier and TypeClassDescriptionclassAcontextavailable to the broadcast side of aBroadcastConnectedStream.classInformation available in an invocation of#onTimer(long, OnTimerContext, Collector).classAcontextavailable to the keyed stream side of aBroadcastConnectedStream(if any). -
Constructor Summary
Constructors -
Method Summary
Modifier and TypeMethodDescriptionvoidonTimer(long timestamp, KeyedBroadcastProcessFunction<KS, IN1, IN2, OUT>.OnTimerContext ctx, org.apache.flink.util.Collector<OUT> out) Called when a timer set usingTimerServicefires.abstract voidprocessBroadcastElement(IN2 value, KeyedBroadcastProcessFunction<KS, IN1, IN2, OUT>.Context ctx, org.apache.flink.util.Collector<OUT> out) This method is called for each element in thebroadcast stream.abstract voidprocessElement(IN1 value, KeyedBroadcastProcessFunction<KS, IN1, IN2, OUT>.ReadOnlyContext ctx, org.apache.flink.util.Collector<OUT> out) This method is called for each element in the (non-broadcast)keyed stream.Methods inherited from class org.apache.flink.api.common.functions.AbstractRichFunction
close, getIterationRuntimeContext, getRuntimeContext, open, setRuntimeContext
-
Constructor Details
-
KeyedBroadcastProcessFunction
public KeyedBroadcastProcessFunction()
-
-
Method Details
-
processElement
public abstract void processElement(IN1 value, KeyedBroadcastProcessFunction<KS, IN1, throws ExceptionIN2, OUT>.ReadOnlyContext ctx, org.apache.flink.util.Collector<OUT> out) This method is called for each element in the (non-broadcast)keyed stream.It can output zero or more elements using the
Collectorparameter, query the current processing/event time, and also query and update the local keyed state. In addition, it can get aTimerServicefor registering timers and querying the time. Finally, it has read-only access to the broadcast state. The context is only valid during the invocation of this method, do not store it.- Parameters:
value- The stream element.ctx- AKeyedBroadcastProcessFunction<KS,that allows querying the timestamp of the element, querying the current processing/event time and iterating the broadcast state with read-only access. The context is only valid during the invocation of this method, do not store it.IN1, IN2, OUT>.ReadOnlyContext out- The collector to emit resulting elements to- Throws:
Exception- The function may throw exceptions which cause the streaming program to fail and go into recovery.
-
processBroadcastElement
public abstract void processBroadcastElement(IN2 value, KeyedBroadcastProcessFunction<KS, IN1, throws ExceptionIN2, OUT>.Context ctx, org.apache.flink.util.Collector<OUT> out) This method is called for each element in thebroadcast stream.It can output zero or more elements using the
Collectorparameter, query the current processing/event time, and also query and update the internalbroadcast state. In addition, it can register afunctionto be applied to all keyed states on the local partition. These can be done through the providedKeyedBroadcastProcessFunction<KS,. The context is only valid during the invocation of this method, do not store it.IN1, IN2, OUT>.Context - Parameters:
value- The stream element.ctx- AKeyedBroadcastProcessFunction<KS,that allows querying the timestamp of the element, querying the current processing/event time and updating the broadcast state. In addition, it allows the registration of aIN1, IN2, OUT>.Context functionto be applied to all keyed state with a givenStateDescriptoron the local partition. The context is only valid during the invocation of this method, do not store it.out- The collector to emit resulting elements to- Throws:
Exception- The function may throw exceptions which cause the streaming program to fail and go into recovery.
-
onTimer
public void onTimer(long timestamp, KeyedBroadcastProcessFunction<KS, IN1, throws ExceptionIN2, OUT>.OnTimerContext ctx, org.apache.flink.util.Collector<OUT> out) Called when a timer set usingTimerServicefires.- Parameters:
timestamp- The timestamp of the firing timer.ctx- AnKeyedBroadcastProcessFunction<KS,that allows querying the timestamp of the firing timer, querying the current processing/event time, iterating the broadcast state with read-only access, querying theIN1, IN2, OUT>.OnTimerContext TimeDomainof the firing timer and getting aTimerServicefor registering timers and querying the time. The context is only valid during the invocation of this method, do not store it.out- The collector for returning result values.- Throws:
Exception- This method may throw exceptions. Throwing an exception will cause the operation to fail and may trigger recovery.
-