Hi Yue,

Thanks for your feedback of the FLIP. I have addressed your questions and made 
a corresponding explanation as follows:

-- About Pattern Updating. If we use PatternProcessoerDiscoverer to update the 
rules, will it increase the load of JM? For example, if the user wants the 
updated rule to take effect immediately,, which means that we need to set a 
shorter check interval or there is another scenario when users rarely update 
the pattern, will the PatternProcessoerDiscoverer be in most of the time Do 
useless checks ? Will a lazy update mode could be used, which the pattern only 
be updated when triggered by the user, and do nothing at other times?

PatternProcessoerDiscoverer is a user-defined interface to discover the 
PatternProcessor updates. Periodically checking the PatternProcessor in the 
database is a implementation of the PatternProcessoerDiscoverer interface, 
which is that periodically querys all the PatternProcessor table in certain 
interval. This implementation indeeds has the useless checks, and could 
directly integrates the changelog of the table. In addition, in addition to the 
implementation of periodically checking the database, there are other 
implementations such as the PatternProcessor that provides Restful services to 
receive updates. 

--  I still have some confusion about how Key Generating Opertator and 
CepOperator (Pattern Matching & Processing Operator) work together. If there 
are N PatternProcessors, will the Key Generating Opertator generate N 
keyedStreams, and then N CepOperator would process each Key separately ? Or 
every CepOperator Task would process all patterns, if so, does the key type in 
each PatternProcessor need to be the same?

Firstly the Pattern Matching & Processing Operator is not the CepOperator at 
present, because CepOperator mechanism is based on the NFAState. Secondly if 
there are N PatternProcessors, the Key Generating Opertator combines all the 
keyedStreams with keyBy() operation, thus the Pattern Matching & Processing 
Operator would process all the patterns. In other words, the KeySelector of the 
PatternProcessor is used for the Key Generating Opertator, and the Pattern and 
PatternProceessFunction of the PatternProcessor are used for the Pattern 
Matching & Processing Operator. Lastly the key type in each PatternProcessor is 
the same, regarded as Object type. 

-- Maybe need to pay attention to it when implementing it .If some Pattern has 
been removed or updated, will the partially matched results in StateBackend 
would be clean up or We rely on state ttl to clean up these expired states.

If certain Pattern has been removed or updated, the partially matched results 
in StateBackend would be clean up until the next checkpoint. The partially 
matched result doesn't depend on the state ttl of the StateBackend. 

4. Will the PatternProcessorManager keep all the active PatternProcessor in 
memory? We have also Support Multiple Rule and Dynamic Rule Changing. But we 
are facing such a problem, some users’ usage scenarios are that they want to 
have their own pattern for each user_id, which means that there could be 
thousands of patterns, which would make the performance of Pattern Matching 
very poor. We are also trying to solve this problem.

The PatternProcessorManager keeps all the active PatternProcessor in memory. 
For scenarios that they want to have their own pattern for each user_id, IMO, 
is it possible to reduce the fine-grained pattern of PatternProcessor to solve 
the performance problem of the Pattern Matching, for example, a pattern 
corresponds to a group of users? The scenarios mentioned above need to be 
solved by case by case.

Best,
Nicholas Jiang

On 2021/12/17 11:43:10 yue ma wrote:
> Glad to see the Community's progress in Flink CEP. After reading this Flip,
> I have few questions, would you please take a look  ?
> 
> 1. About Pattern Updating. If we use PatternProcessoerDiscoverer to update
> the rules, will it increase the load of JM? For example, if the user wants
> the updated rule to take effect immediately,, which means that we need to
> set a shorter check interval  or there is another scenario when users
> rarely update the pattern, will the PatternProcessoerDiscoverer be in most
> of the time Do useless checks ? Will a lazy update mode could be used,
> which the pattern only be updated when triggered by the user, and do
> nothing at other times ?
> 
> 2.   I still have some confusion about how Key Generating Opertator and
> CepOperator (Pattern Matching & Processing Operator) work together. If
> there are N PatternProcessors, will the Key Generating Opertator generate N
> keyedStreams, and then N CepOperator would process each Key separately ? Or
> every CepOperator Task would process all patterns, if so, does the key type
> in each PatternProcessor need to be the same ?
> 
> 3. Maybe need to pay attention to it when implementing it .If some Pattern
> has been removed or updateed  ,will the partially matched results in
> StateBackend would be clean up or We rely on state ttl to clean up these
> expired states.
> 
> 4. Will the PatternProcessorManager keep all the active PatternProcessor in
> memory ? We have also Support Multiple Rule and Dynamic Rule Changing .
> But we are facing such a problem, some users’ usage scenarios are that they
> want to have their own pattern for each user_id, which means that there
> could be thousands of patterns, which would make the performance of Pattern
> Matching very poor. We are also trying to solve this problem.
> 
> Yunfeng Zhou <flink.zhouyunf...@gmail.com> 于2021年12月10日周五 19:16写道:
> 
> > Hi all,
> >
> > I'm opening this thread to propose the design to support multiple rule &
> > dynamic rule changing in the Flink-CEP project, as described in FLIP-200
> > [1]
> > .
> >
> > Currently Flink CEP only supports having a single pattern inside a
> > CepOperator and does not support changing the pattern dynamically. In order
> > to reduce resource consumption and to experience shorter downtime during
> > pattern updates, there is a growing need in the production environment that
> > expects CEP to support having multiple patterns in one operator and to
> > support dynamically changing them. Therefore I propose to add certain
> > infrastructure as described in FLIP-200 to support these functionalities.
> >
> > Please feel free to reply to this email thread. Looking forward to your
> > feedback!
> >
> > [1]
> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=195730308
> >
> > Best regards,
> >
> > Yunfeng
> >
> 

Reply via email to