Class ProcessFunction<I,O>

java.lang.Object
org.apache.flink.api.common.functions.AbstractRichFunction
org.apache.flink.streaming.api.functions.ProcessFunction<I,O>
Type Parameters:
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

@PublicEvolving public abstract class ProcessFunction<I,O> extends org.apache.flink.api.common.functions.AbstractRichFunction
A function that processes elements of a stream.

For every element in the input stream #processElement(Object, Context, Collector) is invoked. This can produce zero or more elements as output. Implementations can also query the time and set timers through the provided ProcessFunction<I,O>.Context. For firing timers #onTimer(long, OnTimerContext, Collector) 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 ProcessFunction is applied on a KeyedStream.

NOTE: A ProcessFunction is always a RichFunction. Therefore, access to the RuntimeContext is always available and setup and teardown methods can be implemented. See RichFunction.open(OpenContext) and RichFunction.close().

See Also:
  • Nested Class Summary

    Nested Classes
    Modifier and Type
    Class
    Description
    class 
    Information available in an invocation of #processElement(Object, Context, Collector) or #onTimer(long, OnTimerContext, Collector).
    class 
    Information available in an invocation of #onTimer(long, OnTimerContext, Collector).
  • Constructor Summary

    Constructors
    Constructor
    Description
     
  • Method Summary

    Modifier and Type
    Method
    Description
    void
    onTimer(long timestamp, ProcessFunction<I,O>.OnTimerContext ctx, org.apache.flink.util.Collector<O> out)
    Called when a timer set using TimerService fires.
    abstract void
    processElement(I value, ProcessFunction<I,O>.Context ctx, org.apache.flink.util.Collector<O> out)
    Process one element from the input stream.

    Methods inherited from class org.apache.flink.api.common.functions.AbstractRichFunction

    close, getIterationRuntimeContext, getRuntimeContext, open, setRuntimeContext

    Methods inherited from class java.lang.Object

    clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
  • Constructor Details

    • ProcessFunction

      public ProcessFunction()
  • Method Details

    • processElement

      public abstract void processElement(I value, ProcessFunction<I,O>.Context ctx, org.apache.flink.util.Collector<O> out) throws Exception
      Process one element from the input stream.

      This function can output zero or more elements using the Collector parameter and also update internal state or set timers using the ProcessFunction<I,O>.Context parameter.

      Parameters:
      value - The input value.
      ctx - A ProcessFunction<I,O>.Context that allows querying the timestamp of the element 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.
      Throws:
      Exception - This method may throw exceptions. Throwing an exception will cause the operation to fail and may trigger recovery.
    • onTimer

      public void onTimer(long timestamp, ProcessFunction<I,O>.OnTimerContext ctx, org.apache.flink.util.Collector<O> out) throws Exception
      Called when a timer set using TimerService fires.
      Parameters:
      timestamp - The timestamp of the firing timer.
      ctx - An ProcessFunction<I,O>.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.
      Throws:
      Exception - This method may throw exceptions. Throwing an exception will cause the operation to fail and may trigger recovery.