You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@flink.apache.org by Nicholas <pr...@163.com> on 2022/05/05 06:27:42 UTC

[DISCUSS] FLIP-228: Support Within between events in CEP Pattern

Hi everyone,




Pattern#withIn interface in CEP defines the maximum time interval in which a matching pattern has to be completed in order to be considered valid, which interval corresponds to the maximum time gap between first and the last event. The interval representing the maximum time gap between events is required to define in the scenario like purchasing good within a maximum of 5 minutes after browsing. 




I would like to start a discussion about FLIP-228[1], in which within between events is proposed in Pattern to support the definition of the maximum time interval in which a completed partial matching pattern is considered valid, which interval represents the maximum time gap between events for partial matching Pattern.




Hence we propose the Pattern#partialWithin interface to define the maximum time interval in which a completed partial matching pattern is considered valid. Please take a look at the FLIP page [1] to get more details. Any feedback about the FLIP-228 would be appreciated!




[1] https://cwiki.apache.org/confluence/display/FLINK/FLIP-228%3A+Support+Within+between+events+in+CEP+Pattern




Best regards,

Nicholas Jiang

Re: [DISCUSS] FLIP-228: Support Within between events in CEP Pattern

Posted by Nicholas Jiang <ni...@apache.org>.
Hi Yue,

Thanks for providing the benefit of this feature. After this feature is merged, you are welcome to try this feature in business scenarios.

Regards,
Nicholas Jiang

On 2022/05/06 11:31:48 yue ma wrote:
> hi Nicholas,
> 
> Thanks for bringing this discussion, we also think it's a useful feature.
> Some fine-grained timeout pattern matching  can be implemented in CEP which
> makes Flink CEP more powerful
> 
> Nicholas <pr...@163.com> 于2022年5月5日周四 14:28写道:
> 
> > Hi everyone,
> >
> >
> >
> >
> > Pattern#withIn interface in CEP defines the maximum time interval in which
> > a matching pattern has to be completed in order to be considered valid,
> > which interval corresponds to the maximum time gap between first and the
> > last event. The interval representing the maximum time gap between events
> > is required to define in the scenario like purchasing good within a maximum
> > of 5 minutes after browsing.
> >
> >
> >
> >
> > I would like to start a discussion about FLIP-228[1], in which within
> > between events is proposed in Pattern to support the definition of the
> > maximum time interval in which a completed partial matching pattern is
> > considered valid, which interval represents the maximum time gap between
> > events for partial matching Pattern.
> >
> >
> >
> >
> > Hence we propose the Pattern#partialWithin interface to define the maximum
> > time interval in which a completed partial matching pattern is considered
> > valid. Please take a look at the FLIP page [1] to get more details. Any
> > feedback about the FLIP-228 would be appreciated!
> >
> >
> >
> >
> > [1]
> > https://cwiki.apache.org/confluence/display/FLINK/FLIP-228%3A+Support+Within+between+events+in+CEP+Pattern
> >
> >
> >
> >
> > Best regards,
> >
> > Nicholas Jiang
> 

Re: [DISCUSS] FLIP-228: Support Within between events in CEP Pattern

Posted by Nicholas Jiang <ni...@apache.org>.
Hi Dian,

Thanks for your feedback about supprting the within between events feature. I have updated the FLIP for introduction of 'Pattern#within(WithinType withInType, Time windowTime)' interface. Regarding your comments, I have the following thoughts:

- Regarding the API, the name `partialWithin` sounds a little weird. Is it possible to find a name which is more intuitive?

I have introduced the 'Pattern#within(WithinType withInType, Time windowTime)' interface to cover the interval corresponds to the maximum time gap between events. With this interface, end-user can define the maximum time interval between the first and last event or between the before and after event for the matching pattern. The within(windowTime) can invoke Pattern#within(WithinType withInType, Time windowTime) with FIRST_AND_LAST within type and there is no incompatibility for within(windowTime). From the user's perspective, the interval corresponds to the maximum time gap in the introduced interface is clear and user-friendly so that there is no need to explain the semantics of the few corner cases mentioned above.

- Besides, this FLIP only describes how the newly introduced API will be used, however, it lacks details about how you will implement it.

I have updated the 'Proposed Changes' and explain the concrete implementation from the perspective of NFA compilation and execution. The main implementation includes constructing the mapping between the name of each computing state and the window time in the compilation phase. In the running phase, the latest timestamp of the current computing state needs to be maintained when computing the next computing state, which timestamp is used to check whether the computation state has timed out.

If there are other questions, any feedback is welcome.

Regards,
Nicholas Jiang

On 2022/05/07 09:16:55 Dian Fu wrote:
> Hi Nicholas,
> 
> Thanks a lot for bringing up this discussion. If I recall it correctly,
> this feature has been requested many times by the users and is among one of
> the most requested features in CEP. So big +1 to this feature overall.
> 
> Regarding the API, the name `partialWithin` sounds a little weird. Is it
> possible to find a name which is more intuitive? Other possible solutions:
> - Reuse the existing `Pattern.within` method and change its semantic to the
> maximum time interval between patterns. Currently `Pattern.within` is used
> to define the maximum time interval between the first event and the last
> event. However, the Pattern object represents only one node in a pattern
> sequence and so it doesn't make much sense to define the maximum time
> interval between the first event and the last event on the Pattern object,
> e.g. we could move it to  `PatternStreamBuilder`. However, if we choose
> this option, we'd better consider how to keep backward compatibility.
> - Introduce a series of methods when appending a new pattern to the
> existing one, e.g. `Pattern.followedBy(Pattern<T, F> group, Time
> timeInterval)`. As timeInterval is a property between patterns and so it
> makes sense to define this property when appending a new pattern. However,
> the drawback is that we need to introduce a series of methods instead of
> only one method.
> 
> We need also to make the semantic clear in a few corner cases, e.g.
> - What's the semantic of `A.followedBy(B).times(3).partialWithin(1 min)`?
> Doesn't it mean that all three B events should occur in 1 minute or only
> the first B event should occur in 1 minute?
> - What's the semantic of
> `A.followedBy(GroupPattern.begin("B").followedBy("C")).partialWithin(1
> min)``? Doesn't it mean that B and C should occur after A in 1 minute?
> 
> Besides, this FLIP only describes how the newly introduced API will be
> used, however, it lacks details about how you will implement it. It doesn't
> need to be very detailed, however, you should describe the basic ideas
> behind it, e.g. how will you support A.notFollowedBy(B).partialWithin(1
> min)? It could make sure that you have considered it thoroughly and also
> makes others confident that this feature could be implemented in a clean
> way.
> 
> Regards,
> Dian
> 
> 
> 
> On Fri, May 6, 2022 at 7:32 PM yue ma <ma...@gmail.com> wrote:
> 
> > hi Nicholas,
> >
> > Thanks for bringing this discussion, we also think it's a useful feature.
> > Some fine-grained timeout pattern matching  can be implemented in CEP which
> > makes Flink CEP more powerful
> >
> > Nicholas <pr...@163.com> 于2022年5月5日周四 14:28写道:
> >
> > > Hi everyone,
> > >
> > >
> > >
> > >
> > > Pattern#withIn interface in CEP defines the maximum time interval in
> > which
> > > a matching pattern has to be completed in order to be considered valid,
> > > which interval corresponds to the maximum time gap between first and the
> > > last event. The interval representing the maximum time gap between events
> > > is required to define in the scenario like purchasing good within a
> > maximum
> > > of 5 minutes after browsing.
> > >
> > >
> > >
> > >
> > > I would like to start a discussion about FLIP-228[1], in which within
> > > between events is proposed in Pattern to support the definition of the
> > > maximum time interval in which a completed partial matching pattern is
> > > considered valid, which interval represents the maximum time gap between
> > > events for partial matching Pattern.
> > >
> > >
> > >
> > >
> > > Hence we propose the Pattern#partialWithin interface to define the
> > maximum
> > > time interval in which a completed partial matching pattern is considered
> > > valid. Please take a look at the FLIP page [1] to get more details. Any
> > > feedback about the FLIP-228 would be appreciated!
> > >
> > >
> > >
> > >
> > > [1]
> > >
> > https://cwiki.apache.org/confluence/display/FLINK/FLIP-228%3A+Support+Within+between+events+in+CEP+Pattern
> > >
> > >
> > >
> > >
> > > Best regards,
> > >
> > > Nicholas Jiang
> >
> 

Re: [DISCUSS] FLIP-228: Support Within between events in CEP Pattern

Posted by Nicholas Jiang <ni...@apache.org>.
Hi Dian, Guys,

Thanks for your suggestion for the `PREVIOUS_AND_CURRENT`. I have updated the naming of WithinType value in the FLIP. 

If there are no other questions, I will start the VOTE thread next Monday.

Regards,
Nicholas Jiang

On 2022/06/10 10:02:46 Dian Fu wrote:
> Hi Nicholas,
> 
> Regarding the naming of `WithinType`, I'm OK with it. For
> `PREVIOUS_AND_NEXT`, I guess `PREVIOUS_AND_CURRENT` makes more sense.
> What's your thought?
> 
> Regards,
> Dian
> 
> On Thu, Jun 9, 2022 at 10:09 AM Nicholas Jiang <ni...@apache.org>
> wrote:
> 
> > Hi Dian,
> >
> > About the indication of the time interval between events matched in the
> > loop. I have updated the FLIP and introduced a series of times interface to
> > specify that this pattern can occur the specified times and interval
> > corresponds to the maximum time gap between previous and next event for
> > each times.
> >
> > The within(withinType, windowTime) is used to configure the same time of
> > the matching window for each times, but the times(int times, windowTimes)
> > can configure the different time interval corresponds to the maximum time
> > gap between previous and next event for each times, which is fully
> > considered for time interval between events matched in the loop or times
> > case.
> >
> > Best,
> > Nicholas Jiang
> >
> > On 2022/06/08 08:11:58 Nicholas Jiang wrote:
> > > Hi Dian,
> > >
> > > Thanks for your feedback about the Public Interface update for
> > supporting the within between events feature. I have left the comments for
> > above points:
> > >
> > > - Regarding the pattern API, should we also introduce APIs such as
> > Pattern.times(int from, int to, Time windowTime) to indicate the time
> > interval between events matched in the loop?
> > >
> > > IMO, we could not introduce the mentioned APIs for indication of the
> > time interval between events. For example Pattern.times(int from, int to,
> > Time windowTime), the user can use Pattern.times(int from, int
> > to).within(BEFORE_AND_AFTER, windowTime) to indicate the time interval
> > between the before and after event.
> > >
> > > - Regarding the naming of the classes, does it make sense to rename
> > `WithinType` to `InternalType` or `WindowType`? For the enum values inside
> > it, the current values(`BEFORE_AND_AFTER` and `FIRST_AND_LAST`) are not
> > intuitive for me. The candidates that come to my mind: -
> > `RELATIVE_TO_FIRST` and `RELATIVE_TO_PREVIOUS` - `WHOLE_MATCH` and
> > `RELATIVE_TO_PREVIOUS`
> > >
> > > IMO, the `WithinType` naming could directly the situation for the time
> > interval. In addtion. the enum values of the `WithinType` could update to
> > `PREVIOUS_AND_NEXT` and `FIRST_AND_LAST` which directly indicate the time
> > interval within the PREVIOUS and NEXT event and within the FIRST and LAST
> > event. `RELATIVE_TO_FIRST` and `RELATIVE_TO_PREVIOUS` are not clear to
> > understand which event is relative to FIRST or PREVIOUS event.
> > >
> > > Best,
> > > Nicholas Jiang
> > >
> > > On 2022/06/06 07:48:22 Dian Fu wrote:
> > > > Hi Nicholas,
> > > >
> > > > Thanks a lot for the update.
> > > >
> > > > Regarding the pattern API, should we also introduce APIs such as
> > > > Pattern.times(int from, int to, Time windowTime) to indicate the time
> > > > interval between events matched in the loop?
> > > >
> > > > Regarding the naming of the classes, does it make sense to rename
> > > > `WithinType` to `InternalType` or `WindowType`? For the enum values
> > inside
> > > > it, the current values(`BEFORE_AND_AFTER` and `FIRST_AND_LAST`) are not
> > > > intuitive for me. The candidates that come to my mind:
> > > > - `RELATIVE_TO_FIRST` and `RELATIVE_TO_PREVIOUS`
> > > > - `WHOLE_MATCH` and `RELATIVE_TO_PREVIOUS`
> > > >
> > > > Regards,
> > > > Dian
> > > >
> > > > On Tue, May 31, 2022 at 2:56 PM Nicholas Jiang <
> > nicholasjiang@apache.org>
> > > > wrote:
> > > >
> > > > > Hi Martijn,
> > > > >
> > > > > Sorry for later reply. This feature is only supported in DataStream
> > and
> > > > > doesn't be supported in MATCH_RECOGNIZE because the SQL syntax of
> > > > > MATCH_RECOGNIZE does not contain the semantics of this feature, which
> > > > > requires modification of the SQL syntax. The support above
> > MATCH_RECOGNIZE
> > > > > is suitable for new FLIP to discuss.
> > > > >
> > > > > Regards,
> > > > > Nicholas Jiang
> > > > >
> > > > > On 2022/05/25 11:36:33 Martijn Visser wrote:
> > > > > > Hi Nicholas,
> > > > > >
> > > > > > Thanks for creating the FLIP, I can imagine that there will be
> > many use
> > > > > > cases who can be created using this new feature.
> > > > > >
> > > > > > The FLIP doesn't mention anything with regards to SQL, could this
> > feature
> > > > > > also be supported when using MATCH_RECOGNIZE?
> > > > > >
> > > > > > Best regards,
> > > > > >
> > > > > > Martijn
> > > > > > https://twitter.com/MartijnVisser82
> > > > > > https://github.com/MartijnVisser
> > > > > >
> > > > > >
> > > > > > On Sat, 7 May 2022 at 11:17, Dian Fu <di...@gmail.com>
> > wrote:
> > > > > >
> > > > > > > Hi Nicholas,
> > > > > > >
> > > > > > > Thanks a lot for bringing up this discussion. If I recall it
> > correctly,
> > > > > > > this feature has been requested many times by the users and is
> > among
> > > > > one of
> > > > > > > the most requested features in CEP. So big +1 to this feature
> > overall.
> > > > > > >
> > > > > > > Regarding the API, the name `partialWithin` sounds a little
> > weird. Is
> > > > > it
> > > > > > > possible to find a name which is more intuitive? Other possible
> > > > > solutions:
> > > > > > > - Reuse the existing `Pattern.within` method and change its
> > semantic
> > > > > to the
> > > > > > > maximum time interval between patterns. Currently
> > `Pattern.within` is
> > > > > used
> > > > > > > to define the maximum time interval between the first event and
> > the
> > > > > last
> > > > > > > event. However, the Pattern object represents only one node in a
> > > > > pattern
> > > > > > > sequence and so it doesn't make much sense to define the maximum
> > time
> > > > > > > interval between the first event and the last event on the
> > Pattern
> > > > > object,
> > > > > > > e.g. we could move it to  `PatternStreamBuilder`. However, if we
> > choose
> > > > > > > this option, we'd better consider how to keep backward
> > compatibility.
> > > > > > > - Introduce a series of methods when appending a new pattern to
> > the
> > > > > > > existing one, e.g. `Pattern.followedBy(Pattern<T, F> group, Time
> > > > > > > timeInterval)`. As timeInterval is a property between patterns
> > and so
> > > > > it
> > > > > > > makes sense to define this property when appending a new pattern.
> > > > > However,
> > > > > > > the drawback is that we need to introduce a series of methods
> > instead
> > > > > of
> > > > > > > only one method.
> > > > > > >
> > > > > > > We need also to make the semantic clear in a few corner cases,
> > e.g.
> > > > > > > - What's the semantic of
> > `A.followedBy(B).times(3).partialWithin(1
> > > > > min)`?
> > > > > > > Doesn't it mean that all three B events should occur in 1 minute
> > or
> > > > > only
> > > > > > > the first B event should occur in 1 minute?
> > > > > > > - What's the semantic of
> > > > > > >
> > `A.followedBy(GroupPattern.begin("B").followedBy("C")).partialWithin(1
> > > > > > > min)``? Doesn't it mean that B and C should occur after A in 1
> > minute?
> > > > > > >
> > > > > > > Besides, this FLIP only describes how the newly introduced API
> > will be
> > > > > > > used, however, it lacks details about how you will implement it.
> > It
> > > > > doesn't
> > > > > > > need to be very detailed, however, you should describe the basic
> > ideas
> > > > > > > behind it, e.g. how will you support
> > A.notFollowedBy(B).partialWithin(1
> > > > > > > min)? It could make sure that you have considered it thoroughly
> > and
> > > > > also
> > > > > > > makes others confident that this feature could be implemented in
> > a
> > > > > clean
> > > > > > > way.
> > > > > > >
> > > > > > > Regards,
> > > > > > > Dian
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Fri, May 6, 2022 at 7:32 PM yue ma <ma...@gmail.com>
> > wrote:
> > > > > > >
> > > > > > > > hi Nicholas,
> > > > > > > >
> > > > > > > > Thanks for bringing this discussion, we also think it's a
> > useful
> > > > > feature.
> > > > > > > > Some fine-grained timeout pattern matching  can be implemented
> > in CEP
> > > > > > > which
> > > > > > > > makes Flink CEP more powerful
> > > > > > > >
> > > > > > > > Nicholas <pr...@163.com> 于2022年5月5日周四 14:28写道:
> > > > > > > >
> > > > > > > > > Hi everyone,
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > Pattern#withIn interface in CEP defines the maximum time
> > interval
> > > > > in
> > > > > > > > which
> > > > > > > > > a matching pattern has to be completed in order to be
> > considered
> > > > > valid,
> > > > > > > > > which interval corresponds to the maximum time gap between
> > first
> > > > > and
> > > > > > > the
> > > > > > > > > last event. The interval representing the maximum time gap
> > between
> > > > > > > events
> > > > > > > > > is required to define in the scenario like purchasing good
> > within a
> > > > > > > > maximum
> > > > > > > > > of 5 minutes after browsing.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > I would like to start a discussion about FLIP-228[1], in
> > which
> > > > > within
> > > > > > > > > between events is proposed in Pattern to support the
> > definition of
> > > > > the
> > > > > > > > > maximum time interval in which a completed partial matching
> > > > > pattern is
> > > > > > > > > considered valid, which interval represents the maximum time
> > gap
> > > > > > > between
> > > > > > > > > events for partial matching Pattern.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > Hence we propose the Pattern#partialWithin interface to
> > define the
> > > > > > > > maximum
> > > > > > > > > time interval in which a completed partial matching pattern
> > is
> > > > > > > considered
> > > > > > > > > valid. Please take a look at the FLIP page [1] to get more
> > > > > details. Any
> > > > > > > > > feedback about the FLIP-228 would be appreciated!
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > [1]
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > >
> > https://cwiki.apache.org/confluence/display/FLINK/FLIP-228%3A+Support+Within+between+events+in+CEP+Pattern
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > Best regards,
> > > > > > > > >
> > > > > > > > > Nicholas Jiang
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> 

Re: [DISCUSS] FLIP-228: Support Within between events in CEP Pattern

Posted by Dian Fu <di...@gmail.com>.
Hi Nicholas,

Regarding the naming of `WithinType`, I'm OK with it. For
`PREVIOUS_AND_NEXT`, I guess `PREVIOUS_AND_CURRENT` makes more sense.
What's your thought?

Regards,
Dian

On Thu, Jun 9, 2022 at 10:09 AM Nicholas Jiang <ni...@apache.org>
wrote:

> Hi Dian,
>
> About the indication of the time interval between events matched in the
> loop. I have updated the FLIP and introduced a series of times interface to
> specify that this pattern can occur the specified times and interval
> corresponds to the maximum time gap between previous and next event for
> each times.
>
> The within(withinType, windowTime) is used to configure the same time of
> the matching window for each times, but the times(int times, windowTimes)
> can configure the different time interval corresponds to the maximum time
> gap between previous and next event for each times, which is fully
> considered for time interval between events matched in the loop or times
> case.
>
> Best,
> Nicholas Jiang
>
> On 2022/06/08 08:11:58 Nicholas Jiang wrote:
> > Hi Dian,
> >
> > Thanks for your feedback about the Public Interface update for
> supporting the within between events feature. I have left the comments for
> above points:
> >
> > - Regarding the pattern API, should we also introduce APIs such as
> Pattern.times(int from, int to, Time windowTime) to indicate the time
> interval between events matched in the loop?
> >
> > IMO, we could not introduce the mentioned APIs for indication of the
> time interval between events. For example Pattern.times(int from, int to,
> Time windowTime), the user can use Pattern.times(int from, int
> to).within(BEFORE_AND_AFTER, windowTime) to indicate the time interval
> between the before and after event.
> >
> > - Regarding the naming of the classes, does it make sense to rename
> `WithinType` to `InternalType` or `WindowType`? For the enum values inside
> it, the current values(`BEFORE_AND_AFTER` and `FIRST_AND_LAST`) are not
> intuitive for me. The candidates that come to my mind: -
> `RELATIVE_TO_FIRST` and `RELATIVE_TO_PREVIOUS` - `WHOLE_MATCH` and
> `RELATIVE_TO_PREVIOUS`
> >
> > IMO, the `WithinType` naming could directly the situation for the time
> interval. In addtion. the enum values of the `WithinType` could update to
> `PREVIOUS_AND_NEXT` and `FIRST_AND_LAST` which directly indicate the time
> interval within the PREVIOUS and NEXT event and within the FIRST and LAST
> event. `RELATIVE_TO_FIRST` and `RELATIVE_TO_PREVIOUS` are not clear to
> understand which event is relative to FIRST or PREVIOUS event.
> >
> > Best,
> > Nicholas Jiang
> >
> > On 2022/06/06 07:48:22 Dian Fu wrote:
> > > Hi Nicholas,
> > >
> > > Thanks a lot for the update.
> > >
> > > Regarding the pattern API, should we also introduce APIs such as
> > > Pattern.times(int from, int to, Time windowTime) to indicate the time
> > > interval between events matched in the loop?
> > >
> > > Regarding the naming of the classes, does it make sense to rename
> > > `WithinType` to `InternalType` or `WindowType`? For the enum values
> inside
> > > it, the current values(`BEFORE_AND_AFTER` and `FIRST_AND_LAST`) are not
> > > intuitive for me. The candidates that come to my mind:
> > > - `RELATIVE_TO_FIRST` and `RELATIVE_TO_PREVIOUS`
> > > - `WHOLE_MATCH` and `RELATIVE_TO_PREVIOUS`
> > >
> > > Regards,
> > > Dian
> > >
> > > On Tue, May 31, 2022 at 2:56 PM Nicholas Jiang <
> nicholasjiang@apache.org>
> > > wrote:
> > >
> > > > Hi Martijn,
> > > >
> > > > Sorry for later reply. This feature is only supported in DataStream
> and
> > > > doesn't be supported in MATCH_RECOGNIZE because the SQL syntax of
> > > > MATCH_RECOGNIZE does not contain the semantics of this feature, which
> > > > requires modification of the SQL syntax. The support above
> MATCH_RECOGNIZE
> > > > is suitable for new FLIP to discuss.
> > > >
> > > > Regards,
> > > > Nicholas Jiang
> > > >
> > > > On 2022/05/25 11:36:33 Martijn Visser wrote:
> > > > > Hi Nicholas,
> > > > >
> > > > > Thanks for creating the FLIP, I can imagine that there will be
> many use
> > > > > cases who can be created using this new feature.
> > > > >
> > > > > The FLIP doesn't mention anything with regards to SQL, could this
> feature
> > > > > also be supported when using MATCH_RECOGNIZE?
> > > > >
> > > > > Best regards,
> > > > >
> > > > > Martijn
> > > > > https://twitter.com/MartijnVisser82
> > > > > https://github.com/MartijnVisser
> > > > >
> > > > >
> > > > > On Sat, 7 May 2022 at 11:17, Dian Fu <di...@gmail.com>
> wrote:
> > > > >
> > > > > > Hi Nicholas,
> > > > > >
> > > > > > Thanks a lot for bringing up this discussion. If I recall it
> correctly,
> > > > > > this feature has been requested many times by the users and is
> among
> > > > one of
> > > > > > the most requested features in CEP. So big +1 to this feature
> overall.
> > > > > >
> > > > > > Regarding the API, the name `partialWithin` sounds a little
> weird. Is
> > > > it
> > > > > > possible to find a name which is more intuitive? Other possible
> > > > solutions:
> > > > > > - Reuse the existing `Pattern.within` method and change its
> semantic
> > > > to the
> > > > > > maximum time interval between patterns. Currently
> `Pattern.within` is
> > > > used
> > > > > > to define the maximum time interval between the first event and
> the
> > > > last
> > > > > > event. However, the Pattern object represents only one node in a
> > > > pattern
> > > > > > sequence and so it doesn't make much sense to define the maximum
> time
> > > > > > interval between the first event and the last event on the
> Pattern
> > > > object,
> > > > > > e.g. we could move it to  `PatternStreamBuilder`. However, if we
> choose
> > > > > > this option, we'd better consider how to keep backward
> compatibility.
> > > > > > - Introduce a series of methods when appending a new pattern to
> the
> > > > > > existing one, e.g. `Pattern.followedBy(Pattern<T, F> group, Time
> > > > > > timeInterval)`. As timeInterval is a property between patterns
> and so
> > > > it
> > > > > > makes sense to define this property when appending a new pattern.
> > > > However,
> > > > > > the drawback is that we need to introduce a series of methods
> instead
> > > > of
> > > > > > only one method.
> > > > > >
> > > > > > We need also to make the semantic clear in a few corner cases,
> e.g.
> > > > > > - What's the semantic of
> `A.followedBy(B).times(3).partialWithin(1
> > > > min)`?
> > > > > > Doesn't it mean that all three B events should occur in 1 minute
> or
> > > > only
> > > > > > the first B event should occur in 1 minute?
> > > > > > - What's the semantic of
> > > > > >
> `A.followedBy(GroupPattern.begin("B").followedBy("C")).partialWithin(1
> > > > > > min)``? Doesn't it mean that B and C should occur after A in 1
> minute?
> > > > > >
> > > > > > Besides, this FLIP only describes how the newly introduced API
> will be
> > > > > > used, however, it lacks details about how you will implement it.
> It
> > > > doesn't
> > > > > > need to be very detailed, however, you should describe the basic
> ideas
> > > > > > behind it, e.g. how will you support
> A.notFollowedBy(B).partialWithin(1
> > > > > > min)? It could make sure that you have considered it thoroughly
> and
> > > > also
> > > > > > makes others confident that this feature could be implemented in
> a
> > > > clean
> > > > > > way.
> > > > > >
> > > > > > Regards,
> > > > > > Dian
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Fri, May 6, 2022 at 7:32 PM yue ma <ma...@gmail.com>
> wrote:
> > > > > >
> > > > > > > hi Nicholas,
> > > > > > >
> > > > > > > Thanks for bringing this discussion, we also think it's a
> useful
> > > > feature.
> > > > > > > Some fine-grained timeout pattern matching  can be implemented
> in CEP
> > > > > > which
> > > > > > > makes Flink CEP more powerful
> > > > > > >
> > > > > > > Nicholas <pr...@163.com> 于2022年5月5日周四 14:28写道:
> > > > > > >
> > > > > > > > Hi everyone,
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > Pattern#withIn interface in CEP defines the maximum time
> interval
> > > > in
> > > > > > > which
> > > > > > > > a matching pattern has to be completed in order to be
> considered
> > > > valid,
> > > > > > > > which interval corresponds to the maximum time gap between
> first
> > > > and
> > > > > > the
> > > > > > > > last event. The interval representing the maximum time gap
> between
> > > > > > events
> > > > > > > > is required to define in the scenario like purchasing good
> within a
> > > > > > > maximum
> > > > > > > > of 5 minutes after browsing.
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > I would like to start a discussion about FLIP-228[1], in
> which
> > > > within
> > > > > > > > between events is proposed in Pattern to support the
> definition of
> > > > the
> > > > > > > > maximum time interval in which a completed partial matching
> > > > pattern is
> > > > > > > > considered valid, which interval represents the maximum time
> gap
> > > > > > between
> > > > > > > > events for partial matching Pattern.
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > Hence we propose the Pattern#partialWithin interface to
> define the
> > > > > > > maximum
> > > > > > > > time interval in which a completed partial matching pattern
> is
> > > > > > considered
> > > > > > > > valid. Please take a look at the FLIP page [1] to get more
> > > > details. Any
> > > > > > > > feedback about the FLIP-228 would be appreciated!
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > [1]
> > > > > > > >
> > > > > > >
> > > > > >
> > > >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-228%3A+Support+Within+between+events+in+CEP+Pattern
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > Best regards,
> > > > > > > >
> > > > > > > > Nicholas Jiang
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] FLIP-228: Support Within between events in CEP Pattern

Posted by Nicholas Jiang <ni...@apache.org>.
Hi Dian,

About the indication of the time interval between events matched in the loop. I have updated the FLIP and introduced a series of times interface to specify that this pattern can occur the specified times and interval corresponds to the maximum time gap between previous and next event for each times. 

The within(withinType, windowTime) is used to configure the same time of the matching window for each times, but the times(int times, windowTimes) can configure the different time interval corresponds to the maximum time gap between previous and next event for each times, which is fully considered for time interval between events matched in the loop or times case.

Best,
Nicholas Jiang

On 2022/06/08 08:11:58 Nicholas Jiang wrote:
> Hi Dian,
> 
> Thanks for your feedback about the Public Interface update for supporting the within between events feature. I have left the comments for above points:
> 
> - Regarding the pattern API, should we also introduce APIs such as Pattern.times(int from, int to, Time windowTime) to indicate the time interval between events matched in the loop?
> 
> IMO, we could not introduce the mentioned APIs for indication of the time interval between events. For example Pattern.times(int from, int to, Time windowTime), the user can use Pattern.times(int from, int to).within(BEFORE_AND_AFTER, windowTime) to indicate the time interval between the before and after event.
> 
> - Regarding the naming of the classes, does it make sense to rename `WithinType` to `InternalType` or `WindowType`? For the enum values inside it, the current values(`BEFORE_AND_AFTER` and `FIRST_AND_LAST`) are not intuitive for me. The candidates that come to my mind: - `RELATIVE_TO_FIRST` and `RELATIVE_TO_PREVIOUS` - `WHOLE_MATCH` and `RELATIVE_TO_PREVIOUS`
> 
> IMO, the `WithinType` naming could directly the situation for the time interval. In addtion. the enum values of the `WithinType` could update to `PREVIOUS_AND_NEXT` and `FIRST_AND_LAST` which directly indicate the time interval within the PREVIOUS and NEXT event and within the FIRST and LAST event. `RELATIVE_TO_FIRST` and `RELATIVE_TO_PREVIOUS` are not clear to understand which event is relative to FIRST or PREVIOUS event.
> 
> Best,
> Nicholas Jiang
> 
> On 2022/06/06 07:48:22 Dian Fu wrote:
> > Hi Nicholas,
> > 
> > Thanks a lot for the update.
> > 
> > Regarding the pattern API, should we also introduce APIs such as
> > Pattern.times(int from, int to, Time windowTime) to indicate the time
> > interval between events matched in the loop?
> > 
> > Regarding the naming of the classes, does it make sense to rename
> > `WithinType` to `InternalType` or `WindowType`? For the enum values inside
> > it, the current values(`BEFORE_AND_AFTER` and `FIRST_AND_LAST`) are not
> > intuitive for me. The candidates that come to my mind:
> > - `RELATIVE_TO_FIRST` and `RELATIVE_TO_PREVIOUS`
> > - `WHOLE_MATCH` and `RELATIVE_TO_PREVIOUS`
> > 
> > Regards,
> > Dian
> > 
> > On Tue, May 31, 2022 at 2:56 PM Nicholas Jiang <ni...@apache.org>
> > wrote:
> > 
> > > Hi Martijn,
> > >
> > > Sorry for later reply. This feature is only supported in DataStream and
> > > doesn't be supported in MATCH_RECOGNIZE because the SQL syntax of
> > > MATCH_RECOGNIZE does not contain the semantics of this feature, which
> > > requires modification of the SQL syntax. The support above MATCH_RECOGNIZE
> > > is suitable for new FLIP to discuss.
> > >
> > > Regards,
> > > Nicholas Jiang
> > >
> > > On 2022/05/25 11:36:33 Martijn Visser wrote:
> > > > Hi Nicholas,
> > > >
> > > > Thanks for creating the FLIP, I can imagine that there will be many use
> > > > cases who can be created using this new feature.
> > > >
> > > > The FLIP doesn't mention anything with regards to SQL, could this feature
> > > > also be supported when using MATCH_RECOGNIZE?
> > > >
> > > > Best regards,
> > > >
> > > > Martijn
> > > > https://twitter.com/MartijnVisser82
> > > > https://github.com/MartijnVisser
> > > >
> > > >
> > > > On Sat, 7 May 2022 at 11:17, Dian Fu <di...@gmail.com> wrote:
> > > >
> > > > > Hi Nicholas,
> > > > >
> > > > > Thanks a lot for bringing up this discussion. If I recall it correctly,
> > > > > this feature has been requested many times by the users and is among
> > > one of
> > > > > the most requested features in CEP. So big +1 to this feature overall.
> > > > >
> > > > > Regarding the API, the name `partialWithin` sounds a little weird. Is
> > > it
> > > > > possible to find a name which is more intuitive? Other possible
> > > solutions:
> > > > > - Reuse the existing `Pattern.within` method and change its semantic
> > > to the
> > > > > maximum time interval between patterns. Currently `Pattern.within` is
> > > used
> > > > > to define the maximum time interval between the first event and the
> > > last
> > > > > event. However, the Pattern object represents only one node in a
> > > pattern
> > > > > sequence and so it doesn't make much sense to define the maximum time
> > > > > interval between the first event and the last event on the Pattern
> > > object,
> > > > > e.g. we could move it to  `PatternStreamBuilder`. However, if we choose
> > > > > this option, we'd better consider how to keep backward compatibility.
> > > > > - Introduce a series of methods when appending a new pattern to the
> > > > > existing one, e.g. `Pattern.followedBy(Pattern<T, F> group, Time
> > > > > timeInterval)`. As timeInterval is a property between patterns and so
> > > it
> > > > > makes sense to define this property when appending a new pattern.
> > > However,
> > > > > the drawback is that we need to introduce a series of methods instead
> > > of
> > > > > only one method.
> > > > >
> > > > > We need also to make the semantic clear in a few corner cases, e.g.
> > > > > - What's the semantic of `A.followedBy(B).times(3).partialWithin(1
> > > min)`?
> > > > > Doesn't it mean that all three B events should occur in 1 minute or
> > > only
> > > > > the first B event should occur in 1 minute?
> > > > > - What's the semantic of
> > > > > `A.followedBy(GroupPattern.begin("B").followedBy("C")).partialWithin(1
> > > > > min)``? Doesn't it mean that B and C should occur after A in 1 minute?
> > > > >
> > > > > Besides, this FLIP only describes how the newly introduced API will be
> > > > > used, however, it lacks details about how you will implement it. It
> > > doesn't
> > > > > need to be very detailed, however, you should describe the basic ideas
> > > > > behind it, e.g. how will you support A.notFollowedBy(B).partialWithin(1
> > > > > min)? It could make sure that you have considered it thoroughly and
> > > also
> > > > > makes others confident that this feature could be implemented in a
> > > clean
> > > > > way.
> > > > >
> > > > > Regards,
> > > > > Dian
> > > > >
> > > > >
> > > > >
> > > > > On Fri, May 6, 2022 at 7:32 PM yue ma <ma...@gmail.com> wrote:
> > > > >
> > > > > > hi Nicholas,
> > > > > >
> > > > > > Thanks for bringing this discussion, we also think it's a useful
> > > feature.
> > > > > > Some fine-grained timeout pattern matching  can be implemented in CEP
> > > > > which
> > > > > > makes Flink CEP more powerful
> > > > > >
> > > > > > Nicholas <pr...@163.com> 于2022年5月5日周四 14:28写道:
> > > > > >
> > > > > > > Hi everyone,
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > Pattern#withIn interface in CEP defines the maximum time interval
> > > in
> > > > > > which
> > > > > > > a matching pattern has to be completed in order to be considered
> > > valid,
> > > > > > > which interval corresponds to the maximum time gap between first
> > > and
> > > > > the
> > > > > > > last event. The interval representing the maximum time gap between
> > > > > events
> > > > > > > is required to define in the scenario like purchasing good within a
> > > > > > maximum
> > > > > > > of 5 minutes after browsing.
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > I would like to start a discussion about FLIP-228[1], in which
> > > within
> > > > > > > between events is proposed in Pattern to support the definition of
> > > the
> > > > > > > maximum time interval in which a completed partial matching
> > > pattern is
> > > > > > > considered valid, which interval represents the maximum time gap
> > > > > between
> > > > > > > events for partial matching Pattern.
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > Hence we propose the Pattern#partialWithin interface to define the
> > > > > > maximum
> > > > > > > time interval in which a completed partial matching pattern is
> > > > > considered
> > > > > > > valid. Please take a look at the FLIP page [1] to get more
> > > details. Any
> > > > > > > feedback about the FLIP-228 would be appreciated!
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > [1]
> > > > > > >
> > > > > >
> > > > >
> > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-228%3A+Support+Within+between+events+in+CEP+Pattern
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > Best regards,
> > > > > > >
> > > > > > > Nicholas Jiang
> > > > > >
> > > > >
> > > >
> > >
> > 
> 

Re: [DISCUSS] FLIP-228: Support Within between events in CEP Pattern

Posted by Martijn Visser <ma...@apache.org>.
Hi Nicholas,

Thanks for clarifying the current feature sparsity between DataStream/Table
and SQL on this topic. I think it's an interesting topic for a future
discussion but let's definitely keep it out of scope for this FLIP. It
would be nice to have a follow-up discussion on this in the future :)

Best regards,

Martijn

Op wo 8 jun. 2022 om 10:12 schreef Nicholas Jiang <nicholasjiang@apache.org
>:

> Hi Dian,
>
> Thanks for your feedback about the Public Interface update for supporting
> the within between events feature. I have left the comments for above
> points:
>
> - Regarding the pattern API, should we also introduce APIs such as
> Pattern.times(int from, int to, Time windowTime) to indicate the time
> interval between events matched in the loop?
>
> IMO, we could not introduce the mentioned APIs for indication of the time
> interval between events. For example Pattern.times(int from, int to, Time
> windowTime), the user can use Pattern.times(int from, int
> to).within(BEFORE_AND_AFTER, windowTime) to indicate the time interval
> between the before and after event.
>
> - Regarding the naming of the classes, does it make sense to rename
> `WithinType` to `InternalType` or `WindowType`? For the enum values inside
> it, the current values(`BEFORE_AND_AFTER` and `FIRST_AND_LAST`) are not
> intuitive for me. The candidates that come to my mind: -
> `RELATIVE_TO_FIRST` and `RELATIVE_TO_PREVIOUS` - `WHOLE_MATCH` and
> `RELATIVE_TO_PREVIOUS`
>
> IMO, the `WithinType` naming could directly the situation for the time
> interval. In addtion. the enum values of the `WithinType` could update to
> `PREVIOUS_AND_NEXT` and `FIRST_AND_LAST` which directly indicate the time
> interval within the PREVIOUS and NEXT event and within the FIRST and LAST
> event. `RELATIVE_TO_FIRST` and `RELATIVE_TO_PREVIOUS` are not clear to
> understand which event is relative to FIRST or PREVIOUS event.
>
> Best,
> Nicholas Jiang
>
> On 2022/06/06 07:48:22 Dian Fu wrote:
> > Hi Nicholas,
> >
> > Thanks a lot for the update.
> >
> > Regarding the pattern API, should we also introduce APIs such as
> > Pattern.times(int from, int to, Time windowTime) to indicate the time
> > interval between events matched in the loop?
> >
> > Regarding the naming of the classes, does it make sense to rename
> > `WithinType` to `InternalType` or `WindowType`? For the enum values
> inside
> > it, the current values(`BEFORE_AND_AFTER` and `FIRST_AND_LAST`) are not
> > intuitive for me. The candidates that come to my mind:
> > - `RELATIVE_TO_FIRST` and `RELATIVE_TO_PREVIOUS`
> > - `WHOLE_MATCH` and `RELATIVE_TO_PREVIOUS`
> >
> > Regards,
> > Dian
> >
> > On Tue, May 31, 2022 at 2:56 PM Nicholas Jiang <nicholasjiang@apache.org
> >
> > wrote:
> >
> > > Hi Martijn,
> > >
> > > Sorry for later reply. This feature is only supported in DataStream and
> > > doesn't be supported in MATCH_RECOGNIZE because the SQL syntax of
> > > MATCH_RECOGNIZE does not contain the semantics of this feature, which
> > > requires modification of the SQL syntax. The support above
> MATCH_RECOGNIZE
> > > is suitable for new FLIP to discuss.
> > >
> > > Regards,
> > > Nicholas Jiang
> > >
> > > On 2022/05/25 11:36:33 Martijn Visser wrote:
> > > > Hi Nicholas,
> > > >
> > > > Thanks for creating the FLIP, I can imagine that there will be many
> use
> > > > cases who can be created using this new feature.
> > > >
> > > > The FLIP doesn't mention anything with regards to SQL, could this
> feature
> > > > also be supported when using MATCH_RECOGNIZE?
> > > >
> > > > Best regards,
> > > >
> > > > Martijn
> > > > https://twitter.com/MartijnVisser82
> > > > https://github.com/MartijnVisser
> > > >
> > > >
> > > > On Sat, 7 May 2022 at 11:17, Dian Fu <di...@gmail.com> wrote:
> > > >
> > > > > Hi Nicholas,
> > > > >
> > > > > Thanks a lot for bringing up this discussion. If I recall it
> correctly,
> > > > > this feature has been requested many times by the users and is
> among
> > > one of
> > > > > the most requested features in CEP. So big +1 to this feature
> overall.
> > > > >
> > > > > Regarding the API, the name `partialWithin` sounds a little weird.
> Is
> > > it
> > > > > possible to find a name which is more intuitive? Other possible
> > > solutions:
> > > > > - Reuse the existing `Pattern.within` method and change its
> semantic
> > > to the
> > > > > maximum time interval between patterns. Currently `Pattern.within`
> is
> > > used
> > > > > to define the maximum time interval between the first event and the
> > > last
> > > > > event. However, the Pattern object represents only one node in a
> > > pattern
> > > > > sequence and so it doesn't make much sense to define the maximum
> time
> > > > > interval between the first event and the last event on the Pattern
> > > object,
> > > > > e.g. we could move it to  `PatternStreamBuilder`. However, if we
> choose
> > > > > this option, we'd better consider how to keep backward
> compatibility.
> > > > > - Introduce a series of methods when appending a new pattern to the
> > > > > existing one, e.g. `Pattern.followedBy(Pattern<T, F> group, Time
> > > > > timeInterval)`. As timeInterval is a property between patterns and
> so
> > > it
> > > > > makes sense to define this property when appending a new pattern.
> > > However,
> > > > > the drawback is that we need to introduce a series of methods
> instead
> > > of
> > > > > only one method.
> > > > >
> > > > > We need also to make the semantic clear in a few corner cases, e.g.
> > > > > - What's the semantic of `A.followedBy(B).times(3).partialWithin(1
> > > min)`?
> > > > > Doesn't it mean that all three B events should occur in 1 minute or
> > > only
> > > > > the first B event should occur in 1 minute?
> > > > > - What's the semantic of
> > > > >
> `A.followedBy(GroupPattern.begin("B").followedBy("C")).partialWithin(1
> > > > > min)``? Doesn't it mean that B and C should occur after A in 1
> minute?
> > > > >
> > > > > Besides, this FLIP only describes how the newly introduced API
> will be
> > > > > used, however, it lacks details about how you will implement it. It
> > > doesn't
> > > > > need to be very detailed, however, you should describe the basic
> ideas
> > > > > behind it, e.g. how will you support
> A.notFollowedBy(B).partialWithin(1
> > > > > min)? It could make sure that you have considered it thoroughly and
> > > also
> > > > > makes others confident that this feature could be implemented in a
> > > clean
> > > > > way.
> > > > >
> > > > > Regards,
> > > > > Dian
> > > > >
> > > > >
> > > > >
> > > > > On Fri, May 6, 2022 at 7:32 PM yue ma <ma...@gmail.com>
> wrote:
> > > > >
> > > > > > hi Nicholas,
> > > > > >
> > > > > > Thanks for bringing this discussion, we also think it's a useful
> > > feature.
> > > > > > Some fine-grained timeout pattern matching  can be implemented
> in CEP
> > > > > which
> > > > > > makes Flink CEP more powerful
> > > > > >
> > > > > > Nicholas <pr...@163.com> 于2022年5月5日周四 14:28写道:
> > > > > >
> > > > > > > Hi everyone,
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > Pattern#withIn interface in CEP defines the maximum time
> interval
> > > in
> > > > > > which
> > > > > > > a matching pattern has to be completed in order to be
> considered
> > > valid,
> > > > > > > which interval corresponds to the maximum time gap between
> first
> > > and
> > > > > the
> > > > > > > last event. The interval representing the maximum time gap
> between
> > > > > events
> > > > > > > is required to define in the scenario like purchasing good
> within a
> > > > > > maximum
> > > > > > > of 5 minutes after browsing.
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > I would like to start a discussion about FLIP-228[1], in which
> > > within
> > > > > > > between events is proposed in Pattern to support the
> definition of
> > > the
> > > > > > > maximum time interval in which a completed partial matching
> > > pattern is
> > > > > > > considered valid, which interval represents the maximum time
> gap
> > > > > between
> > > > > > > events for partial matching Pattern.
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > Hence we propose the Pattern#partialWithin interface to define
> the
> > > > > > maximum
> > > > > > > time interval in which a completed partial matching pattern is
> > > > > considered
> > > > > > > valid. Please take a look at the FLIP page [1] to get more
> > > details. Any
> > > > > > > feedback about the FLIP-228 would be appreciated!
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > [1]
> > > > > > >
> > > > > >
> > > > >
> > >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-228%3A+Support+Within+between+events+in+CEP+Pattern
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > Best regards,
> > > > > > >
> > > > > > > Nicholas Jiang
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] FLIP-228: Support Within between events in CEP Pattern

Posted by Nicholas Jiang <ni...@apache.org>.
Hi Dian,

Thanks for your feedback about the Public Interface update for supporting the within between events feature. I have left the comments for above points:

- Regarding the pattern API, should we also introduce APIs such as Pattern.times(int from, int to, Time windowTime) to indicate the time interval between events matched in the loop?

IMO, we could not introduce the mentioned APIs for indication of the time interval between events. For example Pattern.times(int from, int to, Time windowTime), the user can use Pattern.times(int from, int to).within(BEFORE_AND_AFTER, windowTime) to indicate the time interval between the before and after event.

- Regarding the naming of the classes, does it make sense to rename `WithinType` to `InternalType` or `WindowType`? For the enum values inside it, the current values(`BEFORE_AND_AFTER` and `FIRST_AND_LAST`) are not intuitive for me. The candidates that come to my mind: - `RELATIVE_TO_FIRST` and `RELATIVE_TO_PREVIOUS` - `WHOLE_MATCH` and `RELATIVE_TO_PREVIOUS`

IMO, the `WithinType` naming could directly the situation for the time interval. In addtion. the enum values of the `WithinType` could update to `PREVIOUS_AND_NEXT` and `FIRST_AND_LAST` which directly indicate the time interval within the PREVIOUS and NEXT event and within the FIRST and LAST event. `RELATIVE_TO_FIRST` and `RELATIVE_TO_PREVIOUS` are not clear to understand which event is relative to FIRST or PREVIOUS event.

Best,
Nicholas Jiang

On 2022/06/06 07:48:22 Dian Fu wrote:
> Hi Nicholas,
> 
> Thanks a lot for the update.
> 
> Regarding the pattern API, should we also introduce APIs such as
> Pattern.times(int from, int to, Time windowTime) to indicate the time
> interval between events matched in the loop?
> 
> Regarding the naming of the classes, does it make sense to rename
> `WithinType` to `InternalType` or `WindowType`? For the enum values inside
> it, the current values(`BEFORE_AND_AFTER` and `FIRST_AND_LAST`) are not
> intuitive for me. The candidates that come to my mind:
> - `RELATIVE_TO_FIRST` and `RELATIVE_TO_PREVIOUS`
> - `WHOLE_MATCH` and `RELATIVE_TO_PREVIOUS`
> 
> Regards,
> Dian
> 
> On Tue, May 31, 2022 at 2:56 PM Nicholas Jiang <ni...@apache.org>
> wrote:
> 
> > Hi Martijn,
> >
> > Sorry for later reply. This feature is only supported in DataStream and
> > doesn't be supported in MATCH_RECOGNIZE because the SQL syntax of
> > MATCH_RECOGNIZE does not contain the semantics of this feature, which
> > requires modification of the SQL syntax. The support above MATCH_RECOGNIZE
> > is suitable for new FLIP to discuss.
> >
> > Regards,
> > Nicholas Jiang
> >
> > On 2022/05/25 11:36:33 Martijn Visser wrote:
> > > Hi Nicholas,
> > >
> > > Thanks for creating the FLIP, I can imagine that there will be many use
> > > cases who can be created using this new feature.
> > >
> > > The FLIP doesn't mention anything with regards to SQL, could this feature
> > > also be supported when using MATCH_RECOGNIZE?
> > >
> > > Best regards,
> > >
> > > Martijn
> > > https://twitter.com/MartijnVisser82
> > > https://github.com/MartijnVisser
> > >
> > >
> > > On Sat, 7 May 2022 at 11:17, Dian Fu <di...@gmail.com> wrote:
> > >
> > > > Hi Nicholas,
> > > >
> > > > Thanks a lot for bringing up this discussion. If I recall it correctly,
> > > > this feature has been requested many times by the users and is among
> > one of
> > > > the most requested features in CEP. So big +1 to this feature overall.
> > > >
> > > > Regarding the API, the name `partialWithin` sounds a little weird. Is
> > it
> > > > possible to find a name which is more intuitive? Other possible
> > solutions:
> > > > - Reuse the existing `Pattern.within` method and change its semantic
> > to the
> > > > maximum time interval between patterns. Currently `Pattern.within` is
> > used
> > > > to define the maximum time interval between the first event and the
> > last
> > > > event. However, the Pattern object represents only one node in a
> > pattern
> > > > sequence and so it doesn't make much sense to define the maximum time
> > > > interval between the first event and the last event on the Pattern
> > object,
> > > > e.g. we could move it to  `PatternStreamBuilder`. However, if we choose
> > > > this option, we'd better consider how to keep backward compatibility.
> > > > - Introduce a series of methods when appending a new pattern to the
> > > > existing one, e.g. `Pattern.followedBy(Pattern<T, F> group, Time
> > > > timeInterval)`. As timeInterval is a property between patterns and so
> > it
> > > > makes sense to define this property when appending a new pattern.
> > However,
> > > > the drawback is that we need to introduce a series of methods instead
> > of
> > > > only one method.
> > > >
> > > > We need also to make the semantic clear in a few corner cases, e.g.
> > > > - What's the semantic of `A.followedBy(B).times(3).partialWithin(1
> > min)`?
> > > > Doesn't it mean that all three B events should occur in 1 minute or
> > only
> > > > the first B event should occur in 1 minute?
> > > > - What's the semantic of
> > > > `A.followedBy(GroupPattern.begin("B").followedBy("C")).partialWithin(1
> > > > min)``? Doesn't it mean that B and C should occur after A in 1 minute?
> > > >
> > > > Besides, this FLIP only describes how the newly introduced API will be
> > > > used, however, it lacks details about how you will implement it. It
> > doesn't
> > > > need to be very detailed, however, you should describe the basic ideas
> > > > behind it, e.g. how will you support A.notFollowedBy(B).partialWithin(1
> > > > min)? It could make sure that you have considered it thoroughly and
> > also
> > > > makes others confident that this feature could be implemented in a
> > clean
> > > > way.
> > > >
> > > > Regards,
> > > > Dian
> > > >
> > > >
> > > >
> > > > On Fri, May 6, 2022 at 7:32 PM yue ma <ma...@gmail.com> wrote:
> > > >
> > > > > hi Nicholas,
> > > > >
> > > > > Thanks for bringing this discussion, we also think it's a useful
> > feature.
> > > > > Some fine-grained timeout pattern matching  can be implemented in CEP
> > > > which
> > > > > makes Flink CEP more powerful
> > > > >
> > > > > Nicholas <pr...@163.com> 于2022年5月5日周四 14:28写道:
> > > > >
> > > > > > Hi everyone,
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > Pattern#withIn interface in CEP defines the maximum time interval
> > in
> > > > > which
> > > > > > a matching pattern has to be completed in order to be considered
> > valid,
> > > > > > which interval corresponds to the maximum time gap between first
> > and
> > > > the
> > > > > > last event. The interval representing the maximum time gap between
> > > > events
> > > > > > is required to define in the scenario like purchasing good within a
> > > > > maximum
> > > > > > of 5 minutes after browsing.
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > I would like to start a discussion about FLIP-228[1], in which
> > within
> > > > > > between events is proposed in Pattern to support the definition of
> > the
> > > > > > maximum time interval in which a completed partial matching
> > pattern is
> > > > > > considered valid, which interval represents the maximum time gap
> > > > between
> > > > > > events for partial matching Pattern.
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > Hence we propose the Pattern#partialWithin interface to define the
> > > > > maximum
> > > > > > time interval in which a completed partial matching pattern is
> > > > considered
> > > > > > valid. Please take a look at the FLIP page [1] to get more
> > details. Any
> > > > > > feedback about the FLIP-228 would be appreciated!
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > [1]
> > > > > >
> > > > >
> > > >
> > https://cwiki.apache.org/confluence/display/FLINK/FLIP-228%3A+Support+Within+between+events+in+CEP+Pattern
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > Best regards,
> > > > > >
> > > > > > Nicholas Jiang
> > > > >
> > > >
> > >
> >
> 

Re: [DISCUSS] FLIP-228: Support Within between events in CEP Pattern

Posted by Dian Fu <di...@gmail.com>.
Hi Martijn,

There are many features available in the DataStream API of CEP not
supported in SQL, e.g. followedBy, notFollowedBy, followedByAny, etc. The
main reason is that the MATCH_RECOGNIZE clause which comes from SQL
standard doesn't define grammars for these semantics.

There are two ways to handle this:
1) Only support the features defined in MATCH_RECOGNIZE
2) Extend MATCH_RECOGNIZE to align with the other features in the
DataStream API of CEP

Regarding to 2), it means that we need to extend MATCH_RECOGNIZE in
calcite firstly and then support it in Flink. Personally I think it's worth
to do it. However, we need to think about it thoroughly, e.g. which
functionalities should be supported, what the grammar will be, etc. This
definitely deserves a separate FLIP and maybe should be discussed in
calcite community instead of Flink.

Regarding to the scope of DataStream API of CEP VS CEP SQL, I don't think
that SQL should align with all the functionalities provided in DataStream
API. DataStream API is more flexibility and so usually should be more
powerful.

Regards,
Dian


On Tue, Jun 7, 2022 at 8:37 PM Martijn Visser <ma...@apache.org>
wrote:

> Hi Nicholas,
>
> It is disappointing that we can't support this in SQL. I am under the
> impression that currently all CEP capabilities are supported in both
> DataStream/Table API as well as SQL. If that's indeed the case, then I
> would rather have this also fixed for SQL to avoid introducing feature
> sparsity.
>
> Best regards,
>
> Martijn
>
> Op ma 6 jun. 2022 om 09:49 schreef Dian Fu <di...@gmail.com>:
>
> > Hi Nicholas,
> >
> > Thanks a lot for the update.
> >
> > Regarding the pattern API, should we also introduce APIs such as
> > Pattern.times(int from, int to, Time windowTime) to indicate the time
> > interval between events matched in the loop?
> >
> > Regarding the naming of the classes, does it make sense to rename
> > `WithinType` to `InternalType` or `WindowType`? For the enum values
> inside
> > it, the current values(`BEFORE_AND_AFTER` and `FIRST_AND_LAST`) are not
> > intuitive for me. The candidates that come to my mind:
> > - `RELATIVE_TO_FIRST` and `RELATIVE_TO_PREVIOUS`
> > - `WHOLE_MATCH` and `RELATIVE_TO_PREVIOUS`
> >
> > Regards,
> > Dian
> >
> > On Tue, May 31, 2022 at 2:56 PM Nicholas Jiang <nicholasjiang@apache.org
> >
> > wrote:
> >
> > > Hi Martijn,
> > >
> > > Sorry for later reply. This feature is only supported in DataStream and
> > > doesn't be supported in MATCH_RECOGNIZE because the SQL syntax of
> > > MATCH_RECOGNIZE does not contain the semantics of this feature, which
> > > requires modification of the SQL syntax. The support above
> > MATCH_RECOGNIZE
> > > is suitable for new FLIP to discuss.
> > >
> > > Regards,
> > > Nicholas Jiang
> > >
> > > On 2022/05/25 11:36:33 Martijn Visser wrote:
> > > > Hi Nicholas,
> > > >
> > > > Thanks for creating the FLIP, I can imagine that there will be many
> use
> > > > cases who can be created using this new feature.
> > > >
> > > > The FLIP doesn't mention anything with regards to SQL, could this
> > feature
> > > > also be supported when using MATCH_RECOGNIZE?
> > > >
> > > > Best regards,
> > > >
> > > > Martijn
> > > > https://twitter.com/MartijnVisser82
> > > > https://github.com/MartijnVisser
> > > >
> > > >
> > > > On Sat, 7 May 2022 at 11:17, Dian Fu <di...@gmail.com> wrote:
> > > >
> > > > > Hi Nicholas,
> > > > >
> > > > > Thanks a lot for bringing up this discussion. If I recall it
> > correctly,
> > > > > this feature has been requested many times by the users and is
> among
> > > one of
> > > > > the most requested features in CEP. So big +1 to this feature
> > overall.
> > > > >
> > > > > Regarding the API, the name `partialWithin` sounds a little weird.
> Is
> > > it
> > > > > possible to find a name which is more intuitive? Other possible
> > > solutions:
> > > > > - Reuse the existing `Pattern.within` method and change its
> semantic
> > > to the
> > > > > maximum time interval between patterns. Currently `Pattern.within`
> is
> > > used
> > > > > to define the maximum time interval between the first event and the
> > > last
> > > > > event. However, the Pattern object represents only one node in a
> > > pattern
> > > > > sequence and so it doesn't make much sense to define the maximum
> time
> > > > > interval between the first event and the last event on the Pattern
> > > object,
> > > > > e.g. we could move it to  `PatternStreamBuilder`. However, if we
> > choose
> > > > > this option, we'd better consider how to keep backward
> compatibility.
> > > > > - Introduce a series of methods when appending a new pattern to the
> > > > > existing one, e.g. `Pattern.followedBy(Pattern<T, F> group, Time
> > > > > timeInterval)`. As timeInterval is a property between patterns and
> so
> > > it
> > > > > makes sense to define this property when appending a new pattern.
> > > However,
> > > > > the drawback is that we need to introduce a series of methods
> instead
> > > of
> > > > > only one method.
> > > > >
> > > > > We need also to make the semantic clear in a few corner cases, e.g.
> > > > > - What's the semantic of `A.followedBy(B).times(3).partialWithin(1
> > > min)`?
> > > > > Doesn't it mean that all three B events should occur in 1 minute or
> > > only
> > > > > the first B event should occur in 1 minute?
> > > > > - What's the semantic of
> > > > >
> > `A.followedBy(GroupPattern.begin("B").followedBy("C")).partialWithin(1
> > > > > min)``? Doesn't it mean that B and C should occur after A in 1
> > minute?
> > > > >
> > > > > Besides, this FLIP only describes how the newly introduced API will
> > be
> > > > > used, however, it lacks details about how you will implement it. It
> > > doesn't
> > > > > need to be very detailed, however, you should describe the basic
> > ideas
> > > > > behind it, e.g. how will you support
> > A.notFollowedBy(B).partialWithin(1
> > > > > min)? It could make sure that you have considered it thoroughly and
> > > also
> > > > > makes others confident that this feature could be implemented in a
> > > clean
> > > > > way.
> > > > >
> > > > > Regards,
> > > > > Dian
> > > > >
> > > > >
> > > > >
> > > > > On Fri, May 6, 2022 at 7:32 PM yue ma <ma...@gmail.com>
> wrote:
> > > > >
> > > > > > hi Nicholas,
> > > > > >
> > > > > > Thanks for bringing this discussion, we also think it's a useful
> > > feature.
> > > > > > Some fine-grained timeout pattern matching  can be implemented in
> > CEP
> > > > > which
> > > > > > makes Flink CEP more powerful
> > > > > >
> > > > > > Nicholas <pr...@163.com> 于2022年5月5日周四 14:28写道:
> > > > > >
> > > > > > > Hi everyone,
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > Pattern#withIn interface in CEP defines the maximum time
> interval
> > > in
> > > > > > which
> > > > > > > a matching pattern has to be completed in order to be
> considered
> > > valid,
> > > > > > > which interval corresponds to the maximum time gap between
> first
> > > and
> > > > > the
> > > > > > > last event. The interval representing the maximum time gap
> > between
> > > > > events
> > > > > > > is required to define in the scenario like purchasing good
> > within a
> > > > > > maximum
> > > > > > > of 5 minutes after browsing.
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > I would like to start a discussion about FLIP-228[1], in which
> > > within
> > > > > > > between events is proposed in Pattern to support the definition
> > of
> > > the
> > > > > > > maximum time interval in which a completed partial matching
> > > pattern is
> > > > > > > considered valid, which interval represents the maximum time
> gap
> > > > > between
> > > > > > > events for partial matching Pattern.
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > Hence we propose the Pattern#partialWithin interface to define
> > the
> > > > > > maximum
> > > > > > > time interval in which a completed partial matching pattern is
> > > > > considered
> > > > > > > valid. Please take a look at the FLIP page [1] to get more
> > > details. Any
> > > > > > > feedback about the FLIP-228 would be appreciated!
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > [1]
> > > > > > >
> > > > > >
> > > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-228%3A+Support+Within+between+events+in+CEP+Pattern
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > Best regards,
> > > > > > >
> > > > > > > Nicholas Jiang
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] FLIP-228: Support Within between events in CEP Pattern

Posted by Martijn Visser <ma...@apache.org>.
Hi Nicholas,

It is disappointing that we can't support this in SQL. I am under the
impression that currently all CEP capabilities are supported in both
DataStream/Table API as well as SQL. If that's indeed the case, then I
would rather have this also fixed for SQL to avoid introducing feature
sparsity.

Best regards,

Martijn

Op ma 6 jun. 2022 om 09:49 schreef Dian Fu <di...@gmail.com>:

> Hi Nicholas,
>
> Thanks a lot for the update.
>
> Regarding the pattern API, should we also introduce APIs such as
> Pattern.times(int from, int to, Time windowTime) to indicate the time
> interval between events matched in the loop?
>
> Regarding the naming of the classes, does it make sense to rename
> `WithinType` to `InternalType` or `WindowType`? For the enum values inside
> it, the current values(`BEFORE_AND_AFTER` and `FIRST_AND_LAST`) are not
> intuitive for me. The candidates that come to my mind:
> - `RELATIVE_TO_FIRST` and `RELATIVE_TO_PREVIOUS`
> - `WHOLE_MATCH` and `RELATIVE_TO_PREVIOUS`
>
> Regards,
> Dian
>
> On Tue, May 31, 2022 at 2:56 PM Nicholas Jiang <ni...@apache.org>
> wrote:
>
> > Hi Martijn,
> >
> > Sorry for later reply. This feature is only supported in DataStream and
> > doesn't be supported in MATCH_RECOGNIZE because the SQL syntax of
> > MATCH_RECOGNIZE does not contain the semantics of this feature, which
> > requires modification of the SQL syntax. The support above
> MATCH_RECOGNIZE
> > is suitable for new FLIP to discuss.
> >
> > Regards,
> > Nicholas Jiang
> >
> > On 2022/05/25 11:36:33 Martijn Visser wrote:
> > > Hi Nicholas,
> > >
> > > Thanks for creating the FLIP, I can imagine that there will be many use
> > > cases who can be created using this new feature.
> > >
> > > The FLIP doesn't mention anything with regards to SQL, could this
> feature
> > > also be supported when using MATCH_RECOGNIZE?
> > >
> > > Best regards,
> > >
> > > Martijn
> > > https://twitter.com/MartijnVisser82
> > > https://github.com/MartijnVisser
> > >
> > >
> > > On Sat, 7 May 2022 at 11:17, Dian Fu <di...@gmail.com> wrote:
> > >
> > > > Hi Nicholas,
> > > >
> > > > Thanks a lot for bringing up this discussion. If I recall it
> correctly,
> > > > this feature has been requested many times by the users and is among
> > one of
> > > > the most requested features in CEP. So big +1 to this feature
> overall.
> > > >
> > > > Regarding the API, the name `partialWithin` sounds a little weird. Is
> > it
> > > > possible to find a name which is more intuitive? Other possible
> > solutions:
> > > > - Reuse the existing `Pattern.within` method and change its semantic
> > to the
> > > > maximum time interval between patterns. Currently `Pattern.within` is
> > used
> > > > to define the maximum time interval between the first event and the
> > last
> > > > event. However, the Pattern object represents only one node in a
> > pattern
> > > > sequence and so it doesn't make much sense to define the maximum time
> > > > interval between the first event and the last event on the Pattern
> > object,
> > > > e.g. we could move it to  `PatternStreamBuilder`. However, if we
> choose
> > > > this option, we'd better consider how to keep backward compatibility.
> > > > - Introduce a series of methods when appending a new pattern to the
> > > > existing one, e.g. `Pattern.followedBy(Pattern<T, F> group, Time
> > > > timeInterval)`. As timeInterval is a property between patterns and so
> > it
> > > > makes sense to define this property when appending a new pattern.
> > However,
> > > > the drawback is that we need to introduce a series of methods instead
> > of
> > > > only one method.
> > > >
> > > > We need also to make the semantic clear in a few corner cases, e.g.
> > > > - What's the semantic of `A.followedBy(B).times(3).partialWithin(1
> > min)`?
> > > > Doesn't it mean that all three B events should occur in 1 minute or
> > only
> > > > the first B event should occur in 1 minute?
> > > > - What's the semantic of
> > > >
> `A.followedBy(GroupPattern.begin("B").followedBy("C")).partialWithin(1
> > > > min)``? Doesn't it mean that B and C should occur after A in 1
> minute?
> > > >
> > > > Besides, this FLIP only describes how the newly introduced API will
> be
> > > > used, however, it lacks details about how you will implement it. It
> > doesn't
> > > > need to be very detailed, however, you should describe the basic
> ideas
> > > > behind it, e.g. how will you support
> A.notFollowedBy(B).partialWithin(1
> > > > min)? It could make sure that you have considered it thoroughly and
> > also
> > > > makes others confident that this feature could be implemented in a
> > clean
> > > > way.
> > > >
> > > > Regards,
> > > > Dian
> > > >
> > > >
> > > >
> > > > On Fri, May 6, 2022 at 7:32 PM yue ma <ma...@gmail.com> wrote:
> > > >
> > > > > hi Nicholas,
> > > > >
> > > > > Thanks for bringing this discussion, we also think it's a useful
> > feature.
> > > > > Some fine-grained timeout pattern matching  can be implemented in
> CEP
> > > > which
> > > > > makes Flink CEP more powerful
> > > > >
> > > > > Nicholas <pr...@163.com> 于2022年5月5日周四 14:28写道:
> > > > >
> > > > > > Hi everyone,
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > Pattern#withIn interface in CEP defines the maximum time interval
> > in
> > > > > which
> > > > > > a matching pattern has to be completed in order to be considered
> > valid,
> > > > > > which interval corresponds to the maximum time gap between first
> > and
> > > > the
> > > > > > last event. The interval representing the maximum time gap
> between
> > > > events
> > > > > > is required to define in the scenario like purchasing good
> within a
> > > > > maximum
> > > > > > of 5 minutes after browsing.
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > I would like to start a discussion about FLIP-228[1], in which
> > within
> > > > > > between events is proposed in Pattern to support the definition
> of
> > the
> > > > > > maximum time interval in which a completed partial matching
> > pattern is
> > > > > > considered valid, which interval represents the maximum time gap
> > > > between
> > > > > > events for partial matching Pattern.
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > Hence we propose the Pattern#partialWithin interface to define
> the
> > > > > maximum
> > > > > > time interval in which a completed partial matching pattern is
> > > > considered
> > > > > > valid. Please take a look at the FLIP page [1] to get more
> > details. Any
> > > > > > feedback about the FLIP-228 would be appreciated!
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > [1]
> > > > > >
> > > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-228%3A+Support+Within+between+events+in+CEP+Pattern
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > Best regards,
> > > > > >
> > > > > > Nicholas Jiang
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] FLIP-228: Support Within between events in CEP Pattern

Posted by Dian Fu <di...@gmail.com>.
Hi Nicholas,

Thanks a lot for the update.

Regarding the pattern API, should we also introduce APIs such as
Pattern.times(int from, int to, Time windowTime) to indicate the time
interval between events matched in the loop?

Regarding the naming of the classes, does it make sense to rename
`WithinType` to `InternalType` or `WindowType`? For the enum values inside
it, the current values(`BEFORE_AND_AFTER` and `FIRST_AND_LAST`) are not
intuitive for me. The candidates that come to my mind:
- `RELATIVE_TO_FIRST` and `RELATIVE_TO_PREVIOUS`
- `WHOLE_MATCH` and `RELATIVE_TO_PREVIOUS`

Regards,
Dian

On Tue, May 31, 2022 at 2:56 PM Nicholas Jiang <ni...@apache.org>
wrote:

> Hi Martijn,
>
> Sorry for later reply. This feature is only supported in DataStream and
> doesn't be supported in MATCH_RECOGNIZE because the SQL syntax of
> MATCH_RECOGNIZE does not contain the semantics of this feature, which
> requires modification of the SQL syntax. The support above MATCH_RECOGNIZE
> is suitable for new FLIP to discuss.
>
> Regards,
> Nicholas Jiang
>
> On 2022/05/25 11:36:33 Martijn Visser wrote:
> > Hi Nicholas,
> >
> > Thanks for creating the FLIP, I can imagine that there will be many use
> > cases who can be created using this new feature.
> >
> > The FLIP doesn't mention anything with regards to SQL, could this feature
> > also be supported when using MATCH_RECOGNIZE?
> >
> > Best regards,
> >
> > Martijn
> > https://twitter.com/MartijnVisser82
> > https://github.com/MartijnVisser
> >
> >
> > On Sat, 7 May 2022 at 11:17, Dian Fu <di...@gmail.com> wrote:
> >
> > > Hi Nicholas,
> > >
> > > Thanks a lot for bringing up this discussion. If I recall it correctly,
> > > this feature has been requested many times by the users and is among
> one of
> > > the most requested features in CEP. So big +1 to this feature overall.
> > >
> > > Regarding the API, the name `partialWithin` sounds a little weird. Is
> it
> > > possible to find a name which is more intuitive? Other possible
> solutions:
> > > - Reuse the existing `Pattern.within` method and change its semantic
> to the
> > > maximum time interval between patterns. Currently `Pattern.within` is
> used
> > > to define the maximum time interval between the first event and the
> last
> > > event. However, the Pattern object represents only one node in a
> pattern
> > > sequence and so it doesn't make much sense to define the maximum time
> > > interval between the first event and the last event on the Pattern
> object,
> > > e.g. we could move it to  `PatternStreamBuilder`. However, if we choose
> > > this option, we'd better consider how to keep backward compatibility.
> > > - Introduce a series of methods when appending a new pattern to the
> > > existing one, e.g. `Pattern.followedBy(Pattern<T, F> group, Time
> > > timeInterval)`. As timeInterval is a property between patterns and so
> it
> > > makes sense to define this property when appending a new pattern.
> However,
> > > the drawback is that we need to introduce a series of methods instead
> of
> > > only one method.
> > >
> > > We need also to make the semantic clear in a few corner cases, e.g.
> > > - What's the semantic of `A.followedBy(B).times(3).partialWithin(1
> min)`?
> > > Doesn't it mean that all three B events should occur in 1 minute or
> only
> > > the first B event should occur in 1 minute?
> > > - What's the semantic of
> > > `A.followedBy(GroupPattern.begin("B").followedBy("C")).partialWithin(1
> > > min)``? Doesn't it mean that B and C should occur after A in 1 minute?
> > >
> > > Besides, this FLIP only describes how the newly introduced API will be
> > > used, however, it lacks details about how you will implement it. It
> doesn't
> > > need to be very detailed, however, you should describe the basic ideas
> > > behind it, e.g. how will you support A.notFollowedBy(B).partialWithin(1
> > > min)? It could make sure that you have considered it thoroughly and
> also
> > > makes others confident that this feature could be implemented in a
> clean
> > > way.
> > >
> > > Regards,
> > > Dian
> > >
> > >
> > >
> > > On Fri, May 6, 2022 at 7:32 PM yue ma <ma...@gmail.com> wrote:
> > >
> > > > hi Nicholas,
> > > >
> > > > Thanks for bringing this discussion, we also think it's a useful
> feature.
> > > > Some fine-grained timeout pattern matching  can be implemented in CEP
> > > which
> > > > makes Flink CEP more powerful
> > > >
> > > > Nicholas <pr...@163.com> 于2022年5月5日周四 14:28写道:
> > > >
> > > > > Hi everyone,
> > > > >
> > > > >
> > > > >
> > > > >
> > > > > Pattern#withIn interface in CEP defines the maximum time interval
> in
> > > > which
> > > > > a matching pattern has to be completed in order to be considered
> valid,
> > > > > which interval corresponds to the maximum time gap between first
> and
> > > the
> > > > > last event. The interval representing the maximum time gap between
> > > events
> > > > > is required to define in the scenario like purchasing good within a
> > > > maximum
> > > > > of 5 minutes after browsing.
> > > > >
> > > > >
> > > > >
> > > > >
> > > > > I would like to start a discussion about FLIP-228[1], in which
> within
> > > > > between events is proposed in Pattern to support the definition of
> the
> > > > > maximum time interval in which a completed partial matching
> pattern is
> > > > > considered valid, which interval represents the maximum time gap
> > > between
> > > > > events for partial matching Pattern.
> > > > >
> > > > >
> > > > >
> > > > >
> > > > > Hence we propose the Pattern#partialWithin interface to define the
> > > > maximum
> > > > > time interval in which a completed partial matching pattern is
> > > considered
> > > > > valid. Please take a look at the FLIP page [1] to get more
> details. Any
> > > > > feedback about the FLIP-228 would be appreciated!
> > > > >
> > > > >
> > > > >
> > > > >
> > > > > [1]
> > > > >
> > > >
> > >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-228%3A+Support+Within+between+events+in+CEP+Pattern
> > > > >
> > > > >
> > > > >
> > > > >
> > > > > Best regards,
> > > > >
> > > > > Nicholas Jiang
> > > >
> > >
> >
>

Re: [DISCUSS] FLIP-228: Support Within between events in CEP Pattern

Posted by Nicholas Jiang <ni...@apache.org>.
Hi Martijn,

Sorry for later reply. This feature is only supported in DataStream and doesn't be supported in MATCH_RECOGNIZE because the SQL syntax of MATCH_RECOGNIZE does not contain the semantics of this feature, which requires modification of the SQL syntax. The support above MATCH_RECOGNIZE is suitable for new FLIP to discuss.

Regards,
Nicholas Jiang

On 2022/05/25 11:36:33 Martijn Visser wrote:
> Hi Nicholas,
> 
> Thanks for creating the FLIP, I can imagine that there will be many use
> cases who can be created using this new feature.
> 
> The FLIP doesn't mention anything with regards to SQL, could this feature
> also be supported when using MATCH_RECOGNIZE?
> 
> Best regards,
> 
> Martijn
> https://twitter.com/MartijnVisser82
> https://github.com/MartijnVisser
> 
> 
> On Sat, 7 May 2022 at 11:17, Dian Fu <di...@gmail.com> wrote:
> 
> > Hi Nicholas,
> >
> > Thanks a lot for bringing up this discussion. If I recall it correctly,
> > this feature has been requested many times by the users and is among one of
> > the most requested features in CEP. So big +1 to this feature overall.
> >
> > Regarding the API, the name `partialWithin` sounds a little weird. Is it
> > possible to find a name which is more intuitive? Other possible solutions:
> > - Reuse the existing `Pattern.within` method and change its semantic to the
> > maximum time interval between patterns. Currently `Pattern.within` is used
> > to define the maximum time interval between the first event and the last
> > event. However, the Pattern object represents only one node in a pattern
> > sequence and so it doesn't make much sense to define the maximum time
> > interval between the first event and the last event on the Pattern object,
> > e.g. we could move it to  `PatternStreamBuilder`. However, if we choose
> > this option, we'd better consider how to keep backward compatibility.
> > - Introduce a series of methods when appending a new pattern to the
> > existing one, e.g. `Pattern.followedBy(Pattern<T, F> group, Time
> > timeInterval)`. As timeInterval is a property between patterns and so it
> > makes sense to define this property when appending a new pattern. However,
> > the drawback is that we need to introduce a series of methods instead of
> > only one method.
> >
> > We need also to make the semantic clear in a few corner cases, e.g.
> > - What's the semantic of `A.followedBy(B).times(3).partialWithin(1 min)`?
> > Doesn't it mean that all three B events should occur in 1 minute or only
> > the first B event should occur in 1 minute?
> > - What's the semantic of
> > `A.followedBy(GroupPattern.begin("B").followedBy("C")).partialWithin(1
> > min)``? Doesn't it mean that B and C should occur after A in 1 minute?
> >
> > Besides, this FLIP only describes how the newly introduced API will be
> > used, however, it lacks details about how you will implement it. It doesn't
> > need to be very detailed, however, you should describe the basic ideas
> > behind it, e.g. how will you support A.notFollowedBy(B).partialWithin(1
> > min)? It could make sure that you have considered it thoroughly and also
> > makes others confident that this feature could be implemented in a clean
> > way.
> >
> > Regards,
> > Dian
> >
> >
> >
> > On Fri, May 6, 2022 at 7:32 PM yue ma <ma...@gmail.com> wrote:
> >
> > > hi Nicholas,
> > >
> > > Thanks for bringing this discussion, we also think it's a useful feature.
> > > Some fine-grained timeout pattern matching  can be implemented in CEP
> > which
> > > makes Flink CEP more powerful
> > >
> > > Nicholas <pr...@163.com> 于2022年5月5日周四 14:28写道:
> > >
> > > > Hi everyone,
> > > >
> > > >
> > > >
> > > >
> > > > Pattern#withIn interface in CEP defines the maximum time interval in
> > > which
> > > > a matching pattern has to be completed in order to be considered valid,
> > > > which interval corresponds to the maximum time gap between first and
> > the
> > > > last event. The interval representing the maximum time gap between
> > events
> > > > is required to define in the scenario like purchasing good within a
> > > maximum
> > > > of 5 minutes after browsing.
> > > >
> > > >
> > > >
> > > >
> > > > I would like to start a discussion about FLIP-228[1], in which within
> > > > between events is proposed in Pattern to support the definition of the
> > > > maximum time interval in which a completed partial matching pattern is
> > > > considered valid, which interval represents the maximum time gap
> > between
> > > > events for partial matching Pattern.
> > > >
> > > >
> > > >
> > > >
> > > > Hence we propose the Pattern#partialWithin interface to define the
> > > maximum
> > > > time interval in which a completed partial matching pattern is
> > considered
> > > > valid. Please take a look at the FLIP page [1] to get more details. Any
> > > > feedback about the FLIP-228 would be appreciated!
> > > >
> > > >
> > > >
> > > >
> > > > [1]
> > > >
> > >
> > https://cwiki.apache.org/confluence/display/FLINK/FLIP-228%3A+Support+Within+between+events+in+CEP+Pattern
> > > >
> > > >
> > > >
> > > >
> > > > Best regards,
> > > >
> > > > Nicholas Jiang
> > >
> >
> 

Re: [DISCUSS] FLIP-228: Support Within between events in CEP Pattern

Posted by Martijn Visser <ma...@apache.org>.
Hi Nicholas,

Thanks for creating the FLIP, I can imagine that there will be many use
cases who can be created using this new feature.

The FLIP doesn't mention anything with regards to SQL, could this feature
also be supported when using MATCH_RECOGNIZE?

Best regards,

Martijn
https://twitter.com/MartijnVisser82
https://github.com/MartijnVisser


On Sat, 7 May 2022 at 11:17, Dian Fu <di...@gmail.com> wrote:

> Hi Nicholas,
>
> Thanks a lot for bringing up this discussion. If I recall it correctly,
> this feature has been requested many times by the users and is among one of
> the most requested features in CEP. So big +1 to this feature overall.
>
> Regarding the API, the name `partialWithin` sounds a little weird. Is it
> possible to find a name which is more intuitive? Other possible solutions:
> - Reuse the existing `Pattern.within` method and change its semantic to the
> maximum time interval between patterns. Currently `Pattern.within` is used
> to define the maximum time interval between the first event and the last
> event. However, the Pattern object represents only one node in a pattern
> sequence and so it doesn't make much sense to define the maximum time
> interval between the first event and the last event on the Pattern object,
> e.g. we could move it to  `PatternStreamBuilder`. However, if we choose
> this option, we'd better consider how to keep backward compatibility.
> - Introduce a series of methods when appending a new pattern to the
> existing one, e.g. `Pattern.followedBy(Pattern<T, F> group, Time
> timeInterval)`. As timeInterval is a property between patterns and so it
> makes sense to define this property when appending a new pattern. However,
> the drawback is that we need to introduce a series of methods instead of
> only one method.
>
> We need also to make the semantic clear in a few corner cases, e.g.
> - What's the semantic of `A.followedBy(B).times(3).partialWithin(1 min)`?
> Doesn't it mean that all three B events should occur in 1 minute or only
> the first B event should occur in 1 minute?
> - What's the semantic of
> `A.followedBy(GroupPattern.begin("B").followedBy("C")).partialWithin(1
> min)``? Doesn't it mean that B and C should occur after A in 1 minute?
>
> Besides, this FLIP only describes how the newly introduced API will be
> used, however, it lacks details about how you will implement it. It doesn't
> need to be very detailed, however, you should describe the basic ideas
> behind it, e.g. how will you support A.notFollowedBy(B).partialWithin(1
> min)? It could make sure that you have considered it thoroughly and also
> makes others confident that this feature could be implemented in a clean
> way.
>
> Regards,
> Dian
>
>
>
> On Fri, May 6, 2022 at 7:32 PM yue ma <ma...@gmail.com> wrote:
>
> > hi Nicholas,
> >
> > Thanks for bringing this discussion, we also think it's a useful feature.
> > Some fine-grained timeout pattern matching  can be implemented in CEP
> which
> > makes Flink CEP more powerful
> >
> > Nicholas <pr...@163.com> 于2022年5月5日周四 14:28写道:
> >
> > > Hi everyone,
> > >
> > >
> > >
> > >
> > > Pattern#withIn interface in CEP defines the maximum time interval in
> > which
> > > a matching pattern has to be completed in order to be considered valid,
> > > which interval corresponds to the maximum time gap between first and
> the
> > > last event. The interval representing the maximum time gap between
> events
> > > is required to define in the scenario like purchasing good within a
> > maximum
> > > of 5 minutes after browsing.
> > >
> > >
> > >
> > >
> > > I would like to start a discussion about FLIP-228[1], in which within
> > > between events is proposed in Pattern to support the definition of the
> > > maximum time interval in which a completed partial matching pattern is
> > > considered valid, which interval represents the maximum time gap
> between
> > > events for partial matching Pattern.
> > >
> > >
> > >
> > >
> > > Hence we propose the Pattern#partialWithin interface to define the
> > maximum
> > > time interval in which a completed partial matching pattern is
> considered
> > > valid. Please take a look at the FLIP page [1] to get more details. Any
> > > feedback about the FLIP-228 would be appreciated!
> > >
> > >
> > >
> > >
> > > [1]
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-228%3A+Support+Within+between+events+in+CEP+Pattern
> > >
> > >
> > >
> > >
> > > Best regards,
> > >
> > > Nicholas Jiang
> >
>

Re: [DISCUSS] FLIP-228: Support Within between events in CEP Pattern

Posted by Dian Fu <di...@gmail.com>.
Hi Nicholas,

Thanks a lot for bringing up this discussion. If I recall it correctly,
this feature has been requested many times by the users and is among one of
the most requested features in CEP. So big +1 to this feature overall.

Regarding the API, the name `partialWithin` sounds a little weird. Is it
possible to find a name which is more intuitive? Other possible solutions:
- Reuse the existing `Pattern.within` method and change its semantic to the
maximum time interval between patterns. Currently `Pattern.within` is used
to define the maximum time interval between the first event and the last
event. However, the Pattern object represents only one node in a pattern
sequence and so it doesn't make much sense to define the maximum time
interval between the first event and the last event on the Pattern object,
e.g. we could move it to  `PatternStreamBuilder`. However, if we choose
this option, we'd better consider how to keep backward compatibility.
- Introduce a series of methods when appending a new pattern to the
existing one, e.g. `Pattern.followedBy(Pattern<T, F> group, Time
timeInterval)`. As timeInterval is a property between patterns and so it
makes sense to define this property when appending a new pattern. However,
the drawback is that we need to introduce a series of methods instead of
only one method.

We need also to make the semantic clear in a few corner cases, e.g.
- What's the semantic of `A.followedBy(B).times(3).partialWithin(1 min)`?
Doesn't it mean that all three B events should occur in 1 minute or only
the first B event should occur in 1 minute?
- What's the semantic of
`A.followedBy(GroupPattern.begin("B").followedBy("C")).partialWithin(1
min)``? Doesn't it mean that B and C should occur after A in 1 minute?

Besides, this FLIP only describes how the newly introduced API will be
used, however, it lacks details about how you will implement it. It doesn't
need to be very detailed, however, you should describe the basic ideas
behind it, e.g. how will you support A.notFollowedBy(B).partialWithin(1
min)? It could make sure that you have considered it thoroughly and also
makes others confident that this feature could be implemented in a clean
way.

Regards,
Dian



On Fri, May 6, 2022 at 7:32 PM yue ma <ma...@gmail.com> wrote:

> hi Nicholas,
>
> Thanks for bringing this discussion, we also think it's a useful feature.
> Some fine-grained timeout pattern matching  can be implemented in CEP which
> makes Flink CEP more powerful
>
> Nicholas <pr...@163.com> 于2022年5月5日周四 14:28写道:
>
> > Hi everyone,
> >
> >
> >
> >
> > Pattern#withIn interface in CEP defines the maximum time interval in
> which
> > a matching pattern has to be completed in order to be considered valid,
> > which interval corresponds to the maximum time gap between first and the
> > last event. The interval representing the maximum time gap between events
> > is required to define in the scenario like purchasing good within a
> maximum
> > of 5 minutes after browsing.
> >
> >
> >
> >
> > I would like to start a discussion about FLIP-228[1], in which within
> > between events is proposed in Pattern to support the definition of the
> > maximum time interval in which a completed partial matching pattern is
> > considered valid, which interval represents the maximum time gap between
> > events for partial matching Pattern.
> >
> >
> >
> >
> > Hence we propose the Pattern#partialWithin interface to define the
> maximum
> > time interval in which a completed partial matching pattern is considered
> > valid. Please take a look at the FLIP page [1] to get more details. Any
> > feedback about the FLIP-228 would be appreciated!
> >
> >
> >
> >
> > [1]
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-228%3A+Support+Within+between+events+in+CEP+Pattern
> >
> >
> >
> >
> > Best regards,
> >
> > Nicholas Jiang
>

Re: [DISCUSS] FLIP-228: Support Within between events in CEP Pattern

Posted by yue ma <ma...@gmail.com>.
hi Nicholas,

Thanks for bringing this discussion, we also think it's a useful feature.
Some fine-grained timeout pattern matching  can be implemented in CEP which
makes Flink CEP more powerful

Nicholas <pr...@163.com> 于2022年5月5日周四 14:28写道:

> Hi everyone,
>
>
>
>
> Pattern#withIn interface in CEP defines the maximum time interval in which
> a matching pattern has to be completed in order to be considered valid,
> which interval corresponds to the maximum time gap between first and the
> last event. The interval representing the maximum time gap between events
> is required to define in the scenario like purchasing good within a maximum
> of 5 minutes after browsing.
>
>
>
>
> I would like to start a discussion about FLIP-228[1], in which within
> between events is proposed in Pattern to support the definition of the
> maximum time interval in which a completed partial matching pattern is
> considered valid, which interval represents the maximum time gap between
> events for partial matching Pattern.
>
>
>
>
> Hence we propose the Pattern#partialWithin interface to define the maximum
> time interval in which a completed partial matching pattern is considered
> valid. Please take a look at the FLIP page [1] to get more details. Any
> feedback about the FLIP-228 would be appreciated!
>
>
>
>
> [1]
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-228%3A+Support+Within+between+events+in+CEP+Pattern
>
>
>
>
> Best regards,
>
> Nicholas Jiang

Re: [DISCUSS] FLIP-228: Support Within between events in CEP Pattern

Posted by yue ma <ma...@gmail.com>.
hi Nicholas ,


Nicholas <pr...@163.com> 于2022年5月5日周四 14:28写道:

> Hi everyone,
>
>
>
>
> Pattern#withIn interface in CEP defines the maximum time interval in which
> a matching pattern has to be completed in order to be considered valid,
> which interval corresponds to the maximum time gap between first and the
> last event. The interval representing the maximum time gap between events
> is required to define in the scenario like purchasing good within a maximum
> of 5 minutes after browsing.
>
>
>
>
> I would like to start a discussion about FLIP-228[1], in which within
> between events is proposed in Pattern to support the definition of the
> maximum time interval in which a completed partial matching pattern is
> considered valid, which interval represents the maximum time gap between
> events for partial matching Pattern.
>
>
>
>
> Hence we propose the Pattern#partialWithin interface to define the maximum
> time interval in which a completed partial matching pattern is considered
> valid. Please take a look at the FLIP page [1] to get more details. Any
> feedback about the FLIP-228 would be appreciated!
>
>
>
>
> [1]
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-228%3A+Support+Within+between+events+in+CEP+Pattern
>
>
>
>
> Best regards,
>
> Nicholas Jiang