[jira] [Commented] (FLINK-6772) Incorrect ordering of matched state events in Flink CEP
[ https://issues.apache.org/jira/browse/FLINK-6772?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16043935#comment-16043935 ] Ted Yu commented on FLINK-6772: --- {code} + for (String key: path.keySet()) { + List events = path.get(key); {code} Instead of calling keySet(), entrySet() should be used. This would avoid the path.get() call. > Incorrect ordering of matched state events in Flink CEP > --- > > Key: FLINK-6772 > URL: https://issues.apache.org/jira/browse/FLINK-6772 > Project: Flink > Issue Type: Bug > Components: CEP >Affects Versions: 1.3.0 >Reporter: Tzu-Li (Gordon) Tai >Assignee: Kostas Kloudas > Labels: flink-rel-1.3.1-blockers > Fix For: 1.3.1 > > > I've stumbled across an unexepected ordering of the matched state events. > Pattern: > {code} > Patternpattern = Pattern > .begin("start") > .where(new IterativeCondition() { > @Override > public boolean filter(String s, Context context) throws > Exception { > return s.startsWith("a-"); > } > }).times(4).allowCombinations() > .followedByAny("end") > .where(new IterativeCondition() { > public boolean filter(String s, Context context) throws > Exception { > return s.startsWith("b-"); > } > }).times(3).consecutive(); > {code} > Input event sequence: > a-1, a-2, a-3, a-4, b-1, b-2, b-3 > On b-3 a matched pattern would be triggered. > Now, in the {{Map }} map passed via {{select}} in > {{PatternSelectFunction}}, the list for the "end" state is: > b-3, b-1, b-2. > Based on the timestamp of the events (simply using processing time), the > correct order should be b-1, b-2, b-3. -- This message was sent by Atlassian JIRA (v6.3.15#6346)
[jira] [Commented] (FLINK-6772) Incorrect ordering of matched state events in Flink CEP
[ https://issues.apache.org/jira/browse/FLINK-6772?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16042718#comment-16042718 ] ASF GitHub Bot commented on FLINK-6772: --- Github user asfgit closed the pull request at: https://github.com/apache/flink/pull/4084 > Incorrect ordering of matched state events in Flink CEP > --- > > Key: FLINK-6772 > URL: https://issues.apache.org/jira/browse/FLINK-6772 > Project: Flink > Issue Type: Bug > Components: CEP >Affects Versions: 1.3.0 >Reporter: Tzu-Li (Gordon) Tai >Assignee: Kostas Kloudas > Labels: flink-rel-1.3.1-blockers > > I've stumbled across an unexepected ordering of the matched state events. > Pattern: > {code} > Patternpattern = Pattern > .begin("start") > .where(new IterativeCondition() { > @Override > public boolean filter(String s, Context context) throws > Exception { > return s.startsWith("a-"); > } > }).times(4).allowCombinations() > .followedByAny("end") > .where(new IterativeCondition() { > public boolean filter(String s, Context context) throws > Exception { > return s.startsWith("b-"); > } > }).times(3).consecutive(); > {code} > Input event sequence: > a-1, a-2, a-3, a-4, b-1, b-2, b-3 > On b-3 a matched pattern would be triggered. > Now, in the {{Map }} map passed via {{select}} in > {{PatternSelectFunction}}, the list for the "end" state is: > b-3, b-1, b-2. > Based on the timestamp of the events (simply using processing time), the > correct order should be b-1, b-2, b-3. -- This message was sent by Atlassian JIRA (v6.3.15#6346)
[jira] [Commented] (FLINK-6772) Incorrect ordering of matched state events in Flink CEP
[ https://issues.apache.org/jira/browse/FLINK-6772?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16041018#comment-16041018 ] ASF GitHub Bot commented on FLINK-6772: --- Github user kl0u commented on the issue: https://github.com/apache/flink/pull/4084 Thanks @dawidwys ! I will let travis have another go on the rebased version and then merge. > Incorrect ordering of matched state events in Flink CEP > --- > > Key: FLINK-6772 > URL: https://issues.apache.org/jira/browse/FLINK-6772 > Project: Flink > Issue Type: Bug > Components: CEP >Affects Versions: 1.3.0 >Reporter: Tzu-Li (Gordon) Tai >Assignee: Kostas Kloudas > Labels: flink-rel-1.3.1-blockers > > I've stumbled across an unexepected ordering of the matched state events. > Pattern: > {code} > Patternpattern = Pattern > .begin("start") > .where(new IterativeCondition() { > @Override > public boolean filter(String s, Context context) throws > Exception { > return s.startsWith("a-"); > } > }).times(4).allowCombinations() > .followedByAny("end") > .where(new IterativeCondition() { > public boolean filter(String s, Context context) throws > Exception { > return s.startsWith("b-"); > } > }).times(3).consecutive(); > {code} > Input event sequence: > a-1, a-2, a-3, a-4, b-1, b-2, b-3 > On b-3 a matched pattern would be triggered. > Now, in the {{Map }} map passed via {{select}} in > {{PatternSelectFunction}}, the list for the "end" state is: > b-3, b-1, b-2. > Based on the timestamp of the events (simply using processing time), the > correct order should be b-1, b-2, b-3. -- This message was sent by Atlassian JIRA (v6.3.15#6346)
[jira] [Commented] (FLINK-6772) Incorrect ordering of matched state events in Flink CEP
[ https://issues.apache.org/jira/browse/FLINK-6772?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16040994#comment-16040994 ] ASF GitHub Bot commented on FLINK-6772: --- Github user dawidwys commented on the issue: https://github.com/apache/flink/pull/4084 LGTM > Incorrect ordering of matched state events in Flink CEP > --- > > Key: FLINK-6772 > URL: https://issues.apache.org/jira/browse/FLINK-6772 > Project: Flink > Issue Type: Bug > Components: CEP >Affects Versions: 1.3.0 >Reporter: Tzu-Li (Gordon) Tai >Assignee: Kostas Kloudas > Labels: flink-rel-1.3.1-blockers > > I've stumbled across an unexepected ordering of the matched state events. > Pattern: > {code} > Patternpattern = Pattern > .begin("start") > .where(new IterativeCondition() { > @Override > public boolean filter(String s, Context context) throws > Exception { > return s.startsWith("a-"); > } > }).times(4).allowCombinations() > .followedByAny("end") > .where(new IterativeCondition() { > public boolean filter(String s, Context context) throws > Exception { > return s.startsWith("b-"); > } > }).times(3).consecutive(); > {code} > Input event sequence: > a-1, a-2, a-3, a-4, b-1, b-2, b-3 > On b-3 a matched pattern would be triggered. > Now, in the {{Map }} map passed via {{select}} in > {{PatternSelectFunction}}, the list for the "end" state is: > b-3, b-1, b-2. > Based on the timestamp of the events (simply using processing time), the > correct order should be b-1, b-2, b-3. -- This message was sent by Atlassian JIRA (v6.3.15#6346)
[jira] [Commented] (FLINK-6772) Incorrect ordering of matched state events in Flink CEP
[ https://issues.apache.org/jira/browse/FLINK-6772?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16040686#comment-16040686 ] ASF GitHub Bot commented on FLINK-6772: --- GitHub user kl0u opened a pull request: https://github.com/apache/flink/pull/4084 [FLINK-6772] [cep] Fix ordering (by timestamp) of matched events. You can merge this pull request into a Git repository by running: $ git pull https://github.com/kl0u/flink cep-fix-ordering Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/4084.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 #4084 commit b3f38aa64addd73b20f6ab51f98f95017cb7e348 Author: kkloudasDate: 2017-05-31T15:48:34Z [FLINK-6772] [cep] Fix ordering (by timestamp) of matched events. > Incorrect ordering of matched state events in Flink CEP > --- > > Key: FLINK-6772 > URL: https://issues.apache.org/jira/browse/FLINK-6772 > Project: Flink > Issue Type: Bug > Components: CEP >Affects Versions: 1.3.0 >Reporter: Tzu-Li (Gordon) Tai >Assignee: Kostas Kloudas > Labels: flink-rel-1.3.1-blockers > > I've stumbled across an unexepected ordering of the matched state events. > Pattern: > {code} > Pattern pattern = Pattern > .begin("start") > .where(new IterativeCondition() { > @Override > public boolean filter(String s, Context context) throws > Exception { > return s.startsWith("a-"); > } > }).times(4).allowCombinations() > .followedByAny("end") > .where(new IterativeCondition() { > public boolean filter(String s, Context context) throws > Exception { > return s.startsWith("b-"); > } > }).times(3).consecutive(); > {code} > Input event sequence: > a-1, a-2, a-3, a-4, b-1, b-2, b-3 > On b-3 a matched pattern would be triggered. > Now, in the {{Map }} map passed via {{select}} in > {{PatternSelectFunction}}, the list for the "end" state is: > b-3, b-1, b-2. > Based on the timestamp of the events (simply using processing time), the > correct order should be b-1, b-2, b-3. -- This message was sent by Atlassian JIRA (v6.3.15#6346)
[jira] [Commented] (FLINK-6772) Incorrect ordering of matched state events in Flink CEP
[ https://issues.apache.org/jira/browse/FLINK-6772?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16030861#comment-16030861 ] Kostas Kloudas commented on FLINK-6772: --- I agree with [~rmetzger] that this can wait till 1.3.1. [~dawidwys] the unique names were introduced for serialization/deserialization. I will have a look to see if this can be bypassed. > Incorrect ordering of matched state events in Flink CEP > --- > > Key: FLINK-6772 > URL: https://issues.apache.org/jira/browse/FLINK-6772 > Project: Flink > Issue Type: Bug > Components: CEP >Affects Versions: 1.3.0 >Reporter: Tzu-Li (Gordon) Tai > > I've stumbled across an unexepected ordering of the matched state events. > Pattern: > {code} > Patternpattern = Pattern > .begin("start") > .where(new IterativeCondition() { > @Override > public boolean filter(String s, Context context) throws > Exception { > return s.startsWith("a-"); > } > }).times(4).allowCombinations() > .followedByAny("end") > .where(new IterativeCondition() { > public boolean filter(String s, Context context) throws > Exception { > return s.startsWith("b-"); > } > }).times(3).consecutive(); > {code} > Input event sequence: > a-1, a-2, a-3, a-4, b-1, b-2, b-3 > On b-3 a matched pattern would be triggered. > Now, in the {{Map }} map passed via {{select}} in > {{PatternSelectFunction}}, the list for the "end" state is: > b-3, b-1, b-2. > Based on the timestamp of the events (simply using processing time), the > correct order should be b-1, b-2, b-3. -- This message was sent by Atlassian JIRA (v6.3.15#6346)
[jira] [Commented] (FLINK-6772) Incorrect ordering of matched state events in Flink CEP
[ https://issues.apache.org/jira/browse/FLINK-6772?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16029631#comment-16029631 ] Robert Metzger commented on FLINK-6772: --- I assume we can fix this in Flink 1.3.1 and it won't affect the 1.3.0 release? > Incorrect ordering of matched state events in Flink CEP > --- > > Key: FLINK-6772 > URL: https://issues.apache.org/jira/browse/FLINK-6772 > Project: Flink > Issue Type: Bug > Components: CEP >Affects Versions: 1.3.0 >Reporter: Tzu-Li (Gordon) Tai > > I've stumbled across an unexepected ordering of the matched state events. > Pattern: > {code} > Patternpattern = Pattern > .begin("start") > .where(new IterativeCondition() { > @Override > public boolean filter(String s, Context context) throws > Exception { > return s.startsWith("a-"); > } > }).times(4).allowCombinations() > .followedByAny("end") > .where(new IterativeCondition() { > public boolean filter(String s, Context context) throws > Exception { > return s.startsWith("b-"); > } > }).times(3).consecutive(); > {code} > Input event sequence: > a-1, a-2, a-3, a-4, b-1, b-2, b-3 > On b-3 a matched pattern would be triggered. > Now, in the {{Map }} map passed via {{select}} in > {{PatternSelectFunction}}, the list for the "end" state is: > b-3, b-1, b-2. > Based on the timestamp of the events (simply using processing time), the > correct order should be b-1, b-2, b-3. -- This message was sent by Atlassian JIRA (v6.3.15#6346)
[jira] [Commented] (FLINK-6772) Incorrect ordering of matched state events in Flink CEP
[ https://issues.apache.org/jira/browse/FLINK-6772?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16029347#comment-16029347 ] Dawid Wysakowicz commented on FLINK-6772: - Ok I've tracked down the issue. It is caused by internalStateName introduced in [FLINK-6604], because of that the order in {{SharedBuffer::extractPatterns}} is not preserved. [~kkl0u] Is the internal unique name necessary? > Incorrect ordering of matched state events in Flink CEP > --- > > Key: FLINK-6772 > URL: https://issues.apache.org/jira/browse/FLINK-6772 > Project: Flink > Issue Type: Bug > Components: CEP >Affects Versions: 1.3.0 >Reporter: Tzu-Li (Gordon) Tai > > I've stumbled across an unexepected ordering of the matched state events. > Pattern: > {code} > Patternpattern = Pattern > .begin("start") > .where(new IterativeCondition() { > @Override > public boolean filter(String s, Context context) throws > Exception { > return s.startsWith("a-"); > } > }).times(4).allowCombinations() > .followedByAny("end") > .where(new IterativeCondition() { > public boolean filter(String s, Context context) throws > Exception { > return s.startsWith("b-"); > } > }).times(3).consecutive(); > {code} > Input event sequence: > a-1, a-2, a-3, a-4, b-1, b-2, b-3 > On b-3 a matched pattern would be triggered. > Now, in the {{Map }} map passed via {{select}} in > {{PatternSelectFunction}}, the list for the "end" state is: > b-3, b-1, b-2. > Based on the timestamp of the events (simply using processing time), the > correct order should be b-1, b-2, b-3. -- This message was sent by Atlassian JIRA (v6.3.15#6346)
[jira] [Commented] (FLINK-6772) Incorrect ordering of matched state events in Flink CEP
[ https://issues.apache.org/jira/browse/FLINK-6772?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16029303#comment-16029303 ] Tzu-Li (Gordon) Tai commented on FLINK-6772: So, for example, {code} public class FlinkCEPTest { public static void main(String[] args) throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(2); DataStream input = env.fromElements( "a-1", "a-2", "a-3", "a-4", "b-1", "b-2", "b-3" ); Patternpattern = Pattern .begin("start") .where(new SimpleCondition() { public boolean filter(String s) throws Exception { return s.startsWith("a-"); } }).times(4).allowCombinations() .followedByAny("randomStateName") .where(new SimpleCondition() { public boolean filter(String s) throws Exception { return s.startsWith("b-"); } }).times(3).consecutive(); CEP.pattern(input, pattern).select(new PatternSelectFunction () { public String select(Map pattern) throws Exception { return pattern.toString(); } }).print(); env.execute(); } } {code} The result is: {code} 1> {randomStateName=[b-3, b-1, b-2], start=[a-1, a-2, a-3, a-4]} {code} > Incorrect ordering of matched state events in Flink CEP > --- > > Key: FLINK-6772 > URL: https://issues.apache.org/jira/browse/FLINK-6772 > Project: Flink > Issue Type: Bug > Components: CEP >Reporter: Tzu-Li (Gordon) Tai > > I've stumbled across an unexepected ordering of the matched state events. > Pattern: > {code} > Pattern pattern = Pattern > .begin("start") > .where(new IterativeCondition() { > @Override > public boolean filter(String s, Context context) throws > Exception { > return s.startsWith("a-"); > } > }).times(4).allowCombinations() > .followedByAny("end") > .where(new IterativeCondition() { > public boolean filter(String s, Context context) throws > Exception { > return s.startsWith("b-"); > } > }).times(3).consecutive(); > {code} > Input event sequence: > a-1, a-2, a-3, a-4, b-1, b-2, b-3 > On b-3 a matched pattern would be triggered. > Now, in the {{Map }} map passed via {{select}} in > {{PatternSelectFunction}}, the list for the "end" state is: > b-3, b-1, b-2. > Based on the timestamp of the events (simply using processing time), the > correct order should be b-1, b-2, b-3. -- This message was sent by Atlassian JIRA (v6.3.15#6346)
[jira] [Commented] (FLINK-6772) Incorrect ordering of matched state events in Flink CEP
[ https://issues.apache.org/jira/browse/FLINK-6772?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16029299#comment-16029299 ] Tzu-Li (Gordon) Tai commented on FLINK-6772: Hmm, I've did another check on this, an had a very odd observation: so, when the {{followedByAny}} state is called "end", then it outputs correctly to be {{[b-1, b-2, b-3]}}, but for example when you rename it to be "middle", or "m" (some random data points here), the result ordering is different. Any clue what might be happening here? I know it sounds a bit bizzare :/ The tests are done with on fresh runs, not restored from checkpoint. > Incorrect ordering of matched state events in Flink CEP > --- > > Key: FLINK-6772 > URL: https://issues.apache.org/jira/browse/FLINK-6772 > Project: Flink > Issue Type: Bug > Components: CEP >Reporter: Tzu-Li (Gordon) Tai > > I've stumbled across an unexepected ordering of the matched state events. > Pattern: > {code} > Patternpattern = Pattern > .begin("start") > .where(new IterativeCondition() { > @Override > public boolean filter(String s, Context context) throws > Exception { > return s.startsWith("a-"); > } > }).times(4).allowCombinations() > .followedByAny("end") > .where(new IterativeCondition() { > public boolean filter(String s, Context context) throws > Exception { > return s.startsWith("b-"); > } > }).times(3).consecutive(); > {code} > Input event sequence: > a-1, a-2, a-3, a-4, b-1, b-2, b-3 > On b-3 a matched pattern would be triggered. > Now, in the {{Map }} map passed via {{select}} in > {{PatternSelectFunction}}, the list for the "end" state is: > b-3, b-1, b-2. > Based on the timestamp of the events (simply using processing time), the > correct order should be b-1, b-2, b-3. -- This message was sent by Atlassian JIRA (v6.3.15#6346)
[jira] [Commented] (FLINK-6772) Incorrect ordering of matched state events in Flink CEP
[ https://issues.apache.org/jira/browse/FLINK-6772?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16029288#comment-16029288 ] Dawid Wysakowicz commented on FLINK-6772: - Hmm, I could not reproduce this issue. I tried the following test: {code} @Test public void test() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(2); DataStream input = env.fromElements( "a-1", "a-2", "a-3", "a-4", "b-1", "b-2", "b-3" ); Patternpattern = Pattern .begin("start") .where(new SimpleCondition() { @Override public boolean filter(String s) throws Exception { return s.startsWith("a-"); } }).times(4).allowCombinations() .followedByAny("end") .where(new SimpleCondition() { public boolean filter(String s) throws Exception { return s.startsWith("b-"); } }).times(3).consecutive(); CEP.pattern(input, pattern).select(new PatternSelectFunction () { @Override public String select(Map pattern) throws Exception { return pattern.toString(); } }).print(); env.execute(); } {code} And the results are as follows: {code} 1> {start=[a-1, a-2, a-3, a-4], end=[b-1, b-2, b-3]} {code} > Incorrect ordering of matched state events in Flink CEP > --- > > Key: FLINK-6772 > URL: https://issues.apache.org/jira/browse/FLINK-6772 > Project: Flink > Issue Type: Bug > Components: CEP >Reporter: Tzu-Li (Gordon) Tai > > I've stumbled across an unexepected ordering of the matched state events. > Pattern: > {code} > Pattern pattern = Pattern > .begin("start") > .where(new IterativeCondition() { > @Override > public boolean filter(String s, Context context) throws > Exception { > return s.startsWith("a-"); > } > }).times(4).allowCombinations() > .followedByAny("end") > .where(new IterativeCondition() { > public boolean filter(String s, Context context) throws > Exception { > return s.startsWith("b-"); > } > }).times(3).consecutive(); > {code} > Input event sequence: > a-1, a-2, a-3, a-4, b-1, b-2, b-3 > On b-3 a matched pattern would be triggered. > Now, in the {{Map }} map passed via {{select}} in > {{PatternSelectFunction}}, the list for the "end" state is: > b-3, b-1, b-2. > Based on the timestamp of the events (simply using processing time), the > correct order should be b-1, b-2, b-3. -- This message was sent by Atlassian JIRA (v6.3.15#6346)