Class CoProcessFunction<IN1,IN2,OUT>
- Type Parameters:
IN1- Type of the first input.IN2- Type of the second input.OUT- Output type.
- All Implemented Interfaces:
Serializable,org.apache.flink.api.common.functions.Function,org.apache.flink.api.common.functions.RichFunction
The function will be called for every element in the input streams and can produce zero or
more output elements. Contrary to the CoFlatMapFunction, this function can also query the
time (both event and processing) and set timers, through the provided CoProcessFunction<IN1,. When
reacting to the firing of set timers the function can emit yet more elements.
An example use-case for connected streams would be the application of a set of rules that
change over time (stream A) to the elements contained in another stream (stream
B). The rules contained in stream A can be stored in the state and wait for new elements
to arrive on stream B. Upon reception of a new element on stream B, the function
can now apply the previously stored rules to the element and directly emit a result, and/or
register a timer that will trigger an action in the future.
- See Also:
-
Nested Class Summary
Nested ClassesModifier and TypeClassDescriptionclassInformation available in an invocation of#processElement1(Object, Context, Collector)/#processElement2(Object, Context, Collector)or#onTimer(long, OnTimerContext, Collector).classInformation available in an invocation of#onTimer(long, OnTimerContext, Collector). -
Constructor Summary
Constructors -
Method Summary
Modifier and TypeMethodDescriptionvoidonTimer(long timestamp, CoProcessFunction<IN1, IN2, OUT>.OnTimerContext ctx, org.apache.flink.util.Collector<OUT> out) Called when a timer set usingTimerServicefires.abstract voidprocessElement1(IN1 value, CoProcessFunction<IN1, IN2, OUT>.Context ctx, org.apache.flink.util.Collector<OUT> out) This method is called for each element in the first of the connected streams.abstract voidprocessElement2(IN2 value, CoProcessFunction<IN1, IN2, OUT>.Context ctx, org.apache.flink.util.Collector<OUT> out) This method is called for each element in the second of the connected streams.Methods inherited from class org.apache.flink.api.common.functions.AbstractRichFunction
close, getIterationRuntimeContext, getRuntimeContext, open, setRuntimeContext
-
Constructor Details
-
CoProcessFunction
public CoProcessFunction()
-
-
Method Details
-
processElement1
public abstract void processElement1(IN1 value, CoProcessFunction<IN1, IN2, throws ExceptionOUT>.Context ctx, org.apache.flink.util.Collector<OUT> out) This method is called for each element in the first of the connected streams.This function can output zero or more elements using the
Collectorparameter and also update internal state or set timers using theCoProcessFunction<IN1,parameter.IN2, OUT>.Context - Parameters:
value- The stream elementctx- ACoProcessFunction<IN1,that allows querying the timestamp of the element, querying theIN2, OUT>.Context 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 to emit resulting elements to- Throws:
Exception- The function may throw exceptions which cause the streaming program to fail and go into recovery.
-
processElement2
public abstract void processElement2(IN2 value, CoProcessFunction<IN1, IN2, throws ExceptionOUT>.Context ctx, org.apache.flink.util.Collector<OUT> out) This method is called for each element in the second of the connected streams.This function can output zero or more elements using the
Collectorparameter and also update internal state or set timers using theCoProcessFunction<IN1,parameter.IN2, OUT>.Context - Parameters:
value- The stream elementctx- ACoProcessFunction<IN1,that allows querying the timestamp of the element, querying theIN2, OUT>.Context 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 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, CoProcessFunction<IN1, IN2, throws ExceptionOUT>.OnTimerContext ctx, org.apache.flink.util.Collector<OUT> out) Called when a timer set usingTimerServicefires.- Parameters:
timestamp- The timestamp of the firing timer.ctx- AnCoProcessFunction<IN1,that allows querying the timestamp of the firing timer, querying theIN2, 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.
-