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 <nicholasji...@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 <dian0511...@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 <mayuefi...@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 <programg...@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
> > > > > >
> > > > >
> > > >
> > >
> > 
> 

Reply via email to