K - Type of the key.IN1 - Type of the first input.IN2 - Type of the second input.OUT - Output type.@PublicEvolving
public abstract class KeyedCoProcessFunction<K,IN1,IN2,OUT>
extends org.apache.flink.api.common.functions.AbstractRichFunction
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 KeyedCoProcessFunction.Context.
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.
| Modifier and Type | Class and Description |
|---|---|
class |
KeyedCoProcessFunction.Context
Information available in an invocation of
processElement1(Object, Context, Collector)/
processElement2(Object, Context, Collector)
or onTimer(long, OnTimerContext, Collector). |
class |
KeyedCoProcessFunction.OnTimerContext
Information available in an invocation of
onTimer(long, OnTimerContext, Collector). |
| Constructor and Description |
|---|
KeyedCoProcessFunction() |
| Modifier and Type | Method and Description |
|---|---|
void |
onTimer(long timestamp,
KeyedCoProcessFunction.OnTimerContext ctx,
org.apache.flink.util.Collector<OUT> out)
Called when a timer set using
TimerService fires. |
abstract void |
processElement1(IN1 value,
KeyedCoProcessFunction.Context ctx,
org.apache.flink.util.Collector<OUT> out)
This method is called for each element in the first of the connected streams.
|
abstract void |
processElement2(IN2 value,
KeyedCoProcessFunction.Context ctx,
org.apache.flink.util.Collector<OUT> out)
This method is called for each element in the second of the connected streams.
|
public abstract void processElement1(IN1 value, KeyedCoProcessFunction.Context ctx, org.apache.flink.util.Collector<OUT> out) throws Exception
This function can output zero or more elements using the Collector parameter
and also update internal state or set timers using the KeyedCoProcessFunction.Context parameter.
value - The stream elementctx - A KeyedCoProcessFunction.Context that allows querying the timestamp of the element,
querying the TimeDomain of the firing timer and getting a
TimerService for 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 toException - The function may throw exceptions which cause the streaming program
to fail and go into recovery.public abstract void processElement2(IN2 value, KeyedCoProcessFunction.Context ctx, org.apache.flink.util.Collector<OUT> out) throws Exception
This function can output zero or more elements using the Collector parameter
and also update internal state or set timers using the KeyedCoProcessFunction.Context parameter.
value - The stream elementctx - A KeyedCoProcessFunction.Context that allows querying the timestamp of the element,
querying the TimeDomain of the firing timer and getting a
TimerService for 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 toException - The function may throw exceptions which cause the streaming program
to fail and go into recovery.public void onTimer(long timestamp,
KeyedCoProcessFunction.OnTimerContext ctx,
org.apache.flink.util.Collector<OUT> out)
throws Exception
TimerService fires.timestamp - The timestamp of the firing timer.ctx - An KeyedCoProcessFunction.OnTimerContext that allows querying the timestamp of the firing timer,
querying the TimeDomain of the firing timer and getting a
TimerService for 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.Exception - This method may throw exceptions. Throwing an exception will cause the operation
to fail and may trigger recovery.Copyright © 2014–2020 The Apache Software Foundation. All rights reserved.