[ https://issues.apache.org/jira/browse/FLINK-7147?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16099835#comment-16099835 ]
ASF GitHub Bot commented on FLINK-7147: --------------------------------------- Github user dawidwys commented on the issue: https://github.com/apache/flink/pull/4296 I've started reviewing it, but realised it is working as I expected only in case where the inner consuming strategy is `STRICT`. Let's have a look at test like this one: @Test public void testGreedyFollowedByInBetween() { List<StreamRecord<Event>> inputEvents = new ArrayList<>(); Event c = new Event(40, "c", 1.0); Event a1 = new Event(41, "a", 2.0); Event a2 = new Event(42, "a", 2.0); Event a3 = new Event(43, "a", 2.0); Event d = new Event(44, "d", 3.0); inputEvents.add(new StreamRecord<>(c, 1)); inputEvents.add(new StreamRecord<>(new Event(1, "dummy", 1111), 2)); inputEvents.add(new StreamRecord<>(a1, 3)); inputEvents.add(new StreamRecord<>(new Event(1, "dummy", 1111), 4)); inputEvents.add(new StreamRecord<>(a2, 5)); inputEvents.add(new StreamRecord<>(new Event(1, "dummy", 1111), 6)); inputEvents.add(new StreamRecord<>(a3, 7)); inputEvents.add(new StreamRecord<>(d, 8)); // c a* d Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() { private static final long serialVersionUID = 5726188262756267490L; @Override public boolean filter(Event value) throws Exception { return value.getName().equals("c"); } }).followedBy("middle").where(new SimpleCondition<Event>() { private static final long serialVersionUID = 5726188262756267490L; @Override public boolean filter(Event value) throws Exception { return value.getName().equals("a"); } }).oneOrMore().optional().greedy().followedBy("end").where(new SimpleCondition<Event>() { private static final long serialVersionUID = 5726188262756267490L; @Override public boolean filter(Event value) throws Exception { return value.getName().equals("d"); } }); NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa); compareMaps(resultingPatterns, Lists.<List<Event>>newArrayList( Lists.newArrayList(c, a1, a2, a3, d) )); } I would expect only that one result but I get: Lists.newArrayList(c, a1, a2, a3, d), Lists.newArrayList(c, a1, a2, d), Lists.newArrayList(c, a1, d), Lists.newArrayList(c, d) Which is the same with or without the `greedy()` applied. > Support greedy quantifier in CEP > -------------------------------- > > Key: FLINK-7147 > URL: https://issues.apache.org/jira/browse/FLINK-7147 > Project: Flink > Issue Type: Sub-task > Components: CEP, Table API & SQL > Reporter: Dian Fu > Assignee: Dian Fu > > Greedy quantifier will try to match the token as many times as possible. For > example, for pattern {{a b* c}} (skip till next is used) and inputs {{a b1 b2 > c}}, if the quantifier for {{b}} is greedy, it will only output {{a b1 b2 c}}. -- This message was sent by Atlassian JIRA (v6.4.14#64029)