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