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

[doris] branch branch-1.2-lts updated (be513e1062 -> 5ce523852a)

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

morningman pushed a change to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git


    from be513e1062 [fix](multi catalog)Set column defualt value for query. (#15415)
     new e2a54f6408 [enhancement](session var) varariable to control whether to rewrite OR to IN or not (#15437)
     new e599532588 [Bug](Decimalv3) coredump of decimalv3 multiply (#15452)
     new 5ce523852a [chore](thirdparty) Support cleaning extracted data before building them (#15458)

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 be/src/vec/core/decimal_comparison.h               |  8 +--
 be/src/vec/data_types/data_type_decimal.h          | 60 ++++++----------------
 be/src/vec/functions/function_binary_arithmetic.h  |  7 ++-
 .../java/org/apache/doris/qe/SessionVariable.java  | 14 +++++
 .../doris/rewrite/ExtractCommonFactorsRule.java    | 51 ++++++++++++++----
 .../org/apache/doris/planner/QueryPlanTest.java    | 17 ++++++
 regression-test/data/decimalv3/test_decimalv3.out  |  3 ++
 .../data/performance_p0/redundant_conjuncts.out    |  2 +-
 .../suites/decimalv3/test_decimalv3.groovy         |  1 +
 .../performance_p0/redundant_conjuncts.groovy      |  1 +
 thirdparty/build-thirdparty.sh                     | 14 ++++-
 11 files changed, 116 insertions(+), 62 deletions(-)


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


[doris] 03/03: [chore](thirdparty) Support cleaning extracted data before building them (#15458)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 5ce523852ab797b977fab3b40a2a6da184ba4991
Author: Adonis Ling <ad...@gmail.com>
AuthorDate: Thu Dec 29 16:01:23 2022 +0800

    [chore](thirdparty) Support cleaning extracted data before building them (#15458)
    
    Currently, we may fail to build the third-party libraries if we keep the outdated extracted data.
    
    Considering the following scenario, Bob added patches to some libraries and Alice updates the codebase and builds
    the third-party libraries. If Alice kept the outdated extracted data, she should fail to build the third-party libraries
    because the patches are not applied due to the outdated `patched_marks`.
    
    This PR introduces a way to clean the outdated data before building the third-party libraries.
---
 thirdparty/build-thirdparty.sh | 14 +++++++++++++-
 1 file changed, 13 insertions(+), 1 deletion(-)

diff --git a/thirdparty/build-thirdparty.sh b/thirdparty/build-thirdparty.sh
index 45200bf7ce..6e64b89f5c 100755
--- a/thirdparty/build-thirdparty.sh
+++ b/thirdparty/build-thirdparty.sh
@@ -49,6 +49,7 @@ usage() {
 Usage: $0 <options>
   Optional options:
      -j                 build thirdparty parallel
+     --clean            clean the extracted data
   "
     exit 1
 }
@@ -58,6 +59,7 @@ if ! OPTS="$(getopt \
     -o '' \
     -o 'h' \
     -l 'help' \
+    -l 'clean' \
     -o 'j:' \
     -- "$@")"; then
     usage
@@ -88,6 +90,10 @@ if [[ "$#" -ne 1 ]]; then
             HELP=1
             shift
             ;;
+        --clean)
+            CLEAN=1
+            shift
+            ;;
         --)
             shift
             break
@@ -102,11 +108,11 @@ fi
 
 if [[ "${HELP}" -eq 1 ]]; then
     usage
-    exit
 fi
 
 echo "Get params:
     PARALLEL            -- ${PARALLEL}
+    CLEAN               -- ${CLEAN}
 "
 
 if [[ ! -f "${TP_DIR}/download-thirdparty.sh" ]]; then
@@ -123,6 +129,12 @@ fi
 
 cd "${TP_DIR}"
 
+if [[ "${CLEAN}" -eq 1 ]] && [[ -d "${TP_SOURCE_DIR}" ]]; then
+    echo 'Clean the extracted data ...'
+    find "${TP_SOURCE_DIR}" -mindepth 1 -maxdepth 1 -type d -exec rm -rf {} \;
+    echo 'Success!'
+fi
+
 # Download thirdparties.
 "${TP_DIR}/download-thirdparty.sh"
 


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


[doris] 01/03: [enhancement](session var) varariable to control whether to rewrite OR to IN or not (#15437)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit e2a54f6408ae07a6b85da3206479a8602c00e7a2
Author: Henry2SS <45...@users.noreply.github.com>
AuthorDate: Thu Dec 29 14:50:32 2022 +0800

    [enhancement](session var) varariable to control whether to rewrite OR to IN or not (#15437)
---
 .../java/org/apache/doris/qe/SessionVariable.java  | 14 ++++++
 .../doris/rewrite/ExtractCommonFactorsRule.java    | 51 +++++++++++++++++-----
 .../org/apache/doris/planner/QueryPlanTest.java    | 17 ++++++++
 .../data/performance_p0/redundant_conjuncts.out    |  2 +-
 .../performance_p0/redundant_conjuncts.groovy      |  1 +
 5 files changed, 74 insertions(+), 11 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java
index f0a556067f..8d8f412aee 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java
@@ -196,6 +196,9 @@ public class SessionVariable implements Serializable, Writable {
 
     //percentage of EXEC_MEM_LIMIT
     public static final String BROADCAST_HASHTABLE_MEM_LIMIT_PERCENTAGE = "broadcast_hashtable_mem_limit_percentage";
+
+    public static final String REWRITE_OR_TO_IN_PREDICATE_THRESHOLD = "rewrite_or_to_in_predicate_threshold";
+
     public static final String NEREIDS_STAR_SCHEMA_SUPPORT = "nereids_star_schema_support";
 
     public static final String NEREIDS_CBO_PENALTY_FACTOR = "nereids_cbo_penalty_factor";
@@ -544,6 +547,9 @@ public class SessionVariable implements Serializable, Writable {
     @VariableMgr.VarAttr(name = NEREIDS_STAR_SCHEMA_SUPPORT)
     private boolean nereidsStarSchemaSupport = true;
 
+    @VariableMgr.VarAttr(name = REWRITE_OR_TO_IN_PREDICATE_THRESHOLD)
+    private int rewriteOrToInPredicateThreshold = 2;
+
     @VariableMgr.VarAttr(name = NEREIDS_CBO_PENALTY_FACTOR)
     private double nereidsCboPenaltyFactor = 0.7;
     @VariableMgr.VarAttr(name = ENABLE_NEREIDS_TRACE)
@@ -661,6 +667,14 @@ public class SessionVariable implements Serializable, Writable {
         this.blockEncryptionMode = blockEncryptionMode;
     }
 
+    public void setRewriteOrToInPredicateThreshold(int threshold) {
+        this.rewriteOrToInPredicateThreshold = threshold;
+    }
+
+    public int getRewriteOrToInPredicateThreshold() {
+        return rewriteOrToInPredicateThreshold;
+    }
+
     public long getMaxExecMemByte() {
         return maxExecMemByte;
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/rewrite/ExtractCommonFactorsRule.java b/fe/fe-core/src/main/java/org/apache/doris/rewrite/ExtractCommonFactorsRule.java
index 5a3bc34c8c..3c808c6100 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/rewrite/ExtractCommonFactorsRule.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/rewrite/ExtractCommonFactorsRule.java
@@ -28,6 +28,7 @@ import org.apache.doris.analysis.SlotRef;
 import org.apache.doris.analysis.TableName;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.planner.PlanNode;
+import org.apache.doris.qe.ConnectContext;
 import org.apache.doris.rewrite.ExprRewriter.ClauseType;
 
 import com.google.common.base.Preconditions;
@@ -43,6 +44,7 @@ import org.apache.logging.log4j.Logger;
 
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashSet;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
@@ -462,6 +464,13 @@ public class ExtractCommonFactorsRule implements ExprRewriteRule {
         boolean isOrToInAllowed = true;
         Set<String> slotSet = new LinkedHashSet<>();
 
+        int rewriteThreshold;
+        if (ConnectContext.get() == null) {
+            rewriteThreshold = 2;
+        } else {
+            rewriteThreshold = ConnectContext.get().getSessionVariable().getRewriteOrToInPredicateThreshold();
+        }
+
         for (int i = 0; i < exprs.size(); i++) {
             Expr predicate = exprs.get(i);
             if (!(predicate instanceof BinaryPredicate) && !(predicate instanceof InPredicate)) {
@@ -492,22 +501,44 @@ public class ExtractCommonFactorsRule implements ExprRewriteRule {
         // isOrToInAllowed : true, means can rewrite
         // slotSet.size : nums of columnName in exprs, should be 1
         if (isOrToInAllowed && slotSet.size() == 1) {
-            // slotRef to get ColumnName
-
-            // SlotRef firstSlot = (SlotRef) exprs.get(0).getChild(0);
-            List<Expr> childrenList = exprs.get(0).getChildren();
-            inPredicate = new InPredicate(exprs.get(0).getChild(0),
-                    childrenList.subList(1, childrenList.size()), false);
-
-            for (int i = 1; i < exprs.size(); i++) {
-                childrenList = exprs.get(i).getChildren();
-                inPredicate.addChildren(childrenList.subList(1, childrenList.size()));
+            if (exprs.size() < rewriteThreshold) {
+                return null;
             }
+
+            // get deduplication list
+            List<Expr> deduplicationExprs = getDeduplicationList(exprs);
+            inPredicate = new InPredicate(deduplicationExprs.get(0),
+                    deduplicationExprs.subList(1, deduplicationExprs.size()), false);
         }
 
         return inPredicate;
     }
 
+    public List<Expr> getDeduplicationList(List<Expr> exprs) {
+        Set<Expr> set = new HashSet<>();
+        List<Expr> deduplicationExprList = new ArrayList<>();
+
+        deduplicationExprList.add(exprs.get(0).getChild(0));
+
+        for (Expr expr : exprs) {
+            if (expr instanceof BinaryPredicate) {
+                if (!set.contains(expr.getChild(1))) {
+                    set.add(expr.getChild(1));
+                    deduplicationExprList.add(expr.getChild(1));
+                }
+            } else {
+                List<Expr> childrenExprs = expr.getChildren();
+                for (Expr childrenExpr : childrenExprs.subList(1, childrenExprs.size())) {
+                    if (!set.contains(childrenExpr)) {
+                        set.add(childrenExpr);
+                        deduplicationExprList.add(childrenExpr);
+                    }
+                }
+            }
+        }
+        return deduplicationExprList;
+    }
+
     /**
      * Convert RangeSet to Compound Predicate
      * @param slotRef: <k1>
diff --git a/fe/fe-core/src/test/java/org/apache/doris/planner/QueryPlanTest.java b/fe/fe-core/src/test/java/org/apache/doris/planner/QueryPlanTest.java
index 8d1ad8b78c..914bc338bc 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/planner/QueryPlanTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/planner/QueryPlanTest.java
@@ -2238,5 +2238,22 @@ public class QueryPlanTest extends TestWithFeService {
         sql = "SELECT * from test1 where (query_time = 1 or query_time = 2) and (scan_bytes = 2 or scan_bytes = 3)";
         explainString = UtFrameUtils.getSQLPlanOrErrorMsg(connectContext, "EXPLAIN " + sql);
         Assert.assertTrue(explainString.contains("PREDICATES: `query_time` IN (1, 2), `scan_bytes` IN (2, 3)"));
+
+        sql = "SELECT * from test1 where query_time = 1 or query_time = 2 or query_time = 3 or query_time = 1";
+        explainString = UtFrameUtils.getSQLPlanOrErrorMsg(connectContext, "EXPLAIN " + sql);
+        Assert.assertTrue(explainString.contains("PREDICATES: `query_time` IN (1, 2, 3)"));
+
+        sql = "SELECT * from test1 where query_time = 1 or query_time = 2 or query_time in (3, 2)";
+        explainString = UtFrameUtils.getSQLPlanOrErrorMsg(connectContext, "EXPLAIN " + sql);
+        Assert.assertTrue(explainString.contains("PREDICATES: `query_time` IN (1, 2, 3)"));
+
+        connectContext.getSessionVariable().setRewriteOrToInPredicateThreshold(100);
+        sql = "SELECT * from test1 where query_time = 1 or query_time = 2 or query_time in (3, 4)";
+        explainString = UtFrameUtils.getSQLPlanOrErrorMsg(connectContext, "EXPLAIN " + sql);
+        Assert.assertTrue(explainString.contains("PREDICATES: (`query_time` = 1 OR `query_time` = 2 OR `query_time` IN (3, 4))"));
+
+        sql = "SELECT * from test1 where (query_time = 1 or query_time = 2) and query_time in (3, 4)";
+        explainString = UtFrameUtils.getSQLPlanOrErrorMsg(connectContext, "EXPLAIN " + sql);
+        Assert.assertTrue(explainString.contains("PREDICATES: (`query_time` = 1 OR `query_time` = 2), `query_time` IN (3, 4)"));
     }
 }
diff --git a/regression-test/data/performance_p0/redundant_conjuncts.out b/regression-test/data/performance_p0/redundant_conjuncts.out
index 7dbabccf37..98178f31aa 100644
--- a/regression-test/data/performance_p0/redundant_conjuncts.out
+++ b/regression-test/data/performance_p0/redundant_conjuncts.out
@@ -23,7 +23,7 @@ PLAN FRAGMENT 0
 
   0:VOlapScanNode
      TABLE: default_cluster:regression_test_performance_p0.redundant_conjuncts(redundant_conjuncts), PREAGGREGATION: OFF. Reason: No AggregateInfo
-     PREDICATES: `k1` IN (1, 2)
+     PREDICATES: (`k1` = 1 OR `k1` = 2)
      partitions=0/1, tablets=0/0, tabletList=
      cardinality=0, avgRowSize=8.0, numNodes=1
 
diff --git a/regression-test/suites/performance_p0/redundant_conjuncts.groovy b/regression-test/suites/performance_p0/redundant_conjuncts.groovy
index 14624a8049..c9ed28b026 100644
--- a/regression-test/suites/performance_p0/redundant_conjuncts.groovy
+++ b/regression-test/suites/performance_p0/redundant_conjuncts.groovy
@@ -39,6 +39,7 @@ suite("redundant_conjuncts") {
     EXPLAIN SELECT v1 FROM redundant_conjuncts WHERE k1 = 1 AND k1 = 1;
     """
 
+    sql "set REWRITE_OR_TO_IN_PREDICATE_THRESHOLD = 100"
     qt_redundant_conjuncts_gnerated_by_extract_common_filter """
     EXPLAIN SELECT v1 FROM redundant_conjuncts WHERE k1 = 1 OR k1 = 2;
     """


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


[doris] 02/03: [Bug](Decimalv3) coredump of decimalv3 multiply (#15452)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit e599532588b5ac8d20cfc22efe89c311a0b2af24
Author: HappenLee <ha...@hotmail.com>
AuthorDate: Thu Dec 29 15:35:17 2022 +0800

    [Bug](Decimalv3) coredump of decimalv3 multiply (#15452)
---
 be/src/vec/core/decimal_comparison.h               |  8 +--
 be/src/vec/data_types/data_type_decimal.h          | 60 ++++++----------------
 be/src/vec/functions/function_binary_arithmetic.h  |  7 ++-
 regression-test/data/decimalv3/test_decimalv3.out  |  3 ++
 .../suites/decimalv3/test_decimalv3.groovy         |  1 +
 5 files changed, 29 insertions(+), 50 deletions(-)

diff --git a/be/src/vec/core/decimal_comparison.h b/be/src/vec/core/decimal_comparison.h
index 1f2bce4200..4c7cfcf765 100644
--- a/be/src/vec/core/decimal_comparison.h
+++ b/be/src/vec/core/decimal_comparison.h
@@ -142,9 +142,11 @@ private:
 
         Shift shift;
         if (decimal0 && decimal1) {
-            auto result_type = decimal_result_type(*decimal0, *decimal1, false, false);
-            shift.a = result_type.scale_factor_for(*decimal0, false);
-            shift.b = result_type.scale_factor_for(*decimal1, false);
+            using Type = std::conditional_t<sizeof(T) >= sizeof(U), T, U>;
+            auto type_ptr = decimal_result_type(*decimal0, *decimal1, false, false, false);
+            const DataTypeDecimal<Type>* result_type = check_decimal<Type>(*type_ptr);
+            shift.a = result_type->scale_factor_for(*decimal0, false);
+            shift.b = result_type->scale_factor_for(*decimal1, false);
         } else if (decimal0) {
             shift.b = decimal0->get_scale_multiplier();
         } else if (decimal1) {
diff --git a/be/src/vec/data_types/data_type_decimal.h b/be/src/vec/data_types/data_type_decimal.h
index ea29954293..c8e08303a1 100644
--- a/be/src/vec/data_types/data_type_decimal.h
+++ b/be/src/vec/data_types/data_type_decimal.h
@@ -219,56 +219,30 @@ private:
 };
 
 template <typename T, typename U>
-typename std::enable_if_t<(sizeof(T) >= sizeof(U)), const DataTypeDecimal<T>> decimal_result_type(
-        const DataTypeDecimal<T>& tx, const DataTypeDecimal<U>& ty, bool is_multiply,
-        bool is_divide) {
+DataTypePtr decimal_result_type(const DataTypeDecimal<T>& tx, const DataTypeDecimal<U>& ty,
+                                bool is_multiply, bool is_divide, bool is_plus_minus) {
+    using Type = std::conditional_t<sizeof(T) >= sizeof(U), T, U>;
     if constexpr (IsDecimalV2<T> && IsDecimalV2<U>) {
-        return DataTypeDecimal<T>(max_decimal_precision<T>(), 9);
+        return std::make_shared<DataTypeDecimal<Type>>((max_decimal_precision<T>(), 9));
     } else {
-        UInt32 scale = (tx.get_scale() > ty.get_scale() ? tx.get_scale() : ty.get_scale());
+        UInt32 scale = std::max(tx.get_scale(), ty.get_scale());
+        auto precision = max_decimal_precision<Type>();
+
+        size_t multiply_precision = tx.get_precision() + ty.get_precision();
+        size_t divide_precision = tx.get_precision() + ty.get_scale();
+        size_t plus_minus_precision =
+                std::max(tx.get_precision() - tx.get_scale(), ty.get_precision() - ty.get_scale()) +
+                scale;
         if (is_multiply) {
             scale = tx.get_scale() + ty.get_scale();
+            precision = std::min(multiply_precision, max_decimal_precision<Decimal128I>());
         } else if (is_divide) {
             scale = tx.get_scale();
+            precision = std::min(divide_precision, max_decimal_precision<Decimal128I>());
+        } else if (is_plus_minus) {
+            precision = std::min(plus_minus_precision, max_decimal_precision<Decimal128I>());
         }
-        return DataTypeDecimal<T>(max_decimal_precision<T>(), scale);
-    }
-}
-
-template <typename T, typename U>
-typename std::enable_if_t<(sizeof(T) < sizeof(U)), const DataTypeDecimal<U>> decimal_result_type(
-        const DataTypeDecimal<T>& tx, const DataTypeDecimal<U>& ty, bool is_multiply,
-        bool is_divide) {
-    if constexpr (IsDecimalV2<T> && IsDecimalV2<U>) {
-        return DataTypeDecimal<U>(max_decimal_precision<U>(), 9);
-    } else {
-        UInt32 scale = (tx.get_scale() > ty.get_scale() ? tx.get_scale() : ty.get_scale());
-        if (is_multiply) {
-            scale = tx.get_scale() + ty.get_scale();
-        } else if (is_divide) {
-            scale = tx.get_scale();
-        }
-        return DataTypeDecimal<U>(max_decimal_precision<U>(), scale);
-    }
-}
-
-template <typename T, typename U>
-const DataTypeDecimal<T> decimal_result_type(const DataTypeDecimal<T>& tx, const DataTypeNumber<U>&,
-                                             bool, bool) {
-    if constexpr (IsDecimalV2<T> && IsDecimalV2<U>) {
-        return DataTypeDecimal<T>(max_decimal_precision<T>(), 9);
-    } else {
-        return DataTypeDecimal<T>(max_decimal_precision<T>(), tx.get_scale());
-    }
-}
-
-template <typename T, typename U>
-const DataTypeDecimal<U> decimal_result_type(const DataTypeNumber<T>&, const DataTypeDecimal<U>& ty,
-                                             bool, bool) {
-    if constexpr (IsDecimalV2<T> && IsDecimalV2<U>) {
-        return DataTypeDecimal<U>(max_decimal_precision<U>(), 9);
-    } else {
-        return DataTypeDecimal<U>(max_decimal_precision<U>(), ty.get_scale());
+        return create_decimal(precision, scale, false);
     }
 }
 
diff --git a/be/src/vec/functions/function_binary_arithmetic.h b/be/src/vec/functions/function_binary_arithmetic.h
index 5c98e72486..2a8da748e3 100644
--- a/be/src/vec/functions/function_binary_arithmetic.h
+++ b/be/src/vec/functions/function_binary_arithmetic.h
@@ -730,10 +730,9 @@ public:
                     if constexpr (!std::is_same_v<ResultDataType, InvalidType>) {
                         if constexpr (IsDataTypeDecimal<LeftDataType> &&
                                       IsDataTypeDecimal<RightDataType>) {
-                            ResultDataType result_type = decimal_result_type(
-                                    left, right, OpTraits::is_multiply, OpTraits::is_division);
-                            type_res = std::make_shared<ResultDataType>(result_type.get_precision(),
-                                                                        result_type.get_scale());
+                            type_res = decimal_result_type(left, right, OpTraits::is_multiply,
+                                                           OpTraits::is_division,
+                                                           OpTraits::is_plus_minus);
                         } else if constexpr (IsDataTypeDecimal<LeftDataType>) {
                             type_res = std::make_shared<LeftDataType>(left.get_precision(),
                                                                       left.get_scale());
diff --git a/regression-test/data/decimalv3/test_decimalv3.out b/regression-test/data/decimalv3/test_decimalv3.out
index 1bb8b045c0..f8d56b4c41 100644
--- a/regression-test/data/decimalv3/test_decimalv3.out
+++ b/regression-test/data/decimalv3/test_decimalv3.out
@@ -2,3 +2,6 @@
 -- !decimalv3 --
 100.000000000000000000
 
+-- !decimalv3 --
+100.00000000000000000000
+
diff --git a/regression-test/suites/decimalv3/test_decimalv3.groovy b/regression-test/suites/decimalv3/test_decimalv3.groovy
index 374e554b93..8b8b010240 100644
--- a/regression-test/suites/decimalv3/test_decimalv3.groovy
+++ b/regression-test/suites/decimalv3/test_decimalv3.groovy
@@ -26,4 +26,5 @@ suite("test_decimalv3") {
 	sql "create view test5_v (amout) as select cast(a*b as decimalv3(38,18)) from test5"
 
 	qt_decimalv3 "select * from test5_v"
+	qt_decimalv3 "select cast(a as decimalv3(12,10)) * cast(b as decimalv3(18,10)) from test5"
 }


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