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 5b09d27d54 [feature-wip](nereids) Made decimal in nereids more 
complete (#15087)
5b09d27d54 is described below

commit 5b09d27d54be28358ce2c51da6e0d85d182b3553
Author: Kikyou1997 <33112463+kikyou1...@users.noreply.github.com>
AuthorDate: Thu Dec 29 13:01:47 2022 +0800

    [feature-wip](nereids) Made decimal in nereids more complete (#15087)
    
    1. Add IntegralDivide operator to support `DIV` semantics
    2. Add more operator rewriter to keep expression type consistent between 
operators
    3. Support the convertion between float type and decimal type.
    
    After this PR, below cases could be executed normaly like the legacy 
optimizer:
      use test_query_db;
      select k1, k5,100000*k5 from test order by k1, k2, k3, k4;
      select avg(k9) as a from test group by k1 having a < 100.0 order by a;
---
 .../doris/nereids/jobs/batch/AnalyzeRulesJob.java  |   7 +-
 .../doris/nereids/parser/LogicalPlanBuilder.java   |   3 +-
 .../org/apache/doris/nereids/rules/RuleType.java   |   3 +
 .../ResolveOrdinalInOrderByAndGroupBy.java         |   9 +-
 .../expression/rewrite/ExpressionRewrite.java      |  52 +++-
 .../expression/rewrite/rules/TypeCoercion.java     |   9 +
 .../doris/nereids/stats/ExpressionEstimation.java  |   3 +-
 .../nereids/trees/expressions/IntegralDivide.java  |  60 ++++
 .../expressions/visitor/ExpressionVisitor.java     |   5 +
 .../nereids/trees/plans/logical/LogicalHaving.java |   5 +
 .../doris/nereids/util/TypeCoercionUtils.java      |  10 +-
 .../rules/analysis/FillUpMissingSlotsTest.java     |  11 +-
 .../doris/nereids/util/TypeCoercionUtilsTest.java  |   2 +-
 .../test_arithmetic_operators.out                  | 310 +++++++++++++++++++++
 .../data/nereids_syntax_p0/test_query_between.out  |  41 +++
 .../nereids_syntax_p0/unary_binary_arithmetic.out  |   2 +-
 .../test_arithmetic_operators.groovy               |  80 ++++++
 .../nereids_syntax_p0/test_query_between.groovy    |  42 +++
 18 files changed, 642 insertions(+), 12 deletions(-)

diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/batch/AnalyzeRulesJob.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/batch/AnalyzeRulesJob.java
index fc7fd78856..9e7cebac75 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/batch/AnalyzeRulesJob.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/batch/AnalyzeRulesJob.java
@@ -31,6 +31,9 @@ import 
org.apache.doris.nereids.rules.analysis.ReplaceExpressionByChildOutput;
 import 
org.apache.doris.nereids.rules.analysis.ResolveOrdinalInOrderByAndGroupBy;
 import org.apache.doris.nereids.rules.analysis.Scope;
 import org.apache.doris.nereids.rules.analysis.UserAuthentication;
+import 
org.apache.doris.nereids.rules.expression.rewrite.ExpressionNormalization;
+import 
org.apache.doris.nereids.rules.expression.rewrite.rules.CharacterLiteralTypeCoercion;
+import org.apache.doris.nereids.rules.expression.rewrite.rules.TypeCoercion;
 import 
org.apache.doris.nereids.rules.rewrite.logical.HideOneRowRelationUnderUnion;
 
 import com.google.common.collect.ImmutableList;
@@ -68,7 +71,9 @@ public class AnalyzeRulesJob extends BatchRulesJob {
                     new ProjectWithDistinctToAggregate(),
                     new ResolveOrdinalInOrderByAndGroupBy(),
                     new ReplaceExpressionByChildOutput(),
-                    new HideOneRowRelationUnderUnion()
+                    new HideOneRowRelationUnderUnion(),
+                    new 
ExpressionNormalization(cascadesContext.getConnectContext(),
+                                
ImmutableList.of(CharacterLiteralTypeCoercion.INSTANCE, TypeCoercion.INSTANCE))
                 )),
                 topDownBatch(ImmutableList.of(
                     new FillUpMissingSlots(),
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 59615e27ea..551a594ca6 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
@@ -122,6 +122,7 @@ import 
org.apache.doris.nereids.trees.expressions.GreaterThan;
 import org.apache.doris.nereids.trees.expressions.GreaterThanEqual;
 import org.apache.doris.nereids.trees.expressions.InPredicate;
 import org.apache.doris.nereids.trees.expressions.InSubquery;
+import org.apache.doris.nereids.trees.expressions.IntegralDivide;
 import org.apache.doris.nereids.trees.expressions.IsNull;
 import org.apache.doris.nereids.trees.expressions.LessThan;
 import org.apache.doris.nereids.trees.expressions.LessThanEqual;
@@ -672,7 +673,7 @@ public class LogicalPlanBuilder extends 
DorisParserBaseVisitor<Object> {
                     case DorisParser.MINUS:
                         return new Subtract(left, right);
                     case DorisParser.DIV:
-                        return new Divide(left, right);
+                        return new IntegralDivide(left, right);
                     case DorisParser.HAT:
                         return new BitXor(left, right);
                     case DorisParser.PIPE:
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 65454c14dd..fb04269bfa 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
@@ -135,6 +135,9 @@ public enum RuleType {
     REWRITE_FILTER_EXPRESSION(RuleTypeClass.REWRITE),
     REWRITE_JOIN_EXPRESSION(RuleTypeClass.REWRITE),
     REWRITE_GENERATE_EXPRESSION(RuleTypeClass.REWRITE),
+    REWRITE_SORT_EXPRESSION(RuleTypeClass.REWRITE),
+    REWRITE_HAVING_EXPRESSSION(RuleTypeClass.REWRITE),
+    REWRITE_REPEAT_EXPRESSSION(RuleTypeClass.REWRITE),
     REORDER_JOIN(RuleTypeClass.REWRITE),
     // Merge Consecutive plan
     MERGE_PROJECTS(RuleTypeClass.REWRITE),
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ResolveOrdinalInOrderByAndGroupBy.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ResolveOrdinalInOrderByAndGroupBy.java
index 9b551f68f2..b04f085cef 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ResolveOrdinalInOrderByAndGroupBy.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ResolveOrdinalInOrderByAndGroupBy.java
@@ -66,6 +66,7 @@ public class ResolveOrdinalInOrderByAndGroupBy implements 
AnalysisRuleFactory {
                         logicalAggregate().then(agg -> {
                             List<NamedExpression> aggOutput = 
agg.getOutputExpressions();
                             List<Expression> groupByWithoutOrd = new 
ArrayList<>();
+                            boolean ordExists = false;
                             for (Expression groupByExpr : 
agg.getGroupByExpressions()) {
                                 groupByExpr = 
FoldConstantRule.INSTANCE.rewrite(groupByExpr);
                                 if (groupByExpr instanceof IntegerLikeLiteral) 
{
@@ -74,11 +75,17 @@ public class ResolveOrdinalInOrderByAndGroupBy implements 
AnalysisRuleFactory {
                                     checkOrd(ord, aggOutput.size());
                                     Expression aggExpr = aggOutput.get(ord - 
1);
                                     groupByWithoutOrd.add(aggExpr);
+                                    ordExists = true;
                                 } else {
                                     groupByWithoutOrd.add(groupByExpr);
                                 }
                             }
-                            return new LogicalAggregate(groupByWithoutOrd, 
agg.getOutputExpressions(), agg.child());
+                            if (ordExists) {
+                                return new LogicalAggregate(groupByWithoutOrd, 
agg.getOutputExpressions(), agg.child());
+                            } else {
+                                return agg;
+                            }
+
                         }))).build();
     }
 
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rewrite/ExpressionRewrite.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rewrite/ExpressionRewrite.java
index dfa270992a..11167739b2 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rewrite/ExpressionRewrite.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rewrite/ExpressionRewrite.java
@@ -17,6 +17,7 @@
 
 package org.apache.doris.nereids.rules.expression.rewrite;
 
+import org.apache.doris.nereids.properties.OrderKey;
 import org.apache.doris.nereids.rules.Rule;
 import org.apache.doris.nereids.rules.RuleType;
 import org.apache.doris.nereids.rules.rewrite.OneRewriteRuleFactory;
@@ -35,6 +36,8 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 
+import java.util.ArrayList;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Objects;
 import java.util.Set;
@@ -66,7 +69,10 @@ public class ExpressionRewrite implements RewriteRuleFactory 
{
                 new ProjectExpressionRewrite().build(),
                 new AggExpressionRewrite().build(),
                 new FilterExpressionRewrite().build(),
-                new JoinExpressionRewrite().build());
+                new JoinExpressionRewrite().build(),
+                new SortExpressionRewrite().build(),
+                new LogicalRepeatRewrite().build(),
+                new HavingExpressionRewrite().build());
     }
 
     private class GenerateExpressionRewrite extends OneRewriteRuleFactory {
@@ -183,4 +189,48 @@ public class ExpressionRewrite implements 
RewriteRuleFactory {
             }).toRule(RuleType.REWRITE_JOIN_EXPRESSION);
         }
     }
+
+    private class SortExpressionRewrite extends OneRewriteRuleFactory {
+
+        @Override
+        public Rule build() {
+            return logicalSort().then(sort -> {
+                List<OrderKey> orderKeys = sort.getOrderKeys();
+                List<OrderKey> rewrittenOrderKeys = new ArrayList<>();
+                for (OrderKey k : orderKeys) {
+                    Expression expression = rewriter.rewrite(k.getExpr());
+                    rewrittenOrderKeys.add(new OrderKey(expression, k.isAsc(), 
k.isNullFirst()));
+                }
+                return sort.withOrderByKey(rewrittenOrderKeys);
+            }).toRule(RuleType.REWRITE_SORT_EXPRESSION);
+        }
+    }
+
+    private class HavingExpressionRewrite extends OneRewriteRuleFactory {
+        @Override
+        public Rule build() {
+            return logicalHaving().then(having -> {
+                Set<Expression> rewrittenExpr = new HashSet<>();
+                for (Expression e : having.getExpressions()) {
+                    rewrittenExpr.add(rewriter.rewrite(e));
+                }
+                return having.withExpressions(rewrittenExpr);
+            }).toRule(RuleType.REWRITE_HAVING_EXPRESSSION);
+        }
+    }
+
+    private class LogicalRepeatRewrite extends OneRewriteRuleFactory {
+        @Override
+        public Rule build() {
+            return logicalRepeat().then(r -> {
+                List<List<Expression>> groupingExprs = new ArrayList<>();
+                for (List<Expression> expressions : r.getGroupingSets()) {
+                    
groupingExprs.add(expressions.stream().map(rewriter::rewrite).collect(Collectors.toList()));
+                }
+                return r.withGroupSetsAndOutput(groupingExprs,
+                        
r.getOutputExpressions().stream().map(rewriter::rewrite).map(e -> 
(NamedExpression) e)
+                                .collect(Collectors.toList()));
+            }).toRule(RuleType.REWRITE_REPEAT_EXPRESSSION);
+        }
+    }
 }
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rewrite/rules/TypeCoercion.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rewrite/rules/TypeCoercion.java
index cee94f8af6..a81aefc986 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rewrite/rules/TypeCoercion.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rewrite/rules/TypeCoercion.java
@@ -29,6 +29,7 @@ import org.apache.doris.nereids.trees.expressions.Cast;
 import org.apache.doris.nereids.trees.expressions.Divide;
 import org.apache.doris.nereids.trees.expressions.Expression;
 import org.apache.doris.nereids.trees.expressions.InPredicate;
+import org.apache.doris.nereids.trees.expressions.IntegralDivide;
 import 
org.apache.doris.nereids.trees.expressions.typecoercion.ImplicitCastInputTypes;
 import org.apache.doris.nereids.types.BigIntType;
 import org.apache.doris.nereids.types.DataType;
@@ -218,4 +219,12 @@ public class TypeCoercion extends 
AbstractExpressionRewriteRule {
             }
         });
     }
+
+    @Override
+    public Expression visitIntegralDivide(IntegralDivide integralDivide, 
ExpressionRewriteContext context) {
+        DataType commonType = BigIntType.INSTANCE;
+        Expression newLeft = 
TypeCoercionUtils.castIfNotSameType(integralDivide.left(), commonType);
+        Expression newRight = 
TypeCoercionUtils.castIfNotSameType(integralDivide.right(), commonType);
+        return integralDivide.withChildren(newLeft, newRight);
+    }
 }
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/ExpressionEstimation.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/ExpressionEstimation.java
index 494dfe9371..c65df40433 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/ExpressionEstimation.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/ExpressionEstimation.java
@@ -25,6 +25,7 @@ import org.apache.doris.nereids.trees.expressions.CaseWhen;
 import org.apache.doris.nereids.trees.expressions.Cast;
 import org.apache.doris.nereids.trees.expressions.Divide;
 import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.IntegralDivide;
 import org.apache.doris.nereids.trees.expressions.Multiply;
 import org.apache.doris.nereids.trees.expressions.SlotReference;
 import org.apache.doris.nereids.trees.expressions.Subtract;
@@ -150,7 +151,7 @@ public class ExpressionEstimation extends 
ExpressionVisitor<ColumnStatistic, Sta
                     
.setNumNulls(numNulls).setDataSize(dataSize).setMinValue(min).setMaxValue(max).setSelectivity(1.0)
                     .setMaxExpr(null).setMinExpr(null).build();
         }
-        if (binaryArithmetic instanceof Divide) {
+        if (binaryArithmetic instanceof Divide || binaryArithmetic instanceof 
IntegralDivide) {
             double min = Math.min(
                     Math.min(
                             Math.min(leftMin / noneZeroDivisor(rightMin), 
leftMin / noneZeroDivisor(rightMax)),
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/IntegralDivide.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/IntegralDivide.java
new file mode 100644
index 0000000000..42d6a54935
--- /dev/null
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/IntegralDivide.java
@@ -0,0 +1,60 @@
+// 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.exceptions.UnboundException;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.coercion.AbstractDataType;
+import org.apache.doris.nereids.types.coercion.NumericType;
+
+import com.google.common.base.Preconditions;
+
+import java.util.List;
+
+/**
+ * A DIV B
+ */
+public class IntegralDivide extends BinaryArithmetic {
+
+    public IntegralDivide(Expression left, Expression right) {
+        super(left, right, Operator.INT_DIVIDE);
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitIntegralDivide(this, context);
+    }
+
+    @Override
+    public AbstractDataType inputType() {
+        return NumericType.INSTANCE;
+    }
+
+    // Divide is implemented as a scalar function which return type is always 
nullable.
+    @Override
+    public boolean nullable() throws UnboundException {
+        return true;
+    }
+
+    @Override
+    public Expression withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 2);
+        return new IntegralDivide(children.get(0), children.get(1));
+    }
+}
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 a2383d5c2c..5fc68638c7 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
@@ -46,6 +46,7 @@ import org.apache.doris.nereids.trees.expressions.GreaterThan;
 import org.apache.doris.nereids.trees.expressions.GreaterThanEqual;
 import org.apache.doris.nereids.trees.expressions.InPredicate;
 import org.apache.doris.nereids.trees.expressions.InSubquery;
+import org.apache.doris.nereids.trees.expressions.IntegralDivide;
 import org.apache.doris.nereids.trees.expressions.IsNull;
 import org.apache.doris.nereids.trees.expressions.LessThan;
 import org.apache.doris.nereids.trees.expressions.LessThanEqual;
@@ -391,6 +392,10 @@ public abstract class ExpressionVisitor<R, C>
         return visit(boundStar, context);
     }
 
+    public R visitIntegralDivide(IntegralDivide integralDivide, C context) {
+        return visitBinaryArithmetic(integralDivide, context);
+    }
+
     /* 
********************************************************************************************
      * Unbound expressions
      * 
********************************************************************************************/
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalHaving.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalHaving.java
index 3a0796b448..d6f0ae6d59 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalHaving.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalHaving.java
@@ -84,6 +84,11 @@ public class LogicalHaving<CHILD_TYPE extends Plan> extends 
LogicalUnary<CHILD_T
         return new LogicalHaving<>(conjuncts, Optional.empty(), 
logicalProperties, child());
     }
 
+    public Plan withExpressions(Set<Expression> expressions) {
+        return new LogicalHaving<Plan>(expressions, Optional.empty(),
+                Optional.of(getLogicalProperties()), child());
+    }
+
     @Override
     public List<Slot> computeOutput() {
         return child().getOutput();
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/TypeCoercionUtils.java 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/TypeCoercionUtils.java
index 85b9f5a666..ccb0796ae6 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/TypeCoercionUtils.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/TypeCoercionUtils.java
@@ -156,6 +156,14 @@ public class TypeCoercionUtils {
                 || leftType instanceof IntegralType && rightType instanceof 
DecimalV2Type) {
             return true;
         }
+        if (leftType instanceof FloatType && rightType instanceof DecimalV2Type
+                || leftType instanceof DecimalV2Type && rightType instanceof 
FloatType) {
+            return true;
+        }
+        if (leftType instanceof DoubleType && rightType instanceof 
DecimalV2Type
+                || leftType instanceof DecimalV2Type && rightType instanceof 
DoubleType) {
+            return true;
+        }
         // TODO: add decimal promotion support
         if (!(leftType instanceof DecimalV2Type)
                 && !(rightType instanceof DecimalV2Type)
@@ -230,7 +238,7 @@ public class TypeCoercionUtils {
                     || (right instanceof DateLikeType && left instanceof 
IntegralType)) {
             tightestCommonType = BigIntType.INSTANCE;
         }
-        return Optional.ofNullable(tightestCommonType);
+        return tightestCommonType == null ? Optional.of(DoubleType.INSTANCE) : 
Optional.of(tightestCommonType);
     }
 
     /**
diff --git 
a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FillUpMissingSlotsTest.java
 
b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FillUpMissingSlotsTest.java
index e325504c23..29d4185932 100644
--- 
a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FillUpMissingSlotsTest.java
+++ 
b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FillUpMissingSlotsTest.java
@@ -32,6 +32,7 @@ import 
org.apache.doris.nereids.trees.expressions.SlotReference;
 import org.apache.doris.nereids.trees.expressions.functions.agg.Count;
 import org.apache.doris.nereids.trees.expressions.functions.agg.Min;
 import org.apache.doris.nereids.trees.expressions.functions.agg.Sum;
+import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral;
 import org.apache.doris.nereids.trees.expressions.literal.Literal;
 import org.apache.doris.nereids.trees.expressions.literal.SmallIntLiteral;
 import org.apache.doris.nereids.trees.expressions.literal.TinyIntLiteral;
@@ -312,14 +313,16 @@ public class FillUpMissingSlotsTest extends 
AnalyzeCheckTestBase implements Patt
 
         ExceptionChecker.expectThrowsWithMsg(
                 AnalysisException.class,
-                "Aggregate functions in having clause can't be nested: sum((a1 
+ avg(a2))).",
+                "Aggregate functions in having clause can't be nested:"
+                        + " sum(cast((cast(a1 as DOUBLE) + avg(cast(a2 as 
DOUBLE))) as SMALLINT)).",
                 () -> PlanChecker.from(connectContext).analyze(
                         "SELECT a1 FROM t1 GROUP BY a1 HAVING SUM(a1 + 
AVG(a2)) > 0"
                 ));
 
         ExceptionChecker.expectThrowsWithMsg(
                 AnalysisException.class,
-                "Aggregate functions in having clause can't be nested: 
sum(((a1 + a2) + avg(a2))).",
+                "Aggregate functions in having clause can't be nested:"
+                        + " sum(cast((cast((a1 + a2) as DOUBLE) + avg(cast(a2 
as DOUBLE))) as INT)).",
                 () -> PlanChecker.from(connectContext).analyze(
                         "SELECT a1 FROM t1 GROUP BY a1 HAVING SUM(a1 + a2 + 
AVG(a2)) > 0"
                 ));
@@ -530,8 +533,8 @@ public class FillUpMissingSlotsTest extends 
AnalyzeCheckTestBase implements Patt
                                         ImmutableList.of(
                                                 new OrderKey(pk, true, true),
                                                 new 
OrderKey(countA11.toSlot(), true, true),
-                                                new OrderKey(new 
Add(sumA1A2.toSlot(), new TinyIntLiteral((byte) 1)), true, true),
-                                                new OrderKey(new 
Add(v1.toSlot(), new TinyIntLiteral((byte) 1)), true, true),
+                                                new OrderKey(new 
Add(sumA1A2.toSlot(), new BigIntLiteral((byte) 1)), true, true),
+                                                new OrderKey(new 
Add(v1.toSlot(), new BigIntLiteral((byte) 1)), true, true),
                                                 new OrderKey(v1.toSlot(), 
true, true)
                                         )
                                 ))
diff --git 
a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/TypeCoercionUtilsTest.java
 
b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/TypeCoercionUtilsTest.java
index cd40e28767..bbd75d48c2 100644
--- 
a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/TypeCoercionUtilsTest.java
+++ 
b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/TypeCoercionUtilsTest.java
@@ -145,7 +145,7 @@ public class TypeCoercionUtilsTest {
         testFindTightestCommonType(BigIntType.INSTANCE, IntegerType.INSTANCE, 
BigIntType.INSTANCE);
         testFindTightestCommonType(StringType.INSTANCE, StringType.INSTANCE, 
IntegerType.INSTANCE);
         testFindTightestCommonType(StringType.INSTANCE, IntegerType.INSTANCE, 
StringType.INSTANCE);
-        testFindTightestCommonType(null, DecimalV2Type.SYSTEM_DEFAULT, 
DecimalV2Type.createDecimalV2Type(2, 1));
+        testFindTightestCommonType(DoubleType.INSTANCE, 
DecimalV2Type.SYSTEM_DEFAULT, DecimalV2Type.createDecimalV2Type(2, 1));
         testFindTightestCommonType(VarcharType.createVarcharType(10), 
CharType.createCharType(8), CharType.createCharType(10));
         testFindTightestCommonType(VarcharType.createVarcharType(10), 
VarcharType.createVarcharType(8), VarcharType.createVarcharType(10));
         testFindTightestCommonType(VarcharType.createVarcharType(10), 
VarcharType.createVarcharType(8), CharType.createCharType(10));
diff --git 
a/regression-test/data/nereids_syntax_p0/test_arithmetic_operators.out 
b/regression-test/data/nereids_syntax_p0/test_arithmetic_operators.out
new file mode 100644
index 0000000000..fbe9d18024
--- /dev/null
+++ b/regression-test/data/nereids_syntax_p0/test_arithmetic_operators.out
@@ -0,0 +1,310 @@
+-- This file is automatically generated. You should know what you did if you 
want to edit this
+-- !arith_op1 --
+1      11011902        5536    11000   1
+2      5505951 5544    11000   1
+3      3670635 5536    10989   1
+
+-- !arith_op2 --
+1      2       123.123 12312300.000
+2      3       1243.500        124350000.000
+3      4       24453.325       2445332500.000
+
+-- !arith_op3 --
+1      123.123 244891.647
+2      1243.500        2469591.000
+3      24453.325       48637663.425
+
+-- !arith_op4 --
+1      123.123 12.3123 779.7379815444947       12596.337364196777      198.9
+2      1243.500        25203.258       981432.375      1567450.5       
40252.248
+3      24453.325       1.930467742125E9        8.935244955E7   7267806.0       
1.57021605E8
+
+-- !arith_op5 --
+1      12.312  0.010000000000000002    0.6333000183105469
+2      124.350 2.0268  78.92500000000001
+3      2445.332        7894.5  365.40000000000003
+
+-- !arith_op6 --
+1      -198.9  -100.1  -1101190.2      -12.312 -0.010000000000000002   
-0.6333000183105469
+2      -198.6  -100.1  -1101190.3      -124.350        -2.0268 
-78.92500000000001
+3      -198.9  -100.2  -1101190.5      -2445.332       -7894.5 
-365.40000000000003
+
+-- !arith_op7 --
+1      1.1356092352936706E19   9.2233720368547763E17   5.8411616798251155E19
+2      1.1469263127828914E20   1.869393044429726E20    7.279546380087632E21
+3      2.2554211401312182E21   7.281391054495003E23    3.370220142266735E22
+
+-- !arith_op8 --
+1      2.1564781210736683E-16  1.08528637465799E-16    1.1939128071597471E-12  
0E-9    1.0842021724855045E-20  6.866252556874047E-19
+2      2.1532255145562118E-16  1.08528637465799E-16    1.1939129155799644E-12  
0E-9    2.1974609631936205E-18  8.557065646341844E-17
+3      2.1564781210736683E-16  1.0863705768304754E-16  1.1939131324203989E-12  
0E-9    8.559234050686815E-15   3.961674738262033E-16
+
+-- !arith_op9 --
+1      9.2233720368547888E16   9.223372036854776E18    9.223372036854776E18
+2      9.2233720368549008E16   9.223372036854776E18    9.223372036854776E18
+3      9.2233720368572208E16   9.223372036854856E18    9.22337203685478E18
+
+-- !arith_op10 --
+1      -9.2233720368547632E16  -9.223372036854776E18   -9.223372036854776E18
+2      -9.2233720368546512E16  -9.223372036854776E18   -9.2233720368547748E18
+3      -9.2233720368523312E16  -9.223372036854697E18   -9.2233720368547717E18
+
+-- !arith_op11 --
+1      123123000.000000000     100000.00000000001      6333000.183105469
+2      1243500000.000000000    2.0268E7        7.8925E8
+3      24453325000.000000000   7.8945E10       3.654E9
+
+-- !arith_op12 --
+1      0.1     198.9   100.1   1101190.2       12.312  0.010000000000000002    
0.6333000183105469
+2      0.2     198.6   100.1   1101190.3       124.350 2.0268  
78.92500000000001
+3      0.3     198.9   100.2   1101190.5       2445.332        7894.5  
365.40000000000003
+
+-- !arith_op13 --
+1      0.1     198.9   100.1   1101190.2       12.312300000    0.01    
0.6333000183105468
+2      0.2     198.6   100.1   1101190.3       124.350000000   2.0268  78.925
+3      0.3     198.9   100.2   1101190.5       2445.332500000  7894.5  365.4
+
+-- !arith_op14 --
+1      0.9     1988.9  1000.9  11011901.9      123.023 0.0     
6.233000183105469
+2      1.9     1985.9  1000.9  11011902.9      1243.400        20.168  789.15
+3      2.9     1988.9  1001.9  11011904.9      24453.225       78944.9 3653.9
+
+-- !arith_op15 --
+1      1.1     1989.1  1001.1  11011902.1      123.223 0.2     
6.433000183105468
+2      2.1     1986.1  1001.1  11011903.1      1243.600        
20.368000000000002      789.35
+3      3.1     1989.1  1002.1  11011905.1      24453.425       78945.1 3654.1
+
+-- !arith_op16 --
+11     1999.0  1002.6  11011902        117.123 -234.56 6.333000183105469
+12     1996.0  1002.6  11011903        1237.500        -214.392        789.25
+13     1999.0  1003.6  11011905        24447.325       78710.34        3654.0
+
+-- !arith_op17 --
+
+-- !arith_op18 --
+245136538.647
+4944121182.000
+146204816255.550
+
+-- !arith_op19 --
+1.5524497441372922E8
+7.908872842616545E13
+4.217497670730171E19
+
+-- !arith_op20 --
+0.001433930695745209
+1.1353823333756908E-7
+9.444209807407214E-12
+
+-- !arith_op21 --
+1      2       0       1.0
+4      4       0       1.0
+9      6       0       1.0
+
+-- !arith_op21 --
+1989   1990    -1988   5.027652086475615E-4
+3972   1988    -1984   0.0010070493454179255
+5967   1992    -1986   0.0015082956259426848
+
+-- !arith_op21 --
+1001   1002    -1000   9.99000999000999E-4
+2002   1003    -999    0.001998001998001998
+3006   1005    -999    0.0029940119760479044
+
+-- !arith_op21 --
+123.123        124.123 -122.123        0.008121959
+2487.000       1245.500        -1241.500       0.001608363
+73359.975      24456.325       -24450.325      0.000122683
+
+-- !arith_op21 --
+0.1    1.1     0.9     10.0
+40.536 22.268  -18.268 0.09867771857114663
+236835.0       78948.0 -78942.0        3.800114003420103E-5
+
+-- !arith_op21 --
+6.333000183105469      7.333000183105469       -5.333000183105469      
0.1579030429633806
+1578.5 791.25  -787.25 0.0025340513145391194
+10962.0        3657.0  -3651.0 8.210180623973727E-4
+
+-- !arith_op21 --
+1989   1990    1988    1989.0
+3972   1988    1984    993.0
+5967   1992    1986    663.0
+
+-- !arith_op21 --
+3956121        3978    0       1.0
+3944196        3972    0       1.0
+3956121        3978    0       1.0
+
+-- !arith_op21 --
+1990989        2990    988     1.9870129870129871
+1987986        2987    985     1.9840159840159841
+1992978        2991    987     1.9850299401197604
+
+-- !arith_op21 --
+244891.647     2112.123        1865.877        16.154577130
+2469591.000    3229.500        742.500 1.597104946
+48637663.425   26442.325       -22464.325      0.081338632
+
+-- !arith_op21 --
+198.9  1989.1  1988.9  19890.0
+40252.248      2006.268        1965.732        97.98697454114861
+1.57021605E8   80934.0 -76956.0        0.02519475584267528
+
+-- !arith_op21 --
+12596.337364196777     1995.3330001831055      1982.6669998168945      
314.069152454164
+1567450.5      2775.25 1196.75 2.516312955337346
+7267806.0      5643.0  -1665.0 0.5443349753694581
+
+-- !arith_op21 --
+1001   1002    1000    1001.0
+2002   1003    999     500.5
+3006   1005    999     334.0
+
+-- !arith_op21 --
+1990989        2990    -988    0.5032679738562091
+1987986        2987    -985    0.5040281973816717
+1992978        2991    -987    0.5037707390648567
+
+-- !arith_op21 --
+1002001        2002    0       1.0
+1002001        2002    0       1.0
+1004004        2004    0       1.0
+
+-- !arith_op21 --
+123246.123     1124.123        877.877 8.130081301
+1244743.500    2244.500        -242.500        0.804985927
+24502231.650   25455.325       -23451.325      0.040976023
+
+-- !arith_op21 --
+100.10000000000001     1001.1  1000.9  10010.0
+20288.268      1021.268        980.732 49.38819814485889
+7.910289E7     79947.0 -77943.0        0.012692380771423142
+
+-- !arith_op21 --
+6339.333183288574      1007.3330001831055      994.6669998168945       
158.06094600634398
+790039.25      1790.25 211.75  1.2682926829268293
+3661308.0      4656.0  -2652.0 0.2742200328407225
+
+-- !arith_op21 --
+123.123        124.123 122.123 123.123000000
+2487.000       1245.500        1241.500        621.750000000
+73359.975      24456.325       24450.325       8151.108333333
+
+-- !arith_op21 --
+244891.647     2112.123        -1865.877       0.061901961
+2469591.000    3229.500        -742.500        0.626132931
+48637663.425   26442.325       22464.325       12.294281046
+
+-- !arith_op21 --
+123246.123     1124.123        -877.877        0.123000000
+1244743.500    2244.500        242.500 1.242257742
+24502231.650   25455.325       23451.325       24.404515968
+
+-- !arith_op21 --
+15159.273      246.246 0.000   1.000000000
+1546292.250    2487.000        0.000   1.000000000
+597965103.555  48906.650       0.000   1.000000000
+
+-- !arith_op21 --
+12.3123        123.223 123.02300000000001      1231.23
+25203.258      1263.768        1223.232        61.35287152161042
+1.930467742125E9       103398.325      -54491.675      0.30975140920894295
+
+-- !arith_op21 --
+779.7379815444947      129.45600018310546      116.78999981689454      
19.44149635878031
+981432.375     2032.75 454.25  1.5755464048146974
+8.935244955E7  28107.325       20799.325       6.692207170224412
+
+-- !arith_op21 --
+0.1    1.1     -0.9    0.1
+40.536 22.268  18.268  10.134
+236835.0       78948.0 78942.0 26315.0
+
+-- !arith_op21 --
+198.9  1989.1  -1988.9 5.027652086475616E-5
+40252.248      2006.268        -1965.732       0.010205438066465258
+1.57021605E8   80934.0 76956.0 39.69079939668175
+
+-- !arith_op21 --
+100.10000000000001     1001.1  -1000.9 9.99000999000999E-5
+20288.268      1021.268        -980.732        0.02024775224775225
+7.910289E7     79947.0 77943.0 78.7874251497006
+
+-- !arith_op21 --
+12.3123        123.223 -123.02300000000001     8.121959341471537E-4
+25203.258      1263.768        -1223.232       0.016299155609167674
+1.930467742125E9       103398.325      54491.675       3.228395320472778
+
+-- !arith_op21 --
+0.010000000000000002   0.2     0.0     1.0
+410.791824     40.536  0.0     1.0
+6.232313025E9  157890.0        0.0     1.0
+
+-- !arith_op21 --
+0.6333000183105469     6.433000183105468       -6.233000183105469      
0.01579030429633806
+15996.519      809.518 -768.982        0.025680076021539436
+2.8846503E8    82599.0 75291.0 21.605090311986864
+
+-- !arith_op21 --
+6.333000183105469      7.333000183105469       5.333000183105469       
6.333000183105469
+1578.5 791.25  787.25  394.625
+10962.0        3657.0  3651.0  1218.0
+
+-- !arith_op21 --
+12596.337364196777     1995.3330001831055      -1982.6669998168945     
0.003184012158424067
+1567450.5      2775.25 -1196.75        0.39740684793554887
+7267806.0      5643.0  1665.0  1.83710407239819
+
+-- !arith_op21 --
+6339.333183288574      1007.3330001831055      -994.6669998168945      
0.006326673509595873
+790039.25      1790.25 -211.75 0.7884615384615384
+3661308.0      4656.0  2652.0  3.6467065868263475
+
+-- !arith_op21 --
+779.7379815444947      129.45600018310546      -116.78999981689454     
0.051436369996714415
+981432.375     2032.75 -454.25 0.6347004422999598
+8.935244955E7  28107.325       -20799.325      0.14942753183871724
+
+-- !arith_op21 --
+0.6333000183105469     6.433000183105468       6.233000183105469       
63.33000183105469
+15996.519      809.518 768.982 38.94069469113874
+2.8846503E8    82599.0 -75291.0        0.04628538856165685
+
+-- !arith_op21 --
+40.1068913192139       12.666000366210938      0.0     1.0
+622915.5625    1578.5  0.0     1.0
+1.3351716E7    7308.0  0.0     1.0
+
+-- !arith_op22 --
+1987.2
+1990.2
+1990.2
+
+-- !arith_op23 --
+124.3
+1244.7
+24454.5
+
+-- !arith_op24 --
+2.3
+
+-- !arith_op25 --
+\N     \N      \N      \N
+
+-- !arith_op26 --
+\N     \N      \N      \N
+
+-- !arith_op27 --
+\N     \N      \N      \N
+
+-- !arith_op28 --
+\N     \N      \N
+\N     \N      \N
+\N     \N      \N
+
+-- !arith_op29 --
+\N     \N      \N
+\N     \N      \N
+\N     \N      \N
+
diff --git a/regression-test/data/nereids_syntax_p0/test_query_between.out 
b/regression-test/data/nereids_syntax_p0/test_query_between.out
new file mode 100644
index 0000000000..c97f6621c7
--- /dev/null
+++ b/regression-test/data/nereids_syntax_p0/test_query_between.out
@@ -0,0 +1,41 @@
+-- This file is automatically generated. You should know what you did if you 
want to edit this
+-- !between1 --
+0
+2
+2
+
+-- !between2 --
+3
+
+-- !between3 --
+1989
+1986
+1989
+
+-- !between4 --
+1001
+1001
+1002
+
+-- !between5 --
+
+-- !between6 --
+false
+false
+
+-- !between7 --
+
+-- !between8 --
+
+-- !between9 --
+
+-- !between10 --
+
+-- !between11 --
+
+-- !between12 --
+
+-- !between13 --
+123.123
+1243.500
+
diff --git a/regression-test/data/nereids_syntax_p0/unary_binary_arithmetic.out 
b/regression-test/data/nereids_syntax_p0/unary_binary_arithmetic.out
index f403badf7b..78e08d7730 100644
--- a/regression-test/data/nereids_syntax_p0/unary_binary_arithmetic.out
+++ b/regression-test/data/nereids_syntax_p0/unary_binary_arithmetic.out
@@ -1,6 +1,6 @@
 -- This file is automatically generated. You should know what you did if you 
want to edit this
 -- !select --
-4.0
+4
 
 -- !select --
 \N
diff --git 
a/regression-test/suites/nereids_syntax_p0/test_arithmetic_operators.groovy 
b/regression-test/suites/nereids_syntax_p0/test_arithmetic_operators.groovy
new file mode 100644
index 0000000000..e7fe4a8c4a
--- /dev/null
+++ b/regression-test/suites/nereids_syntax_p0/test_arithmetic_operators.groovy
@@ -0,0 +1,80 @@
+// 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("nereids_test_arithmetic_operators", "query,p0") {
+    def tableName = "test"
+
+    sql "use test_query_db"
+    sql "SET enable_vectorized_engine=true"
+    sql "SET enable_nereids_planner=true"
+    sql "SET enable_fallback_to_original_planner=false"
+
+    qt_arith_op1 "select k1, k4 div k1, k4 div k2, k4 div k3, k4 div k4 \
+                   from ${tableName} order by k1, k2, k3, k4"
+    qt_arith_op2 "select k1, k1+ '1', k5,100000*k5 from ${tableName} order by 
k1, k2, k3, k4"
+    qt_arith_op3 "select k1,k5,k2*k5 from ${tableName} order by k1, k2, k3, k4"
+    qt_arith_op4 "select k1,k5,k8*k5,k5*k9,k2*k9,k2*k8 from ${tableName} order 
by k1, k2, k3, k4"
+    qt_arith_op5 "select k1, k5*0.1, k8*0.1, k9*0.1 from ${tableName} order by 
k1, k2, k3, k4"
+    qt_arith_op6 "select k1, k2*(-0.1), k3*(-0.1), k4*(-0.1), \
+                   k5*(-0.1), k8*(-0.1), k9*(-0.1) from  ${tableName} order by 
k1, k2, k3, k4"
+    qt_arith_op7 "select k1, k5*(9223372036854775807/100), 
k8*9223372036854775807, \
+                   k9*9223372036854775807 from  ${tableName} order by k1, k2, 
k3, k4"
+    qt_arith_op8 "select k1, k2/9223372036854775807, k3/9223372036854775807, \
+                   k4/9223372036854775807,k5/9223372036854775807, \
+                   k8/9223372036854775807,k9/9223372036854775807 \
+                   from  ${tableName} order by k1, k2, k3, k4"
+    qt_arith_op9 "select k1, k5+9223372036854775807/100, 
k8+9223372036854775807, \
+                   k9+9223372036854775807 from  ${tableName} order by k1, k2, 
k3, k4"
+    qt_arith_op10 "select k1, k5-9223372036854775807/100, 
k8-9223372036854775807, \
+                   k9-9223372036854775807 from  ${tableName} order by k1, k2, 
k3, k4"
+    qt_arith_op11 "select k1, k5/0.000001, k8/0.000001, \
+                   k9/0.000001 from  ${tableName} order by k1, k2, k3, k4"
+    qt_arith_op12 "select k1, k1*0.1, k2*0.1, k3*0.1, k4*0.1, k5*0.1, k8*0.1, 
k9*0.1 \
+                   from ${tableName} order by k1, k2, k3, k4"
+    qt_arith_op13 "select k1, k1/10, k2/10, k3/10, k4/10, k5/10, k8/10, k9/10 \
+                   from ${tableName} order by k1, k2, k3, k4"
+    qt_arith_op14 "select k1, k1-0.1, k2-0.1, k3-0.1, k4-0.1, k5-0.1, k8-0.1, 
k9-0.1 \
+                   from ${tableName} order by k1, k2, k3, k4"
+    qt_arith_op15 "select k1, k1+0.1, k2+0.1, k3+0.1, k4+0.1, k5+0.1, k8+0.1, 
k9+0.1 \
+                   from ${tableName} order by k1, k2, k3, k4"
+    qt_arith_op16 "select k1+10, k2+10.0, k3+1.6, k4*1, k5-6, k8-234.66, k9-0 \
+                   from ${tableName} order by k1, k2, k3, k4"
+    qt_arith_op17 "select * from ${tableName} where k1+k9<0 order by k1, k2, 
k3, k4"
+    qt_arith_op18 "select k1*k2*k3*k5 from ${tableName} order by k1, k2, k3, 
k4"
+    qt_arith_op19 "select k1*k2*k3*k5*k8*k9 from ${tableName} order by k1, k2, 
k3, k4"
+    qt_arith_op20 "select k1*10000/k4/k8/k9 from ${tableName} order by k1, k2, 
k3, k4"
+    
+    for( i in [1, 2, 3, 5, 8, 9]) {
+        for( j in [1, 2, 3, 5, 8, 9]) {
+            qt_arith_op21 "select k${i}*k${j}, k${i}+k${j}, k${i}-k${j}, 
k${i}/k${j} from ${tableName} \
+                           where abs(k${i})<9223372036854775807 and k${j}<>0 
and\
+                           abs(k${i})<922337203685477580 order by k1, k2, k3, 
k4"
+        }
+    }
+    
+    qt_arith_op22 "select 1.1*1.1 + k2 from ${tableName} order by 1 limit 10"
+    qt_arith_op23 "select 1.1*1.1 + k5 from ${tableName} order by 1 limit 10"
+    qt_arith_op24 "select 1.1*1.1+1.1"
+
+    // divide mod zero
+    qt_arith_op25 "select 10.2 / 0.0, 10.2 / 0, 10.2 % 0.0, 10.2 % 0"
+    qt_arith_op26 "select 0.0 / 0.0, 0.0 / 0, 0.0 % 0.0, 0.0 % 0"
+    qt_arith_op27 "select -10.2 / 0.0, -10.2 / 0, -10.2 % 0.0, -10.2 % 0"
+    qt_arith_op28 "select k5 / 0, k8 / 0, k9 / 0 from ${tableName} order by 
k1,k2,k3,k4"
+    qt_arith_op29 "select k5 % 0, k8 % 0, k9 % 0 from ${tableName} order by 
k1,k2,k3,k4"
+}
diff --git a/regression-test/suites/nereids_syntax_p0/test_query_between.groovy 
b/regression-test/suites/nereids_syntax_p0/test_query_between.groovy
new file mode 100644
index 0000000000..d3b4b18306
--- /dev/null
+++ b/regression-test/suites/nereids_syntax_p0/test_query_between.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("nereids_test_query_between", "query,p0") {
+    sql"use test_query_db"
+
+    sql "SET enable_vectorized_engine=true"
+    sql "SET enable_nereids_planner=true"
+    sql "SET enable_fallback_to_original_planner=false"
+
+    def tableName = "test"
+    qt_between1 "select if(k1 between 1 and 2, 2, 0) as wj from ${tableName} 
order by wj"
+    qt_between2 "select k1 from ${tableName} where k1 between 3 and 4 order by 
k1, k2, k3, k4"
+    qt_between3 "select k2 from ${tableName} where k2 between 1980 and 1990 
order by k1, k2, k3, k4"
+    qt_between4 "select k3 from ${tableName} where k3 between 1000 and 2000 
order by k1, k2, k3, k4"
+    qt_between5 "select k4 from ${tableName} where k4 between -100000000 and 0 
order by k1, k2, k3, k4"
+    qt_between6 "select k6 from ${tableName} where lower(k6) between 'f' and 
'false' order by k1, k2, k3, k4"
+    qt_between7 "select k7 from ${tableName} where lower(k7) between 'a' and 
'g' order by k1, k2, k3, k4"
+    qt_between8 "select k8 from ${tableName} where k8 between -2 and 0 order 
by k1, k2, k3, k4"
+    qt_between9 """select k10 from ${tableName} where k10 between \"2015-04-02 
00:00:00\"
+                and \"9999-12-31 12:12:12\" order by k1, k2, k3, k4"""
+    qt_between10 """select k11 from ${tableName} where k11 between 
\"2015-04-02 00:00:00\"
+                and \"9999-12-31 12:12:12\" order by k1, k2, k3, k4"""
+    qt_between11 """select k10 from ${tableName} where k10 between 
\"2015-04-02\"
+                and \"9999-12-31\" order by k1, k2, k3, k4"""
+    qt_between12 "select k9 from ${tableName} where k9 between -1 and 6.333 
order by k1, k2, k3, k4"
+    qt_between13 "select k5 from ${tableName} where k5 between 0 and 1243.5 
order by k1, k2, k3, k4"
+}


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


Reply via email to