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