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/04/17 09:47:47 UTC

[doris] branch branch-1.2-lts updated: [Bug](DECIMALV3) report failure if a decimal value is overflow (#18336) (#18733)

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


The following commit(s) were added to refs/heads/branch-1.2-lts by this push:
     new 3a91f083fd [Bug](DECIMALV3) report failure if a decimal value is overflow (#18336) (#18733)
3a91f083fd is described below

commit 3a91f083fdb8ef43465dd3f82038a3917091b22f
Author: Gabriel <ga...@gmail.com>
AuthorDate: Mon Apr 17 17:47:39 2023 +0800

    [Bug](DECIMALV3) report failure if a decimal value is overflow (#18336) (#18733)
    
    pick #18336
---
 be/src/runtime/decimalv2_value.cpp                 |   2 +-
 be/src/util/string_parser.hpp                      |  16 +-
 be/src/vec/data_types/data_type_decimal.cpp        |  11 +-
 be/src/vec/data_types/data_type_decimal.h          |   2 +-
 be/src/vec/exprs/vexpr.cpp                         | 116 ++++-----
 be/src/vec/exprs/vliteral.cpp                      |  59 +++--
 be/src/vec/io/io_helper.h                          |  12 +-
 be/src/vec/sink/vtablet_sink.cpp                   |  70 ++++++
 be/src/vec/sink/vtablet_sink.h                     |  11 +
 .../java/org/apache/doris/catalog/ScalarType.java  |   6 +-
 .../org/apache/doris/analysis/ArithmeticExpr.java  |   9 +-
 .../org/apache/doris/planner/QueryPlanTest.java    |   4 +-
 .../data/datatype_p0/decimalv3/test_overflow.out   |  19 --
 regression-test/data/insert_p0/insert.out          | 182 +++++++-------
 .../operator/test_arithmetic_operators.out         | 261 +++------------------
 .../conditional_functions/test_nullif.out          |   2 +-
 .../decimalv3/test_data/test_overflow.csv          |   2 +
 .../datatype_p0/decimalv3/test_overflow.groovy     |  56 -----
 .../suites/primary_index/test_pk_uk_case.groovy    |   8 +-
 .../suites/query_p0/aggregate/aggregate.groovy     |   2 +-
 .../operator/test_arithmetic_operators.groovy      |  22 +-
 .../conditional_functions/test_nullif.groovy       |   4 +-
 22 files changed, 361 insertions(+), 515 deletions(-)

diff --git a/be/src/runtime/decimalv2_value.cpp b/be/src/runtime/decimalv2_value.cpp
index 3d7f56c047..bc3934d3b7 100644
--- a/be/src/runtime/decimalv2_value.cpp
+++ b/be/src/runtime/decimalv2_value.cpp
@@ -353,7 +353,7 @@ int DecimalV2Value::parse_from_str(const char* decimal_str, int32_t length) {
 
     _value = StringParser::string_to_decimal<__int128>(decimal_str, length, PRECISION, SCALE,
                                                        &result);
-    if (result == StringParser::PARSE_FAILURE) {
+    if (result != StringParser::PARSE_SUCCESS) {
         error = E_DEC_BAD_NUM;
     }
     return error;
diff --git a/be/src/util/string_parser.hpp b/be/src/util/string_parser.hpp
index 4fb0449cfb..eb0ab53060 100644
--- a/be/src/util/string_parser.hpp
+++ b/be/src/util/string_parser.hpp
@@ -614,19 +614,17 @@ inline T StringParser::string_to_decimal(const char* s, int len, int type_precis
             // an exponent will be made later.
             if (LIKELY(type_precision > precision)) {
                 value = (value * 10) + (c - '0'); // Benchmarks are faster with parenthesis...
+            } else {
+                *result = StringParser::PARSE_OVERFLOW;
+                value = is_negative ? vectorized::min_decimal_value<vectorized::Decimal<T>>(
+                                              type_precision)
+                                    : vectorized::max_decimal_value<vectorized::Decimal<T>>(
+                                              type_precision);
+                return value;
             }
             DCHECK(value >= 0); // For some reason //DCHECK_GE doesn't work with __int128.
             ++precision;
             scale += found_dot;
-            if (precision > type_precision) {
-                if constexpr (std::is_same_v<int32_t, T>) {
-                    value = vectorized::max_decimal_value<vectorized::Decimal32>();
-                } else if constexpr (std::is_same_v<int64_t, T>) {
-                    value = vectorized::max_decimal_value<vectorized::Decimal64>();
-                } else {
-                    value = vectorized::max_decimal_value<vectorized::Decimal128>();
-                }
-            }
         } else if (c == '.' && LIKELY(!found_dot)) {
             found_dot = 1;
         } else if ((c == 'e' || c == 'E') && LIKELY(!found_exponent)) {
diff --git a/be/src/vec/data_types/data_type_decimal.cpp b/be/src/vec/data_types/data_type_decimal.cpp
index 5d00e65033..4d3a1147de 100644
--- a/be/src/vec/data_types/data_type_decimal.cpp
+++ b/be/src/vec/data_types/data_type_decimal.cpp
@@ -150,14 +150,11 @@ MutableColumnPtr DataTypeDecimal<T>::create_column() const {
 }
 
 template <typename T>
-T DataTypeDecimal<T>::parse_from_string(const std::string& str) const {
+bool DataTypeDecimal<T>::parse_from_string(const std::string& str, T* res) const {
     StringParser::ParseResult result = StringParser::PARSE_SUCCESS;
-    T value = StringParser::string_to_decimal<__int128>(str.c_str(), str.size(), precision, scale,
-                                                        &result);
-    if (result != StringParser::PARSE_SUCCESS) {
-        LOG(WARNING) << "Failed to parse string of decimal";
-    }
-    return value;
+    *res = StringParser::string_to_decimal<__int128>(str.c_str(), str.size(), precision, scale,
+                                                     &result);
+    return result == StringParser::PARSE_SUCCESS;
 }
 
 DataTypePtr create_decimal(UInt64 precision_value, UInt64 scale_value, bool use_v2) {
diff --git a/be/src/vec/data_types/data_type_decimal.h b/be/src/vec/data_types/data_type_decimal.h
index c644fbcfc0..7b16f27391 100644
--- a/be/src/vec/data_types/data_type_decimal.h
+++ b/be/src/vec/data_types/data_type_decimal.h
@@ -215,7 +215,7 @@ public:
 
     static T get_scale_multiplier(UInt32 scale);
 
-    T parse_from_string(const std::string& str) const;
+    bool parse_from_string(const std::string& str, T* res) const;
 
 private:
     const UInt32 precision;
diff --git a/be/src/vec/exprs/vexpr.cpp b/be/src/vec/exprs/vexpr.cpp
index 89ddc94a63..f584816bf3 100644
--- a/be/src/vec/exprs/vexpr.cpp
+++ b/be/src/vec/exprs/vexpr.cpp
@@ -111,63 +111,67 @@ void VExpr::close(doris::RuntimeState* state, VExprContext* context,
 
 Status VExpr::create_expr(doris::ObjectPool* pool, const doris::TExprNode& texpr_node,
                           VExpr** expr) {
-    switch (texpr_node.node_type) {
-    case TExprNodeType::BOOL_LITERAL:
-    case TExprNodeType::INT_LITERAL:
-    case TExprNodeType::LARGE_INT_LITERAL:
-    case TExprNodeType::FLOAT_LITERAL:
-    case TExprNodeType::DECIMAL_LITERAL:
-    case TExprNodeType::DATE_LITERAL:
-    case TExprNodeType::STRING_LITERAL:
-    case TExprNodeType::JSON_LITERAL:
-    case TExprNodeType::NULL_LITERAL: {
-        *expr = pool->add(new VLiteral(texpr_node));
-        return Status::OK();
-    }
-    case TExprNodeType::ARRAY_LITERAL: {
-        *expr = pool->add(new VArrayLiteral(texpr_node));
-        return Status::OK();
-    }
-    case doris::TExprNodeType::SLOT_REF: {
-        *expr = pool->add(new VSlotRef(texpr_node));
-        break;
-    }
-    case doris::TExprNodeType::COMPOUND_PRED: {
-        *expr = pool->add(new VcompoundPred(texpr_node));
-        break;
-    }
-    case doris::TExprNodeType::ARITHMETIC_EXPR:
-    case doris::TExprNodeType::BINARY_PRED:
-    case doris::TExprNodeType::FUNCTION_CALL:
-    case doris::TExprNodeType::COMPUTE_FUNCTION_CALL: {
-        *expr = pool->add(new VectorizedFnCall(texpr_node));
-        break;
-    }
-    case doris::TExprNodeType::CAST_EXPR: {
-        *expr = pool->add(new VCastExpr(texpr_node));
-        break;
-    }
-    case doris::TExprNodeType::IN_PRED: {
-        *expr = pool->add(new VInPredicate(texpr_node));
-        break;
-    }
-    case doris::TExprNodeType::CASE_EXPR: {
-        if (!texpr_node.__isset.case_expr) {
-            return Status::InternalError("Case expression not set in thrift node");
+    try {
+        switch (texpr_node.node_type) {
+        case TExprNodeType::BOOL_LITERAL:
+        case TExprNodeType::INT_LITERAL:
+        case TExprNodeType::LARGE_INT_LITERAL:
+        case TExprNodeType::FLOAT_LITERAL:
+        case TExprNodeType::DECIMAL_LITERAL:
+        case TExprNodeType::DATE_LITERAL:
+        case TExprNodeType::STRING_LITERAL:
+        case TExprNodeType::JSON_LITERAL:
+        case TExprNodeType::NULL_LITERAL: {
+            *expr = pool->add(new VLiteral(texpr_node));
+            return Status::OK();
         }
-        *expr = pool->add(new VCaseExpr(texpr_node));
-        break;
-    }
-    case TExprNodeType::INFO_FUNC: {
-        *expr = pool->add(new VInfoFunc(texpr_node));
-        break;
-    }
-    case TExprNodeType::TUPLE_IS_NULL_PRED: {
-        *expr = pool->add(new VTupleIsNullPredicate(texpr_node));
-        break;
-    }
-    default:
-        return Status::InternalError("Unknown expr node type: {}", texpr_node.node_type);
+        case TExprNodeType::ARRAY_LITERAL: {
+            *expr = pool->add(new VArrayLiteral(texpr_node));
+            return Status::OK();
+        }
+        case doris::TExprNodeType::SLOT_REF: {
+            *expr = pool->add(new VSlotRef(texpr_node));
+            break;
+        }
+        case doris::TExprNodeType::COMPOUND_PRED: {
+            *expr = pool->add(new VcompoundPred(texpr_node));
+            break;
+        }
+        case doris::TExprNodeType::ARITHMETIC_EXPR:
+        case doris::TExprNodeType::BINARY_PRED:
+        case doris::TExprNodeType::FUNCTION_CALL:
+        case doris::TExprNodeType::COMPUTE_FUNCTION_CALL: {
+            *expr = pool->add(new VectorizedFnCall(texpr_node));
+            break;
+        }
+        case doris::TExprNodeType::CAST_EXPR: {
+            *expr = pool->add(new VCastExpr(texpr_node));
+            break;
+        }
+        case doris::TExprNodeType::IN_PRED: {
+            *expr = pool->add(new VInPredicate(texpr_node));
+            break;
+        }
+        case doris::TExprNodeType::CASE_EXPR: {
+            if (!texpr_node.__isset.case_expr) {
+                return Status::InternalError("Case expression not set in thrift node");
+            }
+            *expr = pool->add(new VCaseExpr(texpr_node));
+            break;
+        }
+        case TExprNodeType::INFO_FUNC: {
+            *expr = pool->add(new VInfoFunc(texpr_node));
+            break;
+        }
+        case TExprNodeType::TUPLE_IS_NULL_PRED: {
+            *expr = pool->add(new VTupleIsNullPredicate(texpr_node));
+            break;
+        }
+        default:
+            return Status::InternalError("Unknown expr node type: {}", texpr_node.node_type);
+        }
+    } catch (const Exception& e) {
+        return Status::Error(e.code());
     }
     if (!(*expr)->data_type()) {
         return Status::InvalidArgument("Unknown expr type: {}", texpr_node.node_type);
diff --git a/be/src/vec/exprs/vliteral.cpp b/be/src/vec/exprs/vliteral.cpp
index f78118ab0f..98c5b0cf55 100644
--- a/be/src/vec/exprs/vliteral.cpp
+++ b/be/src/vec/exprs/vliteral.cpp
@@ -143,8 +143,15 @@ void VLiteral::init(const TExprNode& node) {
         case TYPE_DECIMALV2: {
             DCHECK_EQ(node.node_type, TExprNodeType::DECIMAL_LITERAL);
             DCHECK(node.__isset.decimal_literal);
-            DecimalV2Value value(node.decimal_literal.value);
-            field = DecimalField<Decimal128>(value.value(), value.scale());
+            DecimalV2Value value;
+            if (value.parse_from_str(node.decimal_literal.value.c_str(),
+                                     node.decimal_literal.value.size()) == E_DEC_OK) {
+                field = DecimalField<Decimal128>(value.value(), value.scale());
+            } else {
+                throw Exception(fmt::format("Invalid decimal(scale: {}) value: {}", value.scale(),
+                                            node.decimal_literal.value),
+                                doris::ErrorCode::INVALID_ARGUMENT);
+            }
             break;
         }
         case TYPE_DECIMAL32: {
@@ -152,11 +159,17 @@ void VLiteral::init(const TExprNode& node) {
             DCHECK(node.__isset.decimal_literal);
             DataTypePtr type_ptr = create_decimal(node.type.types[0].scalar_type.precision,
                                                   node.type.types[0].scalar_type.scale, false);
-            auto val = typeid_cast<const DataTypeDecimal<Decimal32>*>(type_ptr.get())
-                               ->parse_from_string(node.decimal_literal.value);
-            auto scale =
-                    typeid_cast<const DataTypeDecimal<Decimal32>*>(type_ptr.get())->get_scale();
-            field = DecimalField<Decimal32>(val, scale);
+            Decimal32 val;
+            if (typeid_cast<const DataTypeDecimal<Decimal32>*>(type_ptr.get())
+                        ->parse_from_string(node.decimal_literal.value, &val)) {
+                auto scale =
+                        typeid_cast<const DataTypeDecimal<Decimal32>*>(type_ptr.get())->get_scale();
+                field = DecimalField<Decimal32>(val, scale);
+            } else {
+                throw Exception(fmt::format("Invalid value: {} for type {}",
+                                            node.decimal_literal.value, type_ptr->get_name()),
+                                doris::ErrorCode::INVALID_ARGUMENT);
+            }
             break;
         }
         case TYPE_DECIMAL64: {
@@ -164,11 +177,17 @@ void VLiteral::init(const TExprNode& node) {
             DCHECK(node.__isset.decimal_literal);
             DataTypePtr type_ptr = create_decimal(node.type.types[0].scalar_type.precision,
                                                   node.type.types[0].scalar_type.scale, false);
-            auto val = typeid_cast<const DataTypeDecimal<Decimal64>*>(type_ptr.get())
-                               ->parse_from_string(node.decimal_literal.value);
-            auto scale =
-                    typeid_cast<const DataTypeDecimal<Decimal64>*>(type_ptr.get())->get_scale();
-            field = DecimalField<Decimal64>(val, scale);
+            Decimal64 val;
+            if (typeid_cast<const DataTypeDecimal<Decimal64>*>(type_ptr.get())
+                        ->parse_from_string(node.decimal_literal.value, &val)) {
+                auto scale =
+                        typeid_cast<const DataTypeDecimal<Decimal64>*>(type_ptr.get())->get_scale();
+                field = DecimalField<Decimal64>(val, scale);
+            } else {
+                throw Exception(fmt::format("Invalid value: {} for type {}",
+                                            node.decimal_literal.value, type_ptr->get_name()),
+                                doris::ErrorCode::INVALID_ARGUMENT);
+            }
             break;
         }
         case TYPE_DECIMAL128I: {
@@ -176,11 +195,17 @@ void VLiteral::init(const TExprNode& node) {
             DCHECK(node.__isset.decimal_literal);
             DataTypePtr type_ptr = create_decimal(node.type.types[0].scalar_type.precision,
                                                   node.type.types[0].scalar_type.scale, false);
-            auto val = typeid_cast<const DataTypeDecimal<Decimal128I>*>(type_ptr.get())
-                               ->parse_from_string(node.decimal_literal.value);
-            auto scale =
-                    typeid_cast<const DataTypeDecimal<Decimal128I>*>(type_ptr.get())->get_scale();
-            field = DecimalField<Decimal128I>(val, scale);
+            Decimal128I val;
+            if (typeid_cast<const DataTypeDecimal<Decimal128I>*>(type_ptr.get())
+                        ->parse_from_string(node.decimal_literal.value, &val)) {
+                auto scale = typeid_cast<const DataTypeDecimal<Decimal128I>*>(type_ptr.get())
+                                     ->get_scale();
+                field = DecimalField<Decimal128I>(val, scale);
+            } else {
+                throw Exception(fmt::format("Invalid value: {} for type {}",
+                                            node.decimal_literal.value, type_ptr->get_name()),
+                                doris::ErrorCode::INVALID_ARGUMENT);
+            }
             break;
         }
         default: {
diff --git a/be/src/vec/io/io_helper.h b/be/src/vec/io/io_helper.h
index 990b67fd62..4c2b9105f4 100644
--- a/be/src/vec/io/io_helper.h
+++ b/be/src/vec/io/io_helper.h
@@ -339,16 +339,18 @@ bool read_decimal_text_impl(T& x, ReadBuffer& buf, UInt32 precision, UInt32 scal
                 (const char*)buf.position(), buf.count(), precision, scale, &result);
         // only to match the is_all_read() check to prevent return null
         buf.position() = buf.end();
-        return result != StringParser::PARSE_FAILURE;
+        return result == StringParser::PARSE_SUCCESS || result == StringParser::PARSE_UNDERFLOW;
     } else {
-        auto dv = binary_cast<Int128, DecimalV2Value>(x.value);
-        auto ans = dv.parse_from_str((const char*)buf.position(), buf.count()) == 0;
+        StringParser::ParseResult result = StringParser::PARSE_SUCCESS;
+
+        x.value = StringParser::string_to_decimal<__int128>(buf.position(), buf.count(),
+                                                            DecimalV2Value::PRECISION,
+                                                            DecimalV2Value::SCALE, &result);
 
         // only to match the is_all_read() check to prevent return null
         buf.position() = buf.end();
 
-        x.value = dv.value();
-        return ans;
+        return result == StringParser::PARSE_SUCCESS || result == StringParser::PARSE_UNDERFLOW;
     }
 }
 
diff --git a/be/src/vec/sink/vtablet_sink.cpp b/be/src/vec/sink/vtablet_sink.cpp
index 016bf1c86a..b1a4e17545 100644
--- a/be/src/vec/sink/vtablet_sink.cpp
+++ b/be/src/vec/sink/vtablet_sink.cpp
@@ -25,6 +25,8 @@
 #include "util/time.h"
 #include "vec/columns/column_array.h"
 #include "vec/core/block.h"
+#include "vec/core/types.h"
+#include "vec/data_types/data_type_decimal.h"
 #include "vec/exprs/vexpr.h"
 #include "vec/exprs/vexpr_context.h"
 
@@ -651,6 +653,33 @@ Status VOlapTableSink::close(RuntimeState* state, Status exec_status) {
     return OlapTableSink::close(state, exec_status);
 }
 
+template <typename DecimalType, bool IsMin>
+DecimalType VOlapTableSink::_get_decimalv3_min_or_max(const TypeDescriptor& type) {
+    std::map<int, typename DecimalType::NativeType>* pmap = nullptr;
+    if constexpr (std::is_same_v<DecimalType, vectorized::Decimal32>) {
+        pmap = IsMin ? &_min_decimal32_val : &_max_decimal32_val;
+    } else if constexpr (std::is_same_v<DecimalType, vectorized::Decimal64>) {
+        pmap = IsMin ? &_min_decimal64_val : &_max_decimal64_val;
+    } else {
+        pmap = IsMin ? &_min_decimal128_val : &_max_decimal128_val;
+    }
+
+    // found
+    auto iter = pmap->find(type.precision);
+    if (iter != pmap->end()) {
+        return iter->second;
+    }
+
+    typename DecimalType::NativeType value;
+    if constexpr (IsMin) {
+        value = vectorized::min_decimal_value<DecimalType>(type.precision);
+    } else {
+        value = vectorized::max_decimal_value<DecimalType>(type.precision);
+    }
+    pmap->emplace(type.precision, value);
+    return value;
+}
+
 Status VOlapTableSink::_validate_column(RuntimeState* state, const TypeDescriptor& type,
                                         bool is_nullable, vectorized::ColumnPtr column,
                                         size_t slot_index, Bitmap* filter_bitmap,
@@ -783,6 +812,47 @@ Status VOlapTableSink::_validate_column(RuntimeState* state, const TypeDescripto
         }
         break;
     }
+    case TYPE_DECIMAL32: {
+#define CHECK_VALIDATION_FOR_DECIMALV3(ColumnDecimalType, DecimalType)                             \
+    auto column_decimal = const_cast<vectorized::ColumnDecimal<vectorized::ColumnDecimalType>*>(   \
+            assert_cast<const vectorized::ColumnDecimal<vectorized::ColumnDecimalType>*>(          \
+                    real_column_ptr.get()));                                                       \
+    for (size_t j = 0; j < column->size(); ++j) {                                                  \
+        auto row = rows ? (*rows)[j] : j;                                                          \
+        if (row == last_invalid_row) {                                                             \
+            continue;                                                                              \
+        }                                                                                          \
+        if (need_to_validate(j, row)) {                                                            \
+            auto dec_val = column_decimal->get_data()[j];                                          \
+            bool invalid = false;                                                                  \
+            const auto& max_decimal =                                                              \
+                    _get_decimalv3_min_or_max<vectorized::DecimalType, false>(type);               \
+            const auto& min_decimal =                                                              \
+                    _get_decimalv3_min_or_max<vectorized::DecimalType, true>(type);                \
+            if (dec_val > max_decimal || dec_val < min_decimal) {                                  \
+                fmt::format_to(error_msg, "{}", "decimal value is not valid for definition");      \
+                fmt::format_to(error_msg, ", value={}", dec_val);                                  \
+                fmt::format_to(error_msg, ", precision={}, scale={}", type.precision, type.scale); \
+                fmt::format_to(error_msg, ", min={}, max={}; ", min_decimal, max_decimal);         \
+                invalid = true;                                                                    \
+            }                                                                                      \
+            if (invalid) {                                                                         \
+                last_invalid_row = row;                                                            \
+                RETURN_IF_ERROR(set_invalid_and_append_error_msg(row));                            \
+            }                                                                                      \
+        }                                                                                          \
+    }
+        CHECK_VALIDATION_FOR_DECIMALV3(Decimal32, Decimal32);
+        break;
+    }
+    case TYPE_DECIMAL64: {
+        CHECK_VALIDATION_FOR_DECIMALV3(Decimal64, Decimal64);
+        break;
+    }
+    case TYPE_DECIMAL128I: {
+        CHECK_VALIDATION_FOR_DECIMALV3(Decimal128I, Decimal128);
+        break;
+    }
     case TYPE_ARRAY: {
         const auto column_array =
                 assert_cast<const vectorized::ColumnArray*>(real_column_ptr.get());
diff --git a/be/src/vec/sink/vtablet_sink.h b/be/src/vec/sink/vtablet_sink.h
index 857c43722b..8e7229349b 100644
--- a/be/src/vec/sink/vtablet_sink.h
+++ b/be/src/vec/sink/vtablet_sink.h
@@ -101,6 +101,10 @@ private:
     // set stop_processing if we want to stop the whole process now.
     Status _validate_data(RuntimeState* state, vectorized::Block* block, Bitmap* filter_bitmap,
                           int* filtered_rows, bool* stop_processing);
+
+    template <typename DecimalType, bool IsMin>
+    DecimalType _get_decimalv3_min_or_max(const TypeDescriptor& type);
+
     Status _validate_column(RuntimeState* state, const TypeDescriptor& type, bool is_nullable,
                             vectorized::ColumnPtr column, size_t slot_index, Bitmap* filter_bitmap,
                             bool* stop_processing, fmt::memory_buffer& error_prefix,
@@ -114,6 +118,13 @@ private:
                        const VOlapTablePartition** partition, uint32_t& tablet_index,
                        bool& stop_processing, bool& is_continue);
 
+    std::map<int, int32_t> _max_decimal32_val;
+    std::map<int, int32_t> _min_decimal32_val;
+    std::map<int, int64_t> _max_decimal64_val;
+    std::map<int, int64_t> _min_decimal64_val;
+    std::map<int, int128_t> _max_decimal128_val;
+    std::map<int, int128_t> _min_decimal128_val;
+
     VOlapTablePartitionParam* _vpartition = nullptr;
     std::vector<vectorized::VExprContext*> _output_vexpr_ctxs;
 };
diff --git a/fe/fe-common/src/main/java/org/apache/doris/catalog/ScalarType.java b/fe/fe-common/src/main/java/org/apache/doris/catalog/ScalarType.java
index c555e9a6ff..263751f5cd 100644
--- a/fe/fe-common/src/main/java/org/apache/doris/catalog/ScalarType.java
+++ b/fe/fe-common/src/main/java/org/apache/doris/catalog/ScalarType.java
@@ -1059,15 +1059,17 @@ public class ScalarType extends Type {
     }
 
     public static ScalarType getAssignmentCompatibleDecimalV2Type(ScalarType t1, ScalarType t2) {
-        int targetPrecision = Math.max(t1.decimalPrecision(), t2.decimalPrecision());
         int targetScale = Math.max(t1.decimalScale(), t2.decimalScale());
+        int targetPrecision = Math.max(t1.decimalPrecision() - t1.decimalScale(), t2.decimalPrecision()
+                - t2.decimalScale()) + targetScale;
         return ScalarType.createDecimalType(PrimitiveType.DECIMALV2,
                 targetPrecision, targetScale);
     }
 
     public static ScalarType getAssignmentCompatibleDecimalV3Type(ScalarType t1, ScalarType t2) {
-        int targetPrecision = Math.max(t1.decimalPrecision(), t2.decimalPrecision());
         int targetScale = Math.max(t1.decimalScale(), t2.decimalScale());
+        int targetPrecision = Math.max(t1.decimalPrecision() - t1.decimalScale(), t2.decimalPrecision()
+                - t2.decimalScale()) + targetScale;
         return ScalarType.createDecimalV3Type(targetPrecision, targetScale);
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ArithmeticExpr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ArithmeticExpr.java
index a6ab76df9c..503b506990 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ArithmeticExpr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ArithmeticExpr.java
@@ -361,13 +361,14 @@ public class ArithmeticExpr extends Expr {
         if (pt1 == PrimitiveType.DOUBLE || pt2 == PrimitiveType.DOUBLE) {
             return Type.DOUBLE;
         } else if (pt1 == PrimitiveType.DECIMALV2 || pt2 == PrimitiveType.DECIMALV2) {
-            return Type.MAX_DECIMALV2_TYPE;
+            return pt1 == PrimitiveType.DECIMALV2 && pt2 == PrimitiveType.DECIMALV2
+                    ? Type.MAX_DECIMALV2_TYPE : Type.DOUBLE;
         } else if (pt1 == PrimitiveType.DECIMAL32 || pt2 == PrimitiveType.DECIMAL32) {
-            return Type.DECIMAL32;
+            return pt1 == PrimitiveType.DECIMAL32 && pt2 == PrimitiveType.DECIMAL32 ? Type.DECIMAL32 : Type.DOUBLE;
         } else if (pt1 == PrimitiveType.DECIMAL64 || pt2 == PrimitiveType.DECIMAL64) {
-            return Type.DECIMAL64;
+            return pt1 == PrimitiveType.DECIMAL64 && pt2 == PrimitiveType.DECIMAL64 ? Type.DECIMAL64 : Type.DOUBLE;
         } else if (pt1 == PrimitiveType.DECIMAL128 || pt2 == PrimitiveType.DECIMAL128) {
-            return Type.DECIMAL128;
+            return pt1 == PrimitiveType.DECIMAL128 && pt2 == PrimitiveType.DECIMAL128 ? Type.DECIMAL128 : Type.DOUBLE;
         } else if (pt1 == PrimitiveType.LARGEINT || pt2 == PrimitiveType.LARGEINT) {
             return Type.LARGEINT;
         } else {
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 8306c6ff14..169a9fdfe3 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
@@ -2074,9 +2074,9 @@ public class QueryPlanTest extends TestWithFeService {
                 + "PROPERTIES (\n"
                 + "\"replication_allocation\" = \"tag.location.default: 1\"\n"
                 + ")");
-        String sql = "explain insert into test.decimal_tb select 1, 10, 1, 1, 1;";
+        String sql = "explain insert into test.decimal_tb select 1, 1, 1, 1, 1;";
         String explainString = getSQLPlanOrErrorMsg(sql);
-        Assert.assertTrue(explainString.contains("1 | 10 | 1 | 1 | 1"));
+        Assert.assertTrue(explainString.contains("1 | 1 | 1 | 1 | 1"));
     }
 
     @Test
diff --git a/regression-test/data/datatype_p0/decimalv3/test_overflow.out b/regression-test/data/datatype_p0/decimalv3/test_overflow.out
deleted file mode 100644
index c9b9873cd7..0000000000
--- a/regression-test/data/datatype_p0/decimalv3/test_overflow.out
+++ /dev/null
@@ -1,19 +0,0 @@
--- This file is automatically generated. You should know what you did if you want to edit this
--- !select_all --
-11111111111111111111.100000000000000000	11111111111111111111.200000000000000000	11111111111111111111.300000000000000000	1.1000000000000000000000000000000000000	1.2000000000000000000000000000000000000	1.3000000000000000000000000000000000000	9
-
--- !select_check_overflow1 --
-\N	\N	\N	99999999999999999999.900000000000000000	\N
-
--- !select_check_overflow2 --
-1.1000000000000000000000000000000000000	111111111111111111111.000000000000000000	\N
-
--- !select_check_overflow3 --
-11111111111111111111.100000000000000000	\N
-
--- !select_not_check_overflow1 --
-99.999999999999999999999999999999999999	99.999999999999999999999999999999999999	1.1111111111111111E21	99999999999999999999.900000000000000000	99999999999999999999.999999999999999999
-
--- !select_not_check_overflow2 --
-1.1000000000000000000000000000000000000	111111111111111111111.000000000000000000	-15.9141183460469231731687303715884105728
-
diff --git a/regression-test/data/insert_p0/insert.out b/regression-test/data/insert_p0/insert.out
index 39756a4193..2ef1dbbbb0 100644
--- a/regression-test/data/insert_p0/insert.out
+++ b/regression-test/data/insert_p0/insert.out
@@ -97,108 +97,108 @@
 4	1.1	a	2016-01-01	2015-01-01T03:15:16	false	-123.22	-123456789012345678.012345678
 4	1.1	a	2016-01-01	2015-01-01T03:15:16	false	-123.22	-123456789012345678.012345678
 4	1.1	a	2016-01-01	2015-01-01T03:15:16	false	-123.22	-123456789012345678.012345678
-12	12.25	String1	1999-01-08	1999-01-08T02:05:06	true	123.22	123456789012345678.900123456
-12	12.25	String1	1999-01-08	1999-01-08T02:05:06	true	123.22	123456789012345678.900123456
-12	12.25	String1	1999-01-08	1999-01-08T02:05:06	true	123.22	123456789012345678.900123456
-12	12.25	String1	1999-01-08	1999-01-08T02:05:06	true	123.22	123456789012345678.900123456
-12	12.25	String1	1999-01-08	1999-01-08T02:05:06	true	123.22	123456789012345678.900123456
-12	12.25	String1	1999-01-08	1999-01-08T02:05:06	true	123.22	123456789012345678.900123456
-12	12.25	String1	1999-01-08	1999-01-08T02:05:06	true	123.22	123456789012345678.900123456
-25	55.52	test	1952-01-05	1989-01-08T04:05:06	false	321.21	-123456789012345678.900123456
-25	55.52	test	1952-01-05	1989-01-08T04:05:06	false	321.21	-123456789012345678.900123456
-25	55.52	test	1952-01-05	1989-01-08T04:05:06	false	321.21	-123456789012345678.900123456
-25	55.52	test	1952-01-05	1989-01-08T04:05:06	false	321.21	-123456789012345678.900123456
+12	12.25	String1	1999-01-08	1999-01-08T02:05:06	true	123.22	\N
+12	12.25	String1	1999-01-08	1999-01-08T02:05:06	true	123.22	\N
+12	12.25	String1	1999-01-08	1999-01-08T02:05:06	true	123.22	\N
+12	12.25	String1	1999-01-08	1999-01-08T02:05:06	true	123.22	\N
+12	12.25	String1	1999-01-08	1999-01-08T02:05:06	true	123.22	\N
+12	12.25	String1	1999-01-08	1999-01-08T02:05:06	true	123.22	\N
+12	12.25	String1	1999-01-08	1999-01-08T02:05:06	true	123.22	\N
+25	55.52	test	1952-01-05	1989-01-08T04:05:06	false	321.21	\N
+25	55.52	test	1952-01-05	1989-01-08T04:05:06	false	321.21	\N
+25	55.52	test	1952-01-05	1989-01-08T04:05:06	false	321.21	\N
+25	55.52	test	1952-01-05	1989-01-08T04:05:06	false	321.21	\N
 50	9.1	abc	2014-01-01	2015-01-01T03:15:16	true	123.22	123456789012345678.012345678
-60	27.25	String1	1999-01-08	1999-01-08T02:05:06	true	61.61	0
-60	27.25	String1	1999-01-08	1999-01-08T02:05:06	true	61.61	0
-100	9.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	10.01	2
-100	9.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	10.01	2
-100	9.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	10.01	2
-100	9.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	10.01	2
-100	9.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	10.01	2
-100	9.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	10.01	2
-100	12.25	testing	1949-07-08	2002-01-07T01:05:06	true	-393.22	-987654321098765432.109876543
-100	12.25	testing	1949-07-08	2002-01-07T01:05:06	true	-393.22	-987654321098765432.109876543
-100	12.25	testing	1949-07-08	2002-01-07T01:05:06	true	-393.22	-987654321098765432.109876543
-100	12.25	testing	1949-07-08	2002-01-07T01:05:06	true	-393.22	-987654321098765432.109876543
-100	12.25	testing	1949-07-08	2002-01-07T01:05:06	true	-393.22	-987654321098765432.109876543
-100	12.25	testing	1949-07-08	2002-01-07T01:05:06	true	-393.22	-987654321098765432.109876543
-100	12.8788	string1	1922-04-02	2010-01-02T02:05:06	true	999.99	-999999999999999999.999999999
-100	12.8788	string1	1922-04-02	2010-01-02T02:05:06	true	999.99	-999999999999999999.999999999
-100	12.8788	string1	1922-04-02	2010-01-02T02:05:06	true	999.99	-999999999999999999.999999999
-100	12.8788	string1	1922-04-02	2010-01-02T02:05:06	true	999.99	-999999999999999999.999999999
-100	12.8788	string1	1922-04-02	2010-01-02T02:05:06	true	999.99	-999999999999999999.999999999
-100	12.8788	string1	1922-04-02	2010-01-02T02:05:06	true	999.99	-999999999999999999.999999999
-100	99.8777	AGAIN	1987-04-09	2010-01-02T04:03:06	true	0	0
-100	99.8777	AGAIN	1987-04-09	2010-01-02T04:03:06	true	0	0
-100	99.8777	AGAIN	1987-04-09	2010-01-02T04:03:06	true	0	0
-100	99.8777	AGAIN	1987-04-09	2010-01-02T04:03:06	true	0	0
-125	70.52000000000001	test	1952-01-05	1989-01-08T04:05:06	false	160.61	0
-125	70.52000000000001	test	1952-01-05	1989-01-08T04:05:06	false	160.61	0
-500	24.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	5.01	2
-500	24.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	5.01	2
-500	27.25	testing	1949-07-08	2002-01-07T01:05:06	true	-196.61	0
-500	27.25	testing	1949-07-08	2002-01-07T01:05:06	true	-196.61	0
-500	27.8788	string1	1922-04-02	2010-01-02T02:05:06	true	500	0
-500	27.8788	string1	1922-04-02	2010-01-02T02:05:06	true	500	0
-500	114.8777	AGAIN	1987-04-09	2010-01-02T04:03:06	true	0	0
-500	114.8777	AGAIN	1987-04-09	2010-01-02T04:03:06	true	0	0
-964	0.245	Again	1936-02-08	2005-01-09T04:05:06	false	333.82	987654321098765432.109876543
-964	0.245	Again	1936-02-08	2005-01-09T04:05:06	false	333.82	987654321098765432.109876543
-964	0.245	Again	1936-02-08	2005-01-09T04:05:06	false	333.82	987654321098765432.109876543
-964	0.245	Again	1936-02-08	2005-01-09T04:05:06	false	333.82	987654321098765432.109876543
-964	0.245	Again	1936-02-08	2005-01-09T04:05:06	false	333.82	987654321098765432.109876543
-964	0.245	Again	1936-02-08	2005-01-09T04:05:06	false	333.82	987654321098765432.109876543
-4820	15.245	Again	1936-02-08	2005-01-09T04:05:06	false	166.91	0
-4820	15.245	Again	1936-02-08	2005-01-09T04:05:06	false	166.91	0
+60	27.25	String1	1999-01-08	1999-01-08T02:05:06	true	61.61	\N
+60	27.25	String1	1999-01-08	1999-01-08T02:05:06	true	61.61	\N
+100	9.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	10.01	2.000000000
+100	9.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	10.01	2.000000000
+100	9.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	10.01	2.000000000
+100	9.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	10.01	2.000000000
+100	9.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	10.01	2.000000000
+100	9.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	10.01	2.000000000
+100	12.25	testing	1949-07-08	2002-01-07T01:05:06	true	-393.22	\N
+100	12.25	testing	1949-07-08	2002-01-07T01:05:06	true	-393.22	\N
+100	12.25	testing	1949-07-08	2002-01-07T01:05:06	true	-393.22	\N
+100	12.25	testing	1949-07-08	2002-01-07T01:05:06	true	-393.22	\N
+100	12.25	testing	1949-07-08	2002-01-07T01:05:06	true	-393.22	\N
+100	12.25	testing	1949-07-08	2002-01-07T01:05:06	true	-393.22	\N
+100	12.8788	string1	1922-04-02	2010-01-02T02:05:06	true	999.99	\N
+100	12.8788	string1	1922-04-02	2010-01-02T02:05:06	true	999.99	\N
+100	12.8788	string1	1922-04-02	2010-01-02T02:05:06	true	999.99	\N
+100	12.8788	string1	1922-04-02	2010-01-02T02:05:06	true	999.99	\N
+100	12.8788	string1	1922-04-02	2010-01-02T02:05:06	true	999.99	\N
+100	12.8788	string1	1922-04-02	2010-01-02T02:05:06	true	999.99	\N
+100	99.8777	AGAIN	1987-04-09	2010-01-02T04:03:06	true	0.00	0E-9
+100	99.8777	AGAIN	1987-04-09	2010-01-02T04:03:06	true	0.00	0E-9
+100	99.8777	AGAIN	1987-04-09	2010-01-02T04:03:06	true	0.00	0E-9
+100	99.8777	AGAIN	1987-04-09	2010-01-02T04:03:06	true	0.00	0E-9
+125	70.52000000000001	test	1952-01-05	1989-01-08T04:05:06	false	160.61	\N
+125	70.52000000000001	test	1952-01-05	1989-01-08T04:05:06	false	160.61	\N
+500	24.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	5.01	2.000000000
+500	24.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	5.01	2.000000000
+500	27.25	testing	1949-07-08	2002-01-07T01:05:06	true	-196.61	\N
+500	27.25	testing	1949-07-08	2002-01-07T01:05:06	true	-196.61	\N
+500	27.8788	string1	1922-04-02	2010-01-02T02:05:06	true	500.00	\N
+500	27.8788	string1	1922-04-02	2010-01-02T02:05:06	true	500.00	\N
+500	114.8777	AGAIN	1987-04-09	2010-01-02T04:03:06	true	0.00	0E-9
+500	114.8777	AGAIN	1987-04-09	2010-01-02T04:03:06	true	0.00	0E-9
+964	0.245	Again	1936-02-08	2005-01-09T04:05:06	false	333.82	\N
+964	0.245	Again	1936-02-08	2005-01-09T04:05:06	false	333.82	\N
+964	0.245	Again	1936-02-08	2005-01-09T04:05:06	false	333.82	\N
+964	0.245	Again	1936-02-08	2005-01-09T04:05:06	false	333.82	\N
+964	0.245	Again	1936-02-08	2005-01-09T04:05:06	false	333.82	\N
+964	0.245	Again	1936-02-08	2005-01-09T04:05:06	false	333.82	\N
+4820	15.245	Again	1936-02-08	2005-01-09T04:05:06	false	166.91	\N
+4820	15.245	Again	1936-02-08	2005-01-09T04:05:06	false	166.91	\N
 5000	67.87	testing	\N	2010-01-02T04:03:06	\N	\N	\N
 5000	67.87	testing	\N	2010-01-02T04:03:06	\N	\N	\N
 5000	67.87	testing	\N	2010-01-02T04:03:06	\N	\N	\N
 5000	67.87	testing	\N	2010-01-02T04:03:06	\N	\N	\N
-5252	12.25	sample	1987-04-09	2010-01-02T04:03:06	true	123	1
-5252	12.25	sample	1987-04-09	2010-01-02T04:03:06	true	123	1
-5252	12.25	sample	1987-04-09	2010-01-02T04:03:06	true	123	1
-5252	12.25	sample	1987-04-09	2010-01-02T04:03:06	true	123	1
-5252	12.25	sample	1987-04-09	2010-01-02T04:03:06	true	123	1
-5252	12.25	sample	1987-04-09	2010-01-02T04:03:06	true	123	1
-5748	67.87	Sample	1987-04-06	2010-01-02T04:03:06	true	181.18	0
-5748	67.87	Sample	1987-04-06	2010-01-02T04:03:06	true	181.18	0
-5748	67.87	Sample	1987-04-06	2010-01-02T04:03:06	true	181.18	0
-5748	67.87	Sample	1987-04-06	2010-01-02T04:03:06	true	181.18	0
-5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	-999.99	0
-5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	-999.99	0
-5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	-999.99	0
-5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	-999.99	0
-5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	123456789012345678.900123456
-5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	123456789012345678.900123456
-5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	123456789012345678.900123456
-5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	123456789012345678.900123456
-5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	123456789012345678.900123456
-5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	123456789012345678.900123456
-5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	123456789012345678.900123456
-5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	123456789012345678.900123456
+5252	12.25	sample	1987-04-09	2010-01-02T04:03:06	true	123.00	1.000000000
+5252	12.25	sample	1987-04-09	2010-01-02T04:03:06	true	123.00	1.000000000
+5252	12.25	sample	1987-04-09	2010-01-02T04:03:06	true	123.00	1.000000000
+5252	12.25	sample	1987-04-09	2010-01-02T04:03:06	true	123.00	1.000000000
+5252	12.25	sample	1987-04-09	2010-01-02T04:03:06	true	123.00	1.000000000
+5252	12.25	sample	1987-04-09	2010-01-02T04:03:06	true	123.00	1.000000000
+5748	67.87	Sample	1987-04-06	2010-01-02T04:03:06	true	181.18	0E-9
+5748	67.87	Sample	1987-04-06	2010-01-02T04:03:06	true	181.18	0E-9
+5748	67.87	Sample	1987-04-06	2010-01-02T04:03:06	true	181.18	0E-9
+5748	67.87	Sample	1987-04-06	2010-01-02T04:03:06	true	181.18	0E-9
+5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	-999.99	0E-9
+5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	-999.99	0E-9
+5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	-999.99	0E-9
+5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	-999.99	0E-9
+5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	\N
+5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	\N
+5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	\N
+5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	\N
+5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	\N
+5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	\N
+5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	\N
+5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	\N
 6000	\N	\N	1987-04-06	\N	true	\N	\N
 6000	\N	\N	1987-04-06	\N	true	\N	\N
 6000	\N	\N	1987-04-06	\N	true	\N	\N
 6000	\N	\N	1987-04-06	\N	true	\N	\N
-8996	98.8777	again	1987-04-09	2010-01-02T04:03:06	false	-0.01	999999999999999999.999999999
-8996	98.8777	again	1987-04-09	2010-01-02T04:03:06	false	-0.01	999999999999999999.999999999
-8996	98.8777	again	1987-04-09	2010-01-02T04:03:06	false	-0.01	999999999999999999.999999999
-8996	98.8777	again	1987-04-09	2010-01-02T04:03:06	false	-0.01	999999999999999999.999999999
+8996	98.8777	again	1987-04-09	2010-01-02T04:03:06	false	-0.01	\N
+8996	98.8777	again	1987-04-09	2010-01-02T04:03:06	false	-0.01	\N
+8996	98.8777	again	1987-04-09	2010-01-02T04:03:06	false	-0.01	\N
+8996	98.8777	again	1987-04-09	2010-01-02T04:03:06	false	-0.01	\N
 25000	82.87	testing	\N	2010-01-02T04:03:06	\N	\N	\N
 25000	82.87	testing	\N	2010-01-02T04:03:06	\N	\N	\N
-26260	27.25	sample	1987-04-09	2010-01-02T04:03:06	true	61.5	1
-26260	27.25	sample	1987-04-09	2010-01-02T04:03:06	true	61.5	1
-28740	82.87	Sample	1987-04-06	2010-01-02T04:03:06	true	90.59	0
-28740	82.87	Sample	1987-04-06	2010-01-02T04:03:06	true	90.59	0
-28740	82.87	sample	1987-04-06	2010-01-02T04:03:06	true	-500	0
-28740	82.87	sample	1987-04-06	2010-01-02T04:03:06	true	-500	0
-28740	82.87	sample	1987-04-06	2010-01-02T04:03:06	true	90.59	0
-28740	82.87	sample	1987-04-06	2010-01-02T04:03:06	true	90.59	0
-28740	82.87	sample	1987-04-06	2010-01-02T04:03:06	true	90.59	0
-28740	82.87	sample	1987-04-06	2010-01-02T04:03:06	true	90.59	0
+26260	27.25	sample	1987-04-09	2010-01-02T04:03:06	true	61.50	1.000000000
+26260	27.25	sample	1987-04-09	2010-01-02T04:03:06	true	61.50	1.000000000
+28740	82.87	Sample	1987-04-06	2010-01-02T04:03:06	true	90.59	0E-9
+28740	82.87	Sample	1987-04-06	2010-01-02T04:03:06	true	90.59	0E-9
+28740	82.87	sample	1987-04-06	2010-01-02T04:03:06	true	-500.00	0E-9
+28740	82.87	sample	1987-04-06	2010-01-02T04:03:06	true	-500.00	0E-9
+28740	82.87	sample	1987-04-06	2010-01-02T04:03:06	true	90.59	\N
+28740	82.87	sample	1987-04-06	2010-01-02T04:03:06	true	90.59	\N
+28740	82.87	sample	1987-04-06	2010-01-02T04:03:06	true	90.59	\N
+28740	82.87	sample	1987-04-06	2010-01-02T04:03:06	true	90.59	\N
 30000	\N	\N	1987-04-06	\N	true	\N	\N
 30000	\N	\N	1987-04-06	\N	true	\N	\N
-44980	113.8777	again	1987-04-09	2010-01-02T04:03:06	false	-0.01	0
-44980	113.8777	again	1987-04-09	2010-01-02T04:03:06	false	-0.01	0
+44980	113.8777	again	1987-04-09	2010-01-02T04:03:06	false	-0.01	\N
+44980	113.8777	again	1987-04-09	2010-01-02T04:03:06	false	-0.01	\N
 
diff --git a/regression-test/data/query_p0/operator/test_arithmetic_operators.out b/regression-test/data/query_p0/operator/test_arithmetic_operators.out
index 60e54c7eef..c1bf2e5b4f 100644
--- a/regression-test/data/query_p0/operator/test_arithmetic_operators.out
+++ b/regression-test/data/query_p0/operator/test_arithmetic_operators.out
@@ -5,40 +5,35 @@
 3	3670635	5536	10989	1
 
 -- !arith_op2 --
-1	2.0	123.123	12312300.000000000
-2	3.0	1243.500	124350000.000000000
-3	4.0	24453.325	2445332500.000000000
+1	2.0	123.123	1.23123E7
+2	3.0	1243.500	1.2435E8
+3	4.0	24453.325	2.4453325E9
 
 -- !arith_op3 --
-1	123.123	244891.647000000
-2	1243.500	2469591.000000000
-3	24453.325	48637663.425000000
+1	123.123	244891.647
+2	1243.500	2469591.0
+3	24453.325	4.8637663425000004E7
 
 -- !arith_op4 --
-1	123.123	12.3123	779.737982640	12596.337364196777	198.9
-2	1243.500	25203.258	981432.353193984	1567450.5	40252.248
-3	24453.325	1.930467742125E9	89352450.776970036	7267806.0	1.57021605E8
+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.312300000	0.010000000000000002	0.633300019
-2	124.350000000	2.0268	78.924998246
-3	2445.332500000	7894.5	365.400005018
+1	12.312300000	0.010000000000000002	0.6333000183105469
+2	124.350000000	2.0268	78.92500000000001
+3	2445.332500000	7894.5	365.40000000000003
 
 -- !arith_op6 --
-1	-198.900000000	-100.100000000	-1101190.200000000	-12.312300000	-0.010000000000000002	-0.633300019
-2	-198.600000000	-100.100000000	-1101190.300000000	-124.350000000	-2.0268	-78.924998246
-3	-198.900000000	-100.200000000	-1101190.500000000	-2445.332500000	-7894.5	-365.400005018
+1	-198.9	-100.10000000000001	-1101190.2	-12.312300000	-0.010000000000000002	-0.6333000183105469
+2	-198.60000000000002	-100.10000000000001	-1101190.3	-124.350000000	-2.0268	-78.92500000000001
+3	-198.9	-100.2	-1101190.5	-2445.332500000	-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
@@ -55,231 +50,51 @@
 3	24453325000.000000000	7.8945E10	3.654E9
 
 -- !arith_op12 --
-1	0.100000000	198.900000000	100.100000000	1101190.200000000	12.312300000	0.010000000000000002	0.633300019
-2	0.200000000	198.600000000	100.100000000	1101190.300000000	124.350000000	2.0268	78.924998246
-3	0.300000000	198.900000000	100.200000000	1101190.500000000	2445.332500000	7894.5	365.400005018
+1	0.1	198.9	100.10000000000001	1101190.2	12.312300000	0.010000000000000002	0.6333000183105469
+2	0.2	198.60000000000002	100.10000000000001	1101190.3	124.350000000	2.0268	78.92500000000001
+3	0.30000000000000004	198.9	100.2	1101190.5	2445.332500000	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
+1	0.1	198.9	100.1	1101190.2	12.3123	0.01	0.6333000183105468
+2	0.2	198.6	100.1	1101190.3	124.35	2.0268	78.925
+3	0.3	198.9	100.2	1101190.5	2445.3325	7894.5	365.4
 
 -- !arith_op14 --
-1	0.900000000	1988.900000000	1000.900000000	11011901.900000000	123.023000000	0.0	6.233000192
-2	1.900000000	1985.900000000	1000.900000000	11011902.900000000	1243.400000000	20.168	789.149982464
-3	2.900000000	1988.900000000	1001.900000000	11011904.900000000	24453.225000000	78944.9	3653.900050176
+1	0.9	1988.9	1000.9	1.10119019E7	123.023000000	0.0	6.233000183105469
+2	1.9	1985.9	1000.9	1.10119029E7	1243.400000000	20.168	789.15
+3	2.9	1988.9	1001.9	1.10119049E7	24453.225000000	78944.9	3653.9
 
 -- !arith_op15 --
-1	1.100000000	1989.100000000	1001.100000000	11011902.100000000	123.223000000	0.2	6.433000192
-2	2.100000000	1986.100000000	1001.100000000	11011903.100000000	1243.600000000	20.368000000000002	789.349982464
-3	3.100000000	1989.100000000	1002.100000000	11011905.100000000	24453.425000000	78945.1	3654.100050176
+1	1.1	1989.1	1001.1	1.10119021E7	123.223000000	0.2	6.433000183105468
+2	2.1	1986.1	1001.1	1.10119031E7	1243.600000000	20.368000000000002	789.35
+3	3.1	1989.1	1002.1	1.10119051E7	24453.425000000	78945.1	3654.1
 
 -- !arith_op16 --
-11	1999.000000000	1002.600000000	11011902	117.123000000	-234.56	6.333000183105469
-12	1996.000000000	1002.600000000	11011903	1237.500000000	-214.392	789.25
-13	1999.000000000	1003.600000000	11011905	24447.325000000	78710.34	3654.0
+11	1999.0	1002.6	11011902	117.123	-234.56	6.333000183105469
+12	1996.0	1002.6	11011903	1237.5	-214.392	789.25
+13	1999.0	1003.6	11011905	24447.325	78710.34	3654.0
 
 -- !arith_op17 --
 
 -- !arith_op18 --
-245136538.647000000
-4944121182.000000000
-146204816255.550000000
+2.45136538647E8
+4.944121182E9
+1.4620481625555002E11
 
 -- !arith_op19 --
 1.5524497441372922E8
 7.908872842616545E13
-4.217497670730171E19
+4.217497670730172E19
 
 -- !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.123000000	124.123000000	-122.123000000	0.008121959
-2487.000000000	1245.500000000	-1241.500000000	0.001608363
-73359.975000000	24456.325000000	-24450.325000000	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.647000000	2112.123000000	1865.877000000	16.154577130
-2469591.000000000	3229.500000000	742.500000000	1.597104946
-48637663.425000000	26442.325000000	-22464.325000000	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.123000000	1124.123000000	877.877000000	8.130081301
-1244743.500000000	2244.500000000	-242.500000000	0.804985927
-24502231.650000000	25455.325000000	-23451.325000000	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.123000000	124.123000000	122.123000000	123.123000000
-2487.000000000	1245.500000000	1241.500000000	621.750000000
-73359.975000000	24456.325000000	24450.325000000	8151.108333333
-
--- !arith_op21 --
-244891.647000000	2112.123000000	-1865.877000000	0.061901961
-2469591.000000000	3229.500000000	-742.500000000	0.626132931
-48637663.425000000	26442.325000000	22464.325000000	12.294281046
-
--- !arith_op21 --
-123246.123000000	1124.123000000	-877.877000000	0.123000000
-1244743.500000000	2244.500000000	242.500000000	1.242257742
-24502231.650000000	25455.325000000	23451.325000000	24.404515968
-
--- !arith_op21 --
-15159.273129000	246.246000000	0E-9	1.000000000
-1546292.250000000	2487.000000000	0E-9	1.000000000
-597965103.555625000	48906.650000000	0E-9	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.737982640	129.456000192	116.789999808	19.44149635878031
-981432.353193984	2032.749982464	454.250017536	1.5755464048146974
-89352450.776970036	28107.325050176	20799.324949824	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.737982640	129.456000192	-116.789999808	0.051436369996714415
-981432.353193984	2032.749982464	-454.250017536	0.6347004422999598
-89352450.776970036	28107.325050176	-20799.324949824	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.210000000
-1990.210000000
-1990.210000000
+1987.21
+1990.21
+1990.21
 
 -- !arith_op23 --
 124.333000000
@@ -287,7 +102,7 @@
 24454.535000000
 
 -- !arith_op24 --
-2.310000000
+2.31
 
 -- !arith_op25 --
 \N	\N	\N	\N
diff --git a/regression-test/data/query_p0/sql_functions/conditional_functions/test_nullif.out b/regression-test/data/query_p0/sql_functions/conditional_functions/test_nullif.out
index 397283ef01..f8def44a3d 100644
--- a/regression-test/data/query_p0/sql_functions/conditional_functions/test_nullif.out
+++ b/regression-test/data/query_p0/sql_functions/conditional_functions/test_nullif.out
@@ -350,7 +350,7 @@ true
 15
 
 -- !if_nullif20 --
-123456789.567890124	1234567890123456789012345678901234567890
+123456789.123456789	123456789012345678
 
 -- !if_nullif21 --
 hello	0
diff --git a/regression-test/suites/datatype_p0/decimalv3/test_data/test_overflow.csv b/regression-test/suites/datatype_p0/decimalv3/test_data/test_overflow.csv
new file mode 100644
index 0000000000..61a1f7c3f5
--- /dev/null
+++ b/regression-test/suites/datatype_p0/decimalv3/test_data/test_overflow.csv
@@ -0,0 +1,2 @@
+3957417026770075.00
+39574170267700.75
diff --git a/regression-test/suites/datatype_p0/decimalv3/test_overflow.groovy b/regression-test/suites/datatype_p0/decimalv3/test_overflow.groovy
deleted file mode 100644
index 01de2ea498..0000000000
--- a/regression-test/suites/datatype_p0/decimalv3/test_overflow.groovy
+++ /dev/null
@@ -1,56 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-
-suite("test_overflow") {
-
-    def table1 = "test_overflow"
-
-    sql "drop table if exists ${table1}"
-
-    sql """
-    CREATE TABLE IF NOT EXISTS test_overflow (
-      `k1` decimalv3(38, 18) NULL COMMENT "",
-      `k2` decimalv3(38, 18) NULL COMMENT "",
-      `k3` decimalv3(38, 18) NULL COMMENT "",
-      `v1` decimalv3(38, 37) NULL COMMENT "",
-      `v2` decimalv3(38, 37) NULL COMMENT "",
-      `v3` decimalv3(38, 37) NULL COMMENT "",
-      `v4` INT NULL COMMENT ""
-    ) ENGINE=OLAP
-    COMMENT "OLAP"
-    DISTRIBUTED BY HASH(`k1`, `k2`, `k3`) BUCKETS 8
-    PROPERTIES (
-    "replication_allocation" = "tag.location.default: 1",
-    "in_memory" = "false",
-    "storage_format" = "V2"
-    )
-    """
-
-    sql """insert into test_overflow values(11111111111111111111.1,11111111111111111111.2,11111111111111111111.3, 1.1,1.2,1.3,9)
-    """
-    qt_select_all "select * from test_overflow order by k1"
-
-    sql " SET check_overflow_for_decimal = true; "
-    qt_select_check_overflow1 "select k1 * k2, k1 * k3, k1 * k2 * k3, k1 * v4, k1*50 from test_overflow;"
-    qt_select_check_overflow2 "select v1, k1*10, v1 +k1*10 from test_overflow"
-    qt_select_check_overflow3 "select `k1`, cast (`k1` as DECIMALV3(38, 36)) from test_overflow;"
-
-    sql " SET check_overflow_for_decimal = false; "
-    qt_select_not_check_overflow1 "select k1 * k2, k1 * k3, k1 * k2 * k3, k1 * v4, k1*50 from test_overflow;"
-    qt_select_not_check_overflow2 "select v1, k1*10, v1 +k1*10 from test_overflow"
-    sql "drop table if exists ${table1}"
-}
diff --git a/regression-test/suites/primary_index/test_pk_uk_case.groovy b/regression-test/suites/primary_index/test_pk_uk_case.groovy
index 9403d5e92c..b8be941689 100644
--- a/regression-test/suites/primary_index/test_pk_uk_case.groovy
+++ b/regression-test/suites/primary_index/test_pk_uk_case.groovy
@@ -95,14 +95,13 @@ suite("test_pk_uk_case") {
     def part_key = rd.nextInt(1000)
     def sub_key = 13
     def line_num = 29
-    def decimal = rd.nextFloat()
+    def decimal = 111.11
     def city = RandomStringUtils.randomAlphabetic(10)
     def name = UUID.randomUUID().toString()
     def date = DateTimeFormatter.ofPattern("yyyy-MM-dd").format(LocalDateTime.now())
     for (int idx = 0; idx < 50; idx++) {
         order_key = rd.nextInt(10)
         part_key = rd.nextInt(10)
-        decimal = rd.nextFloat()
         city = RandomStringUtils.randomAlphabetic(10)
         name = UUID.randomUUID().toString()
         sql """ INSERT INTO ${tableNamePk} VALUES
@@ -116,7 +115,6 @@ suite("test_pk_uk_case") {
 
         order_key = rd.nextInt(10)
         part_key = rd.nextInt(10)
-        decimal = rd.nextFloat()
         city = RandomStringUtils.randomAlphabetic(10)
         name = UUID.randomUUID().toString()
         sql """ INSERT INTO ${tableNamePk} VALUES
@@ -130,7 +128,6 @@ suite("test_pk_uk_case") {
 
         order_key = rd.nextInt(10)
         part_key = rd.nextInt(10)
-        decimal = rd.nextFloat()
         city = RandomStringUtils.randomAlphabetic(10)
         name = UUID.randomUUID().toString()
         sql """ INSERT INTO ${tableNamePk} VALUES
@@ -144,7 +141,6 @@ suite("test_pk_uk_case") {
 
         order_key = rd.nextInt(10)
         part_key = rd.nextInt(10)
-        decimal = rd.nextFloat()
         city = RandomStringUtils.randomAlphabetic(10)
         name = UUID.randomUUID().toString()
         sql """ INSERT INTO ${tableNamePk} VALUES
@@ -158,7 +154,6 @@ suite("test_pk_uk_case") {
         
         order_key = rd.nextInt(10)
         part_key = rd.nextInt(10)
-        decimal = rd.nextFloat()
         city = RandomStringUtils.randomAlphabetic(10)
         name = UUID.randomUUID().toString()
         sql """ INSERT INTO ${tableNamePk} VALUES
@@ -173,7 +168,6 @@ suite("test_pk_uk_case") {
         // insert batch key 
         order_key = rd.nextInt(10)
         part_key = rd.nextInt(10)
-        decimal = rd.nextFloat()
         city = RandomStringUtils.randomAlphabetic(10)
         name = UUID.randomUUID().toString()
         sql """ INSERT INTO ${tableNamePk} VALUES
diff --git a/regression-test/suites/query_p0/aggregate/aggregate.groovy b/regression-test/suites/query_p0/aggregate/aggregate.groovy
index ef7d74fa65..68ac834b94 100644
--- a/regression-test/suites/query_p0/aggregate/aggregate.groovy
+++ b/regression-test/suites/query_p0/aggregate/aggregate.groovy
@@ -104,7 +104,7 @@ suite("aggregate") {
         }
     }
 
-    sql "insert into ${tableName2} values (12, 12.25, 'String1', '1999-01-08', '1999-01-08 02:05:06', '1999-01-08', '1999-01-08 02:05:06.111111', null, '1999-01-08 02:05:06.111111', 'true', null, 12345678901234567890.0123456789);"
+    sql "insert into ${tableName2} values (12, 12.25, 'String1', '1999-01-08', '1999-01-08 02:05:06', '1999-01-08', '1999-01-08 02:05:06.111111', null, '1999-01-08 02:05:06.111111', 'true', null, 123456789012345678.012345678);"
 
     sql " sync "
     qt_aggregate """ select max(upper(c_string)), min(upper(c_string)) from ${tableName} """
diff --git a/regression-test/suites/query_p0/operator/test_arithmetic_operators.groovy b/regression-test/suites/query_p0/operator/test_arithmetic_operators.groovy
index 46d5408503..3b8c5eb919 100644
--- a/regression-test/suites/query_p0/operator/test_arithmetic_operators.groovy
+++ b/regression-test/suites/query_p0/operator/test_arithmetic_operators.groovy
@@ -31,10 +31,10 @@ suite("test_arithmetic_operators", "query,p0") {
 		    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_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, \
@@ -56,13 +56,13 @@ suite("test_arithmetic_operators", "query,p0") {
     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"
-        }
-    }
+    // 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"
diff --git a/regression-test/suites/query_p0/sql_functions/conditional_functions/test_nullif.groovy b/regression-test/suites/query_p0/sql_functions/conditional_functions/test_nullif.groovy
index 16b6864809..e754d6d0e8 100644
--- a/regression-test/suites/query_p0/sql_functions/conditional_functions/test_nullif.groovy
+++ b/regression-test/suites/query_p0/sql_functions/conditional_functions/test_nullif.groovy
@@ -127,8 +127,8 @@ suite("test_nullif") {
             logger.info(exception.message)
         }
     }
-    qt_if_nullif20 """select ifnull(123456789.5678901234567890,2),
-        ifnull("1234567890123456789012345678901234567890",2)"""
+    qt_if_nullif20 """select ifnull(123456789.123456789,2),
+        ifnull("123456789012345678",2)"""
     qt_if_nullif21 """select IFNULL("hello", "doris"), IFNULL(NULL,0)"""
     qt_if_nullif22 """select ifnull("null",2), ifnull("NULL",2), ifnull("null","2019-09-09 00:00:00"),
         ifnull(NULL, concat("NUL", "LL"))"""


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