[ https://issues.apache.org/jira/browse/FLINK-7147?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16116461#comment-16116461 ]
ASF GitHub Bot commented on FLINK-7147: --------------------------------------- Github user dawidwys commented on the issue: https://github.com/apache/flink/pull/4296 There is also one more problem. When we have optional after `greedy` it does not work well. E.g. have a look at this test case: @Test public void testGreedyZeroOrMoreBeforeOptional2() { 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 d = new Event(43, "d", 3.0); Event a3 = new Event(42, "a", 2.0); Event e = new Event(44, "e", 3.0); inputEvents.add(new StreamRecord<>(c, 1)); inputEvents.add(new StreamRecord<>(a1, 2)); inputEvents.add(new StreamRecord<>(a2, 3)); inputEvents.add(new StreamRecord<>(d, 4)); inputEvents.add(new StreamRecord<>(a3, 5)); inputEvents.add(new StreamRecord<>(e, 6)); // c a* d e 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("middle1").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("middle2").where(new SimpleCondition<Event>() { private static final long serialVersionUID = 5726188262756267490L; @Override public boolean filter(Event value) throws Exception { return value.getName().equals("d"); } }).optional().followedBy("end").where(new SimpleCondition<Event>() { private static final long serialVersionUID = 5726188262756267490L; @Override public boolean filter(Event value) throws Exception { return value.getName().equals("e"); } }); 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, e), Lists.newArrayList(c, a1, a2, d, e) )); } Right know it also returns `c a1 a2 e`, which I think is not correct. I don't think there is an easy way to fix it right now. I would suggest restricting on the Pattern level that greedy must not be followed by an `Optional` patten. I would like to hear opinions on that, @kl0u. > 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)