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 2023/01/10 18:55:53 UTC

[doris] branch master updated: [Fix](Nereids) fix type coercion for binary arithmetic (#15185)

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 bc34a44f06 [Fix](Nereids) fix type coercion for binary arithmetic (#15185)
bc34a44f06 is described below

commit bc34a44f065a6f80fe378094c0e560d1c4951f82
Author: mch_ucchi <41...@users.noreply.github.com>
AuthorDate: Wed Jan 11 02:55:44 2023 +0800

    [Fix](Nereids) fix type coercion for binary arithmetic (#15185)
    
    support sql like: select true + 1 + '2.0' and prevent select true + 1 + 'x';
---
 .../expression/rewrite/rules/TypeCoercion.java     |  20 +-
 .../doris/nereids/util/TypeCoercionUtils.java      |   6 +-
 .../rules/expression/rewrite/TypeCoercionTest.java |  10 +
 regression-test/data/nereids_syntax_p0/cast.out    | 268 +++++++++++++++++++++
 .../suites/nereids_syntax_p0/cast.groovy           | 196 +++++++++++++++
 5 files changed, 492 insertions(+), 8 deletions(-)

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 fae6a7e4bd..2cf38efb1b 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
@@ -32,6 +32,7 @@ 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.literal.StringLikeLiteral;
 import org.apache.doris.nereids.trees.expressions.typecoercion.ImplicitCastInputTypes;
 import org.apache.doris.nereids.types.BigIntType;
 import org.apache.doris.nereids.types.DataType;
@@ -46,6 +47,7 @@ import org.apache.doris.nereids.util.TypeCoercionUtils;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableList.Builder;
 
+import java.math.BigDecimal;
 import java.util.List;
 import java.util.Optional;
 import java.util.stream.Collectors;
@@ -84,6 +86,15 @@ public class TypeCoercion extends AbstractExpressionRewriteRule {
         if (binaryOperator instanceof ImplicitCastInputTypes) {
             List<AbstractDataType> expectedInputTypes = ((ImplicitCastInputTypes) binaryOperator).expectedInputTypes();
             if (!expectedInputTypes.isEmpty()) {
+                binaryOperator.children().stream().filter(e -> e instanceof StringLikeLiteral)
+                        .forEach(expr -> {
+                            try {
+                                new BigDecimal(((StringLikeLiteral) expr).getStringValue());
+                            } catch (NumberFormatException e) {
+                                throw new IllegalStateException(String.format(
+                                        "string literal %s cannot be cast to double", expr.toSql()));
+                            }
+                        });
                 binaryOperator = (BinaryOperator) visitImplicitCastInputTypes(binaryOperator, expectedInputTypes,
                         context);
             }
@@ -111,13 +122,14 @@ public class TypeCoercion extends AbstractExpressionRewriteRule {
     public Expression visitDivide(Divide divide, ExpressionRewriteContext context) {
         Expression left = rewrite(divide.left(), context);
         Expression right = rewrite(divide.right(), context);
+
         DataType t1 = TypeCoercionUtils.getNumResultType(left.getDataType());
         DataType t2 = TypeCoercionUtils.getNumResultType(right.getDataType());
         DataType commonType = TypeCoercionUtils.findCommonNumericsType(t1, t2);
-        if (divide.getLegacyOperator() == Operator.DIVIDE) {
-            if (commonType.isBigIntType() || commonType.isLargeIntType()) {
-                commonType = DoubleType.INSTANCE;
-            }
+
+        if (divide.getLegacyOperator() == Operator.DIVIDE
+                && (commonType.isBigIntType() || commonType.isLargeIntType())) {
+            commonType = DoubleType.INSTANCE;
         }
         Expression newLeft = TypeCoercionUtils.castIfNotSameType(left, commonType);
         Expression newRight = TypeCoercionUtils.castIfNotSameType(right, commonType);
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 405f98d153..11584d74d7 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
@@ -148,10 +148,8 @@ public class TypeCoercionUtils {
      * return ture if two type could do type coercion.
      */
     public static boolean canHandleTypeCoercion(DataType leftType, DataType rightType) {
-        if (leftType instanceof DecimalV2Type && rightType instanceof NullType) {
-            return true;
-        }
-        if (leftType instanceof NullType && rightType instanceof DecimalV2Type) {
+        if (leftType instanceof DecimalV2Type && rightType instanceof NullType
+                || leftType instanceof NullType && rightType instanceof DecimalV2Type) {
             return true;
         }
         if (leftType instanceof DecimalV2Type && rightType instanceof IntegralType
diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/rewrite/TypeCoercionTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/rewrite/TypeCoercionTest.java
index 824b6d16d8..15ba413fd9 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/rewrite/TypeCoercionTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/rewrite/TypeCoercionTest.java
@@ -19,6 +19,7 @@ package org.apache.doris.nereids.rules.expression.rewrite;
 
 import org.apache.doris.nereids.exceptions.AnalysisException;
 import org.apache.doris.nereids.rules.expression.rewrite.rules.TypeCoercion;
+import org.apache.doris.nereids.trees.expressions.Add;
 import org.apache.doris.nereids.trees.expressions.CaseWhen;
 import org.apache.doris.nereids.trees.expressions.Cast;
 import org.apache.doris.nereids.trees.expressions.Divide;
@@ -186,6 +187,15 @@ public class TypeCoercionTest extends ExpressionRewriteTestHelper {
         Expression expected = new Divide(new Cast(Literal.of((short) 1), DoubleType.INSTANCE),
                 new Cast(Literal.of(10L), DoubleType.INSTANCE));
         assertRewrite(actual, expected);
+
+        Expression actual1 = new Add(new IntegerLiteral(1), new Add(BooleanLiteral.TRUE, new StringLiteral("2")));
+        Expression expected1 = new Add(new Cast(new IntegerLiteral(1), DoubleType.INSTANCE), new Add(
+                new Cast(BooleanLiteral.TRUE, DoubleType.INSTANCE),
+                new Cast(new StringLiteral("2"), DoubleType.INSTANCE)));
+        assertRewrite(actual1, expected1);
+
+        Expression actual2 = new Add(new IntegerLiteral(1), new Add(BooleanLiteral.TRUE, new StringLiteral("x")));
+        Assertions.assertThrows(IllegalStateException.class, () -> assertRewrite(actual2, null));
     }
 
     private DataType checkAndGetDataType(Expression expression) {
diff --git a/regression-test/data/nereids_syntax_p0/cast.out b/regression-test/data/nereids_syntax_p0/cast.out
new file mode 100644
index 0000000000..b52dc5ac96
--- /dev/null
+++ b/regression-test/data/nereids_syntax_p0/cast.out
@@ -0,0 +1,268 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !orderBy1 --
+1	1989-03-21
+2	1901-12-31
+3	2012-03-14
+
+-- !orderBy2 --
+1	0.1
+2	20.268
+3	78945.0
+
+-- !orderBy3 --
+11011902	1989-03-21
+11011903	1901-12-31
+11011905	2012-03-14
+
+-- !orderBy4 --
+
+-- !group1 --
+123.123000000
+
+-- !group2 --
+24453.325000000
+
+-- !group3 --
+8606.649333333
+
+-- !group4 --
+25819.948000000
+
+-- !group5 --
+3
+
+-- !group6 --
+123.123000000
+1243.500000000
+
+-- !group7 --
+123.123000000
+1243.500000000
+24453.325000000
+
+-- !group8 --
+123.123000000
+1243.500000000
+24453.325000000
+
+-- !group9 --
+123.123000000
+1243.500000000
+24453.325000000
+
+-- !group10 --
+1
+1
+1
+
+-- !group11 --
+true	0.1	0.1	1	0.1	0.1
+false	39482.634	78965.268	2	20.268	78945.0
+
+-- !group12 --
+1986	20.268
+1989	39472.55
+
+-- !group13 --
+1986	20.268
+1989	78945.1
+
+-- !group14 --
+1986	1
+1989	2
+
+-- !group15 --
+1986	20.268
+1989	0.1
+
+-- !group16 --
+1986	20.268
+1989	78945.0
+
+-- !group17 --
+true	0.1
+
+-- !group18 --
+true	0.1
+
+-- !group19 --
+true	1
+
+-- !group20 --
+true	0.1
+
+-- !group21 --
+true	0.1
+
+-- !group22 --
+1986	20.268
+1989	39472.55
+
+-- !group23 --
+1986	20.268
+1989	78945.1
+
+-- !group24 --
+1986	1
+1989	2
+
+-- !group25 --
+1986	20.268
+1989	0.1
+
+-- !group26 --
+1986	20.268
+1989	78945.0
+
+-- !group27 --
+1
+1
+1
+
+-- !group28 --
+123.123	246.246000000	1
+1243.500	2487.000000000	1
+24453.325	48906.650000000	1
+
+-- !group29 --
+0	0	1
+1	0	1
+2	0	1
+
+-- !group30 --
+1	1	0	0	2000-01-01T00:00	1
+
+-- !group31 --
+1
+
+-- !group32 --
+0	\N	\N	\N	\N
+
+-- !group33 --
+3	1	3.0
+
+-- !group34 --
+1901-12-31	1
+1989-03-21	1
+
+-- !group35 --
+1	2
+4	4
+
+-- !group36 --
+1
+
+-- !group37 --
+3	11928
+
+-- !group38 --
+
+-- !group39 --
+1901	12	2
+1989	3	1
+2012	3	3
+
+-- !group40 --
+
+-- !group41 --
+123.123000000
+1243.500000000
+24453.325000000
+
+-- !group42 --
+
+-- !group43 --
+78945.0
+
+-- !group44 --
+6.333000183105469
+
+-- !order8 --
+1	1989
+2	1986
+3	1989
+
+-- !order9 --
+1	1989
+2	1986
+3	1989
+
+-- !order10 --
+1	1
+2	1
+3	1
+
+-- !order11 --
+
+-- !order12 --
+\N
+10
+6
+12
+3
+
+-- !order13 --
+6
+10
+12
+
+-- !order14 --
+6
+10
+12
+
+-- !order15 --
+
+-- !order16 --
+
+-- !orderBy_withNull_1 --
+11012025.123000000
+11013146.500000000
+11036358.325000000
+
+-- !orderBy_withNull_2 --
+123.123	\N
+1243.500	\N
+24453.325	\N
+
+-- !orderBy_withNull_3 --
+\N	\N
+\N	3
+\N	6
+\N	10
+\N	12
+
+-- !orderBy_withNull_4 --
+11012025.123000000	1
+11013146.500000000	2
+11036358.325000000	3
+
+-- !orderBy_withNull_5 --
+\N
+
+-- !orderBy_withNull_6 --
+\N	3
+
+-- !orderBy_withNull_7 --
+\N	3
+
+-- !orderBy_withNull_8 --
+11012025.123000000	\N
+11013146.500000000	\N
+11036358.325000000	\N
+
+-- !orderBy_withNull_9 --
+11012025.123000000
+11013146.500000000
+11036358.325000000
+
+-- !orderBy_withNull_10 --
+\N	\N
+\N	3
+\N	6
+\N	10
+\N	12
+
+-- !group31 --
+1
+
diff --git a/regression-test/suites/nereids_syntax_p0/cast.groovy b/regression-test/suites/nereids_syntax_p0/cast.groovy
new file mode 100644
index 0000000000..1f035256f4
--- /dev/null
+++ b/regression-test/suites/nereids_syntax_p0/cast.groovy
@@ -0,0 +1,196 @@
+// 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("cast") {
+    def tableName1 ="test"
+    def tableName2 ="baseall"
+
+    sql 'set enable_nereids_planner=true'
+    sql 'set enable_fallback_to_original_planner=false'
+
+    sql """
+    drop table if exists test
+    """
+
+    sql """
+    drop table if exists baseall
+    """
+
+    sql """
+    CREATE TABLE IF NOT EXISTS `test` (                                                      
+        `k0` boolean NULL,                                                        
+        `k1` tinyint(4) NULL,                                                     
+        `k2` smallint(6) NULL,                                                   
+        `k3` int(11) NULL,                                                        
+        `k4` bigint(20) NULL,                                                     
+        `k5` decimal(9, 3) NULL,                                                  
+        `k6` char(5) NULL,                                                        
+        `k10` date NULL,                                                          
+        `k11` datetime NULL,                                                      
+        `k7` varchar(20) NULL,                                                    
+        `k8` double MAX NULL,                                                     
+        `k9` float SUM NULL,                                                      
+        `k12` text REPLACE_IF_NOT_NULL NULL,                                      
+        `k13` largeint(40) REPLACE NULL                                           
+    ) ENGINE=OLAP                                                               
+    AGGREGATE KEY(`k0`, `k1`, `k2`, `k3`, `k4`, `k5`, `k6`, `k10`, `k11`, `k7`) 
+    COMMENT 'OLAP'                                                              
+    DISTRIBUTED BY HASH(`k1`) BUCKETS 5                                         
+    PROPERTIES (                                                                
+        "replication_allocation" = "tag.location.default: 1",                       
+        "in_memory" = "false",                                                      
+        "storage_format" = "V2",                                                    
+        "disable_auto_compaction" = "false"                                         
+    );
+    """
+
+    sql """
+    CREATE TABLE IF NOT EXISTS `baseall` (                                                    
+        `k0` boolean NULL,                                                        
+        `k1` tinyint(4) NULL,                                                     
+        `k2` smallint(6) NULL,                                                    
+        `k3` int(11) NULL,                                                        
+        `k4` bigint(20) NULL,                                                     
+        `k5` decimal(9, 3) NULL,                                                  
+        `k6` char(5) NULL,                                                        
+        `k10` date NULL,                                                          
+        `k11` datetime NULL,                                                      
+        `k7` varchar(20) NULL,                                                    
+        `k8` double MAX NULL,                                                     
+        `k9` float SUM NULL,                                                      
+        `k12` text REPLACE NULL,                                                  
+        `k13` largeint(40) REPLACE NULL                                           
+        ) ENGINE=OLAP                                                               
+        AGGREGATE KEY(`k0`, `k1`, `k2`, `k3`, `k4`, `k5`, `k6`, `k10`, `k11`, `k7`) 
+        COMMENT 'OLAP'                                                              
+        DISTRIBUTED BY HASH(`k1`) BUCKETS 5                                         
+        PROPERTIES (                                                                
+            "replication_allocation" = "tag.location.default: 1",                       
+            "in_memory" = "false",                                                      
+            "storage_format" = "V2",                                                    
+            "disable_auto_compaction" = "false"                                         
+        );
+    """
+
+    sql """
+        insert into test values
+        (0, 1, 1989, 1001, 11011902, 123.123, 'true', '1989-03-21', '1989-03-21 13:00:00', 'wangjuoo4', 0.1, 6.333, 'string12345', 170141183460469231731687303715884105727),
+        (0, 2, 1986, 1001, 11011903, 1243.500, 'false', '1901-12-31', '1989-03-21 13:00:00', 'wangynnsf', 20.268, 789.25, 'string12345', -170141183460469231731687303715884105727),
+        (0, 3, 1989, 1002, 11011905, 24453.325, 'false', '2012-03-14', '2000-01-01 00:00:00', 'yunlj8@nk', 78945.0, 3654.0, 'string12345', 0);
+    """
+
+    sql """
+        insert into baseall values 
+        (1, 10, 1991, 5014, 9223372036854775807, -258.369, 'false', '2015-04-02', '2013-04-02 15:16:52', 'wangynnsf', -123456.54, 0.235, 'string12345', -11011903),
+        (1, 12, 32767, -2147483647, 9223372036854775807, 243.325, 'false', '1991-08-11', '2013-04-02 15:16:52', 'lifsno', -564.898, 3.1415927, 'string12345', 1701604692317316873037158),  
+        (0, 6, 32767, 3021, 123456, 604587.000, 'true', '2014-11-11', '2015-03-13 12:36:38', 'yanavnd', 0.1, 80699.0, 'string12345', 20220104),    
+        (null, null, null, null, null, null, null, null, null, null, null, null, null, null),
+        (0, 3, 1989, 1002, 11011905, 24453.325, 'false', '2012-03-14', '2000-01-01 00:00:00', 'yunlj8@nk', 78945.0, 3654.0, 'string12345', 0);
+    """
+
+    // order by
+    qt_orderBy1 "select k1, k10 from test order by 1, 2 limit 1000"
+    qt_orderBy2 "select k1, k8 from test order by 1, 2 desc limit 1000"
+    qt_orderBy3 "select k4, k10 from (select k4, k10 from test order by 1, 2 limit 1000000) as i order by 1, 2 limit 1000"
+    qt_orderBy4 "select * from test where k1<-1000 order by k1"
+
+    // group
+    qt_group1 "select min(k5) from test"
+    qt_group2 "select max(k5) from test"
+    qt_group3 "select avg(k5) from test"
+    qt_group4 "select sum(k5) from test"
+    qt_group5 "select count(k5) from test"
+    qt_group6 "select min(k5) from test group by k2 order by min(k5)"
+    qt_group7 "select max(k5) from test group by k1 order by max(k5)"
+    qt_group8 "select avg(k5) from test group by k1 order by avg(k5)"
+    qt_group9 "select sum(k5) from test group by k1 order by sum(k5)"
+    qt_group10 "select count(k5) from test group by k1 order by count(k5)"
+    qt_group11 "select lower(k6), avg(k8), sum(k8),count(k8),  min(k8), max(k8) from test group by lower(k6) order by avg(k8), sum(k8),count(k8),  min(k8), max(k8)" 
+    qt_group12 "select k2, avg(k8) from test group by k2 order by k2, avg(k8)" 
+    qt_group13 "select k2, sum(k8) from test group by k2 order by k2, sum(k8)" 
+    qt_group14 "select k2, count(k8) from test group by k2 order by k2, count(k8)" 
+    qt_group15 "select k2, min(k8) from test group by k2 order by k2, min(k8)" 
+    qt_group16 "select k2, max(k8) from test group by k2 order by k2, max(k8)" 
+    qt_group17 "select k6, avg(k8) from test group by k6 having k6='true' order by k6, avg(k8)" 
+    qt_group18 "select k6, sum(k8) from test group by k6 having k6='true' order by k6, sum(k8)" 
+    qt_group19 "select k6, count(k8) from test group by k6 having k6='true' order by k6, count(k8)" 
+    qt_group20 "select k6, min(k8) from test group by k6 having k6='true' order by k6, min(k8)" 
+    qt_group21 "select k6, max(k8) from test group by k6 having k6='true' order by k6, max(k8)" 
+    qt_group22 "select k2, avg(k8) from test group by k2 having k2<=1989 order by k2, avg(k8)" 
+    qt_group23 "select k2, sum(k8) from test group by k2 having k2<=1989 order by k2, sum(k8)" 
+    qt_group24 "select k2, count(k8) from test group by k2 having k2<=1989 order by k2, count(k8)" 
+    qt_group25 "select k2, min(k8) from test group by k2 having k2<=1989 order by k2, min(k8)" 
+    qt_group26 "select k2, max(k8) from test group by k2 having k2<=1989 order by k2, max(k8)" 
+    qt_group27 "select count(ALL *) from test where k5 is not null group by k1%10 order by 1"
+    qt_group28 "select k5, k5 * 2, count(*) from test group by 1, 2 order by 1, 2,3"
+    qt_group29 "select k1 % 3, k2 % 3, count(*) from test where k4 > 0 group by 2, 1 order by 1, 2, 3"
+    qt_group30 "select k1 % 2, k2 % 2, k3 % 3, k4 % 3, k11, count(*) from test where (k11 = '2015-03-13 12:36:38' or k11 = '2000-01-01 00:00:00') and k5 is not null group by 1, 2, 3, 4, 5 order by 1, 2, 3, 4, 5" 
+    qt_group31 "select count(*) from test where (k11='2015-03-13 12:36:38' or k11 = '2000-01-01 00:00:00') and k5 is not null group by k1%2, k2%2, k3%3, k4%3, k11%2 order by 1"
+    qt_group32 "select count(*), min(k1), max(k1), sum(k1), avg(k1) from test where k1=10000 order by 1"
+    qt_group33 "select k1 % 7, count(*), avg(k1) from test where k4 > 0 group by 1 having avg(k1) > 2 or count(*) > 5 order by 1, 2, 3"
+    qt_group34 "select k10, count(*) from test where k5 is not null group by k10 having k10 < cast('2010-01-01 01:05:20' as datetime) order by 1, 2"
+    qt_group35 "select k1 * k1, k1 + k1 as c from test group by k1 * k1, k1 + k1, k1 * k1 having (c) < 5 order by 1, 2 limit 10"
+    qt_group36 "select 1 from (select count(k4) c from test having min(k1) is not null) as t where c is not null"
+    qt_group37 "select count(k1), sum(k1 * k2) from test order by 1, 2"
+    qt_group38 "select k1 % 2, k2 + 1, k3 from test where k3 > 10000 group by 1,2,3 order by 1,2,3" 
+    qt_group39 "select extract(year from k10) as wj, extract(month from k10) as dyk, sum(k1) from test group by 1, 2 order by 1, 2, 3"
+
+    // with having
+    qt_group40 "select avg(k1) as a from test group by k2 having a > 10 order by a"
+    qt_group41 "select avg(k5) as a from test group by k1 having a > 100 order by a"
+    qt_group42 "select sum(k5) as a from test group by k1 having a < 100.0 order by a"
+    qt_group43 "select sum(k8) as a from test group by k1 having a > 100 order by a"
+    qt_group44 "select avg(k9) as a from test group by k1 having a < 100.0 order by a"
+
+    // order 2
+    qt_order8 "select k1, k2 from (select k1, max(k2) as k2 from test where k1 > 0 group by k1 order by k1)a where k1 > 0 and k1 < 10 order by k1"
+    qt_order9 "select k1, k2 from (select k1, max(k2) as k2 from test where k1 > 0 group by k1 order by k1)a left join (select k1 as k3, k2 as k4 from baseall) b on a.k1 = b.k3 where k1 > 0 and k1 < 10 order by k1, k2"
+    qt_order10 "select k1, count(*) from test group by 1 order by 1 limit 10"
+    qt_order11 "select a.k1, b.k1, a.k6 from baseall a join test b on a.k1 = b.k1 where a.k2 > 0 and a.k1 + b.k1 > 20 and b.k6 = 'false' order by a.k1"
+    qt_order12 "select k1 from baseall order by k1 % 5, k1"
+    qt_order13 "select k1 from (select k1, k2 from baseall order by k1 limit 10) a where k1 > 5 order by k1 limit 10"
+    qt_order14 "select k1 from (select k1, k2 from baseall order by k1) a where k1 > 5 order by k1 limit 10"
+    qt_order15 "select k1 from (select k1, k2 from baseall order by k1 limit 10 offset 3) a where k1 > 5 order by k1 limit 5 offset 2"
+    qt_order16 "select a.k1, a.k2, b.k1 from baseall a join (select * from test where k6 = 'false' order by k1 limit 3 offset 2) b on a.k1 = b.k1 where a.k2 > 0 order by 1"
+
+    qt_orderBy_withNull_1 "select k4 + k5 from test order by 1 nulls first"
+
+    // NULL结果
+    qt_orderBy_withNull_2 "select k5, k5 + k6 from test where lower(k6) not like 'na%' and upper(k6) not like 'INF%' order by k5 nulls first"
+
+    // null 和非null
+    qt_orderBy_withNull_3 " select a.k1 ak1, b.k1 bk1 from test a right join baseall b on a.k1=b.k1 and b.k1>10 order by ak1 desc nulls first, bk1"
+
+    // NULL列group by
+    qt_orderBy_withNull_4 "select k5 + k4 as nu, sum(k1) from test group by nu order by nu nulls first"
+    qt_orderBy_withNull_5 "select k6 + k5 as nu from test group by nu"
+    qt_orderBy_withNull_6 "select k6 + k5 as nu, sum(1) from test  group by nu order by nu  desc limit 5"
+    qt_orderBy_withNull_7 "select k6 + k5 as nu, sum(1) from test  group by nu order by nu limit 5"
+
+    qt_orderBy_withNull_8 "select k4 + k5 as sum, k5 + k6 as nu from test  where lower(k6) not like 'na%' and upper(k6) not like 'INF%' order by sum nulls last"
+    qt_orderBy_withNull_9 "select k4 + k5 as nu from test order by nu nulls last"
+
+    //null 和非null
+    qt_orderBy_withNull_10 " select a.k1 ak1, b.k1 bk1 from test a right join baseall b on a.k1=b.k1 and b.k1 > 10 order by ak1 nulls last, bk1"
+
+    qt_group31 "select count(*) from test where (k11='2015-03-13 12:36:38' or k11 = '2000-01-01 00:00:00') and k5 is not null group by k1%2, k2%2, k3%3, k4%3, k11%2 order by count(*)"
+
+    test {
+        sql "select true + 1 + 'x'"
+        exception "string literal 'x' cannot be cast to double"
+    }
+}


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