[ 
https://issues.apache.org/jira/browse/FLINK-7169?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16121203#comment-16121203
 ] 

ASF GitHub Bot commented on FLINK-7169:
---------------------------------------

Github user dawidwys commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4331#discussion_r132384194
  
    --- Diff: 
flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java
 ---
    @@ -150,6 +160,59 @@ long getWindowTime() {
                }
     
                /**
    +            * Check pattern after match skip strategy.
    +            */
    +           private void checkPatternSkipStrategy() {
    +                   AfterMatchSkipStrategy afterMatchSkipStrategy = 
currentPattern.getAfterMatchSkipStrategy();
    +                   if (afterMatchSkipStrategy.getStrategy() == 
AfterMatchSkipStrategy.SkipStrategy.SKIP_TO_FIRST ||
    +                           afterMatchSkipStrategy.getStrategy() == 
AfterMatchSkipStrategy.SkipStrategy.SKIP_TO_LAST) {
    +                           Pattern<T, ?> pattern = currentPattern;
    +                           while 
(!pattern.getName().equals(afterMatchSkipStrategy.getPatternName())) {
    +                                   pattern = pattern.getPrevious();
    +                           }
    +                           // pattern name match check.
    +                           if (pattern == null) {
    +                                   throw new 
MalformedPatternException("the pattern name specified in AfterMatchSkipStrategy 
" +
    +                                           "can not be found in the given 
Pattern");
    +                           } else {
    +                                   // can not be used with optional states.
    +                                   if 
(pattern.getQuantifier().hasProperty(Quantifier.QuantifierProperty.OPTIONAL)) {
    +                                           throw new 
MalformedPatternException("the AfterMatchSkipStrategy "
    +                                                   + 
afterMatchSkipStrategy.getStrategy() + " can not be used with optional 
pattern");
    +                                   }
    +                           }
    +
    +                           // start position check.
    +                           if (pattern.getPrevious() == null) {
    --- End diff --
    
    I personally see no reason for a semantic with RuntimeException. I can't 
think of any use-case for it. Maybe let's finish this PR without the switch and 
exceptions and open a JIRA with the switch, ideally with some use-case's for 
that semantic, so we can further agree on that and see if anyone needs it.


> Support AFTER MATCH SKIP function in CEP library API
> ----------------------------------------------------
>
>                 Key: FLINK-7169
>                 URL: https://issues.apache.org/jira/browse/FLINK-7169
>             Project: Flink
>          Issue Type: Sub-task
>          Components: CEP
>            Reporter: Yueting Chen
>            Assignee: Yueting Chen
>             Fix For: 1.4.0
>
>
> In order to support Oracle's MATCH_RECOGNIZE on top of the CEP library, we 
> need to support AFTER MATCH SKIP function in CEP API.
> There're four options in AFTER MATCH SKIP, listed as follows:
> 1. AFTER MATCH SKIP TO NEXT ROW: resume pattern matching at the row after the 
> first row of the current match.
> 2. AFTER MATCH SKIP PAST LAST ROW: resume pattern matching at the next row 
> after the last row of the current match.
> 3. AFTER MATCH SKIP TO FIST *RPV*: resume pattern matching at the first row 
> that is mapped to the row pattern variable RPV.
> 4. AFTER MATCH SKIP TO LAST *RPV*: resume pattern matching at the last row 
> that is mapped to the row pattern variable RPV.
> I think we can introduce a new function to `CEP` class, which takes a new 
> parameter as AfterMatchSKipStrategy.
> The new API may looks like this
> {code}
> public static <T> PatternStream<T> pattern(DataStream<T> input, Pattern<T, ?> 
> pattern, AfterMatchSkipStrategy afterMatchSkipStrategy) 
> {code}
> We can also make `SKIP TO NEXT ROW` as the default option, because that's 
> what CEP library behaves currently.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to