Class DeclaringAsyncKeyedProcessFunction<K,I,O>
- Type Parameters:
K- Type of the key.I- Type of the input elements.O- Type of the output elements.
- All Implemented Interfaces:
Serializable,org.apache.flink.api.common.functions.Function,org.apache.flink.api.common.functions.RichFunction
AsyncKeyedProcessOperator using the async state access and declares the processing logic as
FLIP-455 describes. This function extends from the non-declaring KeyedProcessFunction for
the convenience of unifying logic in the operator AsyncKeyedProcessOperator. Maybe later
we could extract the common part KeyedProcessFunction and this class to a base class, and
deviate from that.
For every element in the input stream the process declareProcess(org.apache.flink.runtime.asyncprocessing.declare.DeclarationContext, org.apache.flink.streaming.api.functions.KeyedProcessFunction.Context, org.apache.flink.util.Collector<O>) declared is invoked.
This can produce zero or more elements as output. Implementations can also query the time and set
timers through the provided KeyedProcessFunction<K,. For firing timers the process declareOnTimer(org.apache.flink.runtime.asyncprocessing.declare.DeclarationContext, org.apache.flink.streaming.api.functions.KeyedProcessFunction.OnTimerContext, org.apache.flink.util.Collector<O>) declared will be invoked. This can again produce zero or more elements as output
and register further timers.
NOTE: Access to keyed state and timers (which are also scoped to a key) is only
available if the DeclaringAsyncKeyedProcessFunction is applied on a KeyedStream.
- See Also:
-
Nested Class Summary
Nested classes/interfaces inherited from class org.apache.flink.streaming.api.functions.KeyedProcessFunction
KeyedProcessFunction.Context, KeyedProcessFunction.OnTimerContext -
Constructor Summary
Constructors -
Method Summary
Modifier and TypeMethodDescriptiondeclareOnTimer(DeclarationContext context, KeyedProcessFunction<K, I, O>.OnTimerContext ctx, org.apache.flink.util.Collector<O> out) Declare a procedure which is called when a timer set usingTimerServicefires.declareProcess(DeclarationContext context, KeyedProcessFunction<K, I, O>.Context ctx, org.apache.flink.util.Collector<O> out) Declare a process for one element from the input stream.voiddeclareVariables(DeclarationContext context) Declaring variables beforedeclareProcess(org.apache.flink.runtime.asyncprocessing.declare.DeclarationContext, org.apache.flink.streaming.api.functions.KeyedProcessFunction.Context, org.apache.flink.util.Collector<O>)anddeclareOnTimer(org.apache.flink.runtime.asyncprocessing.declare.DeclarationContext, org.apache.flink.streaming.api.functions.KeyedProcessFunction.OnTimerContext, org.apache.flink.util.Collector<O>).final voidonTimer(long timestamp, KeyedProcessFunction<K, I, O>.OnTimerContext ctx, org.apache.flink.util.Collector<O> out) Override and finalize this method.final voidprocessElement(I value, KeyedProcessFunction<K, I, O>.Context ctx, org.apache.flink.util.Collector<O> out) Override and finalize this method.Methods inherited from class org.apache.flink.api.common.functions.AbstractRichFunction
close, getIterationRuntimeContext, getRuntimeContext, open, setRuntimeContext
-
Constructor Details
-
DeclaringAsyncKeyedProcessFunction
public DeclaringAsyncKeyedProcessFunction()
-
-
Method Details
-
processElement
public final void processElement(I value, KeyedProcessFunction<K, I, throws ExceptionO>.Context ctx, org.apache.flink.util.Collector<O> out) Override and finalize this method. Please usedeclareProcess(org.apache.flink.runtime.asyncprocessing.declare.DeclarationContext, org.apache.flink.streaming.api.functions.KeyedProcessFunction.Context, org.apache.flink.util.Collector<O>)instead.- Specified by:
processElementin classKeyedProcessFunction<K,I, O> - Parameters:
value- The input value.ctx- AKeyedProcessFunction<K,that allows querying the timestamp of the element and getting aI, O>.Context TimerServicefor 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.
-
onTimer
public final void onTimer(long timestamp, KeyedProcessFunction<K, I, throws ExceptionO>.OnTimerContext ctx, org.apache.flink.util.Collector<O> out) Override and finalize this method. Please usedeclareOnTimer(org.apache.flink.runtime.asyncprocessing.declare.DeclarationContext, org.apache.flink.streaming.api.functions.KeyedProcessFunction.OnTimerContext, org.apache.flink.util.Collector<O>)instead.- Overrides:
onTimerin classKeyedProcessFunction<K,I, O> - Parameters:
timestamp- The timestamp of the firing timer.ctx- AnKeyedProcessFunction<K,that allows querying the timestamp, theI, O>.OnTimerContext TimeDomain, and the key of 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.
-
declareVariables
Declaring variables beforedeclareProcess(org.apache.flink.runtime.asyncprocessing.declare.DeclarationContext, org.apache.flink.streaming.api.functions.KeyedProcessFunction.Context, org.apache.flink.util.Collector<O>)anddeclareOnTimer(org.apache.flink.runtime.asyncprocessing.declare.DeclarationContext, org.apache.flink.streaming.api.functions.KeyedProcessFunction.OnTimerContext, org.apache.flink.util.Collector<O>). -
declareProcess
public abstract org.apache.flink.util.function.ThrowingConsumer<I,Exception> declareProcess(DeclarationContext context, KeyedProcessFunction<K, I, throws DeclarationExceptionO>.Context ctx, org.apache.flink.util.Collector<O> out) Declare a process for one element from the input stream.This function can output zero or more elements using the
Collectorparameter and also update internal state or set timers using theKeyedProcessFunction<K,parameter.I, O>.Context - Parameters:
context- the context that provides useful methods to define named callbacks.ctx- AKeyedProcessFunction<K,that allows querying the timestamp of the element and getting aI, O>.Context TimerServicefor 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.- Returns:
- the whole processing logic just like
processElement. - Throws:
DeclarationException
-
declareOnTimer
public org.apache.flink.util.function.ThrowingConsumer<Long,Exception> declareOnTimer(DeclarationContext context, KeyedProcessFunction<K, I, throws DeclarationExceptionO>.OnTimerContext ctx, org.apache.flink.util.Collector<O> out) Declare a procedure which is called when a timer set usingTimerServicefires.- Parameters:
context- the context that provides useful methods to define named callbacks.ctx- AnKeyedProcessFunction<K,that allows querying the timestamp, theI, O>.OnTimerContext TimeDomain, and the key of 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 processor for processing timestamps.- Throws:
DeclarationException
-