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 <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