diff --git a/flink-libraries/flink-cep/pom.xml b/flink-libraries/flink-cep/pom.xml index 6f0e2c32ef852..a4f5d46377444 100644 --- a/flink-libraries/flink-cep/pom.xml +++ b/flink-libraries/flink-cep/pom.xml @@ -38,6 +38,7 @@ under the License. chill ArraysAsListSerializer -->--add-opens=java.base/java.util=ALL-UNNAMED + 4.13.1 @@ -63,6 +64,13 @@ under the License. flink-shaded-guava + + + org.antlr + antlr4-runtime + ${antlr4.version} + + @@ -124,6 +132,27 @@ under the License. + + + + org.antlr + antlr4-maven-plugin + ${antlr4.version} + + + ${project.build.directory}/generated-sources/antlr4 + + true + true + + + + + antlr4 + + + + diff --git a/flink-libraries/flink-cep/src/main/antlr4/org/apache/flink/cep/dsl/grammar/CepDsl.g4 b/flink-libraries/flink-cep/src/main/antlr4/org/apache/flink/cep/dsl/grammar/CepDsl.g4 new file mode 100644 index 0000000000000..f5e4a29eefa45 --- /dev/null +++ b/flink-libraries/flink-cep/src/main/antlr4/org/apache/flink/cep/dsl/grammar/CepDsl.g4 @@ -0,0 +1,475 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// Grammar for Flink CEP DSL +grammar CepDsl; + +startPatternExpressionRule : patternExpression EOF; +patternExpression : skipStrategy? patternFilterExpression (followedByOrNext)* timeWindow?; +skipStrategy: MOD ((s=SKIP_NO_SKIP) | (s=SKIP_SKIP_PAST_LAST) | s=SKIP_SKIP_TO_FIRST k=LBRACK stringconstant m=RBRACK | s=SKIP_SKIP_TO_LAST k=LBRACK stringconstant m=RBRACK); +followedByOrNext : followedBy | followedByAny | notFollowedBy | (f=LNOT)? patternFilterExpression; +followedBy: f=FOLLOWED_BY patternFilterExpression; +followedByAny: f=FOLLOWED_BY_ANY patternFilterExpression; +notFollowedBy: f=NOT_FOLLOWED_BY patternFilterExpression; +timeWindow: WITHIN c=numberconstant(u=HOUR_SHORT | u=MINUTE_SHORT | u=SECOND_SHORT | u=MILLSECONDS_SHORT); +patternFilterExpression + : patternFilterExpressionOptional | patternFilterExpressionMandatory; +patternFilterExpressionMandatory + : (i=IDENT EQUALS)? classIdentifier quantifier? expressionList? stopCondition?; +patternFilterExpressionOptional + : (i=IDENT EQUALS)? classIdentifier quantifier? expressionList? QUESTION; +quantifier: plus_quantifier | star_quantifier | number_quantifier | number_quantifier_greedy; +number_quantifier_greedy: s=LCURLY numberconstant upper_bound? t=RCURLY QUESTION; +number_quantifier: s=LCURLY numberconstant upper_bound? t=RCURLY; +star_quantifier: r=STAR; +plus_quantifier: q=PLUS; +upper_bound: z=COMMA (upper_bound_unlimited | upper_bound_limited); +upper_bound_limited: numberconstant; +upper_bound_unlimited: k=PLUS; +classIdentifier : i1=escapableStr (DOT i2=escapableStr)*; +escapableStr : i1=IDENT | i3=TICKED_STRING_LITERAL; + +stopCondition : (left=LBRACK expression? right=RBRACK); +expressionList : (left=LPAREN expression? right=RPAREN); + +expression : evalOrExpression; + +evalOrExpression : evalAndExpression (op=OR_EXPR evalAndExpression)*; + +evalAndExpression : negatedExpression (op=AND_EXPR negatedExpression)*; + +negatedExpression : evalEqualsExpression + | NOT_EXPR evalEqualsExpression; + +evalEqualsExpression : evalRelationalExpression ( + (eq=EQUALS + | is=IS + | isnot=IS NOT_EXPR + | ne=NOT_EQUAL + ) + ( + evalRelationalExpression + | (expressionList) + ) + )*; + +evalRelationalExpression : concatenationExpr ( + ( + ( + (r=LT|r=GT|r=LE|r=GE) + ( + concatenationExpr + | ( expressionList) + ) + + )* + ) + | (n=NOT_EXPR)? + ( + // Represent the greedy NOT prefix using the token type by + // testing 'n' and setting the token type accordingly. + (in=IN_SET + (l=LPAREN | l=LBRACK) expression // brackets are for inclusive/exclusive + ( + ( col=COLON (expression) ) // range + | + ( (COMMA expression)* ) // list of values + ) + (r=RPAREN | r=RBRACK) + ) ) + ); + +concatenationExpr : additiveExpression ( c=LOR additiveExpression ( LOR additiveExpression)* )?; +additiveExpression : multiplyExpression ( (PLUS|MINUS) multiplyExpression )*; + +multiplyExpression : unaryExpression ( (STAR|DIV|MOD) unaryExpression )*; + +unaryExpression : MINUS eventProperty + | constant + | eventProperty; + +eventProperty : eventPropertyAtomic (DOT eventPropertyAtomic)*; + +eventPropertyAtomic : eventPropertyIdent ( + lb=LBRACK ni=number RBRACK (q=QUESTION)? + | + lp=LPAREN (s=STRING_LITERAL | s=QUOTED_STRING_LITERAL) RPAREN (q=QUESTION)? + | + q1=QUESTION + )?; + +eventPropertyIdent : ipi=keywordAllowedIdent (ESCAPECHAR DOT ipi2=keywordAllowedIdent?)*; + +constant : numberconstant + | stringconstant + | t=BOOLEAN_TRUE + | f=BOOLEAN_FALSE + | nu=VALUE_NULL; + +numberconstant : (m=MINUS | p=PLUS)? number; + +stringconstant : sl=STRING_LITERAL + | qsl=QUOTED_STRING_LITERAL; + + + +keywordAllowedIdent : i1=IDENT + | i2=TICKED_STRING_LITERAL + | AT + | ESCAPE + | SUM + | AVG + | MAX + | MIN + | UNTIL + | WEEKDAY + | LW + | INSTANCEOF + | TYPEOF + | CAST; + +number : IntegerLiteral | FloatingPointLiteral; + +// Tokens +SKIP_NO_SKIP: 'no_skip'; +SKIP_SKIP_PAST_LAST: 'skip_past_last'; +SKIP_SKIP_TO_FIRST: 'skip_to_first'; +SKIP_SKIP_TO_LAST: 'skip_to_last'; +IN_SET:'in'; +BETWEEN:'between'; +LIKE:'like'; +REGEXP:'regexp'; +ESCAPE:'escape'; +OR_EXPR:'or'; +AND_EXPR:'and'; +NOT_EXPR:'not'; +WHERE:'where'; +AS:'as'; +SUM:'sum'; +AVG:'avg'; +MAX:'max'; +MIN:'min'; +ON:'on'; +IS:'is'; +WEEKDAY:'weekday'; +LW:'lastweekday'; +INSTANCEOF:'instanceof'; +TYPEOF:'typeof'; +CAST:'cast'; +CURRENT_TIMESTAMP:'current_timestamp'; +UNTIL:'until'; +AT:'at'; +TIMEPERIOD_YEAR:'year'; +TIMEPERIOD_YEARS:'years'; +TIMEPERIOD_MONTH:'month'; +TIMEPERIOD_MONTHS:'months'; +TIMEPERIOD_WEEK:'week'; +TIMEPERIOD_WEEKS:'weeks'; +TIMEPERIOD_DAY:'day'; +TIMEPERIOD_DAYS:'days'; +TIMEPERIOD_HOUR:'hour'; +TIMEPERIOD_HOURS:'hours'; +TIMEPERIOD_MINUTE:'minute'; +TIMEPERIOD_MINUTES:'minutes'; +TIMEPERIOD_SEC:'sec'; +TIMEPERIOD_SECOND:'second'; +TIMEPERIOD_SECONDS:'seconds'; +TIMEPERIOD_MILLISEC:'msec'; +TIMEPERIOD_MILLISECOND:'millisecond'; +TIMEPERIOD_MILLISECONDS:'milliseconds'; +TIMEPERIOD_MICROSEC:'usec'; +TIMEPERIOD_MICROSECOND:'microsecond'; +TIMEPERIOD_MICROSECONDS:'microseconds'; +BOOLEAN_TRUE:'true'; +BOOLEAN_FALSE:'false'; +VALUE_NULL:'null'; +WITHIN: 'within'; +HOUR_SHORT: 'h'; +MINUTE_SHORT: 'm'; +SECOND_SHORT: 's'; +MILLSECONDS_SHORT: 'ms'; + + +// Operators +NOT_FOLLOWED_BY : '!->'; +FOLLOWED_BY : '->'; +FOLLOWED_BY_ANY : '->>'; +GOES : '=>'; +EQUALS : '='; +QUESTION : '?'; +LPAREN : '('; +RPAREN : ')'; +LBRACK : '['; +RBRACK : ']'; +LCURLY : '{'; +RCURLY : '}'; +COLON : ':'; +COMMA : ','; +LNOT : '!'; +BNOT : '~'; +NOT_EQUAL : '!='; +DIV : '/'; +PLUS : '+'; +MINUS : '-'; +DEC : '--'; +STAR : '*'; +MOD : '%'; +GE : '>='; +GT : '>'; +LE : '<='; +LT : '<'; +BXOR : '^'; +BOR : '|'; +LOR : '||'; +BAND : '&'; +BAND_ASSIGN : '&='; +LAND : '&&'; +SEMI : ';'; +DOT : '.'; +NUM_LONG : '\u18FF'; // assign bogus unicode characters so the token exists +NUM_DOUBLE : '\u18FE'; +NUM_FLOAT : '\u18FD'; +ESCAPECHAR : '\\'; +ESCAPEBACKTICK : '`'; +ATCHAR : '@'; +HASHCHAR : '#'; + +// Whitespace -- ignored +WS : ( ' ' + | '\t' + | '\f' + // handle newlines + | ( + '\r' // Macintosh + | '\n' // Unix (the right way) + ) + )+ + -> channel(HIDDEN) + ; + +// Single-line comments +SL_COMMENT + : '//' + (~('\n'|'\r'))* ('\n'|'\r'('\n')?)? + -> channel(HIDDEN) + ; + +// multiple-line comments +ML_COMMENT + : '/*' (.)*? '*/' + -> channel(HIDDEN) + ; + +TICKED_STRING_LITERAL + : '`' ( EscapeSequence | ~('`'|'\\') )* '`' + ; + +QUOTED_STRING_LITERAL + : '\'' ( EscapeSequence | ~('\''|'\\') )* '\'' + ; + +STRING_LITERAL + : '"' ( EscapeSequence | ~('\\'|'"') )* '"' + ; + +fragment +EscapeSequence : '\\' + ( 'n' + | 'r' + | 't' + | 'b' + | 'f' + | '"' + | '\'' + | '\\' + | UnicodeEscape + | OctalEscape + | . // unknown, leave as it is + ) + ; + +// an identifier. Note that testLiterals is set to true! This means +// that after we match the rule, we look in the literals table to see +// if it's a literal or really an identifer +IDENT + : ('a'..'z'|'_'|'$') ('a'..'z'|'_'|'0'..'9'|'$')* + ; +IntegerLiteral + : DecimalIntegerLiteral + | HexIntegerLiteral + | OctalIntegerLiteral + | BinaryIntegerLiteral + ; + +FloatingPointLiteral + : DecimalFloatingPointLiteral + | HexadecimalFloatingPointLiteral + ; +fragment +OctalEscape + : '\\' ('0'..'3') ('0'..'7') ('0'..'7') + | '\\' ('0'..'7') ('0'..'7') + | '\\' ('0'..'7') + ; +fragment +UnicodeEscape + : '\\' 'u' HexDigit HexDigit HexDigit HexDigit + ; + +fragment +DecimalIntegerLiteral + : DecimalNumeral IntegerTypeSuffix? + ; +fragment +HexIntegerLiteral + : HexNumeral IntegerTypeSuffix? + ; +fragment +OctalIntegerLiteral + : OctalNumeral IntegerTypeSuffix? + ; +fragment +BinaryIntegerLiteral + : BinaryNumeral IntegerTypeSuffix? + ; +fragment +IntegerTypeSuffix + : [lL] + ; +fragment +DecimalNumeral + : '0' + | ('0')* NonZeroDigit (Digits? | Underscores Digits) + ; +fragment +Digits + : Digit (DigitOrUnderscore* Digit)? + ; +fragment +Digit + : '0' + | NonZeroDigit + ; +fragment +NonZeroDigit + : [1-9] + ; +fragment +DigitOrUnderscore + : Digit + | '_' + ; +fragment +Underscores + : '_'+ + ; +fragment +HexNumeral + : '0' [xX] HexDigits + ; +fragment +HexDigits + : HexDigit (HexDigitOrUnderscore* HexDigit)? + ; +fragment +HexDigit + : [0-9a-fA-F] + ; +fragment +HexDigitOrUnderscore + : HexDigit + | '_' + ; +fragment +OctalNumeral + : '0' Underscores? OctalDigits + ; +fragment +OctalDigits + : OctalDigit (OctalDigitOrUnderscore* OctalDigit)? + ; +fragment +OctalDigit + : [0-7] + ; +fragment +OctalDigitOrUnderscore + : OctalDigit + | '_' + ; +fragment +BinaryNumeral + : '0' [bB] BinaryDigits + ; +fragment +BinaryDigits + : BinaryDigit (BinaryDigitOrUnderscore* BinaryDigit)? + ; +fragment +BinaryDigit + : [01] + ; +fragment +BinaryDigitOrUnderscore + : BinaryDigit + | '_' + ; +fragment +DecimalFloatingPointLiteral + : Digits '.' Digits? ExponentPart? FloatTypeSuffix? + | '.' Digits ExponentPart? FloatTypeSuffix? + | Digits ExponentPart FloatTypeSuffix? + | Digits FloatTypeSuffix + ; +fragment +ExponentPart + : ExponentIndicator SignedInteger + ; +fragment +ExponentIndicator + : [eE] + ; +fragment +SignedInteger + : Sign? Digits + ; +fragment +Sign + : [+-] + ; +fragment +FloatTypeSuffix + : [fFdD] + ; +fragment +HexadecimalFloatingPointLiteral + : HexSignificand BinaryExponent FloatTypeSuffix? + ; +fragment +HexSignificand + : HexNumeral '.'? + | '0' [xX] HexDigits? '.' HexDigits + ; +fragment +BinaryExponent + : BinaryExponentIndicator SignedInteger + ; +fragment +BinaryExponentIndicator + : [pP] + ; diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/api/DslCompiler.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/api/DslCompiler.java new file mode 100644 index 0000000000000..4344754ac7786 --- /dev/null +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/api/DslCompiler.java @@ -0,0 +1,229 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cep.dsl.api; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.cep.CEP; +import org.apache.flink.cep.PatternStream; +import org.apache.flink.cep.dsl.exception.DslCompilationException; +import org.apache.flink.cep.dsl.grammar.CepDslLexer; +import org.apache.flink.cep.dsl.grammar.CepDslParser; +import org.apache.flink.cep.dsl.pattern.DslPatternTranslator; +import org.apache.flink.cep.dsl.util.CaseInsensitiveInputStream; +import org.apache.flink.cep.dsl.util.ReflectiveEventAdapter; +import org.apache.flink.cep.pattern.Pattern; +import org.apache.flink.streaming.api.datastream.DataStream; + +import org.antlr.v4.runtime.CommonTokenStream; +import org.antlr.v4.runtime.RecognitionException; +import org.antlr.v4.runtime.tree.ParseTree; +import org.antlr.v4.runtime.tree.ParseTreeWalker; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Main API for compiling DSL expressions into Flink CEP PatternStreams. + * + *

This class provides static factory methods to compile DSL pattern expressions into {@link + * PatternStream} objects that can be used with Flink's CEP library. + * + *

Basic Usage: + * + *

{@code
+ * DataStream sensorData = ...;
+ *
+ * // Simple pattern matching
+ * PatternStream pattern = DslCompiler.compile(
+ *     "HighTemp(temperature > 100)",
+ *     sensorData
+ * );
+ *
+ * // Pattern with event correlation
+ * PatternStream correlatedPattern = DslCompiler.compile(
+ *     "Start(id > 0) -> End(id = Start.id and value > 50)",
+ *     sensorData
+ * );
+ * }
+ * + *

Advanced Usage with Custom Adapter: + * + *

{@code
+ * EventAdapter adapter = new MyCustomAdapter();
+ * PatternStream pattern = DslCompiler.compile(
+ *     "Alert(severity > 5)",
+ *     customEventStream,
+ *     adapter
+ * );
+ * }
+ * + *

Builder API: + * + *

{@code
+ * PatternStream pattern = DslCompiler.builder()
+ *     .withStrictTypeMatching()
+ *     .withEventAdapter(customAdapter)
+ *     .compile("A(x > 10) -> B(y < 5)", dataStream);
+ * }
+ * + * @see DslCompilerBuilder + * @see EventAdapter + */ +@PublicEvolving +public class DslCompiler { + + private static final Logger LOG = LoggerFactory.getLogger(DslCompiler.class); + + // Private constructor to prevent instantiation + private DslCompiler() {} + + /** + * Compile a DSL expression using the default reflective event adapter. + * + *

This method uses reflection to automatically access POJO fields and getters. It's suitable + * for most use cases where events are simple Java objects. + * + * @param dslExpression The DSL pattern expression to compile + * @param dataStream The input data stream + * @param The type of events in the stream + * @return A PatternStream configured with the compiled pattern + * @throws DslCompilationException if the DSL expression is invalid + */ + public static PatternStream compile(String dslExpression, DataStream dataStream) { + return compile(dslExpression, dataStream, new ReflectiveEventAdapter<>(), false); + } + + /** + * Compile a DSL expression with a custom event adapter. + * + *

Use this method when you need custom logic for extracting attributes from events, or when + * working with non-POJO event types (e.g., Maps, custom data structures). + * + * @param dslExpression The DSL pattern expression to compile + * @param dataStream The input data stream + * @param eventAdapter Custom adapter for extracting event attributes + * @param The type of events in the stream + * @return A PatternStream configured with the compiled pattern + * @throws DslCompilationException if the DSL expression is invalid + */ + public static PatternStream compile( + String dslExpression, DataStream dataStream, EventAdapter eventAdapter) { + return compile(dslExpression, dataStream, eventAdapter, false); + } + + /** + * Compile a DSL expression with full configuration options. + * + *

This method provides complete control over DSL compilation, including strict type + * matching. + * + * @param dslExpression The DSL pattern expression to compile + * @param dataStream The input data stream + * @param eventAdapter Custom adapter for extracting event attributes + * @param strictTypeMatching Whether to enforce strict event type matching (event type in DSL + * must match actual event type) + * @param The type of events in the stream + * @return A PatternStream configured with the compiled pattern + * @throws DslCompilationException if the DSL expression is invalid + */ + public static PatternStream compile( + String dslExpression, + DataStream dataStream, + EventAdapter eventAdapter, + boolean strictTypeMatching) { + + LOG.info("Compiling DSL expression: {}", dslExpression); + + try { + // Step 1: Parse DSL expression + Pattern pattern = parseDsl(dslExpression, eventAdapter, strictTypeMatching); + + // Step 2: Create PatternStream + PatternStream patternStream = CEP.pattern(dataStream, pattern).inEventTime(); + + LOG.info("Successfully compiled DSL expression into pattern: {}", pattern.getName()); + return patternStream; + + } catch (RecognitionException e) { + throw new DslCompilationException( + "Failed to parse DSL expression: " + e.getMessage(), e); + } catch (Exception e) { + throw new DslCompilationException( + "Failed to compile DSL expression: " + e.getMessage(), e); + } + } + + /** + * Create a builder for more complex configuration. + * + *

The builder pattern allows for fluent, readable configuration of DSL compilation options. + * + * @param The type of events + * @return A new DslCompilerBuilder + */ + public static DslCompilerBuilder builder() { + return new DslCompilerBuilder<>(); + } + + /** + * Parse a DSL expression into a Flink Pattern. + * + * @param dslExpression The DSL expression to parse + * @param eventAdapter The event adapter for attribute extraction + * @param strictTypeMatching Whether to enforce strict type matching + * @param The event type + * @return The compiled Pattern + * @throws DslCompilationException if parsing fails + */ + static Pattern parseDsl( + String dslExpression, EventAdapter eventAdapter, boolean strictTypeMatching) { + + // Create case-insensitive input stream + CaseInsensitiveInputStream inputStream = new CaseInsensitiveInputStream(dslExpression); + + // Create lexer and parser + CepDslLexer lexer = new CepDslLexer(inputStream); + CommonTokenStream tokens = new CommonTokenStream(lexer); + CepDslParser parser = new CepDslParser(tokens); + + // Parse the expression + ParseTree parseTree = parser.startPatternExpressionRule(); + + // Check for syntax errors + if (parser.getNumberOfSyntaxErrors() > 0) { + throw new DslCompilationException( + String.format( + "DSL expression contains %d syntax error(s)", + parser.getNumberOfSyntaxErrors())); + } + + // Walk the parse tree and build the pattern + ParseTreeWalker walker = ParseTreeWalker.DEFAULT; + DslPatternTranslator translator = + new DslPatternTranslator<>(eventAdapter, strictTypeMatching); + walker.walk(translator, parseTree); + + Pattern pattern = translator.getPattern(); + if (pattern == null) { + throw new DslCompilationException( + "Failed to translate DSL expression into a valid pattern"); + } + + return pattern; + } +} diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/api/DslCompilerBuilder.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/api/DslCompilerBuilder.java new file mode 100644 index 0000000000000..d99eb0c6da82c --- /dev/null +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/api/DslCompilerBuilder.java @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cep.dsl.api; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.cep.PatternStream; +import org.apache.flink.cep.dsl.util.ReflectiveEventAdapter; +import org.apache.flink.streaming.api.datastream.DataStream; + +/** + * Builder for configuring and compiling DSL expressions. + * + *

This builder provides a fluent API for configuring DSL compilation options before compiling an + * expression into a PatternStream. + * + *

Example usage: + * + *

{@code
+ * PatternStream pattern = DslCompiler.builder()
+ *     .withStrictTypeMatching()
+ *     .withEventAdapter(customAdapter)
+ *     .compile("A(x > 10) -> B(y < 5)", dataStream);
+ * }
+ * + * @param The type of events in the stream + */ +@PublicEvolving +public class DslCompilerBuilder { + + private EventAdapter eventAdapter; + private boolean strictTypeMatching = false; + + /** Package-private constructor, use {@link DslCompiler#builder()} to create instances. */ + DslCompilerBuilder() {} + + /** + * Set a custom event adapter for attribute extraction. + * + *

If not set, {@link ReflectiveEventAdapter} will be used by default. + * + * @param adapter The event adapter to use + * @return this builder for fluent chaining + */ + public DslCompilerBuilder withEventAdapter(EventAdapter adapter) { + this.eventAdapter = adapter; + return this; + } + + /** + * Enable strict event type matching. + * + *

When enabled, the DSL will check that the event type name (obtained from {@link + * EventAdapter#getEventType(Object)}) matches the pattern name in the DSL expression. + * + *

For example, with strict matching enabled, the pattern {@code Sensor(temperature > 100)} + * will only match events where {@code getEventType()} returns "Sensor". + * + *

Default is false (lenient matching). + * + * @return this builder for fluent chaining + */ + public DslCompilerBuilder withStrictTypeMatching() { + this.strictTypeMatching = true; + return this; + } + + /** + * Disable strict event type matching (default behavior). + * + *

When disabled, the DSL will not check event types, allowing patterns to match any event + * regardless of its type. + * + * @return this builder for fluent chaining + */ + public DslCompilerBuilder withLenientTypeMatching() { + this.strictTypeMatching = false; + return this; + } + + /** + * Compile the DSL expression with the configured options. + * + * @param dslExpression The DSL pattern expression to compile + * @param dataStream The input data stream + * @return A PatternStream configured with the compiled pattern + */ + public PatternStream compile(String dslExpression, DataStream dataStream) { + EventAdapter adapter = + eventAdapter != null ? eventAdapter : new ReflectiveEventAdapter<>(); + + return DslCompiler.compile(dslExpression, dataStream, adapter, strictTypeMatching); + } +} diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/api/EventAdapter.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/api/EventAdapter.java new file mode 100644 index 0000000000000..3c7aaa062ad62 --- /dev/null +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/api/EventAdapter.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cep.dsl.api; + +import org.apache.flink.annotation.PublicEvolving; + +import java.io.Serializable; +import java.util.Optional; + +/** + * Adapter interface for extracting attributes from events. + * + *

This interface allows the CEP DSL to work with any event type without requiring specific + * interfaces to be implemented. Users can provide custom adapters for their event types, or use the + * built-in {@link org.apache.flink.cep.dsl.util.ReflectiveEventAdapter} which uses Java reflection + * to access POJO fields and getters. + * + *

Example usage with custom adapter: + * + *

{@code
+ * EventAdapter adapter = new EventAdapter() {
+ *     @Override
+ *     public Optional getAttribute(MyEvent event, String attributeName) {
+ *         return Optional.ofNullable(event.getProperty(attributeName));
+ *     }
+ *
+ *     @Override
+ *     public String getEventType(MyEvent event) {
+ *         return event.getClass().getSimpleName();
+ *     }
+ * };
+ *
+ * PatternStream pattern = DslCompiler.compile(
+ *     "Alert(severity > 5)",
+ *     dataStream,
+ *     adapter
+ * );
+ * }
+ *
+ * @param  The type of events to adapt
+ */
+@PublicEvolving
+public interface EventAdapter extends Serializable {
+
+    /**
+     * Extract an attribute value from an event.
+     *
+     * 

This method is called by the DSL evaluation engine to access event attributes referenced + * in pattern expressions (e.g., {@code temperature > 100}). + * + * @param event The event to extract from + * @param attributeName The name of the attribute to extract + * @return The attribute value wrapped in an Optional, or {@link Optional#empty()} if the + * attribute doesn't exist + */ + Optional getAttribute(T event, String attributeName); + + /** + * Get the event type name for type matching in DSL patterns. + * + *

When strict type matching is enabled, the DSL uses this method to verify that events match + * the expected type specified in the pattern (e.g., {@code Sensor(temperature > 100)} expects + * events with type "Sensor"). + * + * @param event The event + * @return The event type name (typically the simple class name) + */ + String getEventType(T event); +} diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/condition/ComparisonOperator.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/condition/ComparisonOperator.java new file mode 100644 index 0000000000000..2f33cfe0a7f8c --- /dev/null +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/condition/ComparisonOperator.java @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cep.dsl.condition; + +import org.apache.flink.annotation.Internal; + +import java.math.BigDecimal; + +/** + * Comparison operators supported by the CEP DSL. + * + *

This enum provides type-safe comparison operations for DSL expressions. All numeric + * comparisons are performed using {@link BigDecimal} to ensure consistent precision across + * different numeric types. + */ +@Internal +public enum ComparisonOperator { + EQUALS("=") { + @Override + public boolean evaluate(Object left, Object right) { + if (left == null || right == null) { + return left == right; + } + return left.equals(right); + } + }, + + NOT_EQUALS("!=") { + @Override + public boolean evaluate(Object left, Object right) { + return !EQUALS.evaluate(left, right); + } + }, + + LESS_THAN("<") { + @Override + public boolean evaluate(Object left, Object right) { + return compareNumbers(left, right) < 0; + } + }, + + LESS_THAN_OR_EQUAL("<=") { + @Override + public boolean evaluate(Object left, Object right) { + return compareNumbers(left, right) <= 0; + } + }, + + GREATER_THAN(">") { + @Override + public boolean evaluate(Object left, Object right) { + return compareNumbers(left, right) > 0; + } + }, + + GREATER_THAN_OR_EQUAL(">=") { + @Override + public boolean evaluate(Object left, Object right) { + return compareNumbers(left, right) >= 0; + } + }; + + private final String symbol; + + ComparisonOperator(String symbol) { + this.symbol = symbol; + } + + /** + * Get the symbol representation of this operator. + * + * @return The operator symbol (e.g., "=", "!=", "<", etc.) + */ + public String getSymbol() { + return symbol; + } + + /** + * Evaluate the comparison between two values. + * + * @param left The left-hand side value + * @param right The right-hand side value + * @return true if the comparison holds, false otherwise + * @throws IllegalArgumentException if numeric comparison is attempted on non-numeric types + */ + public abstract boolean evaluate(Object left, Object right); + + /** + * Find an operator by its symbol. + * + * @param symbol The operator symbol to search for + * @return The matching operator + * @throws IllegalArgumentException if no operator with the given symbol exists + */ + public static ComparisonOperator fromSymbol(String symbol) { + for (ComparisonOperator op : values()) { + if (op.symbol.equals(symbol)) { + return op; + } + } + throw new IllegalArgumentException("Unknown operator symbol: " + symbol); + } + + /** + * Compare two numbers using BigDecimal for consistent precision. + * + * @param left The left-hand side value (must be a Number) + * @param right The right-hand side value (must be a Number) + * @return negative if left < right, zero if equal, positive if left > right + * @throws IllegalArgumentException if either value is not a Number + */ + private static int compareNumbers(Object left, Object right) { + if (!(left instanceof Number) || !(right instanceof Number)) { + throw new IllegalArgumentException( + String.format( + "Cannot compare non-numeric values: %s (%s) and %s (%s)", + left, + left != null ? left.getClass().getName() : "null", + right, + right != null ? right.getClass().getName() : "null")); + } + + BigDecimal leftBd = toBigDecimal((Number) left); + BigDecimal rightBd = toBigDecimal((Number) right); + return leftBd.compareTo(rightBd); + } + + /** + * Convert a Number to BigDecimal with appropriate precision. + * + * @param number The number to convert + * @return The number as a BigDecimal + */ + private static BigDecimal toBigDecimal(Number number) { + if (number instanceof BigDecimal) { + return (BigDecimal) number; + } else if (number instanceof Double || number instanceof Float) { + // Use string conversion to avoid precision issues + return new BigDecimal(number.toString()); + } else { + // Integer, Long, Short, Byte + return BigDecimal.valueOf(number.longValue()); + } + } +} diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/condition/DslCondition.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/condition/DslCondition.java new file mode 100644 index 0000000000000..be35a4b37d4c6 --- /dev/null +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/condition/DslCondition.java @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cep.dsl.condition; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.cep.dsl.api.EventAdapter; +import org.apache.flink.cep.pattern.conditions.IterativeCondition; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; + +/** + * Flink CEP condition implementation for DSL expressions. + * + *

This condition evaluates a list of {@link DslExpression}s combined with either AND or OR + * logic. It also supports optional event type matching when strict type matching is enabled. + * + *

Features: + * + *

    + *
  • Short-circuit evaluation (AND stops on first false, OR stops on first true) + *
  • Event type filtering (optional) + *
  • Support for complex nested expressions + *
  • Event correlation across patterns + *
+ */ +@Internal +public class DslCondition extends IterativeCondition { + + private static final long serialVersionUID = 1L; + private static final Logger LOG = LoggerFactory.getLogger(DslCondition.class); + + private final List> expressions; + private final LogicalOperator logicalOperator; + private final EventAdapter eventAdapter; + private final String eventTypePattern; + + /** + * Create a condition with only event type matching (no expressions). + * + * @param eventAdapter The event adapter for attribute extraction + * @param eventTypePattern The expected event type (null to skip type checking) + */ + public DslCondition(EventAdapter eventAdapter, String eventTypePattern) { + this(eventAdapter, eventTypePattern, new ArrayList<>(), LogicalOperator.AND); + } + + /** + * Create a condition with expressions and optional event type matching. + * + * @param eventAdapter The event adapter for attribute extraction + * @param eventTypePattern The expected event type (null to skip type checking) + * @param expressions The list of expressions to evaluate + * @param logicalOperator The logical operator combining expressions (AND or OR) + */ + public DslCondition( + EventAdapter eventAdapter, + String eventTypePattern, + List> expressions, + LogicalOperator logicalOperator) { + this.eventAdapter = eventAdapter; + this.eventTypePattern = eventTypePattern; + this.expressions = expressions; + this.logicalOperator = logicalOperator; + } + + @Override + public boolean filter(T event, Context context) throws Exception { + // Step 1: Check event type if specified + if (eventTypePattern != null) { + String actualType = eventAdapter.getEventType(event); + if (!matchesEventType(actualType, eventTypePattern)) { + LOG.trace( + "Event type mismatch: expected '{}', got '{}'", + eventTypePattern, + actualType); + return false; + } + } + + // Step 2: Evaluate expressions + if (expressions.isEmpty()) { + // No expressions means accept all events (of the right type) + return true; + } + + return evaluateExpressions(event, context); + } + + /** + * Evaluate all expressions with the configured logical operator. + * + * @param event The event to evaluate + * @param context The pattern context + * @return true if expressions evaluate to true according to the logical operator + */ + private boolean evaluateExpressions(T event, Context context) { + if (logicalOperator == LogicalOperator.AND) { + // Short-circuit AND: stop on first false + for (DslExpression expr : expressions) { + if (!expr.evaluate(event, eventAdapter, context)) { + LOG.trace("AND expression failed: {}", expr); + return false; + } + } + LOG.trace("All AND expressions passed ({} expressions)", expressions.size()); + return true; + } else { + // Short-circuit OR: stop on first true + for (DslExpression expr : expressions) { + if (expr.evaluate(event, eventAdapter, context)) { + LOG.trace("OR expression succeeded: {}", expr); + return true; + } + } + LOG.trace("All OR expressions failed ({} expressions)", expressions.size()); + return false; + } + } + + /** + * Check if the actual event type matches the expected pattern. + * + *

Uses case-insensitive comparison. + * + * @param actualType The actual event type from the event + * @param pattern The expected event type pattern + * @return true if types match + */ + private boolean matchesEventType(String actualType, String pattern) { + if (actualType == null || pattern == null) { + return false; + } + return actualType.equalsIgnoreCase(pattern); + } + + /** Logical operator for combining expressions. */ + public enum LogicalOperator { + /** All expressions must be true. */ + AND, + /** At least one expression must be true. */ + OR + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + if (eventTypePattern != null) { + sb.append("EventType=").append(eventTypePattern); + if (!expressions.isEmpty()) { + sb.append(" AND "); + } + } + if (!expressions.isEmpty()) { + sb.append("("); + for (int i = 0; i < expressions.size(); i++) { + if (i > 0) { + sb.append(" ").append(logicalOperator).append(" "); + } + sb.append(expressions.get(i)); + } + sb.append(")"); + } + return sb.toString(); + } +} diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/condition/DslExpression.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/condition/DslExpression.java new file mode 100644 index 0000000000000..c779ce5d2cef3 --- /dev/null +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/condition/DslExpression.java @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cep.dsl.condition; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.cep.dsl.api.EventAdapter; +import org.apache.flink.cep.pattern.conditions.IterativeCondition; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Serializable; +import java.util.Optional; + +/** + * Represents a single condition expression in the DSL. + * + *

An expression consists of: + * + *

    + *
  • An attribute name (e.g., "temperature") + *
  • A comparison operator (e.g., >, <, =) + *
  • Either a constant value (e.g., 100) or a reference to another event (e.g., A.id) + *
+ * + *

Examples: + * + *

    + *
  • temperature > 100 (constant comparison) + *
  • A.id = B.id (event correlation) + *
  • status != 'error' (string comparison) + *
+ */ +@Internal +public class DslExpression implements Serializable { + + private static final long serialVersionUID = 1L; + private static final Logger LOG = LoggerFactory.getLogger(DslExpression.class); + + private final String attribute; + private final ComparisonOperator operator; + private final Object constantValue; + private final String referencePattern; + private final String referenceAttribute; + + /** + * Create an expression for constant comparison (e.g., temperature > 100). + * + * @param attribute The attribute name to compare + * @param operator The comparison operator + * @param value The constant value to compare against + */ + public DslExpression(String attribute, ComparisonOperator operator, Object value) { + this.attribute = attribute; + this.operator = operator; + this.constantValue = value; + this.referencePattern = null; + this.referenceAttribute = null; + } + + /** + * Create an expression for event correlation (e.g., A.id = B.id). + * + * @param attribute The attribute name on the current event + * @param operator The comparison operator + * @param refPattern The name of the pattern to reference + * @param refAttribute The attribute name on the referenced pattern + */ + public DslExpression( + String attribute, ComparisonOperator operator, String refPattern, String refAttribute) { + this.attribute = attribute; + this.operator = operator; + this.constantValue = null; + this.referencePattern = refPattern; + this.referenceAttribute = refAttribute; + } + + /** + * Evaluate this expression against an event and pattern context. + * + * @param event The current event to evaluate + * @param adapter The event adapter for attribute extraction + * @param context The pattern context providing access to previously matched events + * @return true if the expression evaluates to true, false otherwise + */ + public boolean evaluate( + T event, EventAdapter adapter, IterativeCondition.Context context) { + + // Get left-hand side value from current event + Optional leftValue = adapter.getAttribute(event, attribute); + if (!leftValue.isPresent()) { + LOG.debug( + "Attribute '{}' not found on event, expression evaluates to false", attribute); + return false; + } + + // Get right-hand side value + Object rightValue; + if (referencePattern != null) { + // Event correlation: get value from referenced pattern + rightValue = getReferenceValue(context, adapter); + if (rightValue == null) { + LOG.debug( + "Reference attribute '{}.{}' not found in context, expression evaluates to false", + referencePattern, + referenceAttribute); + return false; + } + } else { + // Constant comparison + rightValue = constantValue; + } + + // Perform comparison + try { + boolean result = operator.evaluate(leftValue.get(), rightValue); + LOG.trace( + "Expression evaluation: {} {} {} = {}", + leftValue.get(), + operator.getSymbol(), + rightValue, + result); + return result; + } catch (Exception e) { + LOG.warn( + "Error evaluating expression: {} {} {}", + leftValue.get(), + operator.getSymbol(), + rightValue, + e); + return false; + } + } + + /** + * Get the value of a referenced attribute from the pattern context. + * + * @param context The pattern context + * @param adapter The event adapter + * @return The referenced value, or null if not found + */ + private Object getReferenceValue( + IterativeCondition.Context context, EventAdapter adapter) { + try { + Iterable events = context.getEventsForPattern(referencePattern); + for (T event : events) { + Optional value = adapter.getAttribute(event, referenceAttribute); + if (value.isPresent()) { + return value.get(); + } + } + } catch (Exception e) { + LOG.warn("Error accessing reference pattern '{}' in context", referencePattern, e); + } + return null; + } + + @Override + public String toString() { + if (referencePattern != null) { + return String.format( + "%s %s %s.%s", + attribute, operator.getSymbol(), referencePattern, referenceAttribute); + } else { + return String.format("%s %s %s", attribute, operator.getSymbol(), constantValue); + } + } +} diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/exception/DslCompilationException.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/exception/DslCompilationException.java new file mode 100644 index 0000000000000..3dedcc31eca5a --- /dev/null +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/exception/DslCompilationException.java @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cep.dsl.exception; + +import org.apache.flink.annotation.PublicEvolving; + +/** + * Exception thrown when DSL compilation fails. + * + *

This exception is thrown during the parsing and translation phase when the DSL expression + * contains syntax errors or semantic issues that prevent it from being compiled into a valid Flink + * Pattern. + * + *

Common causes: + * + *

    + *
  • Invalid syntax (e.g., unmatched parentheses, invalid operators) + *
  • Unknown pattern names in event correlation + *
  • Invalid quantifiers or time window specifications + *
+ */ +@PublicEvolving +public class DslCompilationException extends RuntimeException { + + private static final long serialVersionUID = 1L; + + private final int line; + private final int column; + private final String dslFragment; + + /** + * Create a compilation exception with a simple message. + * + * @param message The error message + */ + public DslCompilationException(String message) { + super(message); + this.line = -1; + this.column = -1; + this.dslFragment = null; + } + + /** + * Create a compilation exception with a message and cause. + * + * @param message The error message + * @param cause The underlying cause + */ + public DslCompilationException(String message, Throwable cause) { + super(message, cause); + this.line = -1; + this.column = -1; + this.dslFragment = null; + } + + /** + * Create a compilation exception with location information. + * + * @param message The error message + * @param line The line number where the error occurred (0-based) + * @param column The column number where the error occurred (0-based) + * @param dslFragment The fragment of DSL that caused the error + */ + public DslCompilationException(String message, int line, int column, String dslFragment) { + super(formatMessage(message, line, column, dslFragment)); + this.line = line; + this.column = column; + this.dslFragment = dslFragment; + } + + /** + * Get the line number where the error occurred. + * + * @return The line number (0-based), or -1 if not available + */ + public int getLine() { + return line; + } + + /** + * Get the column number where the error occurred. + * + * @return The column number (0-based), or -1 if not available + */ + public int getColumn() { + return column; + } + + /** + * Get the DSL fragment that caused the error. + * + * @return The DSL fragment, or null if not available + */ + public String getDslFragment() { + return dslFragment; + } + + private static String formatMessage(String message, int line, int column, String fragment) { + StringBuilder sb = new StringBuilder(); + sb.append("DSL compilation error at line ").append(line + 1); + sb.append(", column ").append(column + 1).append(": "); + sb.append(message); + + if (fragment != null && !fragment.isEmpty()) { + sb.append("\n ").append(fragment); + sb.append("\n "); + for (int i = 0; i < column; i++) { + sb.append(" "); + } + sb.append("^"); + } + + return sb.toString(); + } +} diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/exception/DslEvaluationException.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/exception/DslEvaluationException.java new file mode 100644 index 0000000000000..ab539b976871b --- /dev/null +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/exception/DslEvaluationException.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cep.dsl.exception; + +import org.apache.flink.annotation.PublicEvolving; + +/** + * Exception thrown during runtime evaluation of DSL expressions. + * + *

This exception is thrown when an expression that compiled successfully fails during runtime + * evaluation, typically due to type mismatches or missing attributes. + * + *

Common causes: + * + *

    + *
  • Type mismatch (e.g., comparing a string with a number) + *
  • Missing attributes on events + *
  • Null values in comparisons + *
  • Referenced pattern not found in context + *
+ */ +@PublicEvolving +public class DslEvaluationException extends RuntimeException { + + private static final long serialVersionUID = 1L; + + private final String expression; + private final Object event; + + /** + * Create an evaluation exception with a simple message. + * + * @param message The error message + */ + public DslEvaluationException(String message) { + super(message); + this.expression = null; + this.event = null; + } + + /** + * Create an evaluation exception with a message and cause. + * + * @param message The error message + * @param cause The underlying cause + */ + public DslEvaluationException(String message, Throwable cause) { + super(message, cause); + this.expression = null; + this.event = null; + } + + /** + * Create an evaluation exception with expression and event context. + * + * @param message The error message + * @param expression The expression that failed + * @param event The event being evaluated + */ + public DslEvaluationException(String message, String expression, Object event) { + super(formatMessage(message, expression, event)); + this.expression = expression; + this.event = event; + } + + /** + * Create an evaluation exception with expression, event, and cause. + * + * @param message The error message + * @param expression The expression that failed + * @param event The event being evaluated + * @param cause The underlying cause + */ + public DslEvaluationException( + String message, String expression, Object event, Throwable cause) { + super(formatMessage(message, expression, event), cause); + this.expression = expression; + this.event = event; + } + + /** + * Get the expression that failed. + * + * @return The expression, or null if not available + */ + public String getExpression() { + return expression; + } + + /** + * Get the event that was being evaluated. + * + * @return The event, or null if not available + */ + public Object getEvent() { + return event; + } + + private static String formatMessage(String message, String expression, Object event) { + StringBuilder sb = new StringBuilder(); + sb.append("DSL evaluation error: ").append(message); + + if (expression != null) { + sb.append("\n Expression: ").append(expression); + } + + if (event != null) { + sb.append("\n Event: ").append(event); + } + + return sb.toString(); + } +} diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/pattern/DslPatternTranslator.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/pattern/DslPatternTranslator.java new file mode 100644 index 0000000000000..2da114e676b43 --- /dev/null +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/pattern/DslPatternTranslator.java @@ -0,0 +1,693 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cep.dsl.pattern; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.cep.dsl.api.EventAdapter; +import org.apache.flink.cep.dsl.condition.ComparisonOperator; +import org.apache.flink.cep.dsl.condition.DslCondition; +import org.apache.flink.cep.dsl.condition.DslExpression; +import org.apache.flink.cep.dsl.exception.DslCompilationException; +import org.apache.flink.cep.dsl.grammar.CepDslBaseListener; +import org.apache.flink.cep.dsl.grammar.CepDslParser; +import org.apache.flink.cep.nfa.aftermatch.AfterMatchSkipStrategy; +import org.apache.flink.cep.pattern.Pattern; + +import org.antlr.v4.runtime.Token; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Stack; +import java.util.function.BiConsumer; +import java.util.function.Function; + +/** + * Translates ANTLR parse tree to Flink Pattern objects. + * + *

This class walks the parse tree generated by ANTLR and constructs the corresponding Flink CEP + * Pattern using the builder API. It handles: + * + *

    + *
  • Pattern sequencing (next, followedBy, followedByAny, notFollowedBy) + *
  • Conditions (where clauses with expressions) + *
  • Quantifiers (*, +, ?, {n}, {n,m}) + *
  • Time windows (within) + *
  • Skip strategies + *
+ */ +@Internal +public class DslPatternTranslator extends CepDslBaseListener { + + private static final Logger LOG = LoggerFactory.getLogger(DslPatternTranslator.class); + + private final EventAdapter eventAdapter; + private final boolean strictTypeMatching; + + // Pattern building state + private Pattern currentPattern; + private AfterMatchSkipStrategy skipStrategy = AfterMatchSkipStrategy.noSkip(); + + // Current pattern node state + private String currentPatternName; + private String currentEventType; + private List> currentExpressions; + private DslCondition.LogicalOperator currentLogicalOp; + private boolean isGreedy = false; + private boolean isOptional = false; + private Integer quantifierLower = null; + private Integer quantifierUpper = null; + + // Expression building stacks + private Stack>> expressionStack = new Stack<>(); + private Stack operatorStack = new Stack<>(); + private Stack, Integer>> numberConsumerStack = new Stack<>(); + private Stack, String>> stringConsumerStack = new Stack<>(); + + // Current expression being built + private String currentAttribute; + private ComparisonOperator currentOperator; + private Object currentValue; + private String currentRefPattern; + private String currentRefAttribute; + private boolean inRelationalExpression = false; + private String + tempClassIdText; // Temporary storage for class identifier when determining pattern name + + // Pattern combiner (determines how patterns are connected) + private Function> patternCombiner; + + // Track used pattern names to avoid conflicts when same event type is used multiple times + private final Map usedPatternNames = new HashMap<>(); + + // Skip strategy helpers + private static final Map> SKIP_STRATEGIES = + new HashMap<>(); + + static { + SKIP_STRATEGIES.put("NO_SKIP", (ignored) -> AfterMatchSkipStrategy.noSkip()); + SKIP_STRATEGIES.put( + "SKIP_PAST_LAST", (ignored) -> AfterMatchSkipStrategy.skipPastLastEvent()); + SKIP_STRATEGIES.put("SKIP_TO_FIRST", AfterMatchSkipStrategy::skipToFirst); + SKIP_STRATEGIES.put("SKIP_TO_LAST", AfterMatchSkipStrategy::skipToLast); + } + + public DslPatternTranslator(EventAdapter eventAdapter, boolean strictTypeMatching) { + this.eventAdapter = eventAdapter; + this.strictTypeMatching = strictTypeMatching; + } + + @Override + public void enterSkipStrategy(CepDslParser.SkipStrategyContext ctx) { + if (ctx.s != null) { + String strategyName = ctx.s.getText().toUpperCase(); + String patternRef = null; + + // Extract pattern reference for SKIP_TO_FIRST/LAST + if (ctx.stringconstant() != null) { + patternRef = extractStringLiteral(ctx.stringconstant().getText()); + } + + Function strategyFactory = + SKIP_STRATEGIES.get(strategyName); + if (strategyFactory != null) { + skipStrategy = strategyFactory.apply(patternRef); + LOG.debug("Set skip strategy: {}", strategyName); + } + } + } + + @Override + public void enterPatternFilterExpression(CepDslParser.PatternFilterExpressionContext ctx) { + // Initialize state for new pattern node + currentPatternName = null; + currentEventType = null; + currentExpressions = new ArrayList<>(); + currentLogicalOp = DslCondition.LogicalOperator.AND; + isGreedy = false; + isOptional = false; + quantifierLower = null; + quantifierUpper = null; + + // Extract pattern name and event type + if (ctx.patternFilterExpressionMandatory() != null) { + CepDslParser.PatternFilterExpressionMandatoryContext mandatory = + ctx.patternFilterExpressionMandatory(); + + if (mandatory.i != null) { + // Explicit name token provided + extractPatternInfo(mandatory.classIdentifier(), mandatory.i); + } else { + // No explicit name - store classId for later determination + tempClassIdText = extractClassIdentifier(mandatory.classIdentifier()); + } + isOptional = false; + } else if (ctx.patternFilterExpressionOptional() != null) { + CepDslParser.PatternFilterExpressionOptionalContext optional = + ctx.patternFilterExpressionOptional(); + extractPatternInfo(optional.classIdentifier(), optional.i); + isOptional = true; + } + } + + @Override + public void exitPatternFilterExpression(CepDslParser.PatternFilterExpressionContext ctx) { + // Handle pattern name detection for cases like "A(price > 0)" where A is pattern name + if (currentPatternName == null + && tempClassIdText != null + && !currentExpressions.isEmpty()) { + // Check if this looks like pattern name syntax (simple identifier with expressions) + CepDslParser.ClassIdentifierContext classId = + ctx.patternFilterExpressionMandatory().classIdentifier(); + if (classId.escapableStr().size() == 1 && classId.escapableStr().get(0).i1 != null) { + // Use the identifier as pattern name, and set event type to null (any type) + currentPatternName = tempClassIdText; + currentEventType = null; + } else { + // Normal case: use classId as event type and generate pattern name + currentEventType = tempClassIdText; + currentPatternName = generateUniquePatternName(currentEventType); + } + tempClassIdText = null; // Clean up + } + + // Build the pattern + buildPattern(); + } + + private void extractPatternInfo(CepDslParser.ClassIdentifierContext classId, Token nameToken) { + // Event type is the class identifier + String classIdText = extractClassIdentifier(classId); + + // Pattern name is either explicitly specified or defaults to event type + if (nameToken != null) { + currentPatternName = nameToken.getText(); + currentEventType = classIdText; + } else { + // Normal case: classId is event type + currentEventType = classIdText; + currentPatternName = generateUniquePatternName(currentEventType); + } + } + + /** + * Generate a unique pattern name based on the event type. If the name is used for the first + * time, return it as-is. Otherwise, append a counter (e.g., TRADE, TRADE_2, TRADE_3). + */ + private String generateUniquePatternName(String baseName) { + Integer count = usedPatternNames.get(baseName); + if (count == null) { + // First use - no suffix needed + usedPatternNames.put(baseName, 1); + return baseName; + } else { + // Subsequent use - add suffix + count++; + usedPatternNames.put(baseName, count); + return baseName + "_" + count; + } + } + + private String extractClassIdentifier(CepDslParser.ClassIdentifierContext ctx) { + if (ctx == null) { + return null; + } + StringBuilder sb = new StringBuilder(); + for (CepDslParser.EscapableStrContext esc : ctx.escapableStr()) { + if (sb.length() > 0) { + sb.append("."); + } + sb.append(extractEscapableStr(esc)); + } + return sb.toString(); + } + + private String extractEscapableStr(CepDslParser.EscapableStrContext ctx) { + if (ctx.i1 != null) { + return ctx.i1.getText(); + } + if (ctx.i3 != null) { + return extractStringLiteral(ctx.i3.getText()); + } + return ""; + } + + private void buildPattern() { + // Create or extend pattern + if (currentPattern == null) { + // First pattern - use begin() + currentPattern = Pattern.begin(currentPatternName, skipStrategy); + LOG.debug("Created initial pattern: {}", currentPatternName); + } else { + // Subsequent patterns - use combiner + if (patternCombiner != null) { + currentPattern = patternCombiner.apply(currentPatternName); + } else { + // Default to next() + currentPattern = currentPattern.next(currentPatternName); + } + LOG.debug("Added pattern node: {}", currentPatternName); + } + + // Add condition if we have expressions or need type matching + // Use currentEventType for matching unless strictTypeMatching explicitly disables it + String typePattern = currentEventType; + LOG.debug( + "Building pattern '{}' with eventType '{}' and {} expressions", + currentPatternName, + typePattern, + currentExpressions.size()); + for (DslExpression expr : currentExpressions) { + LOG.debug(" Expression: {}", expr); + } + DslCondition condition = + new DslCondition<>(eventAdapter, typePattern, currentExpressions, currentLogicalOp); + currentPattern = currentPattern.where(condition); + + // Apply quantifiers + applyQuantifiers(); + + // Reset combiner for next pattern + patternCombiner = null; + } + + private void applyQuantifiers() { + // Handle specific quantifier types + if (quantifierLower != null && quantifierUpper != null) { + // Range quantifier {n,m} + if (quantifierUpper == Integer.MAX_VALUE) { + currentPattern = currentPattern.timesOrMore(quantifierLower); + } else { + currentPattern = currentPattern.times(quantifierLower, quantifierUpper); + } + } else if (quantifierLower != null) { + // Exact quantifier {n} + currentPattern = currentPattern.times(quantifierLower); + } + + // Apply optional modifier + if (isOptional) { + currentPattern = currentPattern.optional(); + } + + // Apply greedy modifier + if (isGreedy) { + currentPattern = currentPattern.greedy(); + } + } + + @Override + public void enterFollowedBy(CepDslParser.FollowedByContext ctx) { + patternCombiner = name -> currentPattern.followedBy(name); + } + + @Override + public void enterFollowedByAny(CepDslParser.FollowedByAnyContext ctx) { + patternCombiner = name -> currentPattern.followedByAny(name); + } + + @Override + public void enterNotFollowedBy(CepDslParser.NotFollowedByContext ctx) { + patternCombiner = name -> currentPattern.notFollowedBy(name); + } + + @Override + public void enterPlus_quantifier(CepDslParser.Plus_quantifierContext ctx) { + quantifierLower = 1; + quantifierUpper = Integer.MAX_VALUE; + } + + @Override + public void enterStar_quantifier(CepDslParser.Star_quantifierContext ctx) { + quantifierLower = 0; + quantifierUpper = Integer.MAX_VALUE; + } + + @Override + public void enterNumber_quantifier(CepDslParser.Number_quantifierContext ctx) { + // Will be handled when number constants are encountered + numberConsumerStack.push( + (t, v) -> { + if (t.quantifierLower == null) { + t.quantifierLower = v; + } else { + t.quantifierUpper = v; + } + }); + } + + @Override + public void exitNumber_quantifier(CepDslParser.Number_quantifierContext ctx) { + numberConsumerStack.pop(); + } + + @Override + public void enterNumber_quantifier_greedy(CepDslParser.Number_quantifier_greedyContext ctx) { + isGreedy = true; + enterNumber_quantifier(null); + } + + @Override + public void exitNumber_quantifier_greedy(CepDslParser.Number_quantifier_greedyContext ctx) { + exitNumber_quantifier(null); + } + + @Override + public void enterUpper_bound_unlimited(CepDslParser.Upper_bound_unlimitedContext ctx) { + quantifierUpper = Integer.MAX_VALUE; + } + + @Override + public void enterEvalOrExpression(CepDslParser.EvalOrExpressionContext ctx) { + // If this OR expression has multiple AND expressions, use OR logic + // Check if there are multiple operands (separated by OR) + System.out.println( + "DEBUG: enterEvalOrExpression with " + + ctx.evalAndExpression().size() + + " AND expressions"); + if (ctx.evalAndExpression().size() > 1) { + currentLogicalOp = DslCondition.LogicalOperator.OR; + System.out.println("DEBUG: Set logical operator to OR"); + } + } + + @Override + public void enterEvalAndExpression(CepDslParser.EvalAndExpressionContext ctx) { + currentLogicalOp = DslCondition.LogicalOperator.AND; + } + + @Override + public void enterEvalEqualsExpression(CepDslParser.EvalEqualsExpressionContext ctx) { + // Determine operator + if (ctx.eq != null) { + currentOperator = ComparisonOperator.EQUALS; + } else if (ctx.ne != null) { + currentOperator = ComparisonOperator.NOT_EQUALS; + } + } + + @Override + public void enterEvalRelationalExpression(CepDslParser.EvalRelationalExpressionContext ctx) { + if (ctx.r != null) { + String op = ctx.r.getText(); + currentOperator = ComparisonOperator.fromSymbol(op); + } + inRelationalExpression = true; + } + + @Override + public void exitEvalRelationalExpression(CepDslParser.EvalRelationalExpressionContext ctx) { + // Only create expressions for actual relational operations (>, <, >=, <=) + // Simple equals expressions are handled in exitEvalEqualsExpression + if (ctx.r != null && currentAttribute != null && currentOperator != null) { + DslExpression expr; + if (currentRefPattern != null) { + // Event reference: currentAttribute > currentRefPattern.currentRefAttribute + expr = + new DslExpression<>( + currentAttribute, + currentOperator, + currentRefPattern, + currentRefAttribute); + } else if (currentValue != null) { + // Constant comparison: currentAttribute > currentValue + expr = new DslExpression<>(currentAttribute, currentOperator, currentValue); + } else { + // Invalid state - skip expression creation + System.out.println( + "DEBUG: Skipping relational expression creation - currentAttribute: " + + currentAttribute + + ", currentOperator: " + + currentOperator + + ", currentValue: " + + currentValue + + ", currentRefPattern: " + + currentRefPattern); + inRelationalExpression = false; + return; + } + currentExpressions.add(expr); + System.out.println("DEBUG: Created relational expression: " + expr); + + // Reset state + currentAttribute = null; + currentOperator = null; + currentValue = null; + currentRefPattern = null; + currentRefAttribute = null; + } + inRelationalExpression = false; + } + + @Override + public void exitEvalEqualsExpression(CepDslParser.EvalEqualsExpressionContext ctx) { + // Create expression for equals operations + System.out.println( + "DEBUG: exitEvalEqualsExpression - currentAttribute: " + + currentAttribute + + ", currentOperator: " + + currentOperator + + ", currentValue: " + + currentValue + + ", currentRefPattern: " + + currentRefPattern); + if (currentAttribute != null && currentOperator != null && currentValue != null) { + DslExpression expr; + if (currentRefPattern != null) { + expr = + new DslExpression<>( + currentAttribute, + currentOperator, + currentRefPattern, + currentRefAttribute); + } else { + expr = new DslExpression<>(currentAttribute, currentOperator, currentValue); + } + currentExpressions.add(expr); + System.out.println("DEBUG: Created equals expression: " + expr); + + // Reset state + currentAttribute = null; + currentOperator = null; + currentValue = null; + currentRefPattern = null; + currentRefAttribute = null; + } else { + System.out.println("DEBUG: Cannot create equals expression - missing components"); + } + } + + @Override + public void enterEventPropertyIdent(CepDslParser.EventPropertyIdentContext ctx) { + String ident = extractEventProperty(ctx); + + // Check if this is part of an event reference (e.g., START.price) + if (ctx.getParent() != null && ctx.getParent().getParent() != null) { + org.antlr.v4.runtime.tree.ParseTree grandParent = ctx.getParent().getParent(); + + // If grandparent has 3 children and middle one is DOT, it's an event reference + if (grandParent.getChildCount() == 3) { + String fullText = grandParent.getText(); + if (fullText.contains(".")) { + String[] parts = fullText.split("\\."); + if (parts.length == 2) { + String refPattern = parts[0]; + String refAttribute = parts[1]; + + // Only process when current ident matches the attribute part (second part) + if (ident.equals(refAttribute)) { + // This is the attribute part of event reference + if (currentAttribute != null && currentOperator != null) { + // Right side reference: price > START.price + currentRefPattern = refPattern; + currentRefAttribute = refAttribute; + } else { + // Left side reference: START.price > 100 + currentAttribute = refAttribute; + } + return; // Skip normal processing - THIS IS CRITICAL! + } else if (ident.equals(refPattern)) { + // This is the pattern name part, skip it + return; // Skip normal processing - THIS IS CRITICAL! + } + } + } + } + } + + // Normal processing for simple attributes + if (currentAttribute == null) { + currentAttribute = ident; + } else if (currentRefPattern == null) { + // Only set if different to avoid duplicate calls + if (!ident.equals(currentAttribute)) { + currentRefPattern = currentAttribute; + currentRefAttribute = ident; + } + } + } + + private String extractEventProperty(CepDslParser.EventPropertyIdentContext ctx) { + // Simple extraction - can be enhanced for complex properties + if (ctx.ipi != null) { + return ctx.ipi.getText(); + } + return ctx.getText(); + } + + @Override + public void exitNumberconstant(CepDslParser.NumberconstantContext ctx) { + try { + int value = Integer.parseInt(ctx.getText()); + if (!numberConsumerStack.isEmpty()) { + numberConsumerStack.peek().accept(this, value); + } else { + currentValue = value; + } + } catch (NumberFormatException e) { + currentValue = Double.parseDouble(ctx.getText()); + } + + // Check if we can create an expression now that we have the value + // Only create if we're not in a relational expression (those are handled in + // exitEvalRelationalExpression) + if (!inRelationalExpression + && currentAttribute != null + && currentOperator != null + && currentValue != null) { + DslExpression expr; + if (currentRefPattern != null) { + expr = + new DslExpression<>( + currentAttribute, + currentOperator, + currentRefPattern, + currentRefAttribute); + } else { + expr = new DslExpression<>(currentAttribute, currentOperator, currentValue); + } + currentExpressions.add(expr); + + // Reset state + currentAttribute = null; + currentOperator = null; + currentValue = null; + currentRefPattern = null; + currentRefAttribute = null; + } + } + + @Override + public void exitStringconstant(CepDslParser.StringconstantContext ctx) { + String value = extractStringLiteral(ctx.getText()); + System.out.println( + "DEBUG: exitStringconstant: ctx.getText()='" + + ctx.getText() + + "', extracted value='" + + value + + "'"); + currentValue = value; + System.out.println("DEBUG: Set currentValue to: " + currentValue); + } + + @Override + public void exitConstant(CepDslParser.ConstantContext ctx) { + if (ctx.t != null) { + currentValue = true; + } else if (ctx.f != null) { + currentValue = false; + } else if (ctx.nu != null) { + currentValue = null; + } + + // Check if we can create an expression now that we have the value + // Only create if we're not in a relational expression (those are handled in + // exitEvalRelationalExpression) + if (!inRelationalExpression + && currentAttribute != null + && currentOperator != null + && currentValue != null) { + DslExpression expr; + if (currentRefPattern != null) { + expr = + new DslExpression<>( + currentAttribute, + currentOperator, + currentRefPattern, + currentRefAttribute); + } else { + expr = new DslExpression<>(currentAttribute, currentOperator, currentValue); + } + currentExpressions.add(expr); + + // Reset state + currentAttribute = null; + currentOperator = null; + currentValue = null; + currentRefPattern = null; + currentRefAttribute = null; + } + } + + @Override + public void enterTimeWindow(CepDslParser.TimeWindowContext ctx) { + // Extract time value and unit + int timeValue = Integer.parseInt(ctx.c.getText()); + String unit = ctx.u.getText(); + + Duration duration; + switch (unit.toLowerCase()) { + case "h": + duration = Duration.ofHours(timeValue); + break; + case "m": + duration = Duration.ofMinutes(timeValue); + break; + case "s": + duration = Duration.ofSeconds(timeValue); + break; + case "ms": + duration = Duration.ofMillis(timeValue); + break; + default: + throw new DslCompilationException("Unknown time unit: " + unit); + } + + if (currentPattern != null) { + currentPattern = currentPattern.within(duration); + LOG.debug("Added time window: {} {}", timeValue, unit); + } + } + + private String extractStringLiteral(String text) { + if (text == null || text.length() < 2) { + return text; + } + // Remove quotes + return text.substring(1, text.length() - 1); + } + + public Pattern getPattern() { + return currentPattern; + } +} diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/util/CaseInsensitiveInputStream.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/util/CaseInsensitiveInputStream.java new file mode 100644 index 0000000000000..5da185b49ccfa --- /dev/null +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/util/CaseInsensitiveInputStream.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cep.dsl.util; + +import org.apache.flink.annotation.Internal; + +import org.antlr.v4.runtime.CharStream; +import org.antlr.v4.runtime.CharStreams; +import org.antlr.v4.runtime.misc.Interval; + +/** + * Case-insensitive character stream for ANTLR parsing. + * + *

This stream converts all characters to lowercase for parsing, making DSL keywords + * case-insensitive (e.g., "AND", "and", "And" are all equivalent). + */ +@Internal +public class CaseInsensitiveInputStream implements CharStream { + + private final CharStream stream; + + public CaseInsensitiveInputStream(String input) { + this.stream = CharStreams.fromString(input); + } + + @Override + public String getText(Interval interval) { + return stream.getText(interval); + } + + @Override + public void consume() { + stream.consume(); + } + + @Override + public int LA(int i) { + int c = stream.LA(i); + if (c <= 0) { + return c; + } + return Character.toLowerCase(c); + } + + @Override + public int mark() { + return stream.mark(); + } + + @Override + public void release(int marker) { + stream.release(marker); + } + + @Override + public int index() { + return stream.index(); + } + + @Override + public void seek(int index) { + stream.seek(index); + } + + @Override + public int size() { + return stream.size(); + } + + @Override + public String getSourceName() { + return stream.getSourceName(); + } +} diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/util/MapEventAdapter.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/util/MapEventAdapter.java new file mode 100644 index 0000000000000..cdbf197f3357c --- /dev/null +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/util/MapEventAdapter.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cep.dsl.util; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.cep.dsl.api.EventAdapter; + +import java.util.Map; +import java.util.Optional; + +/** + * Event adapter for Map-based events. + * + *

This adapter supports events represented as {@code Map}, where attribute names + * are map keys. This is useful for dynamic or schema-less event types. + * + *

Example usage: + * + *

{@code
+ * DataStream> events = ...;
+ * MapEventAdapter adapter = new MapEventAdapter();
+ *
+ * PatternStream> pattern = DslCompiler.compile(
+ *     "Alert(severity > 5 and type = 'error')",
+ *     events,
+ *     adapter
+ * );
+ * }
+ * + *

By default, the event type is extracted from a special key {@code _eventType}. If this key is + * not present, the type "Event" is returned. You can customize the event type key by using {@link + * #MapEventAdapter(String)}. + */ +@PublicEvolving +public class MapEventAdapter implements EventAdapter> { + + private static final long serialVersionUID = 1L; + + /** Default key used to store event type in the map. */ + public static final String DEFAULT_EVENT_TYPE_KEY = "_eventType"; + + private final String eventTypeKey; + + /** Creates a MapEventAdapter with the default event type key. */ + public MapEventAdapter() { + this(DEFAULT_EVENT_TYPE_KEY); + } + + /** + * Creates a MapEventAdapter with a custom event type key. + * + * @param eventTypeKey The key in the map that contains the event type + */ + public MapEventAdapter(String eventTypeKey) { + this.eventTypeKey = eventTypeKey; + } + + @Override + public Optional getAttribute(Map event, String attributeName) { + if (event == null) { + return Optional.empty(); + } + + // Handle nested attributes (e.g., "user.name") + if (attributeName.contains(".")) { + return getNestedAttribute(event, attributeName); + } + + return Optional.ofNullable(event.get(attributeName)); + } + + @Override + public String getEventType(Map event) { + if (event == null) { + return "null"; + } + + Object typeValue = event.get(eventTypeKey); + if (typeValue != null) { + return typeValue.toString(); + } + + // Default to "Event" if no type key is present + return "Event"; + } + + /** + * Handle nested attribute access for Map events. + * + *

For example, "user.name" will first get the "user" object from the map, and then extract + * the "name" attribute from it. + */ + @SuppressWarnings("unchecked") + private Optional getNestedAttribute(Map event, String attributeName) { + String[] parts = attributeName.split("\\.", 2); + String first = parts[0]; + String rest = parts[1]; + + Object intermediate = event.get(first); + if (intermediate == null) { + return Optional.empty(); + } + + // If intermediate is also a Map, continue with MapEventAdapter + if (intermediate instanceof Map) { + return getNestedAttribute((Map) intermediate, rest); + } + + // Otherwise, use reflection for nested object + ReflectiveEventAdapter reflectiveAdapter = new ReflectiveEventAdapter<>(); + return reflectiveAdapter.getAttribute(intermediate, rest); + } +} diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/util/ReflectiveEventAdapter.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/util/ReflectiveEventAdapter.java new file mode 100644 index 0000000000000..02bdc7c16e9f8 --- /dev/null +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/dsl/util/ReflectiveEventAdapter.java @@ -0,0 +1,239 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cep.dsl.util; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.cep.dsl.api.EventAdapter; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Serializable; +import java.lang.reflect.Field; +import java.lang.reflect.Method; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; + +/** + * Event adapter that uses Java reflection to access POJO fields and getters. + * + *

This adapter automatically discovers and caches field accessors for performance. It supports: + * + *

    + *
  • Public getter methods (e.g., {@code getTemperature()} for attribute "temperature") + *
  • Direct field access (if no getter is found) + *
  • Nested attributes using dot notation (e.g., "sensor.location.city") + *
+ * + *

Thread-safe and optimized for repeated access to the same attributes. + * + * @param The type of events to adapt + */ +@PublicEvolving +public class ReflectiveEventAdapter implements EventAdapter { + + private static final long serialVersionUID = 1L; + private static final Logger LOG = LoggerFactory.getLogger(ReflectiveEventAdapter.class); + + // Cache for field accessors (not serializable, will be rebuilt after deserialization) + private transient ConcurrentHashMap cache; + + @Override + public Optional getAttribute(T event, String attributeName) { + if (event == null) { + return Optional.empty(); + } + + initCache(); + + // Handle nested attributes (e.g., "user.name") + if (attributeName.contains(".")) { + return getNestedAttribute(event, attributeName); + } + + // Get or create accessor for this attribute + String cacheKey = event.getClass().getName() + "." + attributeName; + FieldAccessor accessor = + cache.computeIfAbsent( + cacheKey, key -> createAccessor(event.getClass(), attributeName)); + + return accessor.get(event); + } + + @Override + public String getEventType(T event) { + if (event == null) { + return "null"; + } + + // Try to get eventType attribute first + Optional eventTypeAttr = getAttribute(event, "eventType"); + if (eventTypeAttr.isPresent()) { + return String.valueOf(eventTypeAttr.get()); + } + + // Fall back to class simple name + return event.getClass().getSimpleName(); + } + + /** Handle nested attribute access (e.g., "sensor.location.city"). */ + private Optional getNestedAttribute(Object event, String attributeName) { + String[] parts = attributeName.split("\\.", 2); + String first = parts[0]; + String rest = parts[1]; + + Optional intermediate = getAttribute((T) event, first); + if (!intermediate.isPresent()) { + return Optional.empty(); + } + + // Recursively access nested attribute + ReflectiveEventAdapter nestedAdapter = new ReflectiveEventAdapter<>(); + return nestedAdapter.getAttribute(intermediate.get(), rest); + } + + /** Initialize the cache if needed (handles deserialization). */ + private void initCache() { + if (cache == null) { + cache = new ConcurrentHashMap<>(); + } + } + + /** Create an accessor for the given class and field name. */ + private FieldAccessor createAccessor(Class clazz, String fieldName) { + // Try getter method first (e.g., getTemperature()) + String getterName = "get" + capitalize(fieldName); + try { + Method method = clazz.getMethod(getterName); + LOG.debug( + "Created method accessor for {}.{} using getter {}", + clazz.getName(), + fieldName, + getterName); + return new MethodAccessor(method); + } catch (NoSuchMethodException e) { + // Try boolean getter (e.g., isActive()) + String booleanGetterName = "is" + capitalize(fieldName); + try { + Method method = clazz.getMethod(booleanGetterName); + LOG.debug( + "Created method accessor for {}.{} using boolean getter {}", + clazz.getName(), + fieldName, + booleanGetterName); + return new MethodAccessor(method); + } catch (NoSuchMethodException e2) { + // Fall back to direct field access + try { + Field field = findField(clazz, fieldName); + field.setAccessible(true); + LOG.debug( + "Created field accessor for {}.{} using direct field access", + clazz.getName(), + fieldName); + return new DirectFieldAccessor(field); + } catch (NoSuchFieldException e3) { + LOG.warn( + "No accessor found for attribute '{}' on class {}", + fieldName, + clazz.getName()); + return new NullAccessor(); + } + } + } + } + + /** Find a field in the class hierarchy. */ + private Field findField(Class clazz, String fieldName) throws NoSuchFieldException { + Class current = clazz; + while (current != null) { + try { + return current.getDeclaredField(fieldName); + } catch (NoSuchFieldException e) { + current = current.getSuperclass(); + } + } + throw new NoSuchFieldException("Field '" + fieldName + "' not found in " + clazz.getName()); + } + + /** Capitalize the first letter of a string. */ + private String capitalize(String str) { + if (str == null || str.isEmpty()) { + return str; + } + return str.substring(0, 1).toUpperCase() + str.substring(1); + } + + // Internal accessor interfaces + + /** Interface for accessing a field/method. */ + private interface FieldAccessor extends Serializable { + Optional get(Object obj); + } + + /** Accessor using a getter method. */ + private static class MethodAccessor implements FieldAccessor { + private static final long serialVersionUID = 1L; + private final Method method; + + MethodAccessor(Method method) { + this.method = method; + } + + @Override + public Optional get(Object obj) { + try { + return Optional.ofNullable(method.invoke(obj)); + } catch (Exception e) { + LOG.warn("Failed to invoke method {} on object", method.getName(), e); + return Optional.empty(); + } + } + } + + /** Accessor using direct field access. */ + private static class DirectFieldAccessor implements FieldAccessor { + private static final long serialVersionUID = 1L; + private final Field field; + + DirectFieldAccessor(Field field) { + this.field = field; + } + + @Override + public Optional get(Object obj) { + try { + return Optional.ofNullable(field.get(obj)); + } catch (Exception e) { + LOG.warn("Failed to access field {} on object", field.getName(), e); + return Optional.empty(); + } + } + } + + /** Null accessor when no field/method is found. */ + private static class NullAccessor implements FieldAccessor { + private static final long serialVersionUID = 1L; + + @Override + public Optional get(Object obj) { + return Optional.empty(); + } + } +} diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/DebugDslTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/DebugDslTest.java new file mode 100644 index 0000000000000..7163b79a26ead --- /dev/null +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/DebugDslTest.java @@ -0,0 +1,102 @@ +package org.apache.flink.cep.dsl; + +import org.apache.flink.cep.PatternStream; +import org.apache.flink.cep.dsl.api.DslCompiler; +import org.apache.flink.cep.dsl.model.StockEvent; +import org.apache.flink.cep.dsl.util.DslTestDataSets; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +import java.util.ArrayList; +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class DebugDslTest { + + @RegisterExtension + private static final MiniClusterExtension MINI_CLUSTER_RESOURCE = new MiniClusterExtension(); + + @Test + public void testEventReferenceSimple() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + // Simple test: two events where second price > first price + DataStream input = + env.fromData( + new StockEvent( + "AAPL", "TRADE", 100.0, 1000, DslTestDataSets.ts(0), "NASDAQ", 0.0), + new StockEvent( + "AAPL", + "TRADE", + 105.0, + 1100, + DslTestDataSets.ts(1), + "NASDAQ", + 0.0)); + + String dslExpression = "A(price > 0) -> B(price > A.price)"; + System.out.println("\n===== TEST: Event Reference ====="); + System.out.println("DSL: " + dslExpression); + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = + patternStream.select( + match -> { + double aPrice = match.get("A").get(0).getPrice(); + double bPrice = match.get("B").get(0).getPrice(); + return aPrice + "->" + bPrice; + }); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + System.out.println("Results: " + results); + assertEquals(1, results.size(), "Should match one pattern"); + assertTrue(results.get(0).contains("100.0")); + assertTrue(results.get(0).contains("105.0")); + } + + @Test + public void testTwoPatterns() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", "TRADE", 100.0, 1000, DslTestDataSets.ts(0), "NASDAQ", 0.0), + new StockEvent( + "AAPL", + "TRADE", + 105.0, + 1100, + DslTestDataSets.ts(1), + "NASDAQ", + 0.0)); + + String dslExpression = "TRADE(price > 0) -> TRADE(price > 0)"; + System.out.println("\n===== TEST: Two TRADE Patterns ====="); + System.out.println("DSL: " + dslExpression); + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = + patternStream.select( + match -> { + String keys = String.join(",", match.keySet()); + return "Keys: " + keys; + }); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + System.out.println("Results: " + results); + System.out.println("Result count: " + results.size()); + } +} diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/DebugEventRefTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/DebugEventRefTest.java new file mode 100644 index 0000000000000..7ec75e166a69c --- /dev/null +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/DebugEventRefTest.java @@ -0,0 +1,93 @@ +package org.apache.flink.cep.dsl; + +import org.apache.flink.cep.PatternStream; +import org.apache.flink.cep.dsl.api.DslCompiler; +import org.apache.flink.cep.dsl.model.StockEvent; +import org.apache.flink.cep.dsl.util.DslTestDataSets; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +import java.util.ArrayList; +import java.util.List; + +public class DebugEventRefTest { + + @RegisterExtension + private static final MiniClusterExtension MINI_CLUSTER_RESOURCE = new MiniClusterExtension(); + + @Test + public void debugSimpleReference() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", "TRADE", 105.0, 1100, DslTestDataSets.ts(0), "NASDAQ", 0.0), + new StockEvent( + "AAPL", + "TRADE", + 110.0, + 1500, + DslTestDataSets.ts(1), + "NASDAQ", + 0.0)); + + // Test 1: Without reference - should work + System.out.println("\n=== Test 1: Without event reference ==="); + String dsl1 = "START(price > 100) -> INCREASE(price > 105)"; + System.out.println("DSL: " + dsl1); + + PatternStream ps1 = DslCompiler.compile(dsl1, input); + List results1 = new ArrayList<>(); + ps1.select( + m -> + m.get("START").get(0).getPrice() + + "->" + + m.get("INCREASE").get(0).getPrice()) + .executeAndCollect() + .forEachRemaining(results1::add); + + System.out.println("Results: " + results1); + System.out.println("Match count: " + results1.size()); + + // Test 2: With constant reference - simplified + System.out.println("\n=== Test 2: With event reference (constant comparison) ==="); + String dsl2 = "START(price > 0) -> INCREASE(price > 100)"; + System.out.println("DSL: " + dsl2); + + PatternStream ps2 = DslCompiler.compile(dsl2, input); + List results2 = new ArrayList<>(); + ps2.select( + m -> + m.get("START").get(0).getPrice() + + "->" + + m.get("INCREASE").get(0).getPrice()) + .executeAndCollect() + .forEachRemaining(results2::add); + + System.out.println("Results: " + results2); + System.out.println("Match count: " + results2.size()); + + // Test 3: With event reference + System.out.println("\n=== Test 3: With event reference ==="); + String dsl3 = "START(price > 100) -> INCREASE(price > START.price)"; + System.out.println("DSL: " + dsl3); + + PatternStream ps3 = DslCompiler.compile(dsl3, input); + List results3 = new ArrayList<>(); + ps3.select( + m -> + m.get("START").get(0).getPrice() + + "->" + + m.get("INCREASE").get(0).getPrice()) + .executeAndCollect() + .forEachRemaining(results3::add); + + System.out.println("Results: " + results3); + System.out.println("Match count: " + results3.size()); + } +} diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/DslCompilerE2ETest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/DslCompilerE2ETest.java new file mode 100644 index 0000000000000..f0fab80fcc7b0 --- /dev/null +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/DslCompilerE2ETest.java @@ -0,0 +1,291 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cep.dsl; + +import org.apache.flink.cep.PatternStream; +import org.apache.flink.cep.dsl.api.DslCompiler; +import org.apache.flink.cep.dsl.model.SensorEvent; +import org.apache.flink.cep.dsl.model.StockEvent; +import org.apache.flink.cep.dsl.model.UserActivityEvent; +import org.apache.flink.cep.dsl.util.DslTestDataSets; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.util.AbstractTestBaseJUnit4; + +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** End-to-end tests for complex DSL scenarios. */ +public class DslCompilerE2ETest extends AbstractTestBaseJUnit4 { + + @Test + public void testComplexFinancialPattern() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = env.fromCollection(DslTestDataSets.priceIncreasePattern()); + + // Complex DSL: Detect sustained price increase with volume + String dslExpression = + "START(symbol = 'AAPL' and price > 150 and volume > 1000) " + + "-> INCREASE1(price > START.price) " + + "-> INCREASE2(price > INCREASE1.price)"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = + patternStream.select( + match -> { + StockEvent start = match.get("START").get(0); + StockEvent increase2 = match.get("INCREASE2").get(0); + double priceGain = + ((increase2.getPrice() - start.getPrice()) / start.getPrice()) + * 100; + return start.getSymbol() + ":" + String.format("%.2f", priceGain); + }); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + assertTrue(results.size() > 0); + assertTrue(results.stream().anyMatch(r -> r.startsWith("AAPL:"))); + } + + @Test + public void testIoTAnomalyDetection() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = env.fromCollection(DslTestDataSets.escalatingValues()); + + String dslExpression = + "NORMAL(status = 'NORMAL' and value < 35) " + + "-> WARNING(status = 'WARNING' and value > NORMAL.value) " + + "-> CRITICAL(status = 'CRITICAL' and value > WARNING.value)"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = + patternStream.select( + match -> + "ANOMALY:" + + match.get("NORMAL").get(0).getValue() + + "->" + + match.get("WARNING").get(0).getValue() + + "->" + + match.get("CRITICAL").get(0).getValue()); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + assertTrue(results.size() > 0); + assertTrue(results.stream().anyMatch(r -> r.startsWith("ANOMALY:"))); + } + + @Test + public void testUserJourneyFunnel() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromCollection(DslTestDataSets.userJourneyDataset()); + + String dslExpression = + "LOGIN(eventType = 'LOGIN') " + + "-> CLICK(eventType = 'CLICK' and duration > 10) " + + "-> PURCHASE(eventType = 'PURCHASE' and count > 0)"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = + patternStream.select( + match -> + "FUNNEL:" + + match.get("LOGIN").get(0).getUserId() + + ":" + + match.get("PURCHASE").get(0).getEventType()); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + assertTrue(results.size() >= 1); + assertTrue(results.stream().anyMatch(r -> r.contains("PURCHASE"))); + } + + @Test + public void testMultiStepPricePattern() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", "TRADE", 100.0, 1000, DslTestDataSets.ts(0), "NASDAQ", 0.0), + new StockEvent( + "AAPL", "TRADE", 102.0, 1100, DslTestDataSets.ts(1), "NASDAQ", 2.0), + new StockEvent( + "AAPL", "TRADE", 104.0, 1200, DslTestDataSets.ts(2), "NASDAQ", 4.0), + new StockEvent( + "AAPL", + "TRADE", + 106.0, + 1300, + DslTestDataSets.ts(3), + "NASDAQ", + 6.0)); + + String dslExpression = + "A(price > 0) " + + "-> B(price > A.price) " + + "-> C(price > B.price) " + + "-> D(price > C.price)"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = patternStream.select(match -> "MATCH"); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + assertEquals(1, results.size()); + } + + @Test + public void testCombinedConditionsPattern() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", "TRADE", 100.0, 1000, DslTestDataSets.ts(0), "NASDAQ", 0.0), + new StockEvent( + "AAPL", "TRADE", 105.0, 1200, DslTestDataSets.ts(1), "NASDAQ", 5.0), + new StockEvent( + "GOOGL", "TRADE", 110.0, 800, DslTestDataSets.ts(2), "NYSE", 1.0), + new StockEvent( + "AAPL", + "TRADE", + 110.0, + 1500, + DslTestDataSets.ts(3), + "NASDAQ", + 10.0)); + + String dslExpression = + "START(symbol = 'AAPL' and price > 100 and volume > 1000) " + + "-> END(symbol = 'AAPL' and price > START.price and volume > START.volume)"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = patternStream.select(match -> match.get("END").get(0)); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + assertTrue(results.size() >= 1); + assertTrue( + results.stream() + .allMatch(e -> e.getSymbol().equals("AAPL") && e.getVolume() > 1200)); + } + + @Test + public void testStringComparisonPattern() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", "TRADE", 100.0, 1000, DslTestDataSets.ts(0), "NASDAQ", 0.0), + new StockEvent( + "GOOGL", + "TRADE", + 100.0, + 1000, + DslTestDataSets.ts(1), + "NASDAQ", + 0.0), + new StockEvent( + "AAPL", "TRADE", 100.0, 1000, DslTestDataSets.ts(2), "NYSE", 0.0)); + + String dslExpression = "TRADE(symbol = 'AAPL' and exchange = 'NASDAQ')"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = + patternStream.select( + match -> + match.get("TRADE").get(0).getSymbol() + + ":" + + match.get("TRADE").get(0).getExchange()); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + assertEquals(1, results.size()); + assertEquals("AAPL:NASDAQ", results.get(0)); + } + + @Test + public void testQuantifierWithSequence() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", "TRADE", 100.0, 1000, DslTestDataSets.ts(0), "NASDAQ", 0.0), + new StockEvent( + "AAPL", "TRADE", 105.0, 1100, DslTestDataSets.ts(1), "NASDAQ", 5.0), + new StockEvent( + "AAPL", + "TRADE", + 110.0, + 1200, + DslTestDataSets.ts(2), + "NASDAQ", + 10.0), + new StockEvent( + "AAPL", + "QUOTE", + 111.0, + 1300, + DslTestDataSets.ts(3), + "NASDAQ", + 11.0)); + + String dslExpression = + "TRADES(eventType = 'TRADE' and symbol = 'AAPL') " + + "-> QUOTE(eventType = 'QUOTE')"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = + patternStream.select( + match -> + match.get("TRADES").get(0).getEventType() + + "->" + + match.get("QUOTE").get(0).getEventType()); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + assertTrue(results.size() >= 1); + assertTrue(results.stream().allMatch(r -> r.contains("QUOTE"))); + } +} diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/DslErrorHandlingTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/DslErrorHandlingTest.java new file mode 100644 index 0000000000000..055a105db82fd --- /dev/null +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/DslErrorHandlingTest.java @@ -0,0 +1,217 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cep.dsl; + +import org.apache.flink.cep.dsl.api.DslCompiler; +import org.apache.flink.cep.dsl.exception.DslCompilationException; +import org.apache.flink.cep.dsl.model.StockEvent; +import org.apache.flink.cep.dsl.util.DslTestDataSets; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.util.AbstractTestBaseJUnit4; + +import org.junit.Test; + +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** Tests for DSL error handling. */ +public class DslErrorHandlingTest extends AbstractTestBaseJUnit4 { + + @Test + public void testSyntaxError() { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", + "TRADE", + 150.0, + 1000, + DslTestDataSets.ts(0), + "NASDAQ", + 0.0)); + + String invalidDsl = "TRADE(price >>>)"; // Invalid syntax + + try { + DslCompiler.compile(invalidDsl, input); + fail("Expected DslCompilationException"); + } catch (DslCompilationException e) { + assertTrue(e.getMessage().contains("Syntax error") || e.getMessage().contains("parse")); + } + } + + @Test + public void testMissingWhereClause() { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", + "TRADE", + 150.0, + 1000, + DslTestDataSets.ts(0), + "NASDAQ", + 0.0)); + + String invalidDsl = "TRADE symbol = 'AAPL'"; // Missing WHERE + + try { + DslCompiler.compile(invalidDsl, input); + fail("Expected DslCompilationException"); + } catch (DslCompilationException e) { + assertTrue( + e.getMessage().contains("WHERE") + || e.getMessage().contains("Syntax") + || e.getMessage().contains("parse")); + } + } + + @Test + public void testEmptyPattern() { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", + "TRADE", + 150.0, + 1000, + DslTestDataSets.ts(0), + "NASDAQ", + 0.0)); + + String invalidDsl = ""; // Empty string + + try { + DslCompiler.compile(invalidDsl, input); + fail("Expected DslCompilationException or IllegalArgumentException"); + } catch (Exception e) { + assertTrue( + e instanceof DslCompilationException || e instanceof IllegalArgumentException); + } + } + + @Test + public void testInvalidOperator() { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", + "TRADE", + 150.0, + 1000, + DslTestDataSets.ts(0), + "NASDAQ", + 0.0)); + + String invalidDsl = "TRADE(price <> 150)"; // Invalid operator + + try { + DslCompiler.compile(invalidDsl, input); + fail("Expected DslCompilationException"); + } catch (DslCompilationException e) { + assertTrue(e.getMessage().contains("Syntax") || e.getMessage().contains("parse")); + } + } + + @Test + public void testUnbalancedParentheses() { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", + "TRADE", + 150.0, + 1000, + DslTestDataSets.ts(0), + "NASDAQ", + 0.0)); + + String invalidDsl = "TRADE((price > 100)"; // Missing closing paren + + try { + DslCompiler.compile(invalidDsl, input); + fail("Expected DslCompilationException"); + } catch (DslCompilationException e) { + assertTrue( + e.getMessage().contains("Syntax") + || e.getMessage().contains("parse") + || e.getMessage().contains("parenthes")); + } + } + + @Test + public void testInvalidQuantifier() { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", + "TRADE", + 150.0, + 1000, + DslTestDataSets.ts(0), + "NASDAQ", + 0.0)); + + String invalidDsl = "TRADE{-1}(price > 0)"; // Negative quantifier + + try { + DslCompiler.compile(invalidDsl, input); + fail("Expected DslCompilationException"); + } catch (DslCompilationException e) { + assertTrue( + e.getMessage().contains("quantifier") + || e.getMessage().contains("Syntax") + || e.getMessage().contains("parse")); + } + } + + @Test + public void testValidPatternNoError() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", + "TRADE", + 150.0, + 1000, + DslTestDataSets.ts(0), + "NASDAQ", + 0.0)); + + String validDsl = "TRADE(price > 100)"; // Valid DSL + + // Should not throw exception + DslCompiler.compile(validDsl, input); + } +} diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/DslEventCorrelationTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/DslEventCorrelationTest.java new file mode 100644 index 0000000000000..8c908502a87aa --- /dev/null +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/DslEventCorrelationTest.java @@ -0,0 +1,206 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cep.dsl; + +import org.apache.flink.cep.PatternStream; +import org.apache.flink.cep.dsl.api.DslCompiler; +import org.apache.flink.cep.dsl.model.StockEvent; +import org.apache.flink.cep.dsl.util.DslTestDataSets; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.util.AbstractTestBaseJUnit4; + +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** Tests for DSL event correlation (cross-event references). */ +public class DslEventCorrelationTest extends AbstractTestBaseJUnit4 { + + @Test + public void testSingleEventReference() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", "TRADE", 100.0, 1000, DslTestDataSets.ts(0), "NASDAQ", 0.0), + new StockEvent( + "AAPL", "TRADE", 105.0, 1200, DslTestDataSets.ts(1), "NASDAQ", 0.0), + new StockEvent( + "AAPL", + "TRADE", + 110.0, + 1500, + DslTestDataSets.ts(2), + "NASDAQ", + 0.0)); + + String dslExpression = "START(price > 100) -> INCREASE(price > START.price)"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = + patternStream.select( + match -> { + double startPrice = match.get("START").get(0).getPrice(); + double increasePrice = match.get("INCREASE").get(0).getPrice(); + return startPrice + "->" + increasePrice; + }); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + // Should match 105->110 + assertTrue(results.size() >= 1); + assertTrue(results.stream().anyMatch(r -> r.contains("105.0") && r.contains("110.0"))); + } + + @Test + public void testChainedReferences() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", "TRADE", 100.0, 1000, DslTestDataSets.ts(0), "NASDAQ", 0.0), + new StockEvent( + "AAPL", "TRADE", 105.0, 1100, DslTestDataSets.ts(1), "NASDAQ", 0.0), + new StockEvent( + "AAPL", + "TRADE", + 111.0, + 1200, + DslTestDataSets.ts(2), + "NASDAQ", + 0.0)); + + String dslExpression = "A(price > 0) -> B(price > A.price) -> C(price > B.price)"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = + patternStream.select( + match -> { + double a = match.get("A").get(0).getPrice(); + double b = match.get("B").get(0).getPrice(); + double c = match.get("C").get(0).getPrice(); + return a + "->" + b + "->" + c; + }); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + assertEquals(1, results.size()); + assertTrue(results.get(0).contains("100.0")); + assertTrue(results.get(0).contains("105.0")); + assertTrue(results.get(0).contains("111.0")); + } + + @Test + public void testComplexCorrelation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromCollection(DslTestDataSets.eventCorrelationDataset()); + + String dslExpression = + "START(price > 0) -> INCREASE(price > START.price and volume > START.volume)"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = patternStream.select(match -> match.get("INCREASE").get(0)); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + assertTrue(results.size() > 0); + // Verify that matched events have both price and volume increases + assertTrue(results.stream().allMatch(e -> e.getPrice() > 100.0 && e.getVolume() > 1000)); + } + + @Test + public void testMultiAttributeReference() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", "TRADE", 100.0, 1000, DslTestDataSets.ts(0), "NASDAQ", 0.0), + new StockEvent( + "AAPL", "TRADE", 105.0, 1200, DslTestDataSets.ts(1), "NASDAQ", 5.0), + new StockEvent( + "AAPL", + "TRADE", + 110.0, + 1500, + DslTestDataSets.ts(2), + "NASDAQ", + 10.0)); + + String dslExpression = "A(price > 0) -> B(price > A.price and volume > A.volume)"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = patternStream.select(match -> match.get("B").get(0)); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + assertTrue(results.size() >= 1); + assertTrue(results.stream().allMatch(e -> e.getPrice() > 100.0 && e.getVolume() > 1000)); + } + + @Test + public void testVolumeIncrease() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", "TRADE", 100.0, 1000, DslTestDataSets.ts(0), "NASDAQ", 0.0), + new StockEvent( + "AAPL", "TRADE", 101.0, 1500, DslTestDataSets.ts(1), "NASDAQ", 1.0), + new StockEvent( + "AAPL", + "TRADE", + 102.0, + 2000, + DslTestDataSets.ts(2), + "NASDAQ", + 2.0)); + + String dslExpression = "LOW(volume > 0) -> HIGH(volume > LOW.volume)"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = + patternStream.select(match -> match.get("HIGH").get(0).getVolume()); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + assertTrue(results.size() >= 1); + assertTrue(results.stream().allMatch(v -> v > 1000)); + } +} diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/DslExampleUsage.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/DslExampleUsage.java new file mode 100644 index 0000000000000..2209c211ff6fd --- /dev/null +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/DslExampleUsage.java @@ -0,0 +1,254 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cep.dsl; + +import org.apache.flink.cep.PatternStream; +import org.apache.flink.cep.dsl.api.DslCompiler; +import org.apache.flink.cep.dsl.api.EventAdapter; +import org.apache.flink.cep.dsl.util.MapEventAdapter; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Example usage of the CEP DSL. + * + *

This class demonstrates various ways to use the DSL with different event types and patterns. + * These are examples only and not executable tests. + */ +public class DslExampleUsage { + + // Example 1: Simple POJO events with basic pattern + public static void simplePojoExample() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + // Create sample data + DataStream sensorData = + env.fromElements( + new SensorReading("sensor1", 95.0, System.currentTimeMillis()), + new SensorReading("sensor1", 105.0, System.currentTimeMillis()), + new SensorReading("sensor1", 110.0, System.currentTimeMillis())); + + // Define pattern using DSL + PatternStream pattern = + DslCompiler.compile("HighTemp(temperature > 100)", sensorData); + + // Process matches + pattern.select( + match -> { + SensorReading reading = match.get("HighTemp").get(0); + return String.format( + "High temperature alert: Sensor %s at %.1f°C", + reading.id, reading.temperature); + }) + .print(); + + env.execute("Simple POJO Example"); + } + + // Example 2: Event correlation + public static void eventCorrelationExample() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream sensorData = + env.fromElements( + new SensorReading("sensor1", 95.0, 1000L), + new SensorReading("sensor1", 105.0, 2000L), + new SensorReading("sensor2", 110.0, 3000L)); + + // Pattern with event correlation + String dsl = + "Start(id = 'sensor1' and temperature > 90) -> " + + "End(id = Start.id and temperature > Start.temperature)"; + + PatternStream pattern = DslCompiler.compile(dsl, sensorData); + + pattern.select( + match -> { + SensorReading start = match.get("Start").get(0); + SensorReading end = match.get("End").get(0); + return String.format( + "Temperature rise detected: %.1f -> %.1f", + start.temperature, end.temperature); + }) + .print(); + + env.execute("Event Correlation Example"); + } + + // Example 3: Map-based events + public static void mapEventExample() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + // Create Map events + Map event1 = new HashMap<>(); + event1.put("_eventType", "Alert"); + event1.put("severity", 7); + event1.put("message", "High CPU usage"); + + Map event2 = new HashMap<>(); + event2.put("_eventType", "Alert"); + event2.put("severity", 9); + event2.put("message", "Critical error"); + + DataStream> alerts = env.fromElements(event1, event2); + + // Use MapEventAdapter + PatternStream> pattern = + DslCompiler.compile("Alert(severity > 5)", alerts, new MapEventAdapter()); + + pattern.select(match -> match.get("Alert").get(0).get("message")).print(); + + env.execute("Map Event Example"); + } + + // Example 4: Complex pattern with quantifiers and time window + public static void complexPatternExample() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream userEvents = + env.fromElements( + new UserEvent("user1", "login", 1000L), + new UserEvent("user1", "browse", 2000L), + new UserEvent("user1", "browse", 3000L), + new UserEvent("user1", "purchase", 4000L)); + + // Complex pattern: login -> multiple browses -> purchase, within 30 seconds + String dsl = + "%SKIP_TO_LAST['Login'] " + + "Login(action = 'login') -> " + + "Browse{1,5}(action = 'browse' and userId = Login.userId) -> " + + "Purchase(action = 'purchase' and userId = Login.userId) " + + "within 30s"; + + PatternStream pattern = DslCompiler.compile(dsl, userEvents); + + pattern.select( + match -> { + UserEvent login = match.get("Login").get(0); + List browses = match.get("Browse"); + UserEvent purchase = match.get("Purchase").get(0); + return String.format( + "User %s: login -> %d browses -> purchase", + login.userId, browses.size()); + }) + .print(); + + env.execute("Complex Pattern Example"); + } + + // Example 5: Builder API with custom adapter + public static void builderApiExample() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream events = env.fromElements(new CustomEvent()); + + // Custom event adapter + EventAdapter customAdapter = + new EventAdapter() { + @Override + public java.util.Optional getAttribute( + CustomEvent event, String attributeName) { + return java.util.Optional.ofNullable(event.getField(attributeName)); + } + + @Override + public String getEventType(CustomEvent event) { + return event.getTypeName(); + } + }; + + // Use builder API + PatternStream pattern = + DslCompiler.builder() + .withStrictTypeMatching() + .withEventAdapter(customAdapter) + .compile("MyEvent(value > 100)", events); + + pattern.select(match -> "Matched: " + match.get("MyEvent").get(0)).print(); + + env.execute("Builder API Example"); + } + + // Example event classes + + /** Simple sensor reading POJO. */ + public static class SensorReading { + public String id; + public double temperature; + public long timestamp; + + public SensorReading(String id, double temperature, long timestamp) { + this.id = id; + this.temperature = temperature; + this.timestamp = timestamp; + } + + public String getId() { + return id; + } + + public double getTemperature() { + return temperature; + } + + public long getTimestamp() { + return timestamp; + } + } + + /** User event POJO. */ + public static class UserEvent { + public String userId; + public String action; + public long timestamp; + + public UserEvent(String userId, String action, long timestamp) { + this.userId = userId; + this.action = action; + this.timestamp = timestamp; + } + + public String getUserId() { + return userId; + } + + public String getAction() { + return action; + } + + public long getTimestamp() { + return timestamp; + } + } + + /** Custom event type. */ + public static class CustomEvent { + public Object getField(String name) { + return null; // Implementation omitted + } + + public String getTypeName() { + return "MyEvent"; + } + } +} diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/DslExpressionEvaluationTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/DslExpressionEvaluationTest.java new file mode 100644 index 0000000000000..4ee60f89d1e45 --- /dev/null +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/DslExpressionEvaluationTest.java @@ -0,0 +1,339 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cep.dsl; + +import org.apache.flink.cep.PatternStream; +import org.apache.flink.cep.dsl.api.DslCompiler; +import org.apache.flink.cep.dsl.model.StockEvent; +import org.apache.flink.cep.dsl.util.DslTestDataSets; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.util.AbstractTestBaseJUnit4; + +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** Tests for DSL expression evaluation (comparison and logical operators). */ +public class DslExpressionEvaluationTest extends AbstractTestBaseJUnit4 { + + @Test + public void testEqualsOperator() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", "TRADE", 150.0, 1000, DslTestDataSets.ts(0), "NASDAQ", 0.0), + new StockEvent( + "AAPL", "TRADE", 155.0, 1200, DslTestDataSets.ts(1), "NASDAQ", 0.0), + new StockEvent( + "AAPL", "TRADE", 150.0, 800, DslTestDataSets.ts(2), "NASDAQ", 0.0)); + + String dslExpression = "TRADE(price = 150.0)"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = + patternStream.select(match -> match.get("TRADE").get(0).getPrice()); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + assertEquals(2, results.size()); + assertTrue(results.stream().allMatch(p -> p == 150.0)); + } + + @Test + public void testNotEqualsOperator() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", "TRADE", 100.0, 1000, DslTestDataSets.ts(0), "NASDAQ", 0.0), + new StockEvent( + "GOOGL", + "TRADE", + 100.0, + 1000, + DslTestDataSets.ts(1), + "NASDAQ", + 0.0), + new StockEvent( + "MSFT", + "TRADE", + 100.0, + 1000, + DslTestDataSets.ts(2), + "NASDAQ", + 0.0)); + + String dslExpression = "TRADE(symbol != 'AAPL')"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = + patternStream.select(match -> match.get("TRADE").get(0).getSymbol()); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + assertEquals(2, results.size()); + assertTrue(results.contains("GOOGL")); + assertTrue(results.contains("MSFT")); + } + + @Test + public void testLessThanOperator() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", "TRADE", 100.0, 800, DslTestDataSets.ts(0), "NASDAQ", 0.0), + new StockEvent( + "AAPL", "TRADE", 100.0, 1200, DslTestDataSets.ts(1), "NASDAQ", 0.0), + new StockEvent( + "AAPL", "TRADE", 100.0, 500, DslTestDataSets.ts(2), "NASDAQ", 0.0)); + + String dslExpression = "TRADE(volume < 1000)"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = + patternStream.select(match -> match.get("TRADE").get(0).getVolume()); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + assertEquals(2, results.size()); + assertTrue(results.stream().allMatch(v -> v < 1000)); + } + + @Test + public void testLessThanOrEqual() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", "TRADE", 100.0, 800, DslTestDataSets.ts(0), "NASDAQ", 0.0), + new StockEvent( + "AAPL", "TRADE", 100.0, 1000, DslTestDataSets.ts(1), "NASDAQ", 0.0), + new StockEvent( + "AAPL", + "TRADE", + 100.0, + 1200, + DslTestDataSets.ts(2), + "NASDAQ", + 0.0)); + + String dslExpression = "TRADE(volume <= 1000)"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = + patternStream.select(match -> match.get("TRADE").get(0).getVolume()); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + assertEquals(2, results.size()); + assertTrue(results.stream().allMatch(v -> v <= 1000)); + } + + @Test + public void testGreaterThanOperator() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", "TRADE", 95.0, 1000, DslTestDataSets.ts(0), "NASDAQ", 0.0), + new StockEvent( + "AAPL", "TRADE", 105.0, 1000, DslTestDataSets.ts(1), "NASDAQ", 0.0), + new StockEvent( + "AAPL", + "TRADE", + 110.0, + 1000, + DslTestDataSets.ts(2), + "NASDAQ", + 0.0)); + + String dslExpression = "TRADE(price > 100.0)"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = + patternStream.select(match -> match.get("TRADE").get(0).getPrice()); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + assertEquals(2, results.size()); + assertTrue(results.stream().allMatch(p -> p > 100.0)); + } + + @Test + public void testGreaterThanOrEqual() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", "TRADE", 95.0, 1000, DslTestDataSets.ts(0), "NASDAQ", 0.0), + new StockEvent( + "AAPL", "TRADE", 100.0, 1000, DslTestDataSets.ts(1), "NASDAQ", 0.0), + new StockEvent( + "AAPL", + "TRADE", + 105.0, + 1000, + DslTestDataSets.ts(2), + "NASDAQ", + 0.0)); + + String dslExpression = "TRADE(price >= 100.0)"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = + patternStream.select(match -> match.get("TRADE").get(0).getPrice()); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + assertEquals(2, results.size()); + assertTrue(results.stream().allMatch(p -> p >= 100.0)); + } + + @Test + public void testStringEquality() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromCollection(DslTestDataSets.comparisonOperatorDataset()); + + String dslExpression = "TRADE(symbol = 'AAPL')"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = + patternStream.select(match -> match.get("TRADE").get(0).getSymbol()); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + assertEquals(4, results.size()); + assertTrue(results.stream().allMatch(s -> s.equals("AAPL"))); + } + + @Test + public void testAndOperator() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = env.fromCollection(DslTestDataSets.logicalOperatorDataset()); + + String dslExpression = "TRADE(price > 100 and volume > 1000)"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = patternStream.select(match -> match.get("TRADE").get(0)); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + assertEquals(2, results.size()); + assertTrue(results.stream().allMatch(e -> e.getPrice() > 100 && e.getVolume() > 1000)); + } + + @Test + public void testOrOperator() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", "TRADE", 100.0, 1000, DslTestDataSets.ts(0), "NASDAQ", 0.0), + new StockEvent( + "GOOGL", + "TRADE", + 100.0, + 1000, + DslTestDataSets.ts(1), + "NASDAQ", + 0.0), + new StockEvent( + "MSFT", + "TRADE", + 100.0, + 1000, + DslTestDataSets.ts(2), + "NASDAQ", + 0.0)) + .assignTimestampsAndWatermarks( + org.apache.flink.api.common.eventtime.WatermarkStrategy.forMonotonousTimestamps() + .withTimestampAssigner((event, timestamp) -> event.getTimestamp())); + + String dslExpression = "TRADE(symbol = 'AAPL' or symbol = 'GOOGL')"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = + patternStream.select(match -> match.get("TRADE").get(0).getSymbol()); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + assertEquals(2, results.size()); + assertTrue(results.contains("AAPL")); + assertTrue(results.contains("GOOGL")); + } + + @Test + public void testComplexLogic() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = env.fromCollection(DslTestDataSets.logicalOperatorDataset()); + + String dslExpression = "TRADE(price > 100 and volume > 1000 or change > 5.0)"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = patternStream.select(match -> match.get("TRADE").get(0)); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + assertEquals(3, results.size()); + assertTrue( + results.stream() + .allMatch( + e -> + (e.getPrice() > 100 && e.getVolume() > 1000) + || e.getChange() > 5.0)); + } +} diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/DslPatternMatchingTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/DslPatternMatchingTest.java new file mode 100644 index 0000000000000..712a87f90c072 --- /dev/null +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/DslPatternMatchingTest.java @@ -0,0 +1,239 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cep.dsl; + +import org.apache.flink.cep.PatternStream; +import org.apache.flink.cep.dsl.api.DslCompiler; +import org.apache.flink.cep.dsl.model.StockEvent; +import org.apache.flink.cep.dsl.model.UserActivityEvent; +import org.apache.flink.cep.dsl.util.DslTestDataSets; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.util.AbstractTestBaseJUnit4; + +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** Tests for DSL pattern sequencing (NEXT, FOLLOWED BY, etc.). */ +public class DslPatternMatchingTest extends AbstractTestBaseJUnit4 { + + @Test + public void testNext() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", "ORDER", 100.0, 1000, DslTestDataSets.ts(0), "NASDAQ", 0.0), + new StockEvent( + "AAPL", "TRADE", 100.0, 1000, DslTestDataSets.ts(1), "NASDAQ", 0.0), + new StockEvent( + "AAPL", + "QUOTE", + 101.0, + 1100, + DslTestDataSets.ts(2), + "NASDAQ", + 0.0)); + + String dslExpression = "ORDER(eventType = 'ORDER') -> TRADE(eventType = 'TRADE')"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = + patternStream.select( + match -> + match.get("ORDER").get(0).getEventType() + + "->" + + match.get("TRADE").get(0).getEventType()); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + assertEquals(1, results.size()); + assertEquals("ORDER->TRADE", results.get(0)); + } + + @Test + public void testNextWithNoise() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", "ORDER", 100.0, 1000, DslTestDataSets.ts(0), "NASDAQ", 0.0), + new StockEvent( + "AAPL", "QUOTE", 100.5, 1000, DslTestDataSets.ts(1), "NASDAQ", 0.0), + new StockEvent( + "AAPL", + "TRADE", + 100.0, + 1000, + DslTestDataSets.ts(2), + "NASDAQ", + 0.0)); + + String dslExpression = "ORDER(eventType = 'ORDER') -> TRADE(eventType = 'TRADE')"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = patternStream.select(match -> "MATCH"); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + // -> is followedBy (relaxed contiguity), so it should match even with QUOTE in between + assertEquals(1, results.size()); + } + + @Test + public void testFollowedBy() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromCollection(DslTestDataSets.userJourneyDataset()); + + String dslExpression = "LOGIN(eventType = 'LOGIN') -> LOGOUT(eventType = 'LOGOUT')"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = + patternStream.select( + match -> + match.get("LOGIN").get(0).getUserId() + + ":" + + match.get("LOGOUT").get(0).getEventType()); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + assertEquals(1, results.size()); + assertEquals("user1:LOGOUT", results.get(0)); + } + + @Test + public void testFollowedByAny() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new UserActivityEvent( + "user1", "LOGIN", "/home", 0, DslTestDataSets.ts(0), "s1", 1), + new UserActivityEvent( + "user1", "CLICK", "/products", 30, DslTestDataSets.ts(1), "s1", 1), + new UserActivityEvent( + "user1", "CLICK", "/cart", 20, DslTestDataSets.ts(2), "s1", 1), + new UserActivityEvent( + "user1", "CLICK", "/checkout", 10, DslTestDataSets.ts(3), "s1", 1)); + + String dslExpression = "LOGIN(eventType = 'LOGIN') ->> CLICK(eventType = 'CLICK')"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = + patternStream.select(match -> match.get("CLICK").get(0).getPage()); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + // Should match multiple CLICK events + assertTrue(results.size() >= 1); + } + + @Test + public void testNotFollowedBy() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new UserActivityEvent( + "user1", "LOGIN", "/home", 0, DslTestDataSets.ts(0), "s1", 1), + new UserActivityEvent( + "user1", "CLICK", "/products", 30, DslTestDataSets.ts(1), "s1", 1), + new UserActivityEvent( + "user1", "LOGOUT", "/home", 0, DslTestDataSets.ts(2), "s1", 1)); + + String dslExpression = + "LOGIN(eventType = 'LOGIN') !-> ERROR(eventType = 'ERROR') within 10s"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = + patternStream.select(match -> match.get("LOGIN").get(0).getUserId()); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + // Note: In a finite stream with executeAndCollect(), the notFollowedBy pattern + // at the end with 'within' requires the time window to expire before emitting a match. + // Since the stream ends at ts(2) = 2s, well before the 10s window expires, + // no match is produced. This is the expected behavior of Flink CEP. + // To trigger a match, either: + // 1. Add an event beyond the 10s window, or + // 2. Use proper watermark strategy to advance event time + assertEquals(0, results.size()); + } + + @Test + public void testComplexSequence() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", "ORDER", 100.0, 1000, DslTestDataSets.ts(0), "NASDAQ", 0.0), + new StockEvent( + "AAPL", "TRADE", 100.0, 1000, DslTestDataSets.ts(1), "NASDAQ", 0.0), + new StockEvent( + "AAPL", "QUOTE", 101.0, 1100, DslTestDataSets.ts(2), "NASDAQ", 0.0), + new StockEvent( + "AAPL", + "TRADE", + 101.0, + 1100, + DslTestDataSets.ts(3), + "NASDAQ", + 0.0)); + + String dslExpression = + "A(eventType = 'ORDER') -> B(eventType = 'TRADE') -> C(eventType = 'TRADE')"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = + patternStream.select( + match -> + match.get("A").get(0).getEventType() + + "->" + + match.get("B").get(0).getEventType() + + "->" + + match.get("C").get(0).getEventType()); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + assertEquals(1, results.size()); + assertEquals("ORDER->TRADE->TRADE", results.get(0)); + } +} diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/DslQuantifierTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/DslQuantifierTest.java new file mode 100644 index 0000000000000..067158e800ee1 --- /dev/null +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/DslQuantifierTest.java @@ -0,0 +1,220 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cep.dsl; + +import org.apache.flink.cep.PatternSelectFunction; +import org.apache.flink.cep.PatternStream; +import org.apache.flink.cep.dsl.api.DslCompiler; +import org.apache.flink.cep.dsl.model.StockEvent; +import org.apache.flink.cep.dsl.util.DslTestDataSets; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.util.AbstractTestBaseJUnit4; + +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** Tests for DSL quantifiers (*, +, ?, {n}, {n,m}). */ +public class DslQuantifierTest extends AbstractTestBaseJUnit4 { + + @Test + public void testOneOrMore() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", "TRADE", 95.0, 1000, DslTestDataSets.ts(0), "NASDAQ", 0.0), + new StockEvent( + "AAPL", "TRADE", 105.0, 1000, DslTestDataSets.ts(1), "NASDAQ", 0.0), + new StockEvent( + "AAPL", "TRADE", 110.0, 1000, DslTestDataSets.ts(2), "NASDAQ", 0.0), + new StockEvent( + "AAPL", "TRADE", 90.0, 1000, DslTestDataSets.ts(3), "NASDAQ", 0.0)); + + String dslExpression = "TRADE+(price > 100)"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream> result = + patternStream.select( + new PatternSelectFunction>() { + @Override + public List select(Map> match) { + return match.get("TRADE"); + } + }); + + List> results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + assertTrue(results.size() > 0); + assertTrue(results.stream().anyMatch(list -> list.size() >= 1)); + } + + @Test + public void testOptional() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", "TRADE", 100.0, 800, DslTestDataSets.ts(0), "NASDAQ", 0.0), + new StockEvent( + "AAPL", "TRADE", 100.0, 1200, DslTestDataSets.ts(1), "NASDAQ", 0.0), + new StockEvent( + "AAPL", "TRADE", 100.0, 500, DslTestDataSets.ts(2), "NASDAQ", 0.0)); + + String dslExpression = "TRADE(volume > 1000)?"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream result = patternStream.select(match -> "MATCH"); + + List results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + assertTrue(results.size() >= 1); + } + + @Test + public void testExactCount() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", "TRADE", 100.0, 1000, DslTestDataSets.ts(0), "NASDAQ", 0.0), + new StockEvent( + "AAPL", "TRADE", 101.0, 1000, DslTestDataSets.ts(1), "NASDAQ", 0.0), + new StockEvent( + "AAPL", "TRADE", 102.0, 1000, DslTestDataSets.ts(2), "NASDAQ", 0.0), + new StockEvent( + "GOOGL", + "TRADE", + 2800.0, + 500, + DslTestDataSets.ts(3), + "NASDAQ", + 0.0)); + + String dslExpression = "TRADE{3}(symbol = 'AAPL')"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream> result = + patternStream.select( + new PatternSelectFunction>() { + @Override + public List select(Map> match) { + return match.get("TRADE"); + } + }); + + List> results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + assertEquals(1, results.size()); + assertEquals(3, results.get(0).size()); + } + + @Test + public void testRangeCount() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = + env.fromData( + new StockEvent( + "AAPL", "TRADE", 100.0, 1000, DslTestDataSets.ts(0), "NASDAQ", 0.0), + new StockEvent( + "AAPL", "TRADE", 101.0, 1000, DslTestDataSets.ts(1), "NASDAQ", 0.0), + new StockEvent( + "AAPL", "TRADE", 102.0, 1000, DslTestDataSets.ts(2), "NASDAQ", 0.0), + new StockEvent( + "GOOGL", + "QUOTE", + 2800.0, + 500, + DslTestDataSets.ts(3), + "NASDAQ", + 0.0), + new StockEvent( + "AAPL", + "TRADE", + 103.0, + 1000, + DslTestDataSets.ts(4), + "NASDAQ", + 0.0)); + + String dslExpression = "TRADE{2,4}(eventType = 'TRADE')"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream> result = + patternStream.select( + new PatternSelectFunction>() { + @Override + public List select(Map> match) { + return match.get("TRADE"); + } + }); + + List> results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + assertTrue(results.size() > 0); + assertTrue(results.stream().allMatch(list -> list.size() >= 2 && list.size() <= 4)); + } + + @Test + public void testQuantifierWithCondition() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream input = env.fromCollection(DslTestDataSets.quantifierDataset()); + + String dslExpression = "TRADE+(symbol = 'AAPL')"; + + PatternStream patternStream = DslCompiler.compile(dslExpression, input); + + DataStream> result = + patternStream.select( + new PatternSelectFunction>() { + @Override + public List select(Map> match) { + return match.get("TRADE"); + } + }); + + List> results = new ArrayList<>(); + result.executeAndCollect().forEachRemaining(results::add); + + assertTrue(results.size() > 0); + assertTrue( + results.stream() + .allMatch( + list -> list.stream().allMatch(e -> e.getSymbol().equals("AAPL")))); + } +} diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/model/SensorEvent.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/model/SensorEvent.java new file mode 100644 index 0000000000000..4573c62a23826 --- /dev/null +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/model/SensorEvent.java @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cep.dsl.model; + +import java.io.Serializable; +import java.util.Objects; + +/** IoT sensor event model for DSL testing. */ +public class SensorEvent implements Serializable { + private static final long serialVersionUID = 1L; + + private String sensorId; + private String eventType; + private double value; + private String unit; + private long timestamp; + private String location; + private String status; + + public SensorEvent() {} + + public SensorEvent( + String sensorId, + String eventType, + double value, + String unit, + long timestamp, + String location, + String status) { + this.sensorId = sensorId; + this.eventType = eventType; + this.value = value; + this.unit = unit; + this.timestamp = timestamp; + this.location = location; + this.status = status; + } + + public String getSensorId() { + return sensorId; + } + + public void setSensorId(String sensorId) { + this.sensorId = sensorId; + } + + public String getEventType() { + return eventType; + } + + public void setEventType(String eventType) { + this.eventType = eventType; + } + + public double getValue() { + return value; + } + + public void setValue(double value) { + this.value = value; + } + + public String getUnit() { + return unit; + } + + public void setUnit(String unit) { + this.unit = unit; + } + + public long getTimestamp() { + return timestamp; + } + + public void setTimestamp(long timestamp) { + this.timestamp = timestamp; + } + + public String getLocation() { + return location; + } + + public void setLocation(String location) { + this.location = location; + } + + public String getStatus() { + return status; + } + + public void setStatus(String status) { + this.status = status; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SensorEvent that = (SensorEvent) o; + return Double.compare(that.value, value) == 0 + && timestamp == that.timestamp + && Objects.equals(sensorId, that.sensorId) + && Objects.equals(eventType, that.eventType) + && Objects.equals(unit, that.unit) + && Objects.equals(location, that.location) + && Objects.equals(status, that.status); + } + + @Override + public int hashCode() { + return Objects.hash(sensorId, eventType, value, unit, timestamp, location, status); + } + + @Override + public String toString() { + return "SensorEvent{" + + "sensorId='" + + sensorId + + '\'' + + ", eventType='" + + eventType + + '\'' + + ", value=" + + value + + ", unit='" + + unit + + '\'' + + ", timestamp=" + + timestamp + + ", location='" + + location + + '\'' + + ", status='" + + status + + '\'' + + '}'; + } +} diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/model/StockEvent.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/model/StockEvent.java new file mode 100644 index 0000000000000..484c875029571 --- /dev/null +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/model/StockEvent.java @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cep.dsl.model; + +import java.io.Serializable; +import java.util.Objects; + +/** Stock trading event model for DSL testing. */ +public class StockEvent implements Serializable { + private static final long serialVersionUID = 1L; + + private String symbol; + private String eventType; + private double price; + private long volume; + private long timestamp; + private String exchange; + private double change; + + public StockEvent() {} + + public StockEvent( + String symbol, + String eventType, + double price, + long volume, + long timestamp, + String exchange, + double change) { + this.symbol = symbol; + this.eventType = eventType; + this.price = price; + this.volume = volume; + this.timestamp = timestamp; + this.exchange = exchange; + this.change = change; + } + + public String getSymbol() { + return symbol; + } + + public void setSymbol(String symbol) { + this.symbol = symbol; + } + + public String getEventType() { + return eventType; + } + + public void setEventType(String eventType) { + this.eventType = eventType; + } + + public double getPrice() { + return price; + } + + public void setPrice(double price) { + this.price = price; + } + + public long getVolume() { + return volume; + } + + public void setVolume(long volume) { + this.volume = volume; + } + + public long getTimestamp() { + return timestamp; + } + + public void setTimestamp(long timestamp) { + this.timestamp = timestamp; + } + + public String getExchange() { + return exchange; + } + + public void setExchange(String exchange) { + this.exchange = exchange; + } + + public double getChange() { + return change; + } + + public void setChange(double change) { + this.change = change; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + StockEvent that = (StockEvent) o; + return Double.compare(that.price, price) == 0 + && volume == that.volume + && timestamp == that.timestamp + && Double.compare(that.change, change) == 0 + && Objects.equals(symbol, that.symbol) + && Objects.equals(eventType, that.eventType) + && Objects.equals(exchange, that.exchange); + } + + @Override + public int hashCode() { + return Objects.hash(symbol, eventType, price, volume, timestamp, exchange, change); + } + + @Override + public String toString() { + return "StockEvent{" + + "symbol='" + + symbol + + '\'' + + ", eventType='" + + eventType + + '\'' + + ", price=" + + price + + ", volume=" + + volume + + ", timestamp=" + + timestamp + + ", exchange='" + + exchange + + '\'' + + ", change=" + + change + + '}'; + } +} diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/model/UserActivityEvent.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/model/UserActivityEvent.java new file mode 100644 index 0000000000000..b7203328a4d03 --- /dev/null +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/model/UserActivityEvent.java @@ -0,0 +1,157 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cep.dsl.model; + +import java.io.Serializable; +import java.util.Objects; + +/** User activity event model for DSL testing. */ +public class UserActivityEvent implements Serializable { + private static final long serialVersionUID = 1L; + + private String userId; + private String eventType; + private String page; + private long duration; + private long timestamp; + private String sessionId; + private int count; + + public UserActivityEvent() {} + + public UserActivityEvent( + String userId, + String eventType, + String page, + long duration, + long timestamp, + String sessionId, + int count) { + this.userId = userId; + this.eventType = eventType; + this.page = page; + this.duration = duration; + this.timestamp = timestamp; + this.sessionId = sessionId; + this.count = count; + } + + public String getUserId() { + return userId; + } + + public void setUserId(String userId) { + this.userId = userId; + } + + public String getEventType() { + return eventType; + } + + public void setEventType(String eventType) { + this.eventType = eventType; + } + + public String getPage() { + return page; + } + + public void setPage(String page) { + this.page = page; + } + + public long getDuration() { + return duration; + } + + public void setDuration(long duration) { + this.duration = duration; + } + + public long getTimestamp() { + return timestamp; + } + + public void setTimestamp(long timestamp) { + this.timestamp = timestamp; + } + + public String getSessionId() { + return sessionId; + } + + public void setSessionId(String sessionId) { + this.sessionId = sessionId; + } + + public int getCount() { + return count; + } + + public void setCount(int count) { + this.count = count; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + UserActivityEvent that = (UserActivityEvent) o; + return duration == that.duration + && timestamp == that.timestamp + && count == that.count + && Objects.equals(userId, that.userId) + && Objects.equals(eventType, that.eventType) + && Objects.equals(page, that.page) + && Objects.equals(sessionId, that.sessionId); + } + + @Override + public int hashCode() { + return Objects.hash(userId, eventType, page, duration, timestamp, sessionId, count); + } + + @Override + public String toString() { + return "UserActivityEvent{" + + "userId='" + + userId + + '\'' + + ", eventType='" + + eventType + + '\'' + + ", page='" + + page + + '\'' + + ", duration=" + + duration + + ", timestamp=" + + timestamp + + ", sessionId='" + + sessionId + + '\'' + + ", count=" + + count + + '}'; + } +} diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/util/DslTestDataSets.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/util/DslTestDataSets.java new file mode 100644 index 0000000000000..bc53de10c29d8 --- /dev/null +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/dsl/util/DslTestDataSets.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cep.dsl.util; + +import org.apache.flink.cep.dsl.model.SensorEvent; +import org.apache.flink.cep.dsl.model.StockEvent; +import org.apache.flink.cep.dsl.model.UserActivityEvent; + +import java.util.Arrays; +import java.util.List; + +/** Predefined test datasets for DSL testing. */ +public class DslTestDataSets { + + private static final long BASE_TIMESTAMP = 1000000000L; + + public static long ts(int offset) { + return BASE_TIMESTAMP + offset * 1000L; + } + + /** Dataset for testing comparison operators. */ + public static List comparisonOperatorDataset() { + return Arrays.asList( + new StockEvent("AAPL", "TRADE", 150.0, 1000, ts(0), "NASDAQ", 1.5), + new StockEvent("AAPL", "TRADE", 155.0, 1200, ts(1), "NASDAQ", 3.3), + new StockEvent("AAPL", "TRADE", 152.0, 800, ts(2), "NASDAQ", -1.9), + new StockEvent("GOOGL", "TRADE", 2800.0, 500, ts(3), "NASDAQ", 0.5), + new StockEvent("AAPL", "TRADE", 160.0, 1500, ts(4), "NASDAQ", 5.2)); + } + + /** Dataset for testing logical operators. */ + public static List logicalOperatorDataset() { + return Arrays.asList( + new StockEvent("AAPL", "TRADE", 95.0, 900, ts(0), "NASDAQ", -2.0), + new StockEvent("AAPL", "TRADE", 105.0, 1100, ts(1), "NASDAQ", 2.5), + new StockEvent("GOOGL", "TRADE", 110.0, 800, ts(2), "NYSE", 1.0), + new StockEvent("MSFT", "TRADE", 90.0, 1200, ts(3), "NASDAQ", 0.5), + new StockEvent("AAPL", "TRADE", 120.0, 1500, ts(4), "NASDAQ", 6.0)); + } + + /** Dataset for testing quantifiers. */ + public static List quantifierDataset() { + return Arrays.asList( + new StockEvent("AAPL", "TRADE", 100.0, 1000, ts(0), "NASDAQ", 0.0), + new StockEvent("AAPL", "TRADE", 105.0, 1100, ts(1), "NASDAQ", 5.0), + new StockEvent("AAPL", "TRADE", 110.0, 1200, ts(2), "NASDAQ", 10.0), + new StockEvent("GOOGL", "QUOTE", 2800.0, 500, ts(3), "NASDAQ", 0.0), + new StockEvent("AAPL", "TRADE", 115.0, 1300, ts(4), "NASDAQ", 15.0)); + } + + /** Dataset with temperature anomaly pattern. */ + public static List temperatureAnomalyDataset() { + return Arrays.asList( + new SensorEvent( + "TEMP_001", "TEMPERATURE", 20.0, "CELSIUS", ts(0), "ZONE_A", "NORMAL"), + new SensorEvent( + "TEMP_001", "TEMPERATURE", 25.0, "CELSIUS", ts(1), "ZONE_A", "NORMAL"), + new SensorEvent( + "TEMP_001", "TEMPERATURE", 35.0, "CELSIUS", ts(2), "ZONE_A", "WARNING"), + new SensorEvent( + "TEMP_001", "TEMPERATURE", 45.0, "CELSIUS", ts(3), "ZONE_A", "CRITICAL"), + new SensorEvent( + "TEMP_001", "TEMPERATURE", 22.0, "CELSIUS", ts(4), "ZONE_A", "NORMAL")); + } + + /** Dataset for user journey testing. */ + public static List userJourneyDataset() { + return Arrays.asList( + new UserActivityEvent("user1", "LOGIN", "/home", 0, ts(0), "session1", 1), + new UserActivityEvent("user1", "CLICK", "/products", 30, ts(1), "session1", 1), + new UserActivityEvent("user1", "CLICK", "/cart", 20, ts(2), "session1", 1), + new UserActivityEvent("user1", "PURCHASE", "/checkout", 60, ts(3), "session1", 1), + new UserActivityEvent("user1", "LOGOUT", "/home", 0, ts(4), "session1", 1)); + } + + /** Dataset for event correlation testing. */ + public static List eventCorrelationDataset() { + return Arrays.asList( + new StockEvent("AAPL", "TRADE", 100.0, 1000, ts(0), "NASDAQ", 0.0), + new StockEvent("AAPL", "TRADE", 105.0, 1200, ts(1), "NASDAQ", 5.0), + new StockEvent("AAPL", "TRADE", 110.0, 1500, ts(2), "NASDAQ", 10.0), + new StockEvent("AAPL", "TRADE", 115.0, 1800, ts(3), "NASDAQ", 15.0), + new StockEvent("AAPL", "TRADE", 108.0, 900, ts(4), "NASDAQ", 8.0)); + } + + /** Dataset for pattern sequencing testing. */ + public static List patternSequencingDataset() { + return Arrays.asList( + new StockEvent("AAPL", "ORDER", 100.0, 1000, ts(0), "NASDAQ", 0.0), + new StockEvent("AAPL", "TRADE", 100.0, 1000, ts(1), "NASDAQ", 0.0), + new StockEvent("GOOGL", "QUOTE", 2800.0, 500, ts(2), "NASDAQ", 0.0), + new StockEvent("AAPL", "QUOTE", 101.0, 1100, ts(3), "NASDAQ", 1.0), + new StockEvent("AAPL", "TRADE", 102.0, 1200, ts(4), "NASDAQ", 2.0)); + } + + /** Dataset for skip strategy testing. */ + public static List skipStrategyDataset() { + return Arrays.asList( + new StockEvent("AAPL", "TRADE", 100.0, 1000, ts(0), "NASDAQ", 0.0), + new StockEvent("AAPL", "TRADE", 101.0, 1100, ts(1), "NASDAQ", 1.0), + new StockEvent("AAPL", "TRADE", 102.0, 1200, ts(2), "NASDAQ", 2.0), + new StockEvent("AAPL", "TRADE", 103.0, 1300, ts(3), "NASDAQ", 3.0), + new StockEvent("AAPL", "TRADE", 104.0, 1400, ts(4), "NASDAQ", 4.0)); + } + + /** Stock dataset with price increase pattern. */ + public static List priceIncreasePattern() { + return Arrays.asList( + new StockEvent("AAPL", "TRADE", 150.0, 1000, ts(0), "NASDAQ", 0.0), + new StockEvent("AAPL", "TRADE", 152.0, 1100, ts(1), "NASDAQ", 1.3), + new StockEvent("AAPL", "TRADE", 155.0, 1200, ts(2), "NASDAQ", 3.3), + new StockEvent("AAPL", "TRADE", 158.0, 1300, ts(3), "NASDAQ", 5.3), + new StockEvent("AAPL", "TRADE", 160.0, 1400, ts(4), "NASDAQ", 6.7)); + } + + /** Sensor dataset with escalating values. */ + public static List escalatingValues() { + return Arrays.asList( + new SensorEvent( + "TEMP_001", "TEMPERATURE", 25.0, "CELSIUS", ts(0), "ZONE_A", "NORMAL"), + new SensorEvent( + "TEMP_001", "TEMPERATURE", 30.0, "CELSIUS", ts(1), "ZONE_A", "NORMAL"), + new SensorEvent( + "TEMP_001", "TEMPERATURE", 38.0, "CELSIUS", ts(2), "ZONE_A", "WARNING"), + new SensorEvent( + "TEMP_001", "TEMPERATURE", 46.0, "CELSIUS", ts(3), "ZONE_A", "CRITICAL"), + new SensorEvent( + "TEMP_001", "TEMPERATURE", 50.0, "CELSIUS", ts(4), "ZONE_A", "CRITICAL")); + } +}