[
https://issues.apache.org/jira/browse/FLINK-8914?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16502031#comment-16502031
]
ASF GitHub Bot commented on FLINK-8914:
---------------------------------------
GitHub user Aitozi opened a pull request:
https://github.com/apache/flink/pull/6124
[FLINK-8914][CEP]Fix wrong semantic when greedy pattern is the head of the
pattern
## What is the purpose of the change
As described in the jira
[FLINK-8914](https://issues.apache.org/jira/projects/FLINK/issues/FLINK-8914)
There is something wrong with `greedy` when it is the head of the pattern.
Because the `NFA` process each `ComputationState` and will produce a new `Start
ComputationState`. So when it runs into the greedy match, other `start runs`
can also be set up
## Brief change log
*(for example:)*
- *Add a new StateType `Greedy` for convenience of distinguishing the
greedy in computations*
- *Remove the redundant start state during process*
## Verifying this change
Add two UT in `GreedyITCase`
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/Aitozi/flink greedyfix
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/flink/pull/6124.patch
To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:
This closes #6124
----
commit 76c1a4516b4bc98043d944335cc7a0aacd359278
Author: minwenjun <minwenjun@...>
Date: 2018-06-05T16:07:55Z
Fix wrong semantic when greedy pattern is the head of the pattern
----
> CEP's greedy() modifier doesn't work
> ------------------------------------
>
> Key: FLINK-8914
> URL: https://issues.apache.org/jira/browse/FLINK-8914
> Project: Flink
> Issue Type: Bug
> Components: CEP
> Affects Versions: 1.4.0, 1.4.1
> Reporter: David Anderson
> Assignee: aitozi
> Priority: Major
>
> When applied to the first or last component of a CEP Pattern, greedy()
> doesn't work correctly. Here's an example:
> {code:java}
> package com.dataartisans.flinktraining.exercises.datastream_java.cep;
> import org.apache.flink.cep.CEP;
> import org.apache.flink.cep.PatternSelectFunction;
> import org.apache.flink.cep.PatternStream;
> import org.apache.flink.cep.pattern.Pattern;
> import org.apache.flink.cep.pattern.conditions.SimpleCondition;
> import org.apache.flink.streaming.api.datastream.DataStream;
> import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
> import java.util.List;
> import java.util.Map;
> public class RunLength {
> public static void main(String[] args) throws Exception {
> StreamExecutionEnvironment env =
> StreamExecutionEnvironment.getExecutionEnvironment();
> env.setParallelism(1);
> DataStream<Integer> input = env.fromElements(1, 1, 1, 1, 1, 0, 1, 1, 1,
> 0);
> Pattern<Integer, ?> onesThenZero = Pattern.<Integer>begin("ones")
> .where(new SimpleCondition<Integer>() {
> @Override
> public boolean filter(Integer value) throws Exception {
> return value == 1;
> }
> })
> .oneOrMore()
> .greedy()
> .consecutive()
> .next("zero")
> .where(new SimpleCondition<Integer>() {
> @Override
> public boolean filter(Integer value) throws Exception {
> return value == 0;
> }
> });
> PatternStream<Integer> patternStream = CEP.pattern(input, onesThenZero);
> // Expected: 5 3
> // Actual: 5 4 3 2 1 3 2 1
> patternStream.select(new LengthOfRun()).print();
> env.execute();
> }
> public static class LengthOfRun implements PatternSelectFunction<Integer,
> Integer> {
> public Integer select(Map<String, List<Integer>> pattern) {
> return pattern.get("ones").size();
> }
> }
> }
> {code}
> The only workaround for now seems to be to rewrite the pattern so that
> greedy() isn't needed – i.e. by bracketing the greedy section with a prefix
> and suffix that both have to be matched.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)