The ProcessFunction is a low-level stream processing operation, giving access to the basic building blocks of all (acyclic) streaming applications:
The ProcessFunction can be thought of as a FlatMapFunction with access to keyed state and timers. It handles events be being invoked for each event received in the input stream(s).
For fault tolerant state, the ProcessFunction gives access to Flink's keyed state, accessible via the RuntimeContext, similar to the way other stateful functions can access keyed state. Like all functions with keyed state, the ProcessFunction needs to be applied onto a KeyedStream:
stream.keyBy("id").process(new MyProcessFunction())
The timers allow applications to react to changes in processing time and in event time. Every call to the function processElement(...) gets a Context object with gives access to the element‘s event time timestamp, and to the TimerService. The TimerService can be used to register callbacks for future event-/processing- time instants. When a timer’s particular time is reached, the onTimer(...) method is called. During that call, all states are again scoped to the key with which the timer was created, allowing timers to perform keyed state manipulation as well.
To realize low-level operations on two inputs, applications can use CoProcessFunction. It relates to ProcessFunction in the same way that CoFlatMapFunction relates to FlatMapFunction: the function is bound to two different inputs and gets individual calls to processElement1(...) and processElement2(...) for records from the two different inputs.
Implementing a low level join typically follows this pattern:
For example, you might be joining customer data to financial trades, while keeping state for the customer data. If you care about having complete and deterministic joins in the face of out-of-order events, you can use a timer to evaluate and emit the join for a trade when the watermark for the customer data stream has passed the time of that trade.
The following example maintains counts per key, and emits a key/count pair whenever a minute passes (in event time) without an update for that key:
ValueState, which is implicitly scoped by key.ProcessFunction increments the counter and sets the last-modification timestampNote: This simple example could have been implemented with session windows. We use ProcessFunction here to illustrate the basic pattern it provides.
{% highlight java %} import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.RichProcessFunction; import org.apache.flink.streaming.api.functions.ProcessFunction.Context; import org.apache.flink.streaming.api.functions.ProcessFunction.OnTimerContext; import org.apache.flink.util.Collector;
// the source data stream DataStream<Tuple2<String, String>> stream = ...;
// apply the process function onto a keyed stream DataStream<Tuple2<String, Long>> result = stream .keyBy(0) .process(new CountWithTimeoutFunction());
/**
The data type stored in the state */ public class CountWithTimestamp {
public String key; public long count; public long lastModified; }
/**
The implementation of the ProcessFunction that maintains the count and timeouts */ public class CountWithTimeoutFunction extends RichProcessFunction<Tuple2<String, String>, Tuple2<String, Long>> {
/** The state that is maintained by this process function */ private ValueState state;
@Override public void open(Configuration parameters) throws Exception { state = getRuntimeContext().getState(new ValueStateDescriptor<>(“myState”, CountWithTimestamp.class)); }
@Override public void processElement(Tuple2<String, Long> value, Context ctx, Collector<Tuple2<String, Long>> out) throws Exception {
// retrieve the current count
CountWithTimestamp current = state.value();
if (current == null) {
current = new CountWithTimestamp();
current.key = value.f0;
}
// update the state's count
current.count++;
// set the state's timestamp to the record's assigned event time timestamp
current.lastModified = ctx.timestamp();
// write the state back
state.update(current);
// schedule the next timer 60 seconds from the current event time
ctx.timerService().registerEventTimeTimer(current.timestamp + 60000);
}
@Override public void onTimer(long timestamp, OnTimerContext ctx, Collector<Tuple2<String, Long>> out) throws Exception {
// get the state for the key that scheduled the timer
CountWithTimestamp result = state.value();
// check if this is an outdated timer or the latest timer
if (timestamp == result.lastModified) {
// emit the state
out.collect(new Tuple2<String, Long>(result.key, result.count));
}
} } {% endhighlight %}
// the source data stream DataStream<Tuple2<String, String>> stream = ...;
// apply the process function onto a keyed stream DataStream<Tuple2<String, Long>> result = stream .keyBy(0) .process(new CountWithTimeoutFunction());
/**
/**
/** The state that is maintained by this process function */ lazy val state: ValueState[CountWithTimestamp] = getRuntimeContext() .getState(new ValueStateDescriptor<>(“myState”, clasOf[CountWithTimestamp]))
override def processElement(value: (String, Long), ctx: Context, out: Collector[(String, Long)]): Unit = { // initialize or retrieve/update the state
val current: CountWithTimestamp = state.value match {
case null =>
CountWithTimestamp(key, 1, ctx.timestamp)
case CountWithTimestamp(key, count, time) =>
CountWithTimestamp(key, count + 1, ctx.timestamp)
}
// write the state back
state.update(current)
// schedule the next timer 60 seconds from the current event time
ctx.timerService.registerEventTimeTimer(current.timestamp + 60000)
}
override def onTimer(timestamp: Long, ctx: OnTimerContext, out: Collector[(String, Long)]): Unit = { state.value match { case CountWithTimestamp(key, count, lastModified) if (lastModified == timestamp) => out.collect((key, count)) case _ => } } } {% endhighlight %}
{% top %}