Hi Yue, Thanks a lot for your suggestions for this FLIP.
1. PatternProcessorDiscoverer is just an interface, which could have implementations that satisfy different expectations. The example implementation proposed in the FLIP would periodically check a database for updates, but another possible implementation could serve as a HTTP server that only makes updates when a HTTP request is received. This might meet the need to have updates take effect immediately, or preserve computation resources when rarely an update would happen. 2. For each CEP.patternProcess() method there will only be one KeyGenerating and one Pattern Matching & Processing Operator generated, but the subtasks (parallelism) of each of the two operators could be more than one. After a keyBy() operation is applied on the input data, each subtask of the downstream Pattern Matching & Processing Operator only needs to handle part of all possible keys. In this case, the generated key only needs to be an `Object` type. 3. Thanks for reminding me of the implementation. We plan to store the pattern processors and their partially matched results in StateBackend, When a pattern is removed or replaced, we can clean out the partially matched results of that pattern immediately from memory. As for those stored in StateBackend, we can wait until the next checkpoint. 4. It might be hard for me to imagine that there needs to be a different pattern for each rule_id. Could you please illustrate that a bit more? Why can't we solve this problem by increasing the flexibility of patterns, like making an if-else judgment in a pattern's IterativeCondition? Best regards, Yunfeng On Fri, Dec 17, 2021 at 7:43 PM yue ma <mayuefi...@gmail.com> 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 > > >