Re: Flink CEP greedy match of single pattern

2020-03-25 Thread Dawid Wysakowicz
Hi Dominik, I think you are hitting a bug. The greedy quantifier does not work well if applied for the last element of a pattern. There is a jira issue to improve support for greedy qualifier[1]. You could work it around with adding an additional state at the end. E.g. :

Re: Flink CEP greedy match of single pattern

2020-03-25 Thread Dominik Wosiński
P.S. So now my pattern looks like this: Pattern.begin[AccelVector](EventPatternName, AfterMatchSkipStrategy.skipPastLastEvent()) .where(_.data() > Threshold) .oneOrMore .greedy .consecutive() .within(Time.minutes(1)) śr., 25 mar 2020 o 10:03 Dominik Wosiński napisał(a): > Hey,

Re: Flink CEP greedy match of single pattern

2020-03-25 Thread Dominik Wosiński
Hey, thanks for the answer. But if I add the *AfterMatchSkipStrategy* it simply seems to emit event by event so in the case described above it does emit: [400], [500] Shouldn't the *greedy* quantifier guarantee that this will be matched as many times as possible thus creating [400, 500] ??

Re: Flink CEP greedy match of single pattern

2020-02-21 Thread Till Rohrmann
Hi Dominik, you can control FlinkCEP's consumption behaviour via the after match skip strategies [1]. They allow you to control how Flink treats events after a match has occurred. If you are interested in the longest possible window of events exceeding your threshold, then you could also add

Flink CEP greedy match of single pattern

2020-02-21 Thread Dominik Wosiński
Hey, I have a question regarding CEP, assume I have a stream of readings from various sensors. The application is running in EventTime, so according to the CEP docs the events are buffered and sorted by timestamp ascending. So, I want to record the situations when reading from the sensor goes