You are viewing an old version of this page. View the current version.

Compare with Current View Page History

« Previous Version 7 Next »

Status

Current state: "Under Discussion"

Discussion thread: https://lists.apache.org/thread/bv37lnpnb75cc3x0ljf49vm3flhy3c3y

JIRAhere (<- link to https://issues.apache.org/jira/browse/FLINK-XXXX)

Released: <Flink Version>

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

In situations like Risk Controlling or Pattern Matching, it is common that users want to change the pattern that events need to match while the pattern is still functioning to provide service. In current Flink CEP, a CEP operator has a fixed CEP pattern and does not support changing it. Thus in order to achieve the goal described above, users have to restart flink jobs and wait relatively long update time.

Another common situation is that one stream of events need to be matched to multiple patterns. While the current Flink CEP does not support multiple pattern in one CEP operator, users have to setup one Flink job or one operator for each pattern they have. This can be a waste of memory and computation resources.

In this FLIP, we propose to add a couple of APIs and classes to Flink CEP in order to support having multiple patterns in one operator and updating patterns dynamically without stopping Flink jobs.

Public Interfaces

We propose to make the following API changes to support dynamic pattern changing in CEP.

Add Rule Interface

The term "rule" defines a certain pattern, how to match the pattern, and how to process the found matches. A rule extends beyond a pattern to also include all the other relevant functions. A brief illustration to their relationship is as below.

The starting point of introducing "Rule" is trying to deal with situations like follows:

  • While one pattern requires input data to be grouped by user_id , another pattern might require data to be grouped by item_id .
  • Users might want to distinguish results that match different patterns and deal with them differently.

Simply having multiple patterns might not be able to deal with situations like above, and the concept Rule  can handle them properly.


We propose to add Rule interface as below. The properties of the rule are required by CEP operator in order to function correctly.

/**
 * Base class for a rule definition.
 *
 * <p>A rule defines a {@link Pattern}, how to match the pattern, and how to process the found
 * matches.
 *
 * @param <IN> Base type of the elements appearing in the pattern.
 * @param <OUT> Type of produced elements based on found matches.
 */
@PublicEvolving
public interface Rule<IN, OUT> extends Serializable, Versioned {
    /**
     * Returns the ID of the rule.
     *
     * @return The ID of the rule.
     */
    String getId();

    /**
     * Returns the scheduled time at which the rule should come into effective.
     *
     * <p>If the scheduled time is earlier than current event/processing time, the rule will
     * immediately become effective.
     *
     * <p>If the rule should always become effective immediately, the scheduled time can be set to
     * {@code Long.MIN_VALUE}: {@value Long#MIN_VALUE}.
     *
     * @return The scheduled time.
     */
    default Long getScheduledTime() {
        return Long.MIN_VALUE;
    }

    /**
     * Returns the {@link Pattern} to be matched.
     *
     * @return The pattern of the rule.
     */
    Pattern<IN, ?> getPattern();

    /**
     * Returns the {@link KeySelector} to extract the key of the elements appearing in the pattern.
     *
     * <p>Only data with the same key can be chained to match a pattern. If extracted key is null,
     * the behavior is to reuse current key if is {@link KeyedStream}, or allocate the same key for
     * all input data if is not {@link KeyedStream}.
     *
     * @return The key selector of the rule.
     */
    @Nullable
    KeySelector<IN, ?> getKeySelector();

    /**
     * Get the {@link PatternProcessFunction} to process the found matches for the pattern.
     *
     * @return The pattern process function of the rule.
     */
    PatternProcessFunction<IN, OUT> getPatternProcessFunction();
}

Given the needs of data exchange and state storage, the Rule must be serializable, which requires that the classes involved in the Rule must also be serializable, for example, make Pattern implement Serializable.

Add RuleManager Interface

We propose to introduce the RuleManager interface. The manager handles the rule updates and provides information about current active rules.

Its possible implementation includes components in Flink job that need to receive rule update notifications, like CepOperator or its OperatorCoordinator. This means that this interface will only have internal implementations. Users only need to use this interface, not to inherit or create new instance of child classes of this interface.

/**
 * This manager handles updated rules and manages current rules.
 *
 * @param <IN> Base type of the elements appearing in the pattern.
 * @param <OUT> Type of produced elements based on found matches.
 */
@PublicEvolving
public interface RuleManager<IN, OUT> {
    /**
     * Deals with the notification that rules are updated.
     *
     * @param rules A list of all latest rules.
     */
    void onRulesUpdated(List<Rule<IN, OUT>> rules);

    /**
     * Returns the current rules managed.
     *
     * @return The current rules.
     */
    List<Rule<IN, OUT>> getCurrentRules();
}

Add RuleDiscoverer Interface

We introduce the RuleDiscoverer interface, which discovers the rule changes and notifies the RuleManager of the collected rule updates. This interface also serves to provide initial rules.

/**
 * Interface that discovers rule changes, notifies {@link RuleManager} of rule updates and provides
 * the initial rules.
 *
 * @param <IN> Base type of the elements appearing in the pattern.
 * @param <OUT> Type of produced elements based on found matches.
 */
@PublicEvolving
public interface RuleDiscoverer<IN, OUT> {
    /**
     * Discover the rule changes.
     *
     * <p>In dynamic rule changing case, this function should be a continuous process to check rule
     * updates and notify the {@link RuleManager}.
     */
    void discoverRuleChanges(RuleManager<IN, OUT> manager);

    /**
     * Returns the rules an operator should be set up with.
     *
     * @return The initial rules.
     */
    List<Rule<IN, OUT>> getInitialRules();
}

Users should be responsible for creating child classes for this interface, as the possible ways to collect rules are unlimited. Still, we can provide several abstract class that can handle most use cases, like the following AbstractPeriodicRuleDiscoverer that periodically checks for updates.

/** Class that periodically checks for rule updates. */
public abstract class AbstractPeriodicRuleDiscoverer<IN, OUT> implements RuleDiscoverer<IN, OUT> {
    private final List<Rule<IN, OUT>> initRules;
    private final long intervalMillis;

    /**
     * Constructor.
     *
     * @param intervalMillis Time interval in milliseconds at which to check updates.
     */
    public AbstractPeriodicRuleDiscoverer(List<Rule<IN, OUT>> initRules, long intervalMillis) {
        this.initRules = initRules;
        this.intervalMillis = intervalMillis;
    }

    /** Checks whether there is rule update. */
    public abstract boolean checkRuleUpdate();

    /** Returns the latest rules. */
    public abstract List<Rule<IN, OUT>> getLatestRules();

    @Override
    public void discoverRuleChanges(RuleManager<IN, OUT> ruleManager) {
        while (true) {
            if (checkRuleUpdate()) {
                List<Rule<IN, OUT>> rules = getLatestRules();
                ruleManager.onRulesUpdated(rules);
            }
            try {
                Thread.sleep(intervalMillis);
            } catch (InterruptedException e) {
                e.printStackTrace();
                break;
            }
        }
    }

    @Override
    public List<Rule<IN, OUT>> getInitialRules() {
        return initRules;
    }
}

Add RuleDiscovererFactory Interface

The RuleDiscovererFactory is introduced to create the RuleDiscoverer with the RuleManager, that requires RuleDiscoverer to notify RuleManager to deal with the rule changes.

/**
 * A factory for {@link RuleDiscoverer}. The created {@link RuleDiscoverer} should notify {@link
 * RuleManager} to deal with the rule changes.
 *
 * @param <IN> Base type of the elements appearing in the pattern.
 * @param <OUT> Type of produced elements based on found matches.
 */
public interface RuleDiscovererFactory<IN, OUT> extends Serializable {
    /**
     * Creates a {@link RuleDiscoverer}.
     *
     * @return A {@link RuleDiscoverer} instance.
     */
    RuleDiscoverer<IN, OUT> createRuleDiscoverer();
}

Add CEP.rule() method

CEP adds the rule() method to apply the dynamic rules onto the input DataStream and create new DataStream containing the processed results of matched CEP rules.

public class CEP {
    /**
     * Creates a {@link DataStream} containing the processed results of matched CEP rules.
     *
     * @param input DataStream containing the input events
     * @param factory Rule discoverer factory to create the {@link RuleDiscoverer}
     * @param <IN> type of the elements appearing in the pattern
     * @param <OUT> type of produced elements based on found matches
     * @return Resulting data stream
     */
    public static <IN, OUT> DataStream<OUT> rule(
            DataStream<IN> input, RuleDiscovererFactory<IN, OUT> factory) {...}
}

Example Usage

This section provides code snippets that shows how users can use the APIs proposed above to change CEP rules dynamically.

Suppose the CEP library provided an implementation of RuleDiscoverer, called PeriodDBRuleDiscoverer, that periodically checks a database for rule updates and initializes rules from a fixed rule set according to the json.

/**
 * Implementation of {@link RuleDiscoverer} that periodically checks a database for rule updates and
 * initializes the {@link Rule} from a fixed rule set according to information retrieved from the database.
 */
public class PeriodDBRuleDiscoverer<IN, OUT> implements RuleDiscoverer<IN, OUT> {

    @Override
    public void discoverRuleChanges() {...}

    @Override
    public List<Rule<IN, OUT>> getInitialRules() {...}
}

To start with, users need to provide classes that implement Rule interface. Suppose some class called DummyRule is created like below.

public class DummyRule implements Rule<Event, Object> {
    
    public DummyRule(int a) {...}

    @Override
    public String getId() {...}
    
    @Override
    public int getVersion() {...}

    @Override
    public Pattern<Event, ?> getPattern() {...}

    @Nullable
    @Override
    public KeySelector<Event, ?> getKeySelector() {...}

    @Override
    public PatternProcessFunction<Event, Object> getPatternProcessFunction() {...}
}

It is recommended that users override the implementations' toString(), equals(), and hashCode() methods. Overriding toString() provides better readability when monitoring the rules through logs, and overriding equals() and hashCode() can help to avoid reloading existing rules during a rule update.

In order to use PeriodDBRuleDiscoverer, users need to create an instance of PeriodDBRuleDiscovererFactory.

PeriodDBRuleDiscovererFactory<Event, Object> factory = new PeriodDBRuleDiscovererFactory();

Then users create DataStream with input data stream and this factory.

DataStream<Event> inputStream = env.fromElements(...);

DataStream<Object> outputStream = CEP.rule(inputStream, factory);

After Flink job has been configured to support dynamic rule and successfully submitted, users can update rules by configuring rules in their database, recording them in a table as follows.

uid

className

constructorParams

1

"org.apache.flink.cep.rule.DummyRule"

[{"type": "int", "value": 0}]

2

"org.apache.flink.cep.rule.DummyRule"

[{"type": "int", "value": 1}]

Information like rule update or exception will be recorded in Flink's log system. Users can check the log to see, for example, whether a rule update succeeded or not.

The general process of the example described in this section and some general implementation idea is shown in the wireframe below. Operator Coordinator on JobManager will read rule updates from users' database and send rule updates to subtasks, and as a result the matched & processed results produced by subtasks are influenced.


Proposed Changes

The general structure that supports dynamic rules in a job graph is shown in the wireframe below. When an input event data comes from upstream operator, it goes through the following process to produce a matched & processed record.

  1. Key Generating Operator computes the key of the input data according to the KeySelector provided by Rules, and attach the key along with the input data. If there are N rules, then there will be N records with the same input data and different keys sent downstream.
  2. Input data is sent to different downstream operator subtasks by calling keyBy() operation on them. keyBy() uses the key generated in the previous step.
  3. The next operator matches the input data to Patterns provided by Rules, and when a matched sequence is generated, the operator calls the corresponding PatternProcessFunction on that sequence and produce the result to downstream.



In the wireframe above, Key Generating Operator and Pattern Matching & Processing Operator uses information from the rules and thus need a RuleDiscoverer. Our design of collecting rules and update them inside operators is shown in the wireframe below. In each of the two operators mentioned, there will be rule managers inside each of their Operator Coordinators. The manager in the OperatorCoordinator serves to discover rule changes and convert the information into rules. Then the subtasks, after receiving the latest rules from Operator Coordinator, will make the update.


Supporting multiple & dynamic rules inside CEP operators

Currently CepOperator uses NFAState and NFA to deal with input data. As it has not supported multiple patterns yet, all input data are dealt with by the same NFAState.

In order to support multiple patterns in one operator, the one NFA needs to be replaced by a list or map of NFA, each corresponds to a Rule. When input data triggers the processElement() method, the data will be matched against all rules in the operator. Adding or deleting rules means adding or deleting entries in the list or map.

Scheduling rules

In order to make the behavior of rule updates more deterministic, users can schedule the time at which an updated rule should come into effect, which is reflected by the getScheduledTime() method in Rule .

When a new set of rules is provided by RuleDiscoverer , they will first be cached in operators. When a watermark arrived at the operator, it will check the timestamp of the watermark against the scheduled time of cached rules and update rules accordingly.

Failover

During checkpoints, the operators mentioned above will store serialized rules and partially matched results inside state backend and restore them during recovery. In this way rules or matches will not be lost during failover.

Common case behavior analysis

When a rule is deleted or replaced, partially matched results related to the rule is directly deprecated and will no longer match to incoming input data.

If there is no rule in a CepOperator, all input data will be deprecated as they cannot match to any of the existing patterns.

If IDs of the List<Rule>  provided by RuleDiscoverer  duplicate with existing rules, versions of the Rule s will be checked. existing rules will be preserved if the versions of the two are the same, or it will be replaced by the newer one otherwise.

Given that the proposed design offers eventual consistency, there could be a period of time just after a rule update happens when inconsistency exists among subtasks of CepOperators, which means they do not work on the same set of rules. This inconsistency is caused by the fact that OperatorCoordinator cannot send OperatorEvent to subtasks simutaneously, and this period of time usually lasts for milliseconds. As can be illustrated in the graph below, fake alerts ("AC" instead of "ABC") might be generated during that period.

Compatibility, Deprecation, and Migration Plan

The changes proposed in this FLIP is backward compatible with existing APIs of Flink CEP. We propose to change the APIs directly without deprecation period.

Current CEP.pattern() methods still exist after changes of this FLIP is introduced, and static CEP patterns created by these methods can still preserve their function, performance and job structure.

Test Plan

We will use unit tests to cover the correctness of introduced features.

Future Work

Service Degradation

As all CEP patterns can be placed in the same operator of the same Flink job in the features we proposed, the error of a single pattern could cause the failure of the whole Flink job, thus affecting the availability of other CEP patterns.

In order to solve this problem, we plan to make CepOperators able to handle exceptions within a rule's scope. When the matching or processing function of a certain rule throws exceptions, the operator should catch this exception, convert it into error messages and sent to side output to call for external solution. The rule that threw the exception will be temporarily disabled on that subtask while other rules work unaffected.

We plan to expose this feature as a configurable property. The default behavior is still causing the whole Flink job to fail.

Table/SQL API Support

Currently we only propose the design for multiple/dynamic rules in DataStream API, but our design can also be extended into Table/SQL API.

In order to achieve the support, we need to add a keyword in SQL like follows, similar to the existing MATCH_RECOGNIZE keyword.

SELECT * FROM input_table RULE(
	`rule_discoverer_factory` = custom_rule_discoverer_factory
)

While MATCH_RECOGNIZE is standard SQL released by the International Organization for Standardization (ISO), we have not found any standard expression format that supports multiple & dynamic match in SQL. Thus the example above is still a naive idea from us and might need further refinement.

As for implementations, we looked into MATCH_RECOGNIZE's implementation and found that it will finally create a CepOperator , which means it can share large proportion of code with the implementation of DataStream API. As illustrated above, the implementation in our proposal will just be adding common StreamOperators and utilizing the existing OperatorCoordinator mechanism. Therefore we believe when we need to support Table/SQL API, most work would be common and have been done in DataStream's implementation.




  • No labels