You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by hu...@apache.org on 2022/07/26 09:52:04 UTC

[doris] branch master updated: [Enhancement](Nereids) add some basic data structure definitions. (#10981)

This is an automated email from the ASF dual-hosted git repository.

huajianlan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/master by this push:
     new 61fb39a5ca [Enhancement](Nereids) add some basic data structure definitions. (#10981)
61fb39a5ca is described below

commit 61fb39a5ca5a52ec6686ef9443c8dd4032f81d52
Author: shee <13...@users.noreply.github.com>
AuthorDate: Tue Jul 26 17:51:59 2022 +0800

    [Enhancement](Nereids) add some basic data structure definitions. (#10981)
    
    add decimal 、date、datetime type
    add DateLiteral
    add Interval for timestamp calculation
---
 .../antlr4/org/apache/doris/nereids/DorisLexer.g4  |   1 +
 .../antlr4/org/apache/doris/nereids/DorisParser.g4 |  14 +-
 .../doris/analysis/TimestampArithmeticExpr.java    |  12 ++
 .../apache/doris/nereids/analyzer/UnboundSlot.java |   5 +
 .../glue/translator/ExpressionTranslator.java      |  23 +++
 .../doris/nereids/parser/LogicalPlanBuilder.java   |  57 +++++++-
 .../org/apache/doris/nereids/rules/RuleType.java   |   2 +
 .../doris/nereids/rules/analysis/BindFunction.java |  33 +++++
 .../rewrite/logical/PushPredicateThroughJoin.java  |   1 +
 .../nereids/trees/expressions/DateLiteral.java     | 161 +++++++++++++++++++++
 .../nereids/trees/expressions/DateTimeLiteral.java | 151 +++++++++++++++++++
 .../nereids/trees/expressions/Expression.java      |  10 +-
 .../nereids/trees/expressions/IntervalLiteral.java |  64 ++++++++
 .../nereids/trees/expressions/NamedExpression.java |   5 -
 .../nereids/trees/expressions/StringLiteral.java   |  26 ++++
 .../trees/expressions/TimestampArithmetic.java     | 132 +++++++++++++++++
 .../expressions/visitor/ExpressionVisitor.java     |  15 ++
 .../apache/doris/nereids/types/BooleanType.java    |   2 +-
 .../org/apache/doris/nereids/types/DataType.java   |  22 +++
 .../types/{NullType.java => DateTimeType.java}     |  14 +-
 .../nereids/types/{NullType.java => DateType.java} |  10 +-
 .../types/{NullType.java => DecimalType.java}      |  28 +++-
 .../apache/doris/nereids/types/IntegerType.java    |   5 +
 .../org/apache/doris/nereids/types/NullType.java   |   2 +-
 .../apache/doris/nereids/types/VarcharType.java    |   2 +-
 .../org/apache/doris/nereids/util/DateUtils.java   | 146 +++++++++++++++++++
 .../expression/rewrite/ExpressionRewriteTest.java  |  13 +-
 .../org/apache/doris/nereids/ssb/SSBUtils.java     |   1 +
 .../trees/expressions/ExpressionParserTest.java    |  30 ++++
 29 files changed, 957 insertions(+), 30 deletions(-)

diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4
index 1f44cc1686..ff20819e56 100644
--- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4
+++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4
@@ -360,6 +360,7 @@ VALUES: 'VALUES';
 VERSION: 'VERSION';
 VIEW: 'VIEW';
 VIEWS: 'VIEWS';
+WEEK: 'WEEK';
 WHEN: 'WHEN';
 WHERE: 'WHERE';
 WINDOW: 'WINDOW';
diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4
index c3086488cc..18ad0089af 100644
--- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4
+++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4
@@ -199,8 +199,8 @@ primaryExpression
     | constant                                                                                 #constantDefault
     | ASTERISK                                                                                 #star
     | qualifiedName DOT ASTERISK                                                               #star
-    | identifier LEFT_PAREN DISTINCT? arguments+=expression
-      (COMMA arguments+=expression)* RIGHT_PAREN                                                #functionCall
+    | identifier LEFT_PAREN (DISTINCT? arguments+=expression
+      (COMMA arguments+=expression)*)? RIGHT_PAREN                                             #functionCall
     | LEFT_PAREN query RIGHT_PAREN                                                             #subqueryExpression
     | identifier                                                                               #columnReference
     | base=primaryExpression DOT fieldName=identifier                                          #dereference
@@ -213,6 +213,8 @@ qualifiedName
 
 constant
     : NULL                                                                                     #nullLiteral
+    | interval                                                                                 #intervalLiteral
+    | identifier STRING                                                                        #typeConstructor
     | number                                                                                   #numericLiteral
     | booleanValue                                                                             #booleanLiteral
     | STRING+                                                                                  #stringLiteral
@@ -230,6 +232,14 @@ whenClause
     : WHEN condition=expression THEN result=expression
     ;
 
+interval
+    : INTERVAL value=expression unit=unitIdentifier
+    ;
+
+unitIdentifier
+    : YEAR | MONTH | WEEK | DAY | HOUR | MINUTE | SECOND
+    ;
+
 // this rule is used for explicitly capturing wrong identifiers such as test-table, which should actually be `test-table`
 // replace identifier with errorCapturingIdentifier where the immediate follow symbol is not an expression, otherwise
 // valid expressions such as "a-b" can be recognized as an identifier
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/TimestampArithmeticExpr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/TimestampArithmeticExpr.java
index 16f6fc4e74..36b34616a0 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/TimestampArithmeticExpr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/TimestampArithmeticExpr.java
@@ -31,6 +31,7 @@ import org.apache.doris.thrift.TExprNodeType;
 import org.apache.doris.thrift.TExprOpcode;
 
 import com.google.common.base.Preconditions;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -417,4 +418,15 @@ public class TimestampArithmeticExpr extends Expr {
             return description;
         }
     }
+
+    @Override
+    public void finalizeImplForNereids() throws AnalysisException {
+        if (StringUtils.isEmpty(funcName)) {
+            throw new AnalysisException("function name is null");
+        }
+        type = getChild(0).getType();
+        opcode = getOpCode();
+        fn = getBuiltinFunction(funcName.toLowerCase(), collectChildReturnTypes(),
+                Function.CompareMode.IS_NONSTRICT_SUPERTYPE_OF);
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundSlot.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundSlot.java
index 60b662865b..e248264ff7 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundSlot.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundSlot.java
@@ -51,6 +51,11 @@ public class UnboundSlot extends Slot implements Unbound {
         }).reduce((left, right) -> left + "." + right).orElse("");
     }
 
+    @Override
+    public boolean isConstant() {
+        return false;
+    }
+
     @Override
     public String toSql() {
         return nameParts.stream().map(Utils::quoteIfNeeded).reduce((left, right) -> left + "." + right).orElse("");
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java
index 72cdb45f31..9146380e7f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java
@@ -30,6 +30,7 @@ import org.apache.doris.analysis.IntLiteral;
 import org.apache.doris.analysis.LikePredicate;
 import org.apache.doris.analysis.NullLiteral;
 import org.apache.doris.analysis.StringLiteral;
+import org.apache.doris.analysis.TimestampArithmeticExpr;
 import org.apache.doris.nereids.exceptions.AnalysisException;
 import org.apache.doris.nereids.trees.expressions.Alias;
 import org.apache.doris.nereids.trees.expressions.And;
@@ -37,6 +38,8 @@ import org.apache.doris.nereids.trees.expressions.Arithmetic;
 import org.apache.doris.nereids.trees.expressions.Between;
 import org.apache.doris.nereids.trees.expressions.BooleanLiteral;
 import org.apache.doris.nereids.trees.expressions.CaseWhen;
+import org.apache.doris.nereids.trees.expressions.DateLiteral;
+import org.apache.doris.nereids.trees.expressions.DateTimeLiteral;
 import org.apache.doris.nereids.trees.expressions.DoubleLiteral;
 import org.apache.doris.nereids.trees.expressions.EqualTo;
 import org.apache.doris.nereids.trees.expressions.Expression;
@@ -51,6 +54,7 @@ import org.apache.doris.nereids.trees.expressions.NullSafeEqual;
 import org.apache.doris.nereids.trees.expressions.Or;
 import org.apache.doris.nereids.trees.expressions.Regexp;
 import org.apache.doris.nereids.trees.expressions.SlotReference;
+import org.apache.doris.nereids.trees.expressions.TimestampArithmetic;
 import org.apache.doris.nereids.trees.expressions.WhenClause;
 import org.apache.doris.nereids.trees.expressions.functions.BoundFunction;
 import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionVisitor;
@@ -173,6 +177,19 @@ public class ExpressionTranslator extends DefaultExpressionVisitor<Expr, PlanTra
         return new FloatLiteral(doubleLiteral.getValue());
     }
 
+    @Override
+    public Expr visitDateLiteral(DateLiteral dateLiteral, PlanTranslatorContext context) {
+        return new org.apache.doris.analysis.DateLiteral(dateLiteral.getYear(), dateLiteral.getMonth(),
+                dateLiteral.getDay(), 0, 0, 0);
+    }
+
+    @Override
+    public Expr visitDateTimeLiteral(DateTimeLiteral dateTimeLiteral, PlanTranslatorContext context) {
+        return new org.apache.doris.analysis.DateLiteral(dateTimeLiteral.getYear(), dateTimeLiteral.getMonth(),
+                dateTimeLiteral.getDay(), dateTimeLiteral.getHour(), dateTimeLiteral.getMinute(),
+                dateTimeLiteral.getSecond());
+    }
+
     @Override
     public Expr visitBetween(Between between, PlanTranslatorContext context) {
         throw new RuntimeException("Unexpected invocation");
@@ -244,4 +261,10 @@ public class ExpressionTranslator extends DefaultExpressionVisitor<Expr, PlanTra
                 arithmetic.child(0).accept(this, context),
                 arithmeticOperator.isBinary() ? arithmetic.child(1).accept(this, context) : null);
     }
+
+    @Override
+    public Expr visitTimestampArithmetic(TimestampArithmetic arithmetic, PlanTranslatorContext context) {
+        return new TimestampArithmeticExpr(arithmetic.getFuncName(), arithmetic.left().accept(this, context),
+                arithmetic.right().accept(this, context), arithmetic.getTimeUnit().toString());
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java
index 2dcad23211..b5d7c2faf6 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java
@@ -18,6 +18,7 @@
 package org.apache.doris.nereids.parser;
 
 
+import org.apache.doris.analysis.ArithmeticExpr.Operator;
 import org.apache.doris.nereids.DorisParser;
 import org.apache.doris.nereids.DorisParser.AggClauseContext;
 import org.apache.doris.nereids.DorisParser.ArithmeticBinaryContext;
@@ -31,6 +32,7 @@ import org.apache.doris.nereids.DorisParser.FromClauseContext;
 import org.apache.doris.nereids.DorisParser.IdentifierListContext;
 import org.apache.doris.nereids.DorisParser.IdentifierSeqContext;
 import org.apache.doris.nereids.DorisParser.IntegerLiteralContext;
+import org.apache.doris.nereids.DorisParser.IntervalContext;
 import org.apache.doris.nereids.DorisParser.JoinCriteriaContext;
 import org.apache.doris.nereids.DorisParser.JoinRelationContext;
 import org.apache.doris.nereids.DorisParser.LogicalBinaryContext;
@@ -55,6 +57,8 @@ import org.apache.doris.nereids.DorisParser.StarContext;
 import org.apache.doris.nereids.DorisParser.StringLiteralContext;
 import org.apache.doris.nereids.DorisParser.SubqueryExpressionContext;
 import org.apache.doris.nereids.DorisParser.TableNameContext;
+import org.apache.doris.nereids.DorisParser.TypeConstructorContext;
+import org.apache.doris.nereids.DorisParser.UnitIdentifierContext;
 import org.apache.doris.nereids.DorisParser.WhereClauseContext;
 import org.apache.doris.nereids.DorisParserBaseVisitor;
 import org.apache.doris.nereids.analyzer.UnboundAlias;
@@ -69,6 +73,8 @@ import org.apache.doris.nereids.trees.expressions.And;
 import org.apache.doris.nereids.trees.expressions.Between;
 import org.apache.doris.nereids.trees.expressions.BooleanLiteral;
 import org.apache.doris.nereids.trees.expressions.CaseWhen;
+import org.apache.doris.nereids.trees.expressions.DateLiteral;
+import org.apache.doris.nereids.trees.expressions.DateTimeLiteral;
 import org.apache.doris.nereids.trees.expressions.Divide;
 import org.apache.doris.nereids.trees.expressions.EqualTo;
 import org.apache.doris.nereids.trees.expressions.Exists;
@@ -77,6 +83,7 @@ import org.apache.doris.nereids.trees.expressions.GreaterThan;
 import org.apache.doris.nereids.trees.expressions.GreaterThanEqual;
 import org.apache.doris.nereids.trees.expressions.InSubquery;
 import org.apache.doris.nereids.trees.expressions.IntegerLiteral;
+import org.apache.doris.nereids.trees.expressions.IntervalLiteral;
 import org.apache.doris.nereids.trees.expressions.LessThan;
 import org.apache.doris.nereids.trees.expressions.LessThanEqual;
 import org.apache.doris.nereids.trees.expressions.Like;
@@ -92,6 +99,7 @@ import org.apache.doris.nereids.trees.expressions.Regexp;
 import org.apache.doris.nereids.trees.expressions.StringLiteral;
 import org.apache.doris.nereids.trees.expressions.SubqueryExpr;
 import org.apache.doris.nereids.trees.expressions.Subtract;
+import org.apache.doris.nereids.trees.expressions.TimestampArithmetic;
 import org.apache.doris.nereids.trees.expressions.WhenClause;
 import org.apache.doris.nereids.trees.plans.JoinType;
 import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate;
@@ -329,8 +337,30 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> {
             Expression left = getExpression(ctx.left);
             Expression right = getExpression(ctx.right);
 
+            int type = ctx.operator.getType();
+            if (left instanceof IntervalLiteral) {
+                if (type != DorisParser.PLUS) {
+                    throw new IllegalArgumentException("Only supported: " + Operator.ADD);
+                }
+                IntervalLiteral interval = (IntervalLiteral) left;
+                return new TimestampArithmetic(Operator.ADD, right, interval.value(), interval.timeUnit(), true);
+            }
+
+            if (right instanceof IntervalLiteral) {
+                Operator op;
+                if (type == DorisParser.PLUS) {
+                    op = Operator.ADD;
+                } else if (type == DorisParser.MINUS) {
+                    op = Operator.SUBTRACT;
+                } else {
+                    throw new IllegalArgumentException("Only supported: " + Operator.ADD + " and " + Operator.SUBTRACT);
+                }
+                IntervalLiteral interval = (IntervalLiteral) right;
+                return new TimestampArithmetic(op, left, interval.value(), interval.timeUnit(), false);
+            }
+
             return ParserUtils.withOrigin(ctx, () -> {
-                switch (ctx.operator.getType()) {
+                switch (type) {
                     case DorisParser.ASTERISK:
                         return new Multiply(left, right);
                     case DorisParser.SLASH:
@@ -406,6 +436,31 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> {
         });
     }
 
+    @Override
+    public Expression visitInterval(IntervalContext ctx) {
+        return new IntervalLiteral(getExpression(ctx.value), visitUnitIdentifier(ctx.unit));
+    }
+
+    @Override
+    public String visitUnitIdentifier(UnitIdentifierContext ctx) {
+        return ctx.getText();
+    }
+
+    @Override
+    public Expression visitTypeConstructor(TypeConstructorContext ctx) {
+        String value = ctx.STRING().getText();
+        value = value.substring(1, value.length() - 1);
+        String type = ctx.identifier().getText().toUpperCase();
+        switch (type) {
+            case "DATE":
+                return new DateLiteral(value);
+            case "DATETIME":
+                return new DateTimeLiteral(value);
+            default:
+                throw new IllegalStateException("Unsupported data type : " + type);
+        }
+    }
+
     @Override
     public Expression visitDereference(DereferenceContext ctx) {
         return ParserUtils.withOrigin(ctx, () -> {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java
index 6d1f55ea32..2485fad8a4 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java
@@ -33,6 +33,8 @@ public enum RuleType {
     BINDING_SORT_SLOT(RuleTypeClass.REWRITE),
     BINDING_PROJECT_FUNCTION(RuleTypeClass.REWRITE),
     BINDING_AGGREGATE_FUNCTION(RuleTypeClass.REWRITE),
+    BINDING_FILTER_FUNCTION(RuleTypeClass.REWRITE),
+
     RESOLVE_PROJECT_ALIAS(RuleTypeClass.REWRITE),
     RESOLVE_AGGREGATE_ALIAS(RuleTypeClass.REWRITE),
     PROJECT_TO_GLOBAL_AGGREGATE(RuleTypeClass.REWRITE),
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindFunction.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindFunction.java
index 2aafcf6254..3a5b2bbeeb 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindFunction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindFunction.java
@@ -17,15 +17,20 @@
 
 package org.apache.doris.nereids.rules.analysis;
 
+import org.apache.doris.analysis.ArithmeticExpr.Operator;
 import org.apache.doris.nereids.analyzer.UnboundFunction;
 import org.apache.doris.nereids.rules.Rule;
 import org.apache.doris.nereids.rules.RuleType;
 import org.apache.doris.nereids.trees.expressions.Expression;
 import org.apache.doris.nereids.trees.expressions.NamedExpression;
+import org.apache.doris.nereids.trees.expressions.TimestampArithmetic;
 import org.apache.doris.nereids.trees.expressions.functions.Substring;
 import org.apache.doris.nereids.trees.expressions.functions.Sum;
 import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionRewriter;
+import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
 import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
+import org.apache.doris.nereids.types.DateTimeType;
+import org.apache.doris.nereids.types.IntegerType;
 
 import com.google.common.collect.ImmutableList;
 
@@ -51,6 +56,12 @@ public class BindFunction implements AnalysisRuleFactory {
                     List<NamedExpression> output = bind(agg.getOutputExpressions());
                     return agg.withGroupByAndOutput(groupBy, output);
                 })
+            ),
+            RuleType.BINDING_FILTER_FUNCTION.build(
+               logicalFilter().then(filter -> {
+                   List<Expression> predicates = bind(filter.getExpressions());
+                   return new LogicalFilter<>(predicates.get(0), filter.child());
+               })
             )
         );
     }
@@ -92,5 +103,27 @@ public class BindFunction implements AnalysisRuleFactory {
             }
             return unboundFunction;
         }
+
+        @Override
+        public Expression visitTimestampArithmetic(TimestampArithmetic arithmetic, Void context) {
+            String funcOpName = null;
+            if (arithmetic.getFuncName() == null) {
+                funcOpName = String.format("%sS_%s", arithmetic.getTimeUnit(),
+                        (arithmetic.getOp() == Operator.ADD) ? "ADD" : "SUB");
+            } else {
+                funcOpName = arithmetic.getFuncName();
+            }
+
+            Expression left = arithmetic.left();
+            Expression right = arithmetic.right();
+
+            if (!arithmetic.left().getDataType().isDateType()) {
+                left = arithmetic.left().castTo(DateTimeType.INSTANCE);
+            }
+            if (!arithmetic.right().getDataType().isIntType()) {
+                right = arithmetic.right().castTo(IntegerType.INSTANCE);
+            }
+            return arithmetic.withFuncName(funcOpName).withChildren(ImmutableList.of(left, right));
+        }
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/PushPredicateThroughJoin.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/PushPredicateThroughJoin.java
index 84103b0da7..0b79ea062c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/PushPredicateThroughJoin.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/PushPredicateThroughJoin.java
@@ -97,6 +97,7 @@ public class PushPredicateThroughJoin extends OneRewriteRuleFactory {
                 if (slots.isEmpty()) {
                     leftPredicates.add(p);
                     rightPredicates.add(p);
+                    continue;
                 }
                 if (leftInput.containsAll(slots)) {
                     leftPredicates.add(p);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/DateLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/DateLiteral.java
new file mode 100644
index 0000000000..889ab30003
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/DateLiteral.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.doris.nereids.trees.expressions;
+
+import org.apache.doris.nereids.exceptions.AnalysisException;
+import org.apache.doris.nereids.types.DataType;
+import org.apache.doris.nereids.types.DateTimeType;
+import org.apache.doris.nereids.types.DateType;
+import org.apache.doris.nereids.util.DateUtils;
+
+import com.google.common.base.Preconditions;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.joda.time.LocalDateTime;
+import org.joda.time.format.DateTimeFormatter;
+
+/**
+ * Date literal in Nereids.
+ */
+public class DateLiteral extends Literal {
+
+    private static final Logger LOG = LogManager.getLogger(DateLiteral.class);
+
+    private static final int DATEKEY_LENGTH = 8;
+
+    private static DateTimeFormatter DATE_FORMATTER = null;
+    private static DateTimeFormatter DATE_FORMATTER_TWO_DIGIT = null;
+    private static DateTimeFormatter DATEKEY_FORMATTER = null;
+
+    protected long year;
+    protected long month;
+    protected long day;
+
+    static {
+        try {
+            DATE_FORMATTER = DateUtils.formatBuilder("%Y-%m-%d").toFormatter();
+            DATEKEY_FORMATTER = DateUtils.formatBuilder("%Y%m%d").toFormatter();
+            DATE_FORMATTER_TWO_DIGIT = DateUtils.formatBuilder("%y-%m-%d").toFormatter();
+        } catch (AnalysisException e) {
+            LOG.error("invalid date format", e);
+            System.exit(-1);
+        }
+    }
+
+    public DateLiteral(String s) throws AnalysisException {
+        super(DateType.INSTANCE);
+        init(s);
+    }
+
+    public DateLiteral(DataType type) throws AnalysisException {
+        super(type);
+    }
+
+    /**
+     * C'tor for date type.
+     */
+    public DateLiteral(long year, long month, long day) {
+        super(DateType.INSTANCE);
+        this.year = year;
+        this.month = month;
+        this.day = day;
+    }
+
+    /**
+     * C'tor for type conversion.
+     */
+    public DateLiteral(DateLiteral other, DataType type) {
+        super(type);
+        this.year = other.year;
+        this.month = other.month;
+        this.day = other.day;
+    }
+
+    private void init(String s) throws AnalysisException {
+        try {
+            LocalDateTime dateTime;
+            if (s.split("-")[0].length() == 2) {
+                dateTime = DATE_FORMATTER_TWO_DIGIT.parseLocalDateTime(s);
+            } else if (s.length() == DATEKEY_LENGTH && !s.contains("-")) {
+                dateTime = DATEKEY_FORMATTER.parseLocalDateTime(s);
+            } else {
+                dateTime = DATE_FORMATTER.parseLocalDateTime(s);
+            }
+            year = dateTime.getYear();
+            month = dateTime.getMonthOfYear();
+            day = dateTime.getDayOfMonth();
+        } catch (Exception ex) {
+            throw new AnalysisException("date literal [" + s + "] is invalid");
+        }
+    }
+
+    @Override
+    protected Expression uncheckedCastTo(DataType targetType) throws AnalysisException {
+        if (getDataType().equals(targetType)) {
+            return this;
+        }
+        if (targetType.isDate()) {
+            if (getDataType().equals(targetType)) {
+                return this;
+            }
+            if (targetType.equals(DateType.INSTANCE)) {
+                return new DateLiteral(this.year, this.month, this.day);
+            } else if (targetType.equals(DateTimeType.INSTANCE)) {
+                return new DateTimeLiteral(this.year, this.month, this.day, 0, 0, 0);
+            } else {
+                throw new AnalysisException("Error date literal type");
+            }
+        }
+        //todo other target type cast
+        return this;
+    }
+
+    public DateLiteral withDataType(DataType type) {
+        Preconditions.checkArgument(type.isDate() || type.isDateTime());
+        return new DateLiteral(this, type);
+    }
+
+    @Override
+    public Long getValue() {
+        return (year * 10000 + month * 100 + day) * 1000000L;
+    }
+
+    @Override
+    public String toSql() {
+        return toString();
+    }
+
+    @Override
+    public String toString() {
+        return String.format("%04d-%02d-%02d", year, month, day);
+    }
+
+    public long getYear() {
+        return year;
+    }
+
+    public long getMonth() {
+        return month;
+    }
+
+    public long getDay() {
+        return day;
+    }
+}
+
+
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/DateTimeLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/DateTimeLiteral.java
new file mode 100644
index 0000000000..130a4e93a0
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/DateTimeLiteral.java
@@ -0,0 +1,151 @@
+// 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.doris.nereids.trees.expressions;
+
+import org.apache.doris.nereids.exceptions.AnalysisException;
+import org.apache.doris.nereids.types.DataType;
+import org.apache.doris.nereids.types.DateTimeType;
+import org.apache.doris.nereids.types.DateType;
+import org.apache.doris.nereids.util.DateUtils;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.joda.time.LocalDateTime;
+import org.joda.time.format.DateTimeFormatter;
+
+/**
+ * date time literal.
+ */
+public class DateTimeLiteral extends DateLiteral {
+    private static final Logger LOG = LogManager.getLogger(DateTimeLiteral.class);
+
+    private static final int DATETIME_TO_MINUTE_STRING_LENGTH = 16;
+    private static final int DATETIME_TO_HOUR_STRING_LENGTH = 13;
+
+    private static DateTimeFormatter DATE_TIME_FORMATTER = null;
+    private static DateTimeFormatter DATE_TIME_FORMATTER_TO_HOUR = null;
+    private static DateTimeFormatter DATE_TIME_FORMATTER_TO_MINUTE = null;
+    private static DateTimeFormatter DATE_TIME_FORMATTER_TWO_DIGIT = null;
+
+    private long hour;
+    private long minute;
+    private long second;
+
+    static {
+        try {
+            DATE_TIME_FORMATTER = DateUtils.formatBuilder("%Y-%m-%d %H:%i:%s").toFormatter();
+            DATE_TIME_FORMATTER_TO_HOUR = DateUtils.formatBuilder("%Y-%m-%d %H").toFormatter();
+            DATE_TIME_FORMATTER_TO_MINUTE = DateUtils.formatBuilder("%Y-%m-%d %H:%i").toFormatter();
+            DATE_TIME_FORMATTER_TWO_DIGIT = DateUtils.formatBuilder("%y-%m-%d %H:%i:%s").toFormatter();
+        } catch (AnalysisException e) {
+            LOG.error("invalid date format", e);
+            System.exit(-1);
+        }
+    }
+
+    public DateTimeLiteral(String s) {
+        super(DateTimeType.INSTANCE);
+        init(s);
+    }
+
+    /**
+     * C'tor data time literal.
+     */
+    public DateTimeLiteral(long year, long month, long day, long hour, long minute, long second) {
+        super(DateTimeType.INSTANCE);
+        this.hour = hour;
+        this.minute = minute;
+        this.second = second;
+        this.year = year;
+        this.month = month;
+        this.day = day;
+    }
+
+    private void init(String s) throws AnalysisException {
+        try {
+            LocalDateTime dateTime;
+            if (s.split("-")[0].length() == 2) {
+                dateTime = DATE_TIME_FORMATTER_TWO_DIGIT.parseLocalDateTime(s);
+            } else {
+                if (s.length() == DATETIME_TO_MINUTE_STRING_LENGTH) {
+                    dateTime = DATE_TIME_FORMATTER_TO_MINUTE.parseLocalDateTime(s);
+                } else if (s.length() == DATETIME_TO_HOUR_STRING_LENGTH) {
+                    dateTime = DATE_TIME_FORMATTER_TO_HOUR.parseLocalDateTime(s);
+                } else {
+                    dateTime = DATE_TIME_FORMATTER.parseLocalDateTime(s);
+                }
+            }
+            year = dateTime.getYear();
+            month = dateTime.getMonthOfYear();
+            day = dateTime.getDayOfMonth();
+            hour = dateTime.getHourOfDay();
+            minute = dateTime.getMinuteOfHour();
+            second = dateTime.getSecondOfMinute();
+        } catch (Exception ex) {
+            throw new AnalysisException("date time literal [" + s + "] is invalid");
+        }
+    }
+
+    @Override
+    protected Expression uncheckedCastTo(DataType targetType) throws AnalysisException {
+        if (getDataType().equals(targetType)) {
+            return this;
+        }
+        if (targetType.isDate()) {
+            if (getDataType().equals(targetType)) {
+                return this;
+            }
+            if (targetType.equals(DateType.INSTANCE)) {
+                return new DateLiteral(this.year, this.month, this.day);
+            } else if (targetType.equals(DateTimeType.INSTANCE)) {
+                return new DateTimeLiteral(this.year, this.month, this.day, this.hour, this.minute, this.second);
+            } else {
+                throw new AnalysisException("Error date literal type");
+            }
+        }
+        //todo other target type cast
+        return this;
+    }
+
+    @Override
+    public Long getValue() {
+        return (year * 10000 + month * 100 + day) * 1000000L + hour * 10000 + minute * 100 + second;
+    }
+
+    @Override
+    public String toSql() {
+        return toString();
+    }
+
+    @Override
+    public String toString() {
+        return String.format("%04d-%02d-%02d %02d:%02d:%02d", year, month, day, hour, minute, second);
+    }
+
+    public long getHour() {
+        return hour;
+    }
+
+    public long getMinute() {
+        return minute;
+    }
+
+    public long getSecond() {
+        return second;
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Expression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Expression.java
index 98b0808e51..e40544303f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Expression.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Expression.java
@@ -17,6 +17,7 @@
 
 package org.apache.doris.nereids.trees.expressions;
 
+import org.apache.doris.nereids.exceptions.AnalysisException;
 import org.apache.doris.nereids.exceptions.UnboundException;
 import org.apache.doris.nereids.trees.AbstractTreeNode;
 import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
@@ -79,9 +80,16 @@ public abstract class Expression extends AbstractTreeNode<Expression> {
      * Whether the expression is a constant.
      */
     public boolean isConstant() {
-        return children().stream().anyMatch(Expression::isConstant);
+        return children().stream().allMatch(Expression::isConstant);
     }
 
+    public final Expression castTo(DataType targetType) throws AnalysisException {
+        return uncheckedCastTo(targetType);
+    }
+
+    protected Expression uncheckedCastTo(DataType targetType) throws AnalysisException {
+        throw new RuntimeException("Do not implement uncheckedCastTo");
+    }
 
     @Override
     public boolean equals(Object o) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/IntervalLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/IntervalLiteral.java
new file mode 100644
index 0000000000..a11d408bcc
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/IntervalLiteral.java
@@ -0,0 +1,64 @@
+// 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.doris.nereids.trees.expressions;
+
+/**
+ * Interval for timestamp calculation.
+ */
+public class IntervalLiteral extends Expression {
+    private final Expression value;
+    private final TimeUnit timeUnit;
+
+    public IntervalLiteral(Expression value, String desc) {
+        this.value = value;
+        this.timeUnit = TimeUnit.valueOf(desc.toUpperCase());
+    }
+
+    public Expression value() {
+        return value;
+    }
+
+    public TimeUnit timeUnit() {
+        return timeUnit;
+    }
+
+    /**
+     * Supported time unit.
+     */
+    public enum TimeUnit {
+        YEAR("YEAR"),                               // YEARS
+        MONTH("MONTH"),                             // MONTHS
+        WEEK("WEEK"),                               // WEEKS
+        DAY("DAY"),                                 // DAYS
+        HOUR("HOUR"),                               // HOURS
+        MINUTE("MINUTE"),                           // MINUTES
+        SECOND("SECOND");                            // SECONDS
+        private final String description;
+
+        TimeUnit(String description) {
+            this.description = description;
+        }
+
+        @Override
+        public String toString() {
+            return description;
+        }
+    }
+}
+
+
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/NamedExpression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/NamedExpression.java
index ec6ebfb50b..2e376720f8 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/NamedExpression.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/NamedExpression.java
@@ -47,11 +47,6 @@ public abstract class NamedExpression extends Expression {
         throw new UnboundException("qualifier");
     }
 
-    @Override
-    public boolean isConstant() {
-        return false;
-    }
-
     /**
      * Get qualified name of NamedExpression.
      *
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/StringLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/StringLiteral.java
index 16ab722464..c8e7ee633e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/StringLiteral.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/StringLiteral.java
@@ -17,7 +17,9 @@
 
 package org.apache.doris.nereids.trees.expressions;
 
+import org.apache.doris.nereids.exceptions.AnalysisException;
 import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.DataType;
 import org.apache.doris.nereids.types.StringType;
 
 /**
@@ -47,6 +49,30 @@ public class StringLiteral extends Literal {
         return visitor.visitStringLiteral(this, context);
     }
 
+    @Override
+    protected Expression uncheckedCastTo(DataType targetType) throws AnalysisException {
+        if (getDataType().equals(targetType)) {
+            return this;
+        }
+        if (targetType.isDateType()) {
+            return convertToDate(targetType);
+        } else if (targetType.isIntType()) {
+            return new IntegerLiteral(Integer.parseInt(value));
+        }
+        //todo other target type cast
+        return this;
+    }
+
+    private DateLiteral convertToDate(DataType targetType) throws AnalysisException {
+        DateLiteral dateLiteral = null;
+        if (targetType.isDate()) {
+            dateLiteral = new DateLiteral(value);
+        } else if (targetType.isDateTime()) {
+            dateLiteral = new DateTimeLiteral(value);
+        }
+        return dateLiteral;
+    }
+
     @Override
     public String toSql() {
         return "'" + value + "'";
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/TimestampArithmetic.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/TimestampArithmetic.java
new file mode 100644
index 0000000000..bdd4a02c0b
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/TimestampArithmetic.java
@@ -0,0 +1,132 @@
+// 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.doris.nereids.trees.expressions;
+
+import org.apache.doris.analysis.ArithmeticExpr.Operator;
+import org.apache.doris.nereids.trees.expressions.IntervalLiteral.TimeUnit;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+
+import com.google.common.base.Preconditions;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.List;
+
+/**
+ * Describes the addition and subtraction of time units from timestamps.
+ * Arithmetic expressions on timestamps are syntactic sugar.
+ * They are executed as function call exprs in the BE.
+ */
+public class TimestampArithmetic extends Expression implements BinaryExpression {
+    private static final Logger LOG = LogManager.getLogger(TimestampArithmetic.class);
+    private final String funcName;
+    private final boolean intervalFirst;
+    private Operator op;
+    private TimeUnit timeUnit;
+
+    public TimestampArithmetic(String funcName, Expression e1, Expression e2, TimeUnit timeUnit) {
+        this(funcName, null, e1, e2, timeUnit, false);
+    }
+
+    public TimestampArithmetic(Operator op, Expression e1, Expression e2, TimeUnit timeUnit, boolean intervalFirst) {
+        this(null, op, e1, e2, timeUnit, intervalFirst);
+
+    }
+
+    /**
+     * Full parameter constructor.
+     */
+    public TimestampArithmetic(String funcName, Operator op, Expression e1, Expression e2, TimeUnit timeUnit,
+            boolean intervalFirst) {
+        super(e1, e2);
+        Preconditions.checkState(op == Operator.ADD || op == Operator.SUBTRACT);
+        this.funcName = funcName;
+        this.op = op;
+        this.intervalFirst = intervalFirst;
+        this.timeUnit = timeUnit;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitTimestampArithmetic(this, context);
+    }
+
+    @Override
+    public Expression withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 2);
+        return new TimestampArithmetic(this.funcName, this.op, children.get(0), children.get(1),
+                this.timeUnit, this.intervalFirst);
+    }
+
+    public Expression withFuncName(String funcName) {
+        return new TimestampArithmetic(funcName, this.op, children.get(0), children.get(1), this.timeUnit,
+                this.intervalFirst);
+    }
+
+    public String getFuncName() {
+        return funcName;
+    }
+
+    public boolean isIntervalFirst() {
+        return intervalFirst;
+    }
+
+    public Operator getOp() {
+        return op;
+    }
+
+    public TimeUnit getTimeUnit() {
+        return timeUnit;
+    }
+
+    @Override
+    public String toString() {
+        return toSql();
+    }
+
+    @Override
+    public String toSql() {
+        StringBuilder strBuilder = new StringBuilder();
+        if (funcName != null) {
+            // Function-call like version.
+            strBuilder.append(funcName).append("(");
+            strBuilder.append(child(0).toSql()).append(", ");
+            strBuilder.append("INTERVAL ");
+            strBuilder.append(child(1).toSql());
+            strBuilder.append(" ").append(timeUnit);
+            strBuilder.append(")");
+            return strBuilder.toString();
+        }
+        if (intervalFirst) {
+            // Non-function-call like version with interval as first operand.
+            strBuilder.append("INTERVAL ");
+            strBuilder.append(child(1).toSql() + " ");
+            strBuilder.append(timeUnit);
+            strBuilder.append(" ").append(op.toString()).append(" ");
+            strBuilder.append(child(0).toSql());
+        } else {
+            // Non-function-call like version with interval as second operand.
+            strBuilder.append(child(0).toSql());
+            strBuilder.append(" " + op.toString() + " ");
+            strBuilder.append("INTERVAL ");
+            strBuilder.append(child(1).toSql() + " ");
+            strBuilder.append(timeUnit);
+        }
+        return strBuilder.toString();
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java
index 95ae89512a..248c32ab29 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java
@@ -30,6 +30,8 @@ import org.apache.doris.nereids.trees.expressions.BooleanLiteral;
 import org.apache.doris.nereids.trees.expressions.CaseWhen;
 import org.apache.doris.nereids.trees.expressions.ComparisonPredicate;
 import org.apache.doris.nereids.trees.expressions.CompoundPredicate;
+import org.apache.doris.nereids.trees.expressions.DateLiteral;
+import org.apache.doris.nereids.trees.expressions.DateTimeLiteral;
 import org.apache.doris.nereids.trees.expressions.Divide;
 import org.apache.doris.nereids.trees.expressions.DoubleLiteral;
 import org.apache.doris.nereids.trees.expressions.EqualTo;
@@ -57,6 +59,7 @@ import org.apache.doris.nereids.trees.expressions.StringLiteral;
 import org.apache.doris.nereids.trees.expressions.StringRegexPredicate;
 import org.apache.doris.nereids.trees.expressions.SubqueryExpr;
 import org.apache.doris.nereids.trees.expressions.Subtract;
+import org.apache.doris.nereids.trees.expressions.TimestampArithmetic;
 import org.apache.doris.nereids.trees.expressions.WhenClause;
 import org.apache.doris.nereids.trees.expressions.functions.AggregateFunction;
 import org.apache.doris.nereids.trees.expressions.functions.BoundFunction;
@@ -140,6 +143,14 @@ public abstract class ExpressionVisitor<R, C> {
         return visit(doubleLiteral, context);
     }
 
+    public R visitDateLiteral(DateLiteral dateLiteral, C context) {
+        return visit(dateLiteral, context);
+    }
+
+    public R visitDateTimeLiteral(DateTimeLiteral dateTimeLiteral, C context) {
+        return visit(dateTimeLiteral, context);
+    }
+
     public R visitBetween(Between between, C context) {
         return visit(between, context);
     }
@@ -220,6 +231,10 @@ public abstract class ExpressionVisitor<R, C> {
         return visit(subqueryExpr, context);
     }
 
+    public R visitTimestampArithmetic(TimestampArithmetic arithmetic, C context) {
+        return visit(arithmetic, context);
+    }
+
     /* ********************************************************************************************
      * Unbound expressions
      * ********************************************************************************************/
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/BooleanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/BooleanType.java
index 9c1e5f566c..b236ba12a4 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/BooleanType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/BooleanType.java
@@ -22,7 +22,7 @@ import org.apache.doris.catalog.Type;
 /**
  * Boolean type in Nereids.
  */
-public class BooleanType extends DataType {
+public class BooleanType extends PrimitiveType {
     public static BooleanType INSTANCE = new BooleanType();
 
     @Override
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java
index 508c85dcc9..8a29991a2a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java
@@ -52,6 +52,12 @@ public abstract class DataType {
                     return VarcharType.createVarcharType(scalarType.getLength());
                 case STRING:
                     return StringType.INSTANCE;
+                case DATE:
+                    return DateType.INSTANCE;
+                case DATETIME:
+                    return DateTimeType.INSTANCE;
+                case DECIMALV2:
+                    return DecimalType.createDecimalType(scalarType.decimalPrecision(), scalarType.decimalScale());
                 case NULL_TYPE:
                     return NullType.INSTANCE;
                 default:
@@ -87,4 +93,20 @@ public abstract class DataType {
     public int hashCode() {
         return 0;
     }
+
+    public boolean isDate() {
+        return this instanceof DateType;
+    }
+
+    public boolean isIntType() {
+        return this instanceof IntegerType;
+    }
+
+    public boolean isDateTime() {
+        return this instanceof DateTimeType;
+    }
+
+    public boolean isDateType() {
+        return isDate() || isDateTime();
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/NullType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DateTimeType.java
similarity index 77%
copy from fe/fe-core/src/main/java/org/apache/doris/nereids/types/NullType.java
copy to fe/fe-core/src/main/java/org/apache/doris/nereids/types/DateTimeType.java
index 6da6be9859..0d05006e50 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/NullType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DateTimeType.java
@@ -20,13 +20,19 @@ package org.apache.doris.nereids.types;
 import org.apache.doris.catalog.Type;
 
 /**
- * Null data type in Nereids.
+ * Datetime type in Nereids.
  */
-public class NullType extends DataType {
-    public static NullType INSTANCE = new NullType();
+public class DateTimeType extends PrimitiveType {
+
+    public static DateTimeType INSTANCE = new DateTimeType();
 
     @Override
     public Type toCatalogDataType() {
-        return Type.NULL;
+        return Type.DATETIME;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        return o instanceof DateTimeType;
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/NullType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DateType.java
similarity index 86%
copy from fe/fe-core/src/main/java/org/apache/doris/nereids/types/NullType.java
copy to fe/fe-core/src/main/java/org/apache/doris/nereids/types/DateType.java
index 6da6be9859..75187b4bab 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/NullType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DateType.java
@@ -20,13 +20,15 @@ package org.apache.doris.nereids.types;
 import org.apache.doris.catalog.Type;
 
 /**
- * Null data type in Nereids.
+ * Date type in Nereids.
  */
-public class NullType extends DataType {
-    public static NullType INSTANCE = new NullType();
+public class DateType extends PrimitiveType {
+
+    public static DateType INSTANCE = new DateType();
 
     @Override
     public Type toCatalogDataType() {
-        return Type.NULL;
+        return Type.DATE;
     }
 }
+
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/NullType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DecimalType.java
similarity index 63%
copy from fe/fe-core/src/main/java/org/apache/doris/nereids/types/NullType.java
copy to fe/fe-core/src/main/java/org/apache/doris/nereids/types/DecimalType.java
index 6da6be9859..f12d550698 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/NullType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DecimalType.java
@@ -20,13 +20,33 @@ package org.apache.doris.nereids.types;
 import org.apache.doris.catalog.Type;
 
 /**
- * Null data type in Nereids.
+ * Decimal type in Nereids.
  */
-public class NullType extends DataType {
-    public static NullType INSTANCE = new NullType();
+public class DecimalType extends FractionalType {
+
+    private int precision;
+    private int scale;
+
+    public DecimalType(int precision, int scale) {
+        this.precision = precision;
+        this.scale = scale;
+    }
+
+    public static DecimalType createDecimalType(int precision, int scale) {
+        return new DecimalType(precision, scale);
+    }
 
     @Override
     public Type toCatalogDataType() {
-        return Type.NULL;
+        return Type.DECIMALV2;
+    }
+
+    public int getPrecision() {
+        return precision;
+    }
+
+    public int getScale() {
+        return scale;
     }
 }
+
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/IntegerType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/IntegerType.java
index 1a95ff7550..aff70cdd0b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/IntegerType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/IntegerType.java
@@ -29,4 +29,9 @@ public class IntegerType extends IntegralType {
     public Type toCatalogDataType() {
         return Type.INT;
     }
+
+    @Override
+    public boolean equals(Object o) {
+        return o instanceof IntegerType;
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/NullType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/NullType.java
index 6da6be9859..cf1cb952bf 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/NullType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/NullType.java
@@ -22,7 +22,7 @@ import org.apache.doris.catalog.Type;
 /**
  * Null data type in Nereids.
  */
-public class NullType extends DataType {
+public class NullType extends PrimitiveType {
     public static NullType INSTANCE = new NullType();
 
     @Override
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/VarcharType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/VarcharType.java
index df77733203..09f72ceef2 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/VarcharType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/VarcharType.java
@@ -25,7 +25,7 @@ import java.util.Objects;
 /**
  * Varchar type in Nereids.
  */
-public class VarcharType extends DataType {
+public class VarcharType extends PrimitiveType {
     private final int len;
 
     public VarcharType(int len) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/DateUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/DateUtils.java
new file mode 100644
index 0000000000..6228fe8ef8
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/DateUtils.java
@@ -0,0 +1,146 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.nereids.util;
+
+import org.apache.doris.nereids.exceptions.AnalysisException;
+
+import org.joda.time.format.DateTimeFormatterBuilder;
+
+/**
+ * date util tools.
+ */
+public class DateUtils {
+
+    /**
+     * format builder.
+     */
+    public static DateTimeFormatterBuilder formatBuilder(String pattern) throws AnalysisException {
+        DateTimeFormatterBuilder builder = new DateTimeFormatterBuilder();
+        boolean escaped = false;
+        for (int i = 0; i < pattern.length(); i++) {
+            char character = pattern.charAt(i);
+            if (escaped) {
+                switch (character) {
+                    case 'a': // %a Abbreviated weekday name (Sun..Sat)
+                        builder.appendDayOfWeekShortText();
+                        break;
+                    case 'b': // %b Abbreviated month name (Jan..Dec)
+                        builder.appendMonthOfYearShortText();
+                        break;
+                    case 'c': // %c Month, numeric (0..12)
+                        builder.appendMonthOfYear(1);
+                        break;
+                    case 'd': // %d Day of the month, numeric (00..31)
+                        builder.appendDayOfMonth(2);
+                        break;
+                    case 'e': // %e Day of the month, numeric (0..31)
+                        builder.appendDayOfMonth(1);
+                        break;
+                    case 'H': // %H Hour (00..23)
+                        builder.appendHourOfDay(2);
+                        break;
+                    case 'h': // %h Hour (01..12)
+                    case 'I': // %I Hour (01..12)
+                        builder.appendClockhourOfHalfday(2);
+                        break;
+                    case 'i': // %i Minutes, numeric (00..59)
+                        builder.appendMinuteOfHour(2);
+                        break;
+                    case 'j': // %j Day of year (001..366)
+                        builder.appendDayOfYear(3);
+                        break;
+                    case 'k': // %k Hour (0..23)
+                        builder.appendHourOfDay(1);
+                        break;
+                    case 'l': // %l Hour (1..12)
+                        builder.appendClockhourOfHalfday(1);
+                        break;
+                    case 'M': // %M Month name (January..December)
+                        builder.appendMonthOfYearText();
+                        break;
+                    case 'm': // %m Month, numeric (00..12)
+                        builder.appendMonthOfYear(2);
+                        break;
+                    case 'p': // %p AM or PM
+                        builder.appendHalfdayOfDayText();
+                        break;
+                    case 'r': // %r Time, 12-hour (hh:mm:ss followed by AM or PM)
+                        builder.appendClockhourOfHalfday(2)
+                                .appendLiteral(':')
+                                .appendMinuteOfHour(2)
+                                .appendLiteral(':')
+                                .appendSecondOfMinute(2)
+                                .appendLiteral(' ')
+                                .appendHalfdayOfDayText();
+                        break;
+                    case 'S': // %S Seconds (00..59)
+                    case 's': // %s Seconds (00..59)
+                        builder.appendSecondOfMinute(2);
+                        break;
+                    case 'T': // %T Time, 24-hour (hh:mm:ss)
+                        builder.appendHourOfDay(2)
+                                .appendLiteral(':')
+                                .appendMinuteOfHour(2)
+                                .appendLiteral(':')
+                                .appendSecondOfMinute(2);
+                        break;
+                    case 'v': // %v Week (01..53), where Monday is the first day of the week; used with %x
+                        builder.appendWeekOfWeekyear(2);
+                        break;
+                    case 'x':
+                        // %x Year for the week, where Monday is the first day of the week,
+                        // numeric, four digits; used with %v
+                        builder.appendWeekyear(4, 4);
+                        break;
+                    case 'W': // %W Weekday name (Sunday..Saturday)
+                        builder.appendDayOfWeekText();
+                        break;
+                    case 'Y': // %Y Year, numeric, four digits
+                        builder.appendYear(4, 4);
+                        break;
+                    case 'y': // %y Year, numeric (two digits)
+                        builder.appendTwoDigitYear(2020);
+                        break;
+                    // TODO(Gabriel): support microseconds in date literal
+                    case 'f': // %f Microseconds (000000..999999)
+                    case 'w': // %w Day of the week (0=Sunday..6=Saturday)
+                    case 'U': // %U Week (00..53), where Sunday is the first day of the week
+                    case 'u': // %u Week (00..53), where Monday is the first day of the week
+                    case 'V': // %V Week (01..53), where Sunday is the first day of the week; used with %X
+                    case 'X': // %X Year for the week where Sunday is the first day of the week,
+                        // numeric, four digits; used with %V
+                    case 'D': // %D Day of the month with English suffix (0th, 1st, 2nd, 3rd, …)
+                        throw new AnalysisException(String.format("%%%s not supported in date format string",
+                                character));
+                    case '%': // %% A literal "%" character
+                        builder.appendLiteral('%');
+                        break;
+                    default: // %<x> The literal character represented by <x>
+                        builder.appendLiteral(character);
+                        break;
+                }
+                escaped = false;
+            } else if (character == '%') {
+                escaped = true;
+            } else {
+                builder.appendLiteral(character);
+            }
+        }
+        return builder;
+    }
+}
diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/rewrite/ExpressionRewriteTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/rewrite/ExpressionRewriteTest.java
index f677c1570a..ced159b336 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/rewrite/ExpressionRewriteTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/rewrite/ExpressionRewriteTest.java
@@ -26,8 +26,8 @@ import org.apache.doris.nereids.rules.expression.rewrite.rules.SimplifyNotExprRu
 import org.apache.doris.nereids.trees.expressions.Expression;
 
 import com.google.common.collect.ImmutableList;
-import org.junit.Assert;
 import org.junit.Test;
+import org.junit.jupiter.api.Assertions;
 
 /**
  * all expr rewrite rule test case.
@@ -64,11 +64,12 @@ public class ExpressionRewriteTest {
     public void testNormalizeExpressionRewrite() {
         executor = new ExpressionRuleExecutor(NormalizeBinaryPredicatesRule.INSTANCE);
 
+        assertRewrite("1 = 1", "1 = 1");
         assertRewrite("2 > x", "x < 2");
-        assertRewrite("2 >= x", "x <= 2");
-        assertRewrite("2 < x", "x > 2");
-        assertRewrite("2 <= x", "x >= 2");
-        assertRewrite("2 = x", "x = 2");
+        assertRewrite("y > x", "y > x");
+        assertRewrite("1 + 2 > x", "x < 1 + 2");
+        assertRewrite("1 + 2 > x + 1", "x + 1 < 1 + 2");
+        assertRewrite("y + 2 > x + 1", "y + 2 > x + 1");
     }
 
     @Test
@@ -152,6 +153,6 @@ public class ExpressionRewriteTest {
         Expression needRewriteExpression = PARSER.parseExpression(expression);
         Expression expectedExpression = PARSER.parseExpression(expected);
         Expression rewrittenExpression = executor.rewrite(needRewriteExpression);
-        Assert.assertEquals(expectedExpression, rewrittenExpression);
+        Assertions.assertEquals(expectedExpression, rewrittenExpression);
     }
 }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/ssb/SSBUtils.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/ssb/SSBUtils.java
index 3433cbefd3..97de88afd3 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/nereids/ssb/SSBUtils.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/ssb/SSBUtils.java
@@ -349,3 +349,4 @@ public class SSBUtils {
                 + ")");
     }
 }
+
diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/ExpressionParserTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/ExpressionParserTest.java
index fafee04bc6..d02a76c0ee 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/ExpressionParserTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/ExpressionParserTest.java
@@ -176,4 +176,34 @@ public class ExpressionParserTest {
         String notExistExpr = "not exists (select * from test where b = 1)";
         assertExpr(notExistExpr);
     }
+
+    @Test
+    public void testInterval() {
+        String interval = "tt > date '1991-05-01' + interval '1' day";
+        assertExpr(interval);
+
+        interval = "tt > '1991-05-01' + interval '1' day";
+        assertExpr(interval);
+
+        interval = "tt > '1991-05-01' + interval 1 day";
+        assertExpr(interval);
+
+        interval = "tt > '1991-05-01' - interval 1 day";
+        assertExpr(interval);
+
+        interval = "tt > date '1991-05-01' - interval '1' day";
+        assertExpr(interval);
+
+        interval = "tt > interval '1' day + '1991-05-01'";
+        assertExpr(interval);
+
+        interval = "tt > interval '1' day + date '1991-05-01'";
+        assertExpr(interval);
+
+        interval = "tt > '1991-05-01'  - interval 2*1 day";
+        assertExpr(interval);
+
+        interval = "tt > now() - interval 1+1 day";
+        assertExpr(interval);
+    }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org