You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by mo...@apache.org on 2022/12/12 13:34:38 UTC

[doris] branch master updated: [feature](nereids) Date add and Date sub related functions (#14753)

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

morrysnow 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 e57419fc9e [feature](nereids) Date add and Date sub related functions (#14753)
e57419fc9e is described below

commit e57419fc9e6d72b805553bd8661eba10de4d6442
Author: minghong <en...@gmail.com>
AuthorDate: Mon Dec 12 21:34:30 2022 +0800

    [feature](nereids) Date add and Date sub related functions (#14753)
    
    ## date_add series
    - DATE_ADD
    - DAYS_ADD
    - ADDDATE
    - TIMESTAMPADD
    
    ## date_sub series
    - DATE_SUB
    - DAYS_SUB
    - SUBDATE
    
    ## NOTE
    1. For DAYS_XXX, time unit is omissible, by default the time unit is DAY
    2. no TIMESTAMPSUB
---
 .../antlr4/org/apache/doris/nereids/DorisLexer.g4  |  7 +-
 .../antlr4/org/apache/doris/nereids/DorisParser.g4 | 20 ++++-
 .../doris/nereids/parser/LogicalPlanBuilder.java   | 94 +++++++++++++++++++++-
 .../trees/expressions/TimestampArithmetic.java     |  4 +
 .../expressions/functions/scalar/DaysAdd.java      | 67 +++++++++++++++
 .../expressions/functions/scalar/DaysSub.java      | 67 +++++++++++++++
 .../expressions/functions/scalar/HoursAdd.java     | 67 +++++++++++++++
 .../expressions/functions/scalar/HoursSub.java     | 67 +++++++++++++++
 .../expressions/functions/scalar/MinutesAdd.java   | 68 ++++++++++++++++
 .../expressions/functions/scalar/MinutesSub.java   | 68 ++++++++++++++++
 .../expressions/functions/scalar/MonthsAdd.java    | 68 ++++++++++++++++
 .../expressions/functions/scalar/MonthsSub.java    | 67 +++++++++++++++
 .../expressions/functions/scalar/SecondsAdd.java   | 68 ++++++++++++++++
 .../expressions/functions/scalar/SecondsSub.java   | 68 ++++++++++++++++
 .../expressions/functions/scalar/YearsAdd.java     | 67 +++++++++++++++
 .../expressions/functions/scalar/YearsSub.java     | 67 +++++++++++++++
 .../nereids/trees/expressions/literal/Literal.java |  4 +-
 .../expressions/visitor/ScalarFunctionVisitor.java | 60 ++++++++++++++
 .../data/nereids_syntax_p0/test_date_add.out       | 49 +++++++++++
 .../data/nereids_syntax_p0/test_date_sub.out       | 55 +++++++++++++
 .../suites/nereids_syntax_p0/test_date_add.groovy  | 41 ++++++++++
 .../suites/nereids_syntax_p0/test_date_sub.groovy  | 42 ++++++++++
 22 files changed, 1178 insertions(+), 7 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 2c54530123..a38a56c160 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
@@ -90,6 +90,7 @@ RIGHT_BRACKET: ']';
 //============================
 //--DORIS-KEYWORD-LIST-START
 ADD: 'ADD';
+ADDDATE:'ADDDATE';
 AFTER: 'AFTER';
 ALL: 'ALL';
 ALTER: 'ALTER';
@@ -143,13 +144,16 @@ CURRENT_TIME: 'CURRENT_TIME';
 CURRENT_TIMESTAMP: 'CURRENT_TIMESTAMP';
 CURRENT_USER: 'CURRENT_USER';
 DAY: 'DAY';
+DAYS_ADD: 'DAYS_ADD';
+DAYS_SUB: 'DAYS_SUB';
 DATA: 'DATA';
 DATABASE: 'DATABASE';
 DATABASES: 'DATABASES';
 DATE: 'DATE';
 DATEADD: 'DATEADD';
-DATE_ADD: 'DATE_ADD';
 DATEDIFF: 'DATEDIFF';
+DATE_ADD: 'DATE_ADD';
+DATE_SUB: 'DATE_SUB';
 DATE_DIFF: 'DATE_DIFF';
 DBPROPERTIES: 'DBPROPERTIES';
 DEFINED: 'DEFINED';
@@ -328,6 +332,7 @@ STATISTICS: 'STATISTICS';
 STORED: 'STORED';
 STRATIFY: 'STRATIFY';
 STRUCT: 'STRUCT';
+SUBDATE: 'SUBDATE';
 SUBSTR: 'SUBSTR';
 SUBSTRING: 'SUBSTRING';
 SUM: 'SUM';
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 4304ab5e95..a5574ec63c 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
@@ -275,6 +275,18 @@ primaryExpression
                 startTimestamp=valueExpression COMMA
                 endTimestamp=valueExpression
             RIGHT_PAREN                                                                        #timestampdiff
+    | name=(TIMESTAMPADD | ADDDATE | DAYS_ADD | DATE_ADD)
+            LEFT_PAREN
+                timestamp=valueExpression COMMA
+                (INTERVAL unitsAmount=valueExpression unit=datetimeUnit
+                | unitsAmount=valueExpression)
+            RIGHT_PAREN                                                                        #date_add
+    | name=(SUBDATE | DAYS_SUB | DATE_SUB)
+            LEFT_PAREN
+                timestamp=valueExpression COMMA
+                (INTERVAL unitsAmount=valueExpression  unit=datetimeUnit
+                | unitsAmount=valueExpression)
+            RIGHT_PAREN                                                                        #date_sub
     | CASE whenClause+ (ELSE elseExpression=expression)? END                                   #searchedCase
     | CASE value=expression whenClause+ (ELSE elseExpression=expression)? END                  #simpleCase
     | name=CAST LEFT_PAREN expression AS identifier RIGHT_PAREN                                #cast
@@ -371,6 +383,7 @@ number
 ansiNonReserved
 //--ANSI-NON-RESERVED-START
     : ADD
+    | ADDDATE
     | AFTER
     | ALTER
     | ANALYZE
@@ -409,11 +422,14 @@ ansiNonReserved
     | DATABASE
     | DATABASES
     | DATE
-    | DATEADD
     | DATE_ADD
     | DATEDIFF
     | DATE_DIFF
     | DAY
+    | DAYS_ADD
+    | DAYS_SUB
+    | DATE_ADD
+    | DATE_SUB
     | DBPROPERTIES
     | DEFINED
     | DELETE
@@ -545,6 +561,7 @@ ansiNonReserved
     | STORED
     | STRATIFY
     | STRUCT
+    | SUBDATE
     | SUBSTR
     | SUBSTRING
     | SUM
@@ -670,7 +687,6 @@ nonReserved
     | DATABASE
     | DATABASES
     | DATE
-    | DATEADD
     | DATE_ADD
     | DATEDIFF
     | DATE_DIFF
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 756e351331..5ee8d839a8 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
@@ -31,10 +31,13 @@ import org.apache.doris.nereids.DorisParser.ColumnReferenceContext;
 import org.apache.doris.nereids.DorisParser.ComparisonContext;
 import org.apache.doris.nereids.DorisParser.CreateRowPolicyContext;
 import org.apache.doris.nereids.DorisParser.CteContext;
+import org.apache.doris.nereids.DorisParser.Date_addContext;
+import org.apache.doris.nereids.DorisParser.Date_subContext;
 import org.apache.doris.nereids.DorisParser.DecimalLiteralContext;
 import org.apache.doris.nereids.DorisParser.DereferenceContext;
 import org.apache.doris.nereids.DorisParser.ExistContext;
 import org.apache.doris.nereids.DorisParser.ExplainContext;
+import org.apache.doris.nereids.DorisParser.ExpressionContext;
 import org.apache.doris.nereids.DorisParser.FromClauseContext;
 import org.apache.doris.nereids.DorisParser.GroupingElementContext;
 import org.apache.doris.nereids.DorisParser.GroupingSetContext;
@@ -126,12 +129,24 @@ import org.apache.doris.nereids.trees.expressions.Subtract;
 import org.apache.doris.nereids.trees.expressions.TVFProperties;
 import org.apache.doris.nereids.trees.expressions.TimestampArithmetic;
 import org.apache.doris.nereids.trees.expressions.WhenClause;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.DaysAdd;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.DaysDiff;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.DaysSub;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursAdd;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursDiff;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursSub;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.MinutesAdd;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.MinutesDiff;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.MinutesSub;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.MonthsAdd;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.MonthsDiff;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.MonthsSub;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondsAdd;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondsDiff;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondsSub;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.YearsAdd;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.YearsDiff;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.YearsSub;
 import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral;
 import org.apache.doris.nereids.trees.expressions.literal.BooleanLiteral;
 import org.apache.doris.nereids.trees.expressions.literal.DateLiteral;
@@ -167,6 +182,8 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalSort;
 import org.apache.doris.nereids.trees.plans.logical.LogicalSubQueryAlias;
 import org.apache.doris.nereids.trees.plans.logical.RelationUtil;
 import org.apache.doris.nereids.types.DataType;
+import org.apache.doris.nereids.types.IntegerType;
+import org.apache.doris.nereids.types.TinyIntType;
 import org.apache.doris.nereids.util.ExpressionUtils;
 import org.apache.doris.policy.PolicyTypeEnum;
 import org.apache.doris.qe.ConnectContext;
@@ -577,10 +594,82 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> {
         } else if ("SECOND".equalsIgnoreCase(unit)) {
             return new SecondsDiff(end, start);
         }
-        throw new ParseException("Unsupported time stamp diff time unit: " + unit, ctx);
+        throw new ParseException("Unsupported time stamp diff time unit: " + unit
+                + ", supported time unit: YEAR/MONTH/DAY/HOUR/MINUTE/SECOND", ctx);
 
     }
 
+    @Override
+    public Expression visitDate_add(Date_addContext ctx) {
+        Expression timeStamp = (Expression) visit(ctx.timestamp);
+        Expression amount = (Expression) visit(ctx.unitsAmount);
+        if (ctx.unit == null) {
+            if ("days_add".equalsIgnoreCase(ctx.name.getText())) {
+                return new DaysAdd(timeStamp, amount);
+            }
+            throw new ParseException("Unsupported signature: " + ctx.name
+                    + " needs time unit (YEAR/MONTH/DAY/HOUR/MINUTE/SECOND)", ctx);
+        }
+
+        if ("DAY".equalsIgnoreCase(ctx.unit.getText())) {
+            return new DaysAdd(timeStamp, amount);
+        }
+        if ("MONTH".equalsIgnoreCase(ctx.unit.getText())) {
+            return new MonthsAdd(timeStamp, amount);
+        }
+        if ("Year".equalsIgnoreCase(ctx.unit.getText())) {
+            return new YearsAdd(timeStamp, amount);
+        }
+        if ("Hour".equalsIgnoreCase(ctx.unit.getText())) {
+            return new HoursAdd(timeStamp, amount);
+        }
+        if ("Minute".equalsIgnoreCase(ctx.unit.getText())) {
+            return new MinutesAdd(timeStamp, amount);
+        }
+        if ("Second".equalsIgnoreCase(ctx.unit.getText())) {
+            return new SecondsAdd(timeStamp, amount);
+        }
+        throw new ParseException("Unsupported time unit: " + ctx.unit
+                + ", supported time unit: YEAR/MONTH/DAY/HOUR/MINUTE/SECOND", ctx);
+    }
+
+    @Override
+    public Expression visitDate_sub(Date_subContext ctx) {
+        Expression timeStamp = (Expression) visit(ctx.timestamp);
+        Expression amount = (Expression) visit(ctx.unitsAmount);
+        if (! (amount.getDataType() instanceof TinyIntType)) {
+            amount = new Cast(amount, IntegerType.INSTANCE);
+        }
+        if (ctx.unit == null) {
+            if ("days_sub".equalsIgnoreCase(ctx.name.getText())) {
+                return new DaysSub(timeStamp, amount);
+            }
+            throw new ParseException("Unsupported signature: " + ctx.name
+                    + " needs time unit (YEAR/MONTH/DAY/HOUR/MINUTE/SECOND)", ctx);
+        }
+
+        if ("DAY".equalsIgnoreCase(ctx.unit.getText())) {
+            return new DaysSub(timeStamp, amount);
+        }
+        if ("MONTH".equalsIgnoreCase(ctx.unit.getText())) {
+            return new MonthsSub(timeStamp, amount);
+        }
+        if ("Year".equalsIgnoreCase(ctx.unit.getText())) {
+            return new YearsSub(timeStamp, amount);
+        }
+        if ("Hour".equalsIgnoreCase(ctx.unit.getText())) {
+            return new HoursSub(timeStamp, amount);
+        }
+        if ("Minute".equalsIgnoreCase(ctx.unit.getText())) {
+            return new MinutesSub(timeStamp, amount);
+        }
+        if ("Second".equalsIgnoreCase(ctx.unit.getText())) {
+            return new SecondsSub(timeStamp, amount);
+        }
+        throw new ParseException("Unsupported time unit: " + ctx.unit
+                + ", supported time unit: YEAR/MONTH/DAY/HOUR/MINUTE/SECOND", ctx);
+    }
+
     /**
      * Create a value based [[CaseWhen]] expression. This has the following SQL form:
      * {{{
@@ -648,7 +737,8 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> {
             //      the function information is obtained by parsing the catalog. This method is more scalable.
             String functionName = ctx.identifier().getText();
             boolean isDistinct = ctx.DISTINCT() != null;
-            List<Expression> params = visit(ctx.expression(), Expression.class);
+            List<ExpressionContext> expressionContexts = ctx.expression();
+            List<Expression> params = visit(expressionContexts, Expression.class);
             for (Expression expression : params) {
                 if (expression instanceof UnboundStar && functionName.equalsIgnoreCase("count") && !isDistinct) {
                     return new UnboundFunction(functionName, false, true, new ArrayList<>());
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
index c3365050b6..9f96fe9edf 100644
--- 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
@@ -48,6 +48,10 @@ import java.util.Objects;
 public class TimestampArithmetic extends Expression implements BinaryExpression, ImplicitCastInputTypes,
         PropagateNullable {
 
+    //the size and order of EXPECTED_INPUT_TYPES must follow the function signature parameters
+    //For example: days_sub('2000-01-01', interval 5 days),
+    // '2000-01-01'->DateTimeType.INSTANCE
+    // 5 -> IntegerType
     private static final List<AbstractDataType> EXPECTED_INPUT_TYPES = ImmutableList.of(
             DateTimeType.INSTANCE,
             IntegerType.INSTANCE
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DaysAdd.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DaysAdd.java
new file mode 100644
index 0000000000..c7bb1b8e09
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DaysAdd.java
@@ -0,0 +1,67 @@
+// 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.functions.scalar;
+
+import org.apache.doris.catalog.FunctionSignature;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable;
+import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
+import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.DateTimeType;
+import org.apache.doris.nereids.types.DateTimeV2Type;
+import org.apache.doris.nereids.types.DateType;
+import org.apache.doris.nereids.types.DateV2Type;
+import org.apache.doris.nereids.types.IntegerType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * ScalarFunction 'days_add'.
+ */
+public class DaysAdd extends ScalarFunction implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNullable {
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeV2Type.INSTANCE).args(DateTimeV2Type.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE)
+    );
+
+    public DaysAdd(Expression arg0, Expression arg1) {
+        super("days_add", arg0, arg1);
+    }
+
+    @Override
+    public DaysAdd withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 2);
+        return new DaysAdd(children.get(0), children.get(1));
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitDaysAdd(this, context);
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DaysSub.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DaysSub.java
new file mode 100644
index 0000000000..001244f226
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DaysSub.java
@@ -0,0 +1,67 @@
+// 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.functions.scalar;
+
+import org.apache.doris.catalog.FunctionSignature;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable;
+import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
+import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.DateTimeType;
+import org.apache.doris.nereids.types.DateTimeV2Type;
+import org.apache.doris.nereids.types.DateType;
+import org.apache.doris.nereids.types.DateV2Type;
+import org.apache.doris.nereids.types.IntegerType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * ScalarFunction 'days_add'.
+ */
+public class DaysSub extends ScalarFunction implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNullable {
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeV2Type.INSTANCE).args(DateTimeV2Type.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE)
+    );
+
+    public DaysSub(Expression arg0, Expression arg1) {
+        super("days_sub", arg0, arg1);
+    }
+
+    @Override
+    public DaysSub withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 2);
+        return new DaysSub(children.get(0), children.get(1));
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitDaysSub(this, context);
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HoursAdd.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HoursAdd.java
new file mode 100644
index 0000000000..0c4820b52d
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HoursAdd.java
@@ -0,0 +1,67 @@
+// 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.functions.scalar;
+
+import org.apache.doris.catalog.FunctionSignature;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable;
+import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
+import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.DateTimeType;
+import org.apache.doris.nereids.types.DateTimeV2Type;
+import org.apache.doris.nereids.types.DateType;
+import org.apache.doris.nereids.types.DateV2Type;
+import org.apache.doris.nereids.types.IntegerType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * ScalarFunction 'days_add'.
+ */
+public class HoursAdd extends ScalarFunction implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNullable {
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeV2Type.INSTANCE).args(DateTimeV2Type.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE)
+    );
+
+    public HoursAdd(Expression arg0, Expression arg1) {
+        super("hours_add", arg0, arg1);
+    }
+
+    @Override
+    public HoursAdd withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 2);
+        return new HoursAdd(children.get(0), children.get(1));
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitHoursAdd(this, context);
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HoursSub.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HoursSub.java
new file mode 100644
index 0000000000..cab530fa54
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HoursSub.java
@@ -0,0 +1,67 @@
+// 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.functions.scalar;
+
+import org.apache.doris.catalog.FunctionSignature;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable;
+import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
+import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.DateTimeType;
+import org.apache.doris.nereids.types.DateTimeV2Type;
+import org.apache.doris.nereids.types.DateType;
+import org.apache.doris.nereids.types.DateV2Type;
+import org.apache.doris.nereids.types.IntegerType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * ScalarFunction 'hours_sub'.
+ */
+public class HoursSub extends ScalarFunction implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNullable {
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeV2Type.INSTANCE).args(DateTimeV2Type.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE)
+    );
+
+    public HoursSub(Expression arg0, Expression arg1) {
+        super("hours_sub", arg0, arg1);
+    }
+
+    @Override
+    public HoursSub withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 2);
+        return new HoursSub(children.get(0), children.get(1));
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitHoursSub(this, context);
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MinutesAdd.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MinutesAdd.java
new file mode 100644
index 0000000000..8dbf28a6bc
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MinutesAdd.java
@@ -0,0 +1,68 @@
+// 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.functions.scalar;
+
+import org.apache.doris.catalog.FunctionSignature;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable;
+import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
+import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.DateTimeType;
+import org.apache.doris.nereids.types.DateTimeV2Type;
+import org.apache.doris.nereids.types.DateType;
+import org.apache.doris.nereids.types.DateV2Type;
+import org.apache.doris.nereids.types.IntegerType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * ScalarFunction 'minutes_add'.
+ */
+public class MinutesAdd extends ScalarFunction
+        implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNullable {
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeV2Type.INSTANCE).args(DateTimeV2Type.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE)
+    );
+
+    public MinutesAdd(Expression arg0, Expression arg1) {
+        super("minutes_add", arg0, arg1);
+    }
+
+    @Override
+    public MinutesAdd withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 2);
+        return new MinutesAdd(children.get(0), children.get(1));
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitMinutesAdd(this, context);
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MinutesSub.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MinutesSub.java
new file mode 100644
index 0000000000..8733843dea
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MinutesSub.java
@@ -0,0 +1,68 @@
+// 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.functions.scalar;
+
+import org.apache.doris.catalog.FunctionSignature;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable;
+import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
+import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.DateTimeType;
+import org.apache.doris.nereids.types.DateTimeV2Type;
+import org.apache.doris.nereids.types.DateType;
+import org.apache.doris.nereids.types.DateV2Type;
+import org.apache.doris.nereids.types.IntegerType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * ScalarFunction 'minutes_sub'.
+ */
+public class MinutesSub extends ScalarFunction
+        implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNullable {
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeV2Type.INSTANCE).args(DateTimeV2Type.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE)
+    );
+
+    public MinutesSub(Expression arg0, Expression arg1) {
+        super("minutes_sub", arg0, arg1);
+    }
+
+    @Override
+    public MinutesSub withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 2);
+        return new MinutesSub(children.get(0), children.get(1));
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitMinutesSub(this, context);
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MonthsAdd.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MonthsAdd.java
new file mode 100644
index 0000000000..4cfc73454e
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MonthsAdd.java
@@ -0,0 +1,68 @@
+// 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.functions.scalar;
+
+import org.apache.doris.catalog.FunctionSignature;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable;
+import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
+import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.DateTimeType;
+import org.apache.doris.nereids.types.DateTimeV2Type;
+import org.apache.doris.nereids.types.DateType;
+import org.apache.doris.nereids.types.DateV2Type;
+import org.apache.doris.nereids.types.IntegerType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * ScalarFunction 'months_add'.
+ */
+public class MonthsAdd extends ScalarFunction
+        implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNullable {
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeV2Type.INSTANCE).args(DateTimeV2Type.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE)
+    );
+
+    public MonthsAdd(Expression arg0, Expression arg1) {
+        super("months_add", arg0, arg1);
+    }
+
+    @Override
+    public MonthsAdd withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 2);
+        return new MonthsAdd(children.get(0), children.get(1));
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitMonthsAdd(this, context);
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MonthsSub.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MonthsSub.java
new file mode 100644
index 0000000000..10df4e2195
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MonthsSub.java
@@ -0,0 +1,67 @@
+// 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.functions.scalar;
+
+import org.apache.doris.catalog.FunctionSignature;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable;
+import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
+import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.DateTimeType;
+import org.apache.doris.nereids.types.DateTimeV2Type;
+import org.apache.doris.nereids.types.DateType;
+import org.apache.doris.nereids.types.DateV2Type;
+import org.apache.doris.nereids.types.IntegerType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * ScalarFunction 'months_sub'.
+ */
+public class MonthsSub extends ScalarFunction implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNullable {
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeV2Type.INSTANCE).args(DateTimeV2Type.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE)
+    );
+
+    public MonthsSub(Expression arg0, Expression arg1) {
+        super("months_sub", arg0, arg1);
+    }
+
+    @Override
+    public MonthsSub withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 2);
+        return new MonthsSub(children.get(0), children.get(1));
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitMonthsSub(this, context);
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SecondsAdd.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SecondsAdd.java
new file mode 100644
index 0000000000..0a23c66d75
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SecondsAdd.java
@@ -0,0 +1,68 @@
+// 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.functions.scalar;
+
+import org.apache.doris.catalog.FunctionSignature;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable;
+import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
+import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.DateTimeType;
+import org.apache.doris.nereids.types.DateTimeV2Type;
+import org.apache.doris.nereids.types.DateType;
+import org.apache.doris.nereids.types.DateV2Type;
+import org.apache.doris.nereids.types.IntegerType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * ScalarFunction 'minutes_add'.
+ */
+public class SecondsAdd extends ScalarFunction
+        implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNullable {
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeV2Type.INSTANCE).args(DateTimeV2Type.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE)
+    );
+
+    public SecondsAdd(Expression arg0, Expression arg1) {
+        super("seconds_add", arg0, arg1);
+    }
+
+    @Override
+    public SecondsAdd withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 2);
+        return new SecondsAdd(children.get(0), children.get(1));
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitSecondsAdd(this, context);
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SecondsSub.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SecondsSub.java
new file mode 100644
index 0000000000..1f6f958315
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SecondsSub.java
@@ -0,0 +1,68 @@
+// 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.functions.scalar;
+
+import org.apache.doris.catalog.FunctionSignature;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable;
+import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
+import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.DateTimeType;
+import org.apache.doris.nereids.types.DateTimeV2Type;
+import org.apache.doris.nereids.types.DateType;
+import org.apache.doris.nereids.types.DateV2Type;
+import org.apache.doris.nereids.types.IntegerType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * ScalarFunction 'Seconds_sub'.
+ */
+public class SecondsSub extends ScalarFunction
+        implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNullable {
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeV2Type.INSTANCE).args(DateTimeV2Type.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE)
+    );
+
+    public SecondsSub(Expression arg0, Expression arg1) {
+        super("Seconds_sub", arg0, arg1);
+    }
+
+    @Override
+    public SecondsSub withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 2);
+        return new SecondsSub(children.get(0), children.get(1));
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitSecondsSub(this, context);
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/YearsAdd.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/YearsAdd.java
new file mode 100644
index 0000000000..26b8db3c0d
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/YearsAdd.java
@@ -0,0 +1,67 @@
+// 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.functions.scalar;
+
+import org.apache.doris.catalog.FunctionSignature;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable;
+import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
+import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.DateTimeType;
+import org.apache.doris.nereids.types.DateTimeV2Type;
+import org.apache.doris.nereids.types.DateType;
+import org.apache.doris.nereids.types.DateV2Type;
+import org.apache.doris.nereids.types.IntegerType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * ScalarFunction 'days_add'.
+ */
+public class YearsAdd extends ScalarFunction implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNullable {
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeV2Type.INSTANCE).args(DateTimeV2Type.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE)
+    );
+
+    public YearsAdd(Expression arg0, Expression arg1) {
+        super("years_add", arg0, arg1);
+    }
+
+    @Override
+    public YearsAdd withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 2);
+        return new YearsAdd(children.get(0), children.get(1));
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitYearsAdd(this, context);
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/YearsSub.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/YearsSub.java
new file mode 100644
index 0000000000..671426da91
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/YearsSub.java
@@ -0,0 +1,67 @@
+// 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.functions.scalar;
+
+import org.apache.doris.catalog.FunctionSignature;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable;
+import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
+import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.DateTimeType;
+import org.apache.doris.nereids.types.DateTimeV2Type;
+import org.apache.doris.nereids.types.DateType;
+import org.apache.doris.nereids.types.DateV2Type;
+import org.apache.doris.nereids.types.IntegerType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * ScalarFunction 'days_add'.
+ */
+public class YearsSub extends ScalarFunction implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNullable {
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeV2Type.INSTANCE).args(DateTimeV2Type.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE)
+    );
+
+    public YearsSub(Expression arg0, Expression arg1) {
+        super("years_sub", arg0, arg1);
+    }
+
+    @Override
+    public YearsSub withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 2);
+        return new YearsSub(children.get(0), children.get(1));
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitYearsSub(this, context);
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/Literal.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/Literal.java
index 307f7d8574..8b376f21ad 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/Literal.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/Literal.java
@@ -87,10 +87,10 @@ public abstract class Literal extends Expression implements LeafExpression, Comp
      * for numeric literal (int/long/double/float), directly convert to double
      * for char/varchar/string, we take first 8 chars as a int64, and convert it to double
      * for other literals, getDouble() is not used.
-     *
+     * <p>
      * And hence, we could express the range of a datatype, and used in stats derive.
      * for example:
-     *'abcxxxxxxxxxxx' is between ('abb', 'zzz')
+     * 'abcxxxxxxxxxxx' is between ('abb', 'zzz')
      *
      * @return double representation of literal.
      */
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
index 97d3779d2e..c4e2366889 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
@@ -76,7 +76,9 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.DayName;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.DayOfMonth;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.DayOfWeek;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.DayOfYear;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.DaysAdd;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.DaysDiff;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.DaysSub;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Dceil;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Degrees;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Dexp;
@@ -110,7 +112,9 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.HllHash;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Hour;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.HourCeil;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.HourFloor;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursAdd;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursDiff;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursSub;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.If;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Initcap;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Instr;
@@ -156,13 +160,17 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.Md5Sum;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Minute;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.MinuteCeil;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.MinuteFloor;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.MinutesAdd;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.MinutesDiff;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.MinutesSub;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.MoneyFormat;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Month;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.MonthCeil;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.MonthFloor;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.MonthName;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.MonthsAdd;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.MonthsDiff;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.MonthsSub;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.MurmurHash332;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.MurmurHash364;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Negative;
@@ -194,7 +202,9 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.ScalarFunctio
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Second;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondCeil;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondFloor;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondsAdd;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondsDiff;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondsSub;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Sign;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Sin;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Sleep;
@@ -254,7 +264,9 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.Year;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.YearCeil;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.YearFloor;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.YearWeek;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.YearsAdd;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.YearsDiff;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.YearsSub;
 
 /** ScalarFunctionVisitor. */
 public interface ScalarFunctionVisitor<R, C> {
@@ -501,6 +513,54 @@ public interface ScalarFunctionVisitor<R, C> {
         return visitScalarFunction(daysDiff, context);
     }
 
+    default R visitDaysAdd(DaysAdd daysAdd, C context) {
+        return visitScalarFunction(daysAdd, context);
+    }
+
+    default R visitDaysSub(DaysSub daysSub, C context) {
+        return visitScalarFunction(daysSub, context);
+    }
+
+    default R visitYearsSub(YearsSub yearsSub, C context) {
+        return visitScalarFunction(yearsSub, context);
+    }
+
+    default R visitMonthsSub(MonthsSub monthsSub, C context) {
+        return visitScalarFunction(monthsSub, context);
+    }
+
+    default R visitHoursSub(HoursSub hoursSub, C context) {
+        return visitScalarFunction(hoursSub, context);
+    }
+
+    default R visitMinutesSub(MinutesSub minutesSub, C context) {
+        return visitScalarFunction(minutesSub, context);
+    }
+
+    default R visitSecondsSub(SecondsSub secondsSub, C context) {
+        return visitScalarFunction(secondsSub, context);
+    }
+
+    default R visitMonthsAdd(MonthsAdd monthsAdd, C context) {
+        return visitScalarFunction(monthsAdd, context);
+    }
+
+    default R visitYearsAdd(YearsAdd yearsAdd, C context) {
+        return visitScalarFunction(yearsAdd, context);
+    }
+
+    default R visitHoursAdd(HoursAdd hoursAdd, C context) {
+        return visitScalarFunction(hoursAdd, context);
+    }
+
+    default R visitMinutesAdd(MinutesAdd minutesAdd, C context) {
+        return visitScalarFunction(minutesAdd, context);
+    }
+
+    default R visitSecondsAdd(SecondsAdd secondsAdd, C context) {
+        return visitScalarFunction(secondsAdd, context);
+    }
+
     default R visitDceil(Dceil dceil, C context) {
         return visitScalarFunction(dceil, context);
     }
diff --git a/regression-test/data/nereids_syntax_p0/test_date_add.out b/regression-test/data/nereids_syntax_p0/test_date_add.out
new file mode 100644
index 0000000000..5e0c38d2c6
--- /dev/null
+++ b/regression-test/data/nereids_syntax_p0/test_date_add.out
@@ -0,0 +1,49 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !select --
+2020-01-03T00:00
+
+-- !select --
+2020-01-03T00:00
+
+-- !select --
+2020-01-03T00:00
+
+-- !select --
+2020-01-03T00:00
+
+-- !select --
+2020-01-01T02:00
+
+-- !select --
+2020-01-01T02:00
+
+-- !select --
+2020-01-01T02:00
+
+-- !select --
+2020-01-01T02:00
+
+-- !select --
+2022-01-01T00:00
+
+-- !select --
+2020-01-01T02:00
+
+-- !select --
+2020-01-01T00:02
+
+-- !select --
+2020-01-01T00:00:02
+
+-- !select --
+2020-01-03T00:00
+
+-- !select --
+2020-01-03T00:00
+
+-- !select --
+2020-01-03T00:00
+
+-- !select --
+2019-12-28T00:00
+
diff --git a/regression-test/data/nereids_syntax_p0/test_date_sub.out b/regression-test/data/nereids_syntax_p0/test_date_sub.out
new file mode 100644
index 0000000000..013bd78d91
--- /dev/null
+++ b/regression-test/data/nereids_syntax_p0/test_date_sub.out
@@ -0,0 +1,55 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !select --
+2018-01-01T00:00
+
+-- !select --
+2018-01-01T00:00
+
+-- !select --
+2018-01-01T00:00
+
+-- !select --
+2018-01-01T00:00
+
+-- !select --
+2018-01-01T00:00
+
+-- !select --
+2019-12-31T23:58
+
+-- !select --
+2019-12-31T23:58
+
+-- !select --
+2019-12-31T23:58
+
+-- !select --
+2019-12-31T23:58
+
+-- !select --
+2019-12-31T23:58
+
+-- !select --
+2019-12-30T00:00
+
+-- !select --
+2019-12-31T22:00
+
+-- !select --
+2019-12-31T23:58
+
+-- !select --
+2019-12-31T23:59:58
+
+-- !select --
+2019-12-30T00:00
+
+-- !select --
+2019-12-30T00:00
+
+-- !select --
+2019-12-30T00:00
+
+-- !select --
+2020-01-05T00:00
+
diff --git a/regression-test/suites/nereids_syntax_p0/test_date_add.groovy b/regression-test/suites/nereids_syntax_p0/test_date_add.groovy
new file mode 100644
index 0000000000..998b6962da
--- /dev/null
+++ b/regression-test/suites/nereids_syntax_p0/test_date_add.groovy
@@ -0,0 +1,41 @@
+// 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.
+
+suite("test_date_add") {
+    sql "set enable_nereids_planner=true"
+    sql "set enable_fallback_to_original_planner=false"
+    sql "set enable_fold_constant_by_be=false"
+    
+    qt_select "select TIMESTAMPADD(cast('2020-01-01' as date), interval 2 day)"
+    qt_select "select TIMESTAMPADD(cast('2020-01-01' as datev2),interval 2 day)"
+    qt_select "select TIMESTAMPADD(cast('2020-01-01' as datetime), interval 2 day)"
+    qt_select "select TIMESTAMPADD(cast('2020-01-01' as datetimev2), interval 2 day)"
+    
+    qt_select "select TIMESTAMPADD(cast('2020-01-01' as date), interval 2 hour)"
+    qt_select "select TIMESTAMPADD(cast('2020-01-01' as datev2),interval 2 hour)"
+    qt_select "select TIMESTAMPADD(cast('2020-01-01' as datetime), interval 2 hour)"
+    qt_select "select TIMESTAMPADD(cast('2020-01-01' as datetimev2), interval 2 hour)"
+    qt_select "select TIMESTAMPADD('2020-01-01' , interval 2 year)"
+
+    qt_select "SELECT DAYS_ADD('2020-01-01', interval 2 hour)"
+    qt_select "SELECT DAYS_ADD('2020-01-01', interval 2 minute)"
+    qt_select "SELECT DAYS_ADD('2020-01-01', interval 2 second)"
+    qt_select "SELECT DATE_ADD('2020-01-01', interval 2 day)"
+    qt_select "SELECT ADDDATE('2020-01-01', interval 2 day)"
+    qt_select "SELECT DAYS_ADD('2020-01-01', 2)"
+    qt_select "SELECT DAYS_ADD('2020-01-01', -4)"
+}
\ No newline at end of file
diff --git a/regression-test/suites/nereids_syntax_p0/test_date_sub.groovy b/regression-test/suites/nereids_syntax_p0/test_date_sub.groovy
new file mode 100644
index 0000000000..3c54d1765c
--- /dev/null
+++ b/regression-test/suites/nereids_syntax_p0/test_date_sub.groovy
@@ -0,0 +1,42 @@
+// 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.
+
+suite("test_date_sub") {
+    sql "set enable_nereids_planner=true"
+    sql "set enable_fallback_to_original_planner=false"
+
+    qt_select "select DAYS_SUB(cast('2020-01-01' as date), interval 2 year)"
+    qt_select "select DAYS_SUB(cast('2020-01-01' as datev2),interval 2 year)"
+    qt_select "select DAYS_SUB(cast('2020-01-01' as datetime), interval 2 year)"
+    qt_select "select DAYS_SUB(cast('2020-01-01' as datetimev2), interval 2 year)"
+    qt_select "select DAYS_SUB('2020-01-01' , interval 2 year)"
+    
+    qt_select "select DAYS_SUB(cast('2020-01-01' as date), interval 2 minute)"
+    qt_select "select DAYS_SUB(cast('2020-01-01' as datev2),interval 2 minute)"
+    qt_select "select DAYS_SUB(cast('2020-01-01' as datetime), interval 2 minute)"
+    qt_select "select DAYS_SUB(cast('2020-01-01' as datetimev2), interval 2 minute)"
+    qt_select "select DAYS_SUB('2020-01-01' , interval 2 minute)"
+
+    qt_select "SELECT DAYS_SUB('2020-01-01', interval 2 day)"
+    qt_select "SELECT DAYS_SUB('2020-01-01', interval 2 hour)"
+    qt_select "SELECT DAYS_SUB('2020-01-01', interval 2 minute)"
+    qt_select "SELECT DAYS_SUB('2020-01-01', interval 2 second)"
+    qt_select "SELECT DATE_SUB('2020-01-01', interval 2 day)"
+    qt_select "SELECT SUBDATE('2020-01-01', interval 2 day)"
+    qt_select "SELECT DAYS_SUB('2020-01-01', 2)"
+    qt_select "SELECT DAYS_SUB('2020-01-01', -4)"
+}
\ No newline at end of file


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