FlinkCEP - Complex event processing for Flink

FlinkCEP is the Complex Event Processing (CEP) library implemented on top of Flink. It allows you to detect event patterns in an endless stream of events, giving you the opportunity to get hold of what’s important in your data.

This page describes the API calls available in Flink CEP. We start by presenting the Pattern API, which allows you to specify the patterns that you want to detect in your stream, before presenting how you can detect and act upon matching event sequences. We then present the assumptions the CEP library makes when dealing with lateness in event time and how you can migrate your job from an older Flink version to Flink-1.3.

Getting Started

If you want to jump right in, set up a Flink program and add the FlinkCEP dependency to the pom.xml of your project.

  1. <dependency>
  2. <groupId>org.apache.flink</groupId>
  3. <artifactId>flink-cep_2.11</artifactId>
  4. <version>1.11.0</version>
  5. </dependency>
  1. <dependency>
  2. <groupId>org.apache.flink</groupId>
  3. <artifactId>flink-cep-scala_2.11</artifactId>
  4. <version>1.11.0</version>
  5. </dependency>

Info FlinkCEP is not part of the binary distribution. See how to link with it for cluster execution here.

Now you can start writing your first CEP program using the Pattern API.

Attention The events in the DataStream to which you want to apply pattern matching must implement proper equals() and hashCode() methods because FlinkCEP uses them for comparing and matching events.

  1. DataStream<Event> input = ...
  2. Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(
  3. new SimpleCondition<Event>() {
  4. @Override
  5. public boolean filter(Event event) {
  6. return event.getId() == 42;
  7. }
  8. }
  9. ).next("middle").subtype(SubEvent.class).where(
  10. new SimpleCondition<SubEvent>() {
  11. @Override
  12. public boolean filter(SubEvent subEvent) {
  13. return subEvent.getVolume() >= 10.0;
  14. }
  15. }
  16. ).followedBy("end").where(
  17. new SimpleCondition<Event>() {
  18. @Override
  19. public boolean filter(Event event) {
  20. return event.getName().equals("end");
  21. }
  22. }
  23. );
  24. PatternStream<Event> patternStream = CEP.pattern(input, pattern);
  25. DataStream<Alert> result = patternStream.process(
  26. new PatternProcessFunction<Event, Alert>() {
  27. @Override
  28. public void processMatch(
  29. Map<String, List<Event>> pattern,
  30. Context ctx,
  31. Collector<Alert> out) throws Exception {
  32. out.collect(createAlertFrom(pattern));
  33. }
  34. });
  1. val input: DataStream[Event] = ...
  2. val pattern = Pattern.begin[Event]("start").where(_.getId == 42)
  3. .next("middle").subtype(classOf[SubEvent]).where(_.getVolume >= 10.0)
  4. .followedBy("end").where(_.getName == "end")
  5. val patternStream = CEP.pattern(input, pattern)
  6. val result: DataStream[Alert] = patternStream.process(
  7. new PatternProcessFunction[Event, Alert]() {
  8. override def processMatch(
  9. `match`: util.Map[String, util.List[Event]],
  10. ctx: PatternProcessFunction.Context,
  11. out: Collector[Alert]): Unit = {
  12. out.collect(createAlertFrom(pattern))
  13. }
  14. })

The Pattern API

The pattern API allows you to define complex pattern sequences that you want to extract from your input stream.

Each complex pattern sequence consists of multiple simple patterns, i.e. patterns looking for individual events with the same properties. From now on, we will call these simple patterns patterns, and the final complex pattern sequence we are searching for in the stream, the pattern sequence. You can see a pattern sequence as a graph of such patterns, where transitions from one pattern to the next occur based on user-specified conditions, e.g. event.getName().equals("end"). A match is a sequence of input events which visits all patterns of the complex pattern graph, through a sequence of valid pattern transitions.

Attention Each pattern must have a unique name, which you use later to identify the matched events.

Attention Pattern names CANNOT contain the character ":".

In the rest of this section we will first describe how to define Individual Patterns, and then how you can combine individual patterns into Complex Patterns.

Individual Patterns

A Pattern can be either a singleton or a looping pattern. Singleton patterns accept a single event, while looping patterns can accept more than one. In pattern matching symbols, the pattern "a b+ c? d" (or "a", followed by one or more "b"’s, optionally followed by a "c", followed by a "d"), a, c?, and d are singleton patterns, while b+ is a looping one. By default, a pattern is a singleton pattern and you can transform it to a looping one by using Quantifiers. Each pattern can have one or more Conditions based on which it accepts events.

Quantifiers

In FlinkCEP, you can specify looping patterns using these methods: pattern.oneOrMore(), for patterns that expect one or more occurrences of a given event (e.g. the b+ mentioned before); and pattern.times(#ofTimes), for patterns that expect a specific number of occurrences of a given type of event, e.g. 4 a’s; and pattern.times(#fromTimes, #toTimes), for patterns that expect a specific minimum number of occurrences and a maximum number of occurrences of a given type of event, e.g. 2-4 as.

You can make looping patterns greedy using the pattern.greedy() method, but you cannot yet make group patterns greedy. You can make all patterns, looping or not, optional using the pattern.optional() method.

For a pattern named start, the following are valid quantifiers:

  1. // expecting 4 occurrences
  2. start.times(4);
  3. // expecting 0 or 4 occurrences
  4. start.times(4).optional();
  5. // expecting 2, 3 or 4 occurrences
  6. start.times(2, 4);
  7. // expecting 2, 3 or 4 occurrences and repeating as many as possible
  8. start.times(2, 4).greedy();
  9. // expecting 0, 2, 3 or 4 occurrences
  10. start.times(2, 4).optional();
  11. // expecting 0, 2, 3 or 4 occurrences and repeating as many as possible
  12. start.times(2, 4).optional().greedy();
  13. // expecting 1 or more occurrences
  14. start.oneOrMore();
  15. // expecting 1 or more occurrences and repeating as many as possible
  16. start.oneOrMore().greedy();
  17. // expecting 0 or more occurrences
  18. start.oneOrMore().optional();
  19. // expecting 0 or more occurrences and repeating as many as possible
  20. start.oneOrMore().optional().greedy();
  21. // expecting 2 or more occurrences
  22. start.timesOrMore(2);
  23. // expecting 2 or more occurrences and repeating as many as possible
  24. start.timesOrMore(2).greedy();
  25. // expecting 0, 2 or more occurrences
  26. start.timesOrMore(2).optional()
  27. // expecting 0, 2 or more occurrences and repeating as many as possible
  28. start.timesOrMore(2).optional().greedy();
  1. // expecting 4 occurrences
  2. start.times(4)
  3. // expecting 0 or 4 occurrences
  4. start.times(4).optional()
  5. // expecting 2, 3 or 4 occurrences
  6. start.times(2, 4)
  7. // expecting 2, 3 or 4 occurrences and repeating as many as possible
  8. start.times(2, 4).greedy()
  9. // expecting 0, 2, 3 or 4 occurrences
  10. start.times(2, 4).optional()
  11. // expecting 0, 2, 3 or 4 occurrences and repeating as many as possible
  12. start.times(2, 4).optional().greedy()
  13. // expecting 1 or more occurrences
  14. start.oneOrMore()
  15. // expecting 1 or more occurrences and repeating as many as possible
  16. start.oneOrMore().greedy()
  17. // expecting 0 or more occurrences
  18. start.oneOrMore().optional()
  19. // expecting 0 or more occurrences and repeating as many as possible
  20. start.oneOrMore().optional().greedy()
  21. // expecting 2 or more occurrences
  22. start.timesOrMore(2)
  23. // expecting 2 or more occurrences and repeating as many as possible
  24. start.timesOrMore(2).greedy()
  25. // expecting 0, 2 or more occurrences
  26. start.timesOrMore(2).optional()
  27. // expecting 0, 2 or more occurrences and repeating as many as possible
  28. start.timesOrMore(2).optional().greedy()

Conditions

For every pattern you can specify a condition that an incoming event has to meet in order to be “accepted” into the pattern e.g. its value should be larger than 5, or larger than the average value of the previously accepted events. You can specify conditions on the event properties via the pattern.where(), pattern.or() or pattern.until() methods. These can be either IterativeConditions or SimpleConditions.

Iterative Conditions: This is the most general type of condition. This is how you can specify a condition that accepts subsequent events based on properties of the previously accepted events or a statistic over a subset of them.

Below is the code for an iterative condition that accepts the next event for a pattern named “middle” if its name starts with “foo”, and if the sum of the prices of the previously accepted events for that pattern plus the price of the current event do not exceed the value of 5.0. Iterative conditions can be powerful, especially in combination with looping patterns, e.g. oneOrMore().

  1. middle.oneOrMore()
  2. .subtype(SubEvent.class)
  3. .where(new IterativeCondition<SubEvent>() {
  4. @Override
  5. public boolean filter(SubEvent value, Context<SubEvent> ctx) throws Exception {
  6. if (!value.getName().startsWith("foo")) {
  7. return false;
  8. }
  9. double sum = value.getPrice();
  10. for (Event event : ctx.getEventsForPattern("middle")) {
  11. sum += event.getPrice();
  12. }
  13. return Double.compare(sum, 5.0) < 0;
  14. }
  15. });
  1. middle.oneOrMore()
  2. .subtype(classOf[SubEvent])
  3. .where(
  4. (value, ctx) => {
  5. lazy val sum = ctx.getEventsForPattern("middle").map(_.getPrice).sum
  6. value.getName.startsWith("foo") && sum + value.getPrice < 5.0
  7. }
  8. )

Attention The call to ctx.getEventsForPattern(...) finds all the previously accepted events for a given potential match. The cost of this operation can vary, so when implementing your condition, try to minimize its use.

Described context gives one access to event time characteristics as well. For more info see Time context.

Simple Conditions: This type of condition extends the aforementioned IterativeCondition class and decides whether to accept an event or not, based only on properties of the event itself.

  1. start.where(new SimpleCondition<Event>() {
  2. @Override
  3. public boolean filter(Event value) {
  4. return value.getName().startsWith("foo");
  5. }
  6. });
  1. start.where(event => event.getName.startsWith("foo"))

Finally, you can also restrict the type of the accepted event to a subtype of the initial event type (here Event) via the pattern.subtype(subClass) method.

  1. start.subtype(SubEvent.class).where(new SimpleCondition<SubEvent>() {
  2. @Override
  3. public boolean filter(SubEvent value) {
  4. return ... // some condition
  5. }
  6. });
  1. start.subtype(classOf[SubEvent]).where(subEvent => ... /* some condition */)

Combining Conditions: As shown above, you can combine the subtype condition with additional conditions. This holds for every condition. You can arbitrarily combine conditions by sequentially calling where(). The final result will be the logical AND of the results of the individual conditions. To combine conditions using OR, you can use the or() method, as shown below.

  1. pattern.where(new SimpleCondition<Event>() {
  2. @Override
  3. public boolean filter(Event value) {
  4. return ... // some condition
  5. }
  6. }).or(new SimpleCondition<Event>() {
  7. @Override
  8. public boolean filter(Event value) {
  9. return ... // or condition
  10. }
  11. });
  1. pattern.where(event => ... /* some condition */).or(event => ... /* or condition */)

Stop condition: In case of looping patterns (oneOrMore() and oneOrMore().optional()) you can also specify a stop condition, e.g. accept events with value larger than 5 until the sum of values is smaller than 50.

To better understand it, have a look at the following example. Given

  • pattern like "(a+ until b)" (one or more "a" until "b")

  • a sequence of incoming events "a1" "c" "a2" "b" "a3"

  • the library will output results: {a1 a2} {a1} {a2} {a3}.

As you can see {a1 a2 a3} or {a2 a3} are not returned due to the stop condition.

Pattern OperationDescription
where(condition)

Defines a condition for the current pattern. To match the pattern, an event must satisfy the condition. Multiple consecutive where() clauses lead to their conditions being ANDed:

  1. pattern.where(new IterativeCondition<Event>() {
  2. @Override
  3. public boolean filter(Event value, Context ctx) throws Exception {
  4. return // some condition
  5. }
  6. });
or(condition)

Adds a new condition which is ORed with an existing one. An event can match the pattern only if it passes at least one of the conditions:

  1. pattern.where(new IterativeCondition<Event>() {
  2. @Override
  3. public boolean filter(Event value, Context ctx) throws Exception {
  4. return // some condition
  5. }
  6. }).or(new IterativeCondition<Event>() {
  7. @Override
  8. public boolean filter(Event value, Context ctx) throws Exception {
  9. return // alternative condition
  10. }
  11. });
until(condition)

Specifies a stop condition for a looping pattern. Meaning if event matching the given condition occurs, no more events will be accepted into the pattern.

Applicable only in conjunction with oneOrMore()

NOTE: It allows for cleaning state for corresponding pattern on event-based condition.

  1. pattern.oneOrMore().until(new IterativeCondition<Event>() {
  2. @Override
  3. public boolean filter(Event value, Context ctx) throws Exception {
  4. return // alternative condition
  5. }
  6. });
subtype(subClass)

Defines a subtype condition for the current pattern. An event can only match the pattern if it is of this subtype:

  1. pattern.subtype(SubEvent.class);
oneOrMore()

Specifies that this pattern expects at least one occurrence of a matching event.

By default a relaxed internal contiguity (between subsequent events) is used. For more info on internal contiguity see consecutive.

NOTE: It is advised to use either until() or within() to enable state clearing

  1. pattern.oneOrMore();
timesOrMore(#times)

Specifies that this pattern expects at least #times occurrences of a matching event.

By default a relaxed internal contiguity (between subsequent events) is used. For more info on internal contiguity see consecutive.

  1. pattern.timesOrMore(2);
times(#ofTimes)

Specifies that this pattern expects an exact number of occurrences of a matching event.

By default a relaxed internal contiguity (between subsequent events) is used. For more info on internal contiguity see consecutive.

  1. pattern.times(2);
times(#fromTimes, #toTimes)

Specifies that this pattern expects occurrences between #fromTimes and #toTimes of a matching event.

By default a relaxed internal contiguity (between subsequent events) is used. For more info on internal contiguity see consecutive.

  1. pattern.times(2, 4);
optional()

Specifies that this pattern is optional, i.e. it may not occur at all. This is applicable to all aforementioned quantifiers.

  1. pattern.oneOrMore().optional();
greedy()

Specifies that this pattern is greedy, i.e. it will repeat as many as possible. This is only applicable to quantifiers and it does not support group pattern currently.

  1. pattern.oneOrMore().greedy();
Pattern OperationDescription
where(condition)

Defines a condition for the current pattern. To match the pattern, an event must satisfy the condition. Multiple consecutive where() clauses lead to their conditions being ANDed:

  1. pattern.where(event => / some condition /)
or(condition)

Adds a new condition which is ORed with an existing one. An event can match the pattern only if it passes at least one of the conditions:

  1. pattern.where(event => / some condition /)
  2. .or(event => / alternative condition /)
until(condition)

Specifies a stop condition for looping pattern. Meaning if event matching the given condition occurs, no more events will be accepted into the pattern.

Applicable only in conjunction with oneOrMore()

NOTE: It allows for cleaning state for corresponding pattern on event-based condition.

  1. pattern.oneOrMore().until(event => / some condition /)
subtype(subClass)

Defines a subtype condition for the current pattern. An event can only match the pattern if it is of this subtype:

  1. pattern.subtype(classOf[SubEvent])
oneOrMore()

Specifies that this pattern expects at least one occurrence of a matching event.

By default a relaxed internal contiguity (between subsequent events) is used. For more info on internal contiguity see consecutive.

NOTE: It is advised to use either until() or within() to enable state clearing

  1. pattern.oneOrMore()
timesOrMore(#times)

Specifies that this pattern expects at least #times occurrences of a matching event.

By default a relaxed internal contiguity (between subsequent events) is used. For more info on internal contiguity see consecutive.

  1. pattern.timesOrMore(2)
times(#ofTimes)

Specifies that this pattern expects an exact number of occurrences of a matching event.

By default a relaxed internal contiguity (between subsequent events) is used. For more info on internal contiguity see consecutive.

  1. pattern.times(2)
times(#fromTimes, #toTimes)

Specifies that this pattern expects occurrences between #fromTimes and #toTimes of a matching event.

By default a relaxed internal contiguity (between subsequent events) is used. For more info on internal contiguity see consecutive.

  1. pattern.times(2, 4)
optional()

Specifies that this pattern is optional, i.e. it may not occur at all. This is applicable to all aforementioned quantifiers.

  1. pattern.oneOrMore().optional()
greedy()

Specifies that this pattern is greedy, i.e. it will repeat as many as possible. This is only applicable to quantifiers and it does not support group pattern currently.

  1. pattern.oneOrMore().greedy()

Combining Patterns

Now that you’ve seen what an individual pattern can look like, it is time to see how to combine them into a full pattern sequence.

A pattern sequence has to start with an initial pattern, as shown below:

  1. Pattern<Event, ?> start = Pattern.<Event>begin("start");
  1. val start : Pattern[Event, _] = Pattern.begin("start")

Next, you can append more patterns to your pattern sequence by specifying the desired contiguity conditions between them. FlinkCEP supports the following forms of contiguity between events:

  1. Strict Contiguity: Expects all matching events to appear strictly one after the other, without any non-matching events in-between.

  2. Relaxed Contiguity: Ignores non-matching events appearing in-between the matching ones.

  3. Non-Deterministic Relaxed Contiguity: Further relaxes contiguity, allowing additional matches that ignore some matching events.

To apply them between consecutive patterns, you can use:

  1. next(), for strict,
  2. followedBy(), for relaxed, and
  3. followedByAny(), for non-deterministic relaxed contiguity.

or

  1. notNext(), if you do not want an event type to directly follow another
  2. notFollowedBy(), if you do not want an event type to be anywhere between two other event types.

Attention A pattern sequence cannot end in notFollowedBy().

Attention A NOT pattern cannot be preceded by an optional one.

  1. // strict contiguity
  2. Pattern<Event, ?> strict = start.next("middle").where(...);
  3. // relaxed contiguity
  4. Pattern<Event, ?> relaxed = start.followedBy("middle").where(...);
  5. // non-deterministic relaxed contiguity
  6. Pattern<Event, ?> nonDetermin = start.followedByAny("middle").where(...);
  7. // NOT pattern with strict contiguity
  8. Pattern<Event, ?> strictNot = start.notNext("not").where(...);
  9. // NOT pattern with relaxed contiguity
  10. Pattern<Event, ?> relaxedNot = start.notFollowedBy("not").where(...);
  1. // strict contiguity
  2. val strict: Pattern[Event, _] = start.next("middle").where(...)
  3. // relaxed contiguity
  4. val relaxed: Pattern[Event, _] = start.followedBy("middle").where(...)
  5. // non-deterministic relaxed contiguity
  6. val nonDetermin: Pattern[Event, _] = start.followedByAny("middle").where(...)
  7. // NOT pattern with strict contiguity
  8. val strictNot: Pattern[Event, _] = start.notNext("not").where(...)
  9. // NOT pattern with relaxed contiguity
  10. val relaxedNot: Pattern[Event, _] = start.notFollowedBy("not").where(...)

Relaxed contiguity means that only the first succeeding matching event will be matched, while with non-deterministic relaxed contiguity, multiple matches will be emitted for the same beginning. As an example, a pattern "a b", given the event sequence "a", "c", "b1", "b2", will give the following results:

  1. Strict Contiguity between "a" and "b": {} (no match), the "c" after "a" causes "a" to be discarded.

  2. Relaxed Contiguity between "a" and "b": {a b1}, as relaxed continuity is viewed as “skip non-matching events till the next matching one”.

  3. Non-Deterministic Relaxed Contiguity between "a" and "b": {a b1}, {a b2}, as this is the most general form.

It’s also possible to define a temporal constraint for the pattern to be valid. For example, you can define that a pattern should occur within 10 seconds via the pattern.within() method. Temporal patterns are supported for both processing and event time.

Attention A pattern sequence can only have one temporal constraint. If multiple such constraints are defined on different individual patterns, then the smallest is applied.

  1. next.within(Time.seconds(10));
  1. next.within(Time.seconds(10))

Contiguity within looping patterns

You can apply the same contiguity condition as discussed in the previous section within a looping pattern. The contiguity will be applied between elements accepted into such a pattern. To illustrate the above with an example, a pattern sequence "a b+ c" ("a" followed by any(non-deterministic relaxed) sequence of one or more "b"’s followed by a "c") with input "a", "b1", "d1", "b2", "d2", "b3" "c" will have the following results:

  1. Strict Contiguity: {a b3 c} – the "d1" after "b1" causes "b1" to be discarded, the same happens for "b2" because of "d2".

  2. Relaxed Contiguity: {a b1 c}, {a b1 b2 c}, {a b1 b2 b3 c}, {a b2 c}, {a b2 b3 c}, {a b3 c} - "d"’s are ignored.

  3. Non-Deterministic Relaxed Contiguity: {a b1 c}, {a b1 b2 c}, {a b1 b3 c}, {a b1 b2 b3 c}, {a b2 c}, {a b2 b3 c}, {a b3 c} - notice the {a b1 b3 c}, which is the result of relaxing contiguity between "b"’s.

For looping patterns (e.g. oneOrMore() and times()) the default is relaxed contiguity. If you want strict contiguity, you have to explicitly specify it by using the consecutive() call, and if you want non-deterministic relaxed contiguity you can use the allowCombinations() call.

Pattern OperationDescription
consecutive()

Works in conjunction with oneOrMore() and times() and imposes strict contiguity between the matching events, i.e. any non-matching element breaks the match (as in next()).

If not applied a relaxed contiguity (as in followedBy()) is used.

E.g. a pattern like:

  1. Pattern.<Event>begin(“start”).where(new SimpleCondition<Event>() {
  2. @Override
  3. public boolean filter(Event value) throws Exception {
  4. return value.getName().equals(“c”);
  5. }
  6. })
  7. .followedBy(“middle”).where(new SimpleCondition<Event>() {
  8. @Override
  9. public boolean filter(Event value) throws Exception {
  10. return value.getName().equals(“a”);
  11. }
  12. }).oneOrMore().consecutive()
  13. .followedBy(“end1”).where(new SimpleCondition<Event>() {
  14. @Override
  15. public boolean filter(Event value) throws Exception {
  16. return value.getName().equals(“b”);
  17. }
  18. });

Will generate the following matches for an input sequence: C D A1 A2 A3 D A4 B

with consecutive applied: {C A1 B}, {C A1 A2 B}, {C A1 A2 A3 B}

without consecutive applied: {C A1 B}, {C A1 A2 B}, {C A1 A2 A3 B}, {C A1 A2 A3 A4 B}

allowCombinations()

Works in conjunction with oneOrMore() and times() and imposes non-deterministic relaxed contiguity between the matching events (as in followedByAny()).

If not applied a relaxed contiguity (as in followedBy()) is used.

E.g. a pattern like:

  1. Pattern.<Event>begin(“start”).where(new SimpleCondition<Event>() {
  2. @Override
  3. public boolean filter(Event value) throws Exception {
  4. return value.getName().equals(“c”);
  5. }
  6. })
  7. .followedBy(“middle”).where(new SimpleCondition<Event>() {
  8. @Override
  9. public boolean filter(Event value) throws Exception {
  10. return value.getName().equals(“a”);
  11. }
  12. }).oneOrMore().allowCombinations()
  13. .followedBy(“end1”).where(new SimpleCondition<Event>() {
  14. @Override
  15. public boolean filter(Event value) throws Exception {
  16. return value.getName().equals(“b”);
  17. }
  18. });

Will generate the following matches for an input sequence: C D A1 A2 A3 D A4 B

with combinations enabled: {C A1 B}, {C A1 A2 B}, {C A1 A3 B}, {C A1 A4 B}, {C A1 A2 A3 B}, {C A1 A2 A4 B}, {C A1 A3 A4 B}, {C A1 A2 A3 A4 B}

without combinations enabled: {C A1 B}, {C A1 A2 B}, {C A1 A2 A3 B}, {C A1 A2 A3 A4 B}

Pattern OperationDescription
consecutive()

Works in conjunction with oneOrMore() and times() and imposes strict contiguity between the matching events, i.e. any non-matching element breaks the match (as in next()).

If not applied a relaxed contiguity (as in followedBy()) is used.

E.g. a pattern like:

  1. Pattern.begin(“start”).where(.getName().equals(“c”))
  2. .followedBy(“middle”).where(.getName().equals(“a”))
  3. .oneOrMore().consecutive()
  4. .followedBy(“end1”).where(.getName().equals(“b”))

Will generate the following matches for an input sequence: C D A1 A2 A3 D A4 B

with consecutive applied: {C A1 B}, {C A1 A2 B}, {C A1 A2 A3 B}

without consecutive applied: {C A1 B}, {C A1 A2 B}, {C A1 A2 A3 B}, {C A1 A2 A3 A4 B}

allowCombinations()

Works in conjunction with oneOrMore() and times() and imposes non-deterministic relaxed contiguity between the matching events (as in followedByAny()).

If not applied a relaxed contiguity (as in followedBy()) is used.

E.g. a pattern like:

  1. Pattern.begin(“start”).where(.getName().equals(“c”))
  2. .followedBy(“middle”).where(.getName().equals(“a”))
  3. .oneOrMore().allowCombinations()
  4. .followedBy(“end1”).where(.getName().equals(“b”))

Will generate the following matches for an input sequence: C D A1 A2 A3 D A4 B

with combinations enabled: {C A1 B}, {C A1 A2 B}, {C A1 A3 B}, {C A1 A4 B}, {C A1 A2 A3 B}, {C A1 A2 A4 B}, {C A1 A3 A4 B}, {C A1 A2 A3 A4 B}

without combinations enabled: {C A1 B}, {C A1 A2 B}, {C A1 A2 A3 B}, {C A1 A2 A3 A4 B}

Groups of patterns

It’s also possible to define a pattern sequence as the condition for begin, followedBy, followedByAny and next. The pattern sequence will be considered as the matching condition logically and a GroupPattern will be returned and it is possible to apply oneOrMore(), times(#ofTimes), times(#fromTimes, #toTimes), optional(), consecutive(), allowCombinations() to the GroupPattern.

  1. Pattern<Event, ?> start = Pattern.begin(
  2. Pattern.<Event>begin("start").where(...).followedBy("start_middle").where(...)
  3. );
  4. // strict contiguity
  5. Pattern<Event, ?> strict = start.next(
  6. Pattern.<Event>begin("next_start").where(...).followedBy("next_middle").where(...)
  7. ).times(3);
  8. // relaxed contiguity
  9. Pattern<Event, ?> relaxed = start.followedBy(
  10. Pattern.<Event>begin("followedby_start").where(...).followedBy("followedby_middle").where(...)
  11. ).oneOrMore();
  12. // non-deterministic relaxed contiguity
  13. Pattern<Event, ?> nonDetermin = start.followedByAny(
  14. Pattern.<Event>begin("followedbyany_start").where(...).followedBy("followedbyany_middle").where(...)
  15. ).optional();
  1. val start: Pattern[Event, _] = Pattern.begin(
  2. Pattern.begin[Event]("start").where(...).followedBy("start_middle").where(...)
  3. )
  4. // strict contiguity
  5. val strict: Pattern[Event, _] = start.next(
  6. Pattern.begin[Event]("next_start").where(...).followedBy("next_middle").where(...)
  7. ).times(3)
  8. // relaxed contiguity
  9. val relaxed: Pattern[Event, _] = start.followedBy(
  10. Pattern.begin[Event]("followedby_start").where(...).followedBy("followedby_middle").where(...)
  11. ).oneOrMore()
  12. // non-deterministic relaxed contiguity
  13. val nonDetermin: Pattern[Event, _] = start.followedByAny(
  14. Pattern.begin[Event]("followedbyany_start").where(...).followedBy("followedbyany_middle").where(...)
  15. ).optional()
Pattern OperationDescription
begin(#name)

Defines a starting pattern:

  1. Pattern<Event, ?> start = Pattern.<Event>begin(“start”);
begin(#pattern_sequence)

Defines a starting pattern:

  1. Pattern<Event, ?> start = Pattern.<Event>begin(
  2. Pattern.<Event>begin(“start”).where(…).followedBy(“middle”).where(…)
  3. );
next(#name)

Appends a new pattern. A matching event has to directly succeed the previous matching event (strict contiguity):

  1. Pattern<Event, ?> next = start.next(“middle”);
next(#pattern_sequence)

Appends a new pattern. A sequence of matching events have to directly succeed the previous matching event (strict contiguity):

  1. Pattern<Event, ?> next = start.next(
  2. Pattern.<Event>begin(“start”).where(…).followedBy(“middle”).where(…)
  3. );
followedBy(#name)

Appends a new pattern. Other events can occur between a matching event and the previous matching event (relaxed contiguity):

  1. Pattern<Event, ?> followedBy = start.followedBy(“middle”);
followedBy(#pattern_sequence)

Appends a new pattern. Other events can occur between a sequence of matching events and the previous matching event (relaxed contiguity):

  1. Pattern<Event, ?> followedBy = start.followedBy(
  2. Pattern.<Event>begin(“start”).where(…).followedBy(“middle”).where(…)
  3. );
followedByAny(#name)

Appends a new pattern. Other events can occur between a matching event and the previous matching event, and alternative matches will be presented for every alternative matching event (non-deterministic relaxed contiguity):

  1. Pattern<Event, ?> followedByAny = start.followedByAny(“middle”);
followedByAny(#pattern_sequence)

Appends a new pattern. Other events can occur between a sequence of matching events and the previous matching event, and alternative matches will be presented for every alternative sequence of matching events (non-deterministic relaxed contiguity):

  1. Pattern<Event, ?> followedByAny = start.followedByAny(
  2. Pattern.<Event>begin(“start”).where(…).followedBy(“middle”).where(…)
  3. );
notNext()

Appends a new negative pattern. A matching (negative) event has to directly succeed the previous matching event (strict contiguity) for the partial match to be discarded:

  1. Pattern<Event, ?> notNext = start.notNext(“not”);
notFollowedBy()

Appends a new negative pattern. A partial matching event sequence will be discarded even if other events occur between the matching (negative) event and the previous matching event (relaxed contiguity):

  1. Pattern<Event, ?> notFollowedBy = start.notFollowedBy(“not”);
within(time)

Defines the maximum time interval for an event sequence to match the pattern. If a non-completed event sequence exceeds this time, it is discarded:

  1. pattern.within(Time.seconds(10));
Pattern OperationDescription
begin(#name)

Defines a starting pattern:

  1. val start = Pattern.beginEvent
begin(#pattern_sequence)

Defines a starting pattern:

  1. val start = Pattern.begin(
  2. Pattern.beginEvent.where(…).followedBy(“middle”).where(…)
  3. )
next(#name)

Appends a new pattern. A matching event has to directly succeed the previous matching event (strict contiguity):

  1. val next = start.next(“middle”)
next(#pattern_sequence)

Appends a new pattern. A sequence of matching events have to directly succeed the previous matching event (strict contiguity):

  1. val next = start.next(
  2. Pattern.beginEvent.where(…).followedBy(“middle”).where(…)
  3. )
followedBy(#name)

Appends a new pattern. Other events can occur between a matching event and the previous matching event (relaxed contiguity) :

  1. val followedBy = start.followedBy(“middle”)
followedBy(#pattern_sequence)

Appends a new pattern. Other events can occur between a sequence of matching events and the previous matching event (relaxed contiguity) :

  1. val followedBy = start.followedBy(
  2. Pattern.beginEvent.where(…).followedBy(“middle”).where(…)
  3. )
followedByAny(#name)

Appends a new pattern. Other events can occur between a matching event and the previous matching event, and alternative matches will be presented for every alternative matching event (non-deterministic relaxed contiguity):

  1. val followedByAny = start.followedByAny(“middle”)
followedByAny(#pattern_sequence)

Appends a new pattern. Other events can occur between a sequence of matching events and the previous matching event, and alternative matches will be presented for every alternative sequence of matching events (non-deterministic relaxed contiguity):

  1. val followedByAny = start.followedByAny(
  2. Pattern.beginEvent.where(…).followedBy(“middle”).where(…)
  3. )
notNext()

Appends a new negative pattern. A matching (negative) event has to directly succeed the previous matching event (strict contiguity) for the partial match to be discarded:

  1. val notNext = start.notNext(“not”)
notFollowedBy()

Appends a new negative pattern. A partial matching event sequence will be discarded even if other events occur between the matching (negative) event and the previous matching event (relaxed contiguity):

  1. val notFollowedBy = start.notFollowedBy(“not”)
within(time)

Defines the maximum time interval for an event sequence to match the pattern. If a non-completed event sequence exceeds this time, it is discarded:

  1. pattern.within(Time.seconds(10))

After Match Skip Strategy

For a given pattern, the same event may be assigned to multiple successful matches. To control to how many matches an event will be assigned, you need to specify the skip strategy called AfterMatchSkipStrategy. There are five types of skip strategies, listed as follows:

  • NO_SKIP: Every possible match will be emitted.
  • SKIP_TO_NEXT: Discards every partial match that started with the same event, emitted match was started.
  • SKIP_PAST_LAST_EVENT: Discards every partial match that started after the match started but before it ended.
  • SKIP_TO_FIRST: Discards every partial match that started after the match started but before the first event of PatternName occurred.
  • SKIP_TO_LAST: Discards every partial match that started after the match started but before the last event of PatternName occurred.

Notice that when using SKIP_TO_FIRST and SKIP_TO_LAST skip strategy, a valid PatternName should also be specified.

For example, for a given pattern b+ c and a data stream b1 b2 b3 c, the differences between these four skip strategies are as follows:

Skip StrategyResultDescription
NO_SKIPb1 b2 b3 c
b2 b3 c
b3 c
After found matching b1 b2 b3 c, the match process will not discard any result.
SKIP_TO_NEXTb1 b2 b3 c
b2 b3 c
b3 c
After found matching b1 b2 b3 c, the match process will not discard any result, because no other match could start at b1.
SKIP_PAST_LAST_EVENTb1 b2 b3 c
After found matching b1 b2 b3 c, the match process will discard all started partial matches.
SKIP_TO_FIRST[b]b1 b2 b3 c
b2 b3 c
b3 c
After found matching b1 b2 b3 c, the match process will try to discard all partial matches started before b1, but there are no such matches. Therefore nothing will be discarded.
SKIP_TO_LAST[b]b1 b2 b3 c
b3 c
After found matching b1 b2 b3 c, the match process will try to discard all partial matches started before b3. There is one such match b2 b3 c

Have a look also at another example to better see the difference between NO_SKIP and SKIP_TO_FIRST: Pattern: (a | b | c) (b | c) c+.greedy d and sequence: a b c1 c2 c3 d Then the results will be:

Skip StrategyResultDescription
NO_SKIPa b c1 c2 c3 d
b c1 c2 c3 d
c1 c2 c3 d
After found matching a b c1 c2 c3 d, the match process will not discard any result.
SKIP_TO_FIRST[c*]a b c1 c2 c3 d
c1 c2 c3 d
After found matching a b c1 c2 c3 d, the match process will discard all partial matches started before c1. There is one such match b c1 c2 c3 d.

To better understand the difference between NO_SKIP and SKIP_TO_NEXT take a look at following example: Pattern: a b+ and sequence: a b1 b2 b3 Then the results will be:

Skip StrategyResultDescription
NO_SKIPa b1
a b1 b2
a b1 b2 b3
After found matching a b1, the match process will not discard any result.
SKIP_TO_NEXTa b1
After found matching a b1, the match process will discard all partial matches started at a. This means neither a b1 b2 nor a b1 b2 b3 could be generated.

To specify which skip strategy to use, just create an AfterMatchSkipStrategy by calling:

FunctionDescription
AfterMatchSkipStrategy.noSkip()Create a NO_SKIP skip strategy
AfterMatchSkipStrategy.skipToNext()Create a SKIP_TO_NEXT skip strategy
AfterMatchSkipStrategy.skipPastLastEvent()Create a SKIP_PAST_LAST_EVENT skip strategy
AfterMatchSkipStrategy.skipToFirst(patternName)Create a SKIP_TO_FIRST skip strategy with the referenced pattern name patternName
AfterMatchSkipStrategy.skipToLast(patternName)Create a SKIP_TO_LAST skip strategy with the referenced pattern name patternName

Then apply the skip strategy to a pattern by calling:

  1. AfterMatchSkipStrategy skipStrategy = ...
  2. Pattern.begin("patternName", skipStrategy);
  1. val skipStrategy = ...
  2. Pattern.begin("patternName", skipStrategy)

Attention For SKIP_TO_FIRST/LAST there are two options how to handle cases when there are no elements mapped to the specified variable. By default a NO_SKIP strategy will be used in this case. The other option is to throw exception in such situation. One can enable this option by:

  1. AfterMatchSkipStrategy.skipToFirst(patternName).throwExceptionOnMiss()
  1. AfterMatchSkipStrategy.skipToFirst(patternName).throwExceptionOnMiss()

Detecting Patterns

After specifying the pattern sequence you are looking for, it is time to apply it to your input stream to detect potential matches. To run a stream of events against your pattern sequence, you have to create a PatternStream. Given an input stream input, a pattern pattern and an optional comparator comparator used to sort events with the same timestamp in case of EventTime or that arrived at the same moment, you create the PatternStream by calling:

  1. DataStream<Event> input = ...
  2. Pattern<Event, ?> pattern = ...
  3. EventComparator<Event> comparator = ... // optional
  4. PatternStream<Event> patternStream = CEP.pattern(input, pattern, comparator);
  1. val input : DataStream[Event] = ...
  2. val pattern : Pattern[Event, _] = ...
  3. var comparator : EventComparator[Event] = ... // optional
  4. val patternStream: PatternStream[Event] = CEP.pattern(input, pattern, comparator)

The input stream can be keyed or non-keyed depending on your use-case.

Attention Applying your pattern on a non-keyed stream will result in a job with parallelism equal to 1.

Selecting from Patterns

Once you have obtained a PatternStream you can apply transformation to detected event sequences. The suggested way of doing that is by PatternProcessFunction.

A PatternProcessFunction has a processMatch method which is called for each matching event sequence. It receives a match in the form of Map<String, List<IN>> where the key is the name of each pattern in your pattern sequence and the value is a list of all accepted events for that pattern (IN is the type of your input elements). The events for a given pattern are ordered by timestamp. The reason for returning a list of accepted events for each pattern is that when using looping patterns (e.g. oneToMany() and times()), more than one event may be accepted for a given pattern.

  1. class MyPatternProcessFunction<IN, OUT> extends PatternProcessFunction<IN, OUT> {
  2. @Override
  3. public void processMatch(Map<String, List<IN>> match, Context ctx, Collector<OUT> out) throws Exception;
  4. IN startEvent = match.get("start").get(0);
  5. IN endEvent = match.get("end").get(0);
  6. out.collect(OUT(startEvent, endEvent));
  7. }
  8. }

The PatternProcessFunction gives access to a Context object. Thanks to it, one can access time related characteristics such as currentProcessingTime or timestamp of current match (which is the timestamp of the last element assigned to the match). For more info see Time context. Through this context one can also emit results to a side-output.

Handling Timed Out Partial Patterns

Whenever a pattern has a window length attached via the within keyword, it is possible that partial event sequences are discarded because they exceed the window length. To act upon a timed out partial match one can use TimedOutPartialMatchHandler interface. The interface is supposed to be used in a mixin style. This mean you can additionally implement this interface with your PatternProcessFunction. The TimedOutPartialMatchHandler provides the additional processTimedOutMatch method which will be called for every timed out partial match.

  1. class MyPatternProcessFunction<IN, OUT> extends PatternProcessFunction<IN, OUT> implements TimedOutPartialMatchHandler<IN> {
  2. @Override
  3. public void processMatch(Map<String, List<IN>> match, Context ctx, Collector<OUT> out) throws Exception;
  4. ...
  5. }
  6. @Override
  7. public void processTimedOutMatch(Map<String, List<IN>> match, Context ctx) throws Exception;
  8. IN startEvent = match.get("start").get(0);
  9. ctx.output(outputTag, T(startEvent));
  10. }
  11. }

Note The processTimedOutMatch does not give one access to the main output. You can still emit results through side-outputs though, through the Context object.

Convenience API

The aforementioned PatternProcessFunction was introduced in Flink 1.8 and since then it is the recommended way to interact with matches. One can still use the old style API like select/flatSelect, which internally will be translated into a PatternProcessFunction.

  1. PatternStream<Event> patternStream = CEP.pattern(input, pattern);
  2. OutputTag<String> outputTag = new OutputTag<String>("side-output"){};
  3. SingleOutputStreamOperator<ComplexEvent> flatResult = patternStream.flatSelect(
  4. outputTag,
  5. new PatternFlatTimeoutFunction<Event, TimeoutEvent>() {
  6. public void timeout(
  7. Map<String, List<Event>> pattern,
  8. long timeoutTimestamp,
  9. Collector<TimeoutEvent> out) throws Exception {
  10. out.collect(new TimeoutEvent());
  11. }
  12. },
  13. new PatternFlatSelectFunction<Event, ComplexEvent>() {
  14. public void flatSelect(Map<String, List<IN>> pattern, Collector<OUT> out) throws Exception {
  15. out.collect(new ComplexEvent());
  16. }
  17. }
  18. );
  19. DataStream<TimeoutEvent> timeoutFlatResult = flatResult.getSideOutput(outputTag);
  1. val patternStream: PatternStream[Event] = CEP.pattern(input, pattern)
  2. val outputTag = OutputTag[String]("side-output")
  3. val result: SingleOutputStreamOperator[ComplexEvent] = patternStream.flatSelect(outputTag){
  4. (pattern: Map[String, Iterable[Event]], timestamp: Long, out: Collector[TimeoutEvent]) =>
  5. out.collect(TimeoutEvent())
  6. } {
  7. (pattern: mutable.Map[String, Iterable[Event]], out: Collector[ComplexEvent]) =>
  8. out.collect(ComplexEvent())
  9. }
  10. val timeoutResult: DataStream[TimeoutEvent] = result.getSideOutput(outputTag)

Time in CEP library

Handling Lateness in Event Time

In CEP the order in which elements are processed matters. To guarantee that elements are processed in the correct order when working in event time, an incoming element is initially put in a buffer where elements are sorted in ascending order based on their timestamp, and when a watermark arrives, all the elements in this buffer with timestamps smaller than that of the watermark are processed. This implies that elements between watermarks are processed in event-time order.

Attention The library assumes correctness of the watermark when working in event time.

To guarantee that elements across watermarks are processed in event-time order, Flink’s CEP library assumes correctness of the watermark, and considers as late elements whose timestamp is smaller than that of the last seen watermark. Late elements are not further processed. Also, you can specify a sideOutput tag to collect the late elements come after the last seen watermark, you can use it like this.

  1. PatternStream<Event> patternStream = CEP.pattern(input, pattern);
  2. OutputTag<String> lateDataOutputTag = new OutputTag<String>("late-data"){};
  3. SingleOutputStreamOperator<ComplexEvent> result = patternStream
  4. .sideOutputLateData(lateDataOutputTag)
  5. .select(
  6. new PatternSelectFunction<Event, ComplexEvent>() {...}
  7. );
  8. DataStream<String> lateData = result.getSideOutput(lateDataOutputTag);
  1. val patternStream: PatternStream[Event] = CEP.pattern(input, pattern)
  2. val lateDataOutputTag = OutputTag[String]("late-data")
  3. val result: SingleOutputStreamOperator[ComplexEvent] = patternStream
  4. .sideOutputLateData(lateDataOutputTag)
  5. .select{
  6. pattern: Map[String, Iterable[ComplexEvent]] => ComplexEvent()
  7. }
  8. val lateData: DataStream[String] = result.getSideOutput(lateDataOutputTag)

Time context

In PatternProcessFunction as well as in IterativeCondition user has access to a context that implements TimeContext as follows:

  1. /**
  2. * Enables access to time related characteristics such as current processing time or timestamp of
  3. * currently processed element. Used in {@link PatternProcessFunction} and
  4. * {@link org.apache.flink.cep.pattern.conditions.IterativeCondition}
  5. */
  6. @PublicEvolving
  7. public interface TimeContext {
  8. /**
  9. * Timestamp of the element currently being processed.
  10. *
  11. * <p>In case of {@link org.apache.flink.streaming.api.TimeCharacteristic#ProcessingTime} this
  12. * will be set to the time when event entered the cep operator.
  13. */
  14. long timestamp();
  15. /** Returns the current processing time. */
  16. long currentProcessingTime();
  17. }

This context gives user access to time characteristics of processed events (incoming records in case of IterativeCondition and matches in case of PatternProcessFunction). Call to TimeContext#currentProcessingTime always gives you the value of current processing time and this call should be preferred to e.g. calling System.currentTimeMillis().

In case of TimeContext#timestamp() the returned value is equal to assigned timestamp in case of EventTime. In ProcessingTime this will equal to the point of time when said event entered cep operator (or when the match was generated in case of PatternProcessFunction). This means that the value will be consistent across multiple calls to that method.

Examples

The following example detects the pattern start, middle(name = "error") -> end(name = "critical") on a keyed data stream of Events. The events are keyed by their ids and a valid pattern has to occur within 10 seconds. The whole processing is done with event time.

  1. StreamExecutionEnvironment env = ...
  2. env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
  3. DataStream<Event> input = ...
  4. DataStream<Event> partitionedInput = input.keyBy(new KeySelector<Event, Integer>() {
  5. @Override
  6. public Integer getKey(Event value) throws Exception {
  7. return value.getId();
  8. }
  9. });
  10. Pattern<Event, ?> pattern = Pattern.<Event>begin("start")
  11. .next("middle").where(new SimpleCondition<Event>() {
  12. @Override
  13. public boolean filter(Event value) throws Exception {
  14. return value.getName().equals("error");
  15. }
  16. }).followedBy("end").where(new SimpleCondition<Event>() {
  17. @Override
  18. public boolean filter(Event value) throws Exception {
  19. return value.getName().equals("critical");
  20. }
  21. }).within(Time.seconds(10));
  22. PatternStream<Event> patternStream = CEP.pattern(partitionedInput, pattern);
  23. DataStream<Alert> alerts = patternStream.select(new PatternSelectFunction<Event, Alert>() {
  24. @Override
  25. public Alert select(Map<String, List<Event>> pattern) throws Exception {
  26. return createAlert(pattern);
  27. }
  28. });
  1. val env : StreamExecutionEnvironment = ...
  2. env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
  3. val input : DataStream[Event] = ...
  4. val partitionedInput = input.keyBy(event => event.getId)
  5. val pattern = Pattern.begin[Event]("start")
  6. .next("middle").where(_.getName == "error")
  7. .followedBy("end").where(_.getName == "critical")
  8. .within(Time.seconds(10))
  9. val patternStream = CEP.pattern(partitionedInput, pattern)
  10. val alerts = patternStream.select(createAlert(_))

Migrating from an older Flink version(pre 1.3)

Migrating to 1.4+

In Flink-1.4 the backward compatibility of CEP library with <= Flink 1.2 was dropped. Unfortunately it is not possible to restore a CEP job that was once run with 1.2.x

Migrating to 1.3.x

The CEP library in Flink-1.3 ships with a number of new features which have led to some changes in the API. Here we describe the changes that you need to make to your old CEP jobs, in order to be able to run them with Flink-1.3. After making these changes and recompiling your job, you will be able to resume its execution from a savepoint taken with the old version of your job, i.e. without having to re-process your past data.

The changes required are:

  1. Change your conditions (the ones in the where(...) clause) to extend the SimpleCondition class instead of implementing the FilterFunction interface.

  2. Change your functions provided as arguments to the select(...) and flatSelect(...) methods to expect a list of events associated with each pattern (List in Java, Iterable in Scala). This is because with the addition of the looping patterns, multiple input events can match a single (looping) pattern.

  3. The followedBy() in Flink 1.1 and 1.2 implied non-deterministic relaxed contiguity (see here). In Flink 1.3 this has changed and followedBy() implies relaxed contiguity, while followedByAny() should be used if non-deterministic relaxed contiguity is required.

Back to top