You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by GitBox <gi...@apache.org> on 2022/05/16 01:12:15 UTC

[GitHub] [incubator-doris] liaoxin01 opened a new pull request, #9582: [refactor](decimalv3) optimize decimal performance and precision

liaoxin01 opened a new pull request, #9582:
URL: https://github.com/apache/incubator-doris/pull/9582

   # Proposed changes
   
   Issue Number: close #9208 
   
   ## Problem Summary:
   
   Describe the overview of changes.
   
   ## Checklist(Required)
   
   1. Does it affect the original behavior: (Yes/No/I Don't know)
   2. Has unit tests been added: (Yes/No/No Need)
   3. Has document been added or modified: (Yes/No/No Need)
   4. Does it need to update dependencies: (Yes/No)
   5. Are there any changes that cannot be rolled back: (Yes/No)
   
   ## Further comments
   
   If this is a relatively large or complex change, kick off the discussion at [dev@doris.apache.org](mailto:dev@doris.apache.org) by explaining why you chose the solution you did and what alternatives you considered, etc...
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] Gabriel39 commented on a diff in pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
Gabriel39 commented on code in PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#discussion_r882395470


##########
be/src/vec/olap/olap_data_convertor.h:
##########
@@ -164,6 +165,45 @@ class OlapBlockDataConvertor {
         PaddedPODArray<decimal12_t> _values;

Review Comment:
   We should use different types to store value for different decimal precision



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] Gabriel39 commented on pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
Gabriel39 commented on PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#issuecomment-1131311445

   We should support schema change on decimal column between different precision and scale which is not done in this PR.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] Gabriel39 commented on pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
Gabriel39 commented on PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#issuecomment-1138728173

   also add new decimal types to `_infos_mapping` in `AggregateFuncResolver`


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] liaoxin01 commented on a diff in pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
liaoxin01 commented on code in PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#discussion_r881118810


##########
be/src/vec/data_types/data_type_factory.cpp:
##########
@@ -26,13 +26,75 @@ namespace doris::vectorized {
 
 DataTypePtr DataTypeFactory::create_data_type(const doris::Field& col_desc) {
     DataTypePtr nested = nullptr;
-    if (col_desc.type() == OLAP_FIELD_TYPE_ARRAY) {
+    switch (col_desc.type()) {

Review Comment:
   Decimal need precision and scale information, but primitive data type don't have that information.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] cambyzju commented on a diff in pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
cambyzju commented on code in PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#discussion_r873300713


##########
be/src/vec/exprs/vliteral.cpp:
##########
@@ -109,8 +109,33 @@ 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());
+            if (config::enable_engine_decimalv3) {
+                DataTypePtr type_ptr = create_decimal(node.type.types[0].scalar_type.precision,
+                                                      node.type.types[0].scalar_type.scale);
+                WhichDataType which(type_ptr->get_type_id());

Review Comment:
   WhichDataType(const DataTypePtr& data_type) support DataTypePtr as input argument.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] cambyzju commented on a diff in pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
cambyzju commented on code in PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#discussion_r875833952


##########
fe/fe-core/src/main/java/org/apache/doris/analysis/TypeDef.java:
##########
@@ -159,15 +159,28 @@ private void analyzeScalarType(ScalarType scalarType)
       case DECIMALV2: {
         int precision = scalarType.decimalPrecision();
         int scale = scalarType.decimalScale();
-        // precision: [1, 27]
-        if (precision < 1 || precision > 27) {
-          throw new AnalysisException("Precision of decimal must between 1 and 27."
-                  + " Precision was set to: " + precision + ".");
-        }
-        // scale: [0, 9]
-        if (scale < 0 || scale > 9) {
-          throw new AnalysisException("Scale of decimal must between 0 and 9."
-                  + " Scale was set to: " + scale + ".");
+        if (Config.enable_storage_decimalv3) {
+            // precision: [1, 38]
+            if (precision < 1 || precision > 38) {
+                throw new AnalysisException("Precision of decimal must between 1 and 27."

Review Comment:
   The error message "between 1 and 27" do not match the if condition(1~38).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] cambyzju commented on a diff in pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
cambyzju commented on code in PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#discussion_r875832952


##########
fe/fe-core/src/main/java/org/apache/doris/analysis/TypeDef.java:
##########
@@ -159,15 +159,28 @@ private void analyzeScalarType(ScalarType scalarType)
       case DECIMALV2: {
         int precision = scalarType.decimalPrecision();
         int scale = scalarType.decimalScale();
-        // precision: [1, 27]
-        if (precision < 1 || precision > 27) {
-          throw new AnalysisException("Precision of decimal must between 1 and 27."
-                  + " Precision was set to: " + precision + ".");
-        }
-        // scale: [0, 9]
-        if (scale < 0 || scale > 9) {
-          throw new AnalysisException("Scale of decimal must between 0 and 9."
-                  + " Scale was set to: " + scale + ".");
+        if (Config.enable_storage_decimalv3) {
+            // precision: [1, 38]
+            if (precision < 1 || precision > 38) {

Review Comment:
   it is better replace 38 with MAX_DECIMAL128_PRECISION.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] cambyzju commented on a diff in pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
cambyzju commented on code in PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#discussion_r875886527


##########
be/src/vec/data_types/data_type_decimal.h:
##########
@@ -106,8 +107,8 @@ class DataTypeDecimal final : public IDataType {
         }
 
         // Now, Doris only support precision:27, scale: 9

Review Comment:
   just remove these lines.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] Gabriel39 commented on a diff in pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
Gabriel39 commented on code in PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#discussion_r873280037


##########
be/src/common/config.h:
##########
@@ -736,6 +736,12 @@ CONF_Validator(string_type_length_soft_limit_bytes,
 // used for olap scanner to save memory, when the size of unused_object_pool
 // is greater than object_pool_buffer_size, release the object in the unused_object_pool.
 CONF_Int32(object_pool_buffer_size, "100");
+
+// decimalv3: Decimal32/Decimal64/Decimal128
+// decimalv2 is converted to decimalv3 for calculation when enable_engine_decimalv3 is true.
+CONF_Bool(enable_engine_decimalv3, "false");

Review Comment:
   ```suggestion
   CONF_Bool(enable_execution_decimalv3, "false");
   ```



##########
be/src/vec/aggregate_functions/aggregate_function_avg.h:
##########
@@ -46,13 +46,15 @@ struct AggregateFunctionAvgData {
             // null is handled in AggregationNode::_get_without_key_result
             return static_cast<ResultT>(sum);
         }
-        // to keep the same result with row vesion; see AggregateFunctions::decimalv2_avg_get_value
-        if constexpr (std::is_same_v<ResultT, Decimal128> && std::is_same_v<T, Decimal128>) {
-            DecimalV2Value decimal_val_count(count, 0);
-            DecimalV2Value decimal_val_sum(static_cast<Int128>(sum));
-            DecimalV2Value cal_ret = decimal_val_sum / decimal_val_count;
-            Decimal128 ret(cal_ret.value());
-            return ret;
+        if (!config::enable_engine_decimalv3) {

Review Comment:
   ```suggestion
           if (!config::enable_execution_decimalv3) {
   ```



##########
be/src/olap/types.h:
##########
@@ -878,6 +893,87 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DECIMAL>
     }
 };
 
+template <>
+struct FieldTypeTraits<OLAP_FIELD_TYPE_DECIMAL32>
+        : public BaseFieldtypeTraits<OLAP_FIELD_TYPE_DECIMAL32> {
+    static void set_to_max(void* buf) {
+        CppType* data = reinterpret_cast<CppType*>(buf);
+        *data = 999999999;
+    }
+    static void set_to_min(void* buf) {
+        CppType* data = reinterpret_cast<CppType*>(buf);
+        *data = -999999999;
+    }
+};
+
+template <>
+struct FieldTypeTraits<OLAP_FIELD_TYPE_DECIMAL64>
+        : public BaseFieldtypeTraits<OLAP_FIELD_TYPE_DECIMAL64> {
+    static void set_to_max(void* buf) {
+        CppType* data = reinterpret_cast<CppType*>(buf);
+        *data = 999999999999999999ll;
+    }
+    static void set_to_min(void* buf) {
+        CppType* data = reinterpret_cast<CppType*>(buf);
+        *data = -999999999999999999ll;
+    }
+};
+
+template <>
+struct FieldTypeTraits<OLAP_FIELD_TYPE_DECIMAL128>
+        : public BaseFieldtypeTraits<OLAP_FIELD_TYPE_DECIMAL128> {
+    static Status from_string(void* buf, const std::string& scan_key) {
+        int128_t value = 0;
+        const char* value_string = scan_key.c_str();
+        char* end = nullptr;
+        value = strtol(value_string, &end, 10);
+        if (*end != 0) {
+            value = 0;
+        } else if (value > LONG_MIN && value < LONG_MAX) {
+            // use strtol result directly
+        } else {
+            bool is_negative = false;
+            if (*value_string == '-' || *value_string == '+') {
+                if (*(value_string++) == '-') {
+                    is_negative = true;
+                }
+            }
+            uint128_t current = 0;
+            uint128_t max_int128 = ~((int128_t)(1) << 127);
+            while (*value_string != 0) {
+                if (current > max_int128 / 10) {
+                    break;
+                }
+                current = current * 10 + (*(value_string++) - '0');
+            }
+            if (*value_string != 0 || (!is_negative && current > max_int128) ||
+                (is_negative && current > max_int128 + 1)) {
+                current = 0;
+            }
+            value = is_negative ? -current : current;
+        }
+
+        *reinterpret_cast<PackedInt128*>(buf) = value;
+        return Status::OK();
+    }
+    static std::string to_string(const void* src) {
+        int128_t value = reinterpret_cast<const PackedInt128*>(src)->value;
+        fmt::memory_buffer buffer;
+        fmt::format_to(buffer, "{}", value);
+        return std::string(buffer.data(), buffer.size());
+    }
+    static void set_to_max(void* buf) {
+        *reinterpret_cast<PackedInt128*>(buf) =
+                static_cast<int128_t>(999999999999999999ll) * 100000000000000000ll * 1000ll +

Review Comment:
   Better to use hexadecimal numbers



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] liaoxin01 commented on a diff in pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
liaoxin01 commented on code in PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#discussion_r877695207


##########
be/src/olap/field.h:
##########
@@ -298,6 +298,11 @@ class Field {
     Field* get_sub_field(int i) const { return _sub_fields[i].get(); }
     size_t get_sub_field_count() const { return _sub_fields.size(); }
 
+    void set_precision(int32_t precision) { _precision = precision; }
+    void set_frac(int32_t frac) { _frac = frac; }
+    int32_t precision() const { return _precision; }
+    int32_t frac() const { return _frac; }

Review Comment:
   yes, frac is the same as scale. All code uses frac in src/olap, so I used frac here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] morningman commented on a diff in pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
morningman commented on code in PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#discussion_r880616200


##########
be/src/vec/functions/function_binary_arithmetic.h:
##########
@@ -527,6 +563,31 @@ struct DecimalBinaryOperation {
             return apply(a, b, is_null);
         }
     }
+
+    template <bool scale_left>
+    static NativeResultType apply_scaled_mod(NativeResultType a, NativeResultType b,
+                                             NativeResultType scale, UInt8& is_null) {
+        if constexpr (OpTraits::is_mod) {
+            if constexpr (check_overflow) {
+                bool overflow = false;
+                if constexpr (scale_left)
+                    overflow |= common::mul_overflow(a, scale, a);
+                else
+                    overflow |= common::mul_overflow(b, scale, b);
+
+                if (overflow) {
+                    LOG(FATAL) << "Decimal math overflow";

Review Comment:
   Better not using FATAL



##########
be/src/vec/columns/column_decimal.cpp:
##########
@@ -130,6 +131,38 @@ void ColumnDecimal<T>::insert_data(const char* src, size_t /*length*/) {
     data.emplace_back(tmp);
 }
 
+template <typename T>
+void ColumnDecimal<T>::insert_many_decimalv2_data(const char* data_ptr, size_t num) {
+    for (int i = 0; i < num; i++) {
+        const char* cur_ptr = data_ptr + sizeof(decimal12_t) * i;
+        int64_t int_value = *(int64_t*)(cur_ptr);
+        int32_t frac_value = *(int32_t*)(cur_ptr + sizeof(int64_t));
+        if (config::enable_execution_decimalv3) {
+            bool is_negative = (int_value < 0 || frac_value < 0);
+            if (is_negative) {
+                int_value = std::abs(int_value);
+                frac_value = std::abs(frac_value);
+            }
+            frac_value /= (DecimalV2Value::ONE_BILLION / get_scale_multiplier());
+            T value = T(int_value) * get_scale_multiplier() + T(frac_value);
+            if (is_negative) {
+                value = -value;
+            }
+            this->insert_data(reinterpret_cast<char*>(&value), 0);

Review Comment:
   Does it mean that in this method, we may insert many decimal with different precision and scale?



##########
be/src/vec/columns/column_decimal.cpp:
##########
@@ -130,6 +131,38 @@ void ColumnDecimal<T>::insert_data(const char* src, size_t /*length*/) {
     data.emplace_back(tmp);
 }
 
+template <typename T>
+void ColumnDecimal<T>::insert_many_decimalv2_data(const char* data_ptr, size_t num) {
+    for (int i = 0; i < num; i++) {
+        const char* cur_ptr = data_ptr + sizeof(decimal12_t) * i;
+        int64_t int_value = *(int64_t*)(cur_ptr);
+        int32_t frac_value = *(int32_t*)(cur_ptr + sizeof(int64_t));
+        if (config::enable_execution_decimalv3) {
+            bool is_negative = (int_value < 0 || frac_value < 0);
+            if (is_negative) {
+                int_value = std::abs(int_value);
+                frac_value = std::abs(frac_value);
+            }
+            frac_value /= (DecimalV2Value::ONE_BILLION / get_scale_multiplier());
+            T value = T(int_value) * get_scale_multiplier() + T(frac_value);
+            if (is_negative) {
+                value = -value;
+            }
+            this->insert_data(reinterpret_cast<char*>(&value), 0);
+        } else {
+            DecimalV2Value decimal_val(int_value, frac_value);
+            this->insert_data(reinterpret_cast<char*>(&decimal_val), 0);
+        }
+    }
+}
+
+template <typename T>
+void ColumnDecimal<T>::insert_many_fix_len_data(const char* data_ptr, size_t num) {

Review Comment:
   where to use this method now?



##########
fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java:
##########
@@ -1017,6 +1030,18 @@ public void analyzeImpl(Analyzer analyzer) throws AnalysisException {
         } else {
             this.type = fn.getReturnType();
         }
+
+        // DECIMAL need to pass precision and scale to be
+        if (DECIMAL_FUNCTION_SET.contains(fn.getFunctionName().getFunction()) && this.type.isDecimalV2()) {
+            if (DECIMAL_SAME_TYPE_SET.contains(fnName.getFunction())) {
+                this.type = argTypes[0];
+            } else if (DECIMAL_WIDER_TYPE_SET.contains(fnName.getFunction())) {
+                this.type = ScalarType.createDecimalV2Type(ScalarType.MAX_DECIMAL128_PRECISION,
+                    ((ScalarType) argTypes[0]).getScalarScale());
+            } else if (STDDEV_FUNCTION_SET.contains(fnName.getFunction())) {
+                this.type = ScalarType.createDecimalV2Type(ScalarType.MAX_DECIMAL128_PRECISION, 9);

Review Comment:
   Define this `9` somewhere



##########
be/src/vec/data_types/data_type_factory.cpp:
##########
@@ -26,13 +26,75 @@ namespace doris::vectorized {
 
 DataTypePtr DataTypeFactory::create_data_type(const doris::Field& col_desc) {
     DataTypePtr nested = nullptr;
-    if (col_desc.type() == OLAP_FIELD_TYPE_ARRAY) {
+    switch (col_desc.type()) {

Review Comment:
   Why move this large `switch..case` out of `_create_primitive_data_type()` method?



##########
be/src/vec/functions/function_binary_arithmetic.h:
##########
@@ -449,6 +469,18 @@ struct DecimalBinaryOperation {
 private:
     /// there's implicit type convertion here
     static NativeResultType apply(NativeResultType a, NativeResultType b) {
+        if (config::enable_execution_decimalv3) {
+            if constexpr (OpTraits::can_overflow && check_overflow) {
+                NativeResultType res;
+                if (Op::template apply<NativeResultType>(a, b, res)) {
+                    LOG(FATAL) << "Decimal math overflow";

Review Comment:
   Better not use FATAL log, it will cause BE crash.
   Could we return a invalid value or null to indicate this error?
   And in what situation can we get here?



##########
be/src/vec/sink/vtablet_sink.cpp:
##########
@@ -518,6 +518,9 @@ Status VOlapTableSink::_validate_data(RuntimeState* state, vectorized::Block* bl
             break;
         }
         case TYPE_DECIMALV2: {
+            if (config::enable_execution_decimalv3) {
+                break;

Review Comment:
   why break for v3? please add comment in code.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] liaoxin01 commented on a diff in pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
liaoxin01 commented on code in PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#discussion_r881123403


##########
be/src/vec/functions/function_binary_arithmetic.h:
##########
@@ -449,6 +469,18 @@ struct DecimalBinaryOperation {
 private:
     /// there's implicit type convertion here
     static NativeResultType apply(NativeResultType a, NativeResultType b) {
+        if (config::enable_execution_decimalv3) {
+            if constexpr (OpTraits::can_overflow && check_overflow) {
+                NativeResultType res;
+                if (Op::template apply<NativeResultType>(a, b, res)) {
+                    LOG(FATAL) << "Decimal math overflow";

Review Comment:
   Decimal addition and multiplication may overflow.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] HappenLee commented on a diff in pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
HappenLee commented on code in PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#discussion_r889981738


##########
be/src/olap/tablet_meta.cpp:
##########
@@ -202,12 +202,24 @@ void TabletMeta::_init_column_from_tcolumn(uint32_t unique_id, const TColumn& tc
     EnumToString(TPrimitiveType, tcolumn.column_type.type, data_type);
     column->set_type(data_type);
 
+    uint32_t length = TabletColumn::get_field_length_by_type(tcolumn.column_type.type,
+                                                             tcolumn.column_type.len);
     if (tcolumn.column_type.type == TPrimitiveType::DECIMALV2) {
         column->set_precision(tcolumn.column_type.precision);
         column->set_frac(tcolumn.column_type.scale);
+        if (config::enable_storage_decimalv3) {
+            if (tcolumn.column_type.precision <= 9) {
+                length = sizeof(int32_t);
+                column->set_type("DECIMAL32");
+            } else if (tcolumn.column_type.precision <= 18) {

Review Comment:
   should use `MAX_DECIMAL64_PRECISION` to replace all code use `18`.
   same to `9` and `38`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] cambyzju commented on a diff in pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
cambyzju commented on code in PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#discussion_r875922111


##########
be/src/vec/sink/mysql_result_writer.cpp:
##########
@@ -419,9 +447,13 @@ Status VMysqlResultWriter::append_block(Block& input_block) {
         }
         case TYPE_DECIMALV2: {
             if (type_ptr->is_nullable()) {
-                status = _add_one_column<PrimitiveType::TYPE_DECIMALV2, true>(column_ptr, result);
+                auto& nested_type =
+                        assert_cast<const DataTypeNullable&>(*type_ptr).get_nested_type();
+                status = _add_one_column<PrimitiveType::TYPE_DECIMALV2, true>(column_ptr, result,

Review Comment:
   why we need nested_type here?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] HappenLee commented on a diff in pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
HappenLee commented on code in PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#discussion_r891422484


##########
be/src/vec/data_types/data_type_factory.cpp:
##########
@@ -26,13 +26,75 @@ namespace doris::vectorized {
 
 DataTypePtr DataTypeFactory::create_data_type(const doris::Field& col_desc) {

Review Comment:
   You should simplify this code by making call `DataTypeFactory::create_data_type(const TabletColumn& col_desc, bool is_nullable)`. do not have two big and same `switch`



##########
be/src/vec/functions/function_binary_arithmetic.h:
##########
@@ -529,6 +565,31 @@ struct DecimalBinaryOperation {
             return apply(a, b, is_null);
         }
     }
+
+    template <bool scale_left>
+    static NativeResultType apply_scaled_mod(NativeResultType a, NativeResultType b,
+                                             NativeResultType scale, UInt8& is_null) {
+        if constexpr (OpTraits::is_mod) {

Review Comment:
   if not `is_mod` return nothing?



##########
be/src/vec/olap/olap_data_convertor.h:
##########
@@ -182,12 +182,56 @@ class OlapBlockDataConvertor {
         Status convert_to_olap() override;
     };
 
+    // For compatibility with decimalv2.
+    // Convert the decimal32/decimal64/decimal128 of the computing
+    // layer to decimal12_t of the storage layer.
+    template <typename T>
     class OlapColumnDataConvertorDecimal
             : public OlapColumnDataConvertorPaddedPODArray<decimal12_t> {
     public:
         Status convert_to_olap() override;
     };
 
+    // decimalv3 don't need to do any convert
+    template <typename T>
+    class OlapColumnDataConvertorDecimalV3 : public OlapColumnDataConvertorBase {

Review Comment:
   should be a subclass of `OlapColumnDataConvertorSimple` only need override the `convert_to_olap` method



##########
be/src/vec/sink/mysql_result_writer.cpp:
##########
@@ -144,6 +145,35 @@ Status VMysqlResultWriter::_add_one_column(const ColumnPtr& column_ptr,
             _buffer.close_dynamic_mode();
             result->result_batch.rows[i].append(_buffer.buf(), _buffer.length());
         }
+    } else if (type == TYPE_DECIMALV2 && config::enable_execution_decimalv3) {

Review Comment:
   why the code here not 234 line like
   ```
   if (config::enable_execution_decimalv3) {
     .....
   } else {
    DecimalV2Value ...
   }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] Gabriel39 commented on pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
Gabriel39 commented on PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#issuecomment-1138350323

   PredicateColumn for DecimalV3 is still not supported, right?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] Gabriel39 commented on a diff in pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
Gabriel39 commented on code in PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#discussion_r882367320


##########
be/src/olap/field.h:
##########
@@ -298,6 +298,11 @@ class Field {
     Field* get_sub_field(int i) const { return _sub_fields[i].get(); }
     size_t get_sub_field_count() const { return _sub_fields.size(); }
 
+    void set_precision(int32_t precision) { _precision = precision; }
+    void set_frac(int32_t frac) { _frac = frac; }
+    int32_t precision() const { return _precision; }
+    int32_t frac() const { return _frac; }

Review Comment:
   I also think `frac` is a little confused for decimal. To keep semantics for decimal, it's better to use `scale` here.



##########
be/src/vec/olap/olap_data_convertor.h:
##########
@@ -164,6 +165,45 @@ class OlapBlockDataConvertor {
         PaddedPODArray<decimal12_t> _values;

Review Comment:
   We should use different types to store value for different decimal precision



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] cambyzju commented on a diff in pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
cambyzju commented on code in PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#discussion_r875855136


##########
be/src/vec/columns/column.h:
##########
@@ -183,6 +183,11 @@ class IColumn : public COW<IColumn> {
     /// All data will be inserted as single element
     virtual void insert_data(const char* pos, size_t length) = 0;
 
+    // just used for ColumnDecimal to be compatible with decimalv2
+    virtual void insert_many_decimalv2_data(const char* pos, size_t num) {

Review Comment:
   `insert_many_decimalv2_data` is not a common function, it is better not add it to IColumn.
   
   While using, you could cast IColumn to ColumnDecimal, and call the function.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] liaoxin01 commented on a diff in pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
liaoxin01 commented on code in PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#discussion_r881113467


##########
be/src/vec/columns/column_decimal.cpp:
##########
@@ -130,6 +131,38 @@ void ColumnDecimal<T>::insert_data(const char* src, size_t /*length*/) {
     data.emplace_back(tmp);
 }
 
+template <typename T>
+void ColumnDecimal<T>::insert_many_decimalv2_data(const char* data_ptr, size_t num) {
+    for (int i = 0; i < num; i++) {
+        const char* cur_ptr = data_ptr + sizeof(decimal12_t) * i;
+        int64_t int_value = *(int64_t*)(cur_ptr);
+        int32_t frac_value = *(int32_t*)(cur_ptr + sizeof(int64_t));
+        if (config::enable_execution_decimalv3) {
+            bool is_negative = (int_value < 0 || frac_value < 0);
+            if (is_negative) {
+                int_value = std::abs(int_value);
+                frac_value = std::abs(frac_value);
+            }
+            frac_value /= (DecimalV2Value::ONE_BILLION / get_scale_multiplier());
+            T value = T(int_value) * get_scale_multiplier() + T(frac_value);
+            if (is_negative) {
+                value = -value;
+            }
+            this->insert_data(reinterpret_cast<char*>(&value), 0);

Review Comment:
   In this method, we can insert many decimal with same precision and scale. This method converts storage layer decimalv2 to decimalv3 for calculation.



##########
be/src/vec/columns/column_decimal.cpp:
##########
@@ -130,6 +131,38 @@ void ColumnDecimal<T>::insert_data(const char* src, size_t /*length*/) {
     data.emplace_back(tmp);
 }
 
+template <typename T>
+void ColumnDecimal<T>::insert_many_decimalv2_data(const char* data_ptr, size_t num) {
+    for (int i = 0; i < num; i++) {
+        const char* cur_ptr = data_ptr + sizeof(decimal12_t) * i;
+        int64_t int_value = *(int64_t*)(cur_ptr);
+        int32_t frac_value = *(int32_t*)(cur_ptr + sizeof(int64_t));
+        if (config::enable_execution_decimalv3) {
+            bool is_negative = (int_value < 0 || frac_value < 0);
+            if (is_negative) {
+                int_value = std::abs(int_value);
+                frac_value = std::abs(frac_value);
+            }
+            frac_value /= (DecimalV2Value::ONE_BILLION / get_scale_multiplier());
+            T value = T(int_value) * get_scale_multiplier() + T(frac_value);
+            if (is_negative) {
+                value = -value;
+            }
+            this->insert_data(reinterpret_cast<char*>(&value), 0);

Review Comment:
   In this method, we can insert many decimal with same precision and scale. This method converts storage layer decimalv2 to decimalv3 for calculation.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] Gabriel39 commented on a diff in pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
Gabriel39 commented on code in PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#discussion_r882768009


##########
be/src/vec/exec/vaggregation_node.cpp:
##########
@@ -145,9 +145,22 @@ void AggregationNode::_init_hash_method(std::vector<VExprContext*>& probe_exprs)
             _agg_data.init(AggregatedDataVariants::Type::int64_key, is_nullable);
             return;
         case TYPE_LARGEINT:
-        case TYPE_DECIMALV2:
-            _agg_data.init(AggregatedDataVariants::Type::int128_key, is_nullable);
+        case TYPE_DECIMALV2: {

Review Comment:
   this code block should also be added to vhashjoinnode



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] liaoxin01 commented on a diff in pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
liaoxin01 commented on code in PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#discussion_r895174675


##########
be/src/olap/field.h:
##########
@@ -340,6 +345,8 @@ class Field {
     uint16_t _index_size;
     bool _is_nullable;
     std::vector<std::unique_ptr<Field>> _sub_fields;
+    int32_t _precision;

Review Comment:
   Just to be consistent with other code.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] liaoxin01 commented on a diff in pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
liaoxin01 commented on code in PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#discussion_r877688404


##########
be/src/vec/sink/mysql_result_writer.cpp:
##########
@@ -419,9 +447,13 @@ Status VMysqlResultWriter::append_block(Block& input_block) {
         }
         case TYPE_DECIMALV2: {
             if (type_ptr->is_nullable()) {
-                status = _add_one_column<PrimitiveType::TYPE_DECIMALV2, true>(column_ptr, result);
+                auto& nested_type =
+                        assert_cast<const DataTypeNullable&>(*type_ptr).get_nested_type();
+                status = _add_one_column<PrimitiveType::TYPE_DECIMALV2, true>(column_ptr, result,

Review Comment:
   used for _add_one_column function to infer the type of decimal column.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] liaoxin01 commented on a diff in pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
liaoxin01 commented on code in PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#discussion_r878672704


##########
be/src/vec/columns/column.h:
##########
@@ -183,6 +183,11 @@ class IColumn : public COW<IColumn> {
     /// All data will be inserted as single element
     virtual void insert_data(const char* pos, size_t length) = 0;
 
+    // just used for ColumnDecimal to be compatible with decimalv2
+    virtual void insert_many_decimalv2_data(const char* pos, size_t num) {

Review Comment:
   This function will be called in bitshuffle_page.h. ColumnDecimal<T> is template class,but we don't know the real type of T in bitshuffle_page.h. so we can't  use static_cast here.
   
   There are several solutions:
   1. use dynamic_cast
   2. change insert_many_decimalv2_data to a template function
   
   I don't think they are better than the current implementation.
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] liaoxin01 commented on pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
liaoxin01 commented on PR #9582:
URL: https://github.com/apache/doris/pull/9582#issuecomment-1177161549

   Will be resubmitted in another pr #10545.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] liaoxin01 closed pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
liaoxin01 closed pull request #9582: [refactor](decimalv3) optimize decimal performance and precision
URL: https://github.com/apache/doris/pull/9582


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] Gabriel39 commented on pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
Gabriel39 commented on PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#issuecomment-1155034819

   DecimalV3 also needs to be processed in runtime filter


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] HappenLee commented on a diff in pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
HappenLee commented on code in PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#discussion_r890833960


##########
be/src/exec/olap_scan_node.cpp:
##########
@@ -630,6 +630,10 @@ Status OlapScanNode::normalize_conjuncts() {
         }
 
         case TYPE_DECIMALV2: {
+            // TODO predicate pushdown for decimalv3
+            if (config::enable_execution_decimalv3) {

Review Comment:
   the behavior may cause some performance problem ? do we have more great way to do the thing



##########
be/src/olap/types.h:
##########
@@ -885,6 +900,87 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DECIMAL>
     }
 };
 
+template <>
+struct FieldTypeTraits<OLAP_FIELD_TYPE_DECIMAL32>
+        : public BaseFieldtypeTraits<OLAP_FIELD_TYPE_DECIMAL32> {
+    static void set_to_max(void* buf) {
+        CppType* data = reinterpret_cast<CppType*>(buf);
+        *data = 999999999;
+    }
+    static void set_to_min(void* buf) {
+        CppType* data = reinterpret_cast<CppType*>(buf);
+        *data = -999999999;
+    }
+};
+
+template <>
+struct FieldTypeTraits<OLAP_FIELD_TYPE_DECIMAL64>
+        : public BaseFieldtypeTraits<OLAP_FIELD_TYPE_DECIMAL64> {
+    static void set_to_max(void* buf) {
+        CppType* data = reinterpret_cast<CppType*>(buf);
+        *data = 999999999999999999ll;
+    }
+    static void set_to_min(void* buf) {
+        CppType* data = reinterpret_cast<CppType*>(buf);
+        *data = -999999999999999999ll;
+    }
+};
+
+template <>
+struct FieldTypeTraits<OLAP_FIELD_TYPE_DECIMAL128>
+        : public BaseFieldtypeTraits<OLAP_FIELD_TYPE_DECIMAL128> {
+    static Status from_string(void* buf, const std::string& scan_key) {

Review Comment:
   why do not use `StringParser` to do `string` to `int128`?



##########
be/src/olap/field.h:
##########
@@ -340,6 +345,8 @@ class Field {
     uint16_t _index_size;
     bool _is_nullable;
     std::vector<std::unique_ptr<Field>> _sub_fields;
+    int32_t _precision;

Review Comment:
   why use int32_t the `precision` and `_scale` may less than `128` ? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] liaoxin01 commented on a diff in pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
liaoxin01 commented on code in PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#discussion_r881554793


##########
be/src/vec/functions/function_binary_arithmetic.h:
##########
@@ -449,6 +469,18 @@ struct DecimalBinaryOperation {
 private:
     /// there's implicit type convertion here
     static NativeResultType apply(NativeResultType a, NativeResultType b) {
+        if (config::enable_execution_decimalv3) {
+            if constexpr (OpTraits::can_overflow && check_overflow) {
+                NativeResultType res;
+                if (Op::template apply<NativeResultType>(a, b, res)) {
+                    LOG(FATAL) << "Decimal math overflow";

Review Comment:
   The column may not be ColumnNullable, so we can't return null.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] cambyzju commented on a diff in pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
cambyzju commented on code in PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#discussion_r875846111


##########
be/src/olap/field.h:
##########
@@ -298,6 +298,11 @@ class Field {
     Field* get_sub_field(int i) const { return _sub_fields[i].get(); }
     size_t get_sub_field_count() const { return _sub_fields.size(); }
 
+    void set_precision(int32_t precision) { _precision = precision; }
+    void set_frac(int32_t frac) { _frac = frac; }
+    int32_t precision() const { return _precision; }
+    int32_t frac() const { return _frac; }

Review Comment:
   Does frac the same as scale? If it is true, why not use scale?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] Gabriel39 commented on a diff in pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
Gabriel39 commented on code in PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#discussion_r873281667


##########
be/src/olap/types.h:
##########
@@ -878,6 +893,87 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DECIMAL>
     }
 };
 
+template <>
+struct FieldTypeTraits<OLAP_FIELD_TYPE_DECIMAL32>
+        : public BaseFieldtypeTraits<OLAP_FIELD_TYPE_DECIMAL32> {
+    static void set_to_max(void* buf) {
+        CppType* data = reinterpret_cast<CppType*>(buf);
+        *data = 999999999;
+    }
+    static void set_to_min(void* buf) {
+        CppType* data = reinterpret_cast<CppType*>(buf);
+        *data = -999999999;
+    }
+};
+
+template <>
+struct FieldTypeTraits<OLAP_FIELD_TYPE_DECIMAL64>
+        : public BaseFieldtypeTraits<OLAP_FIELD_TYPE_DECIMAL64> {
+    static void set_to_max(void* buf) {
+        CppType* data = reinterpret_cast<CppType*>(buf);
+        *data = 999999999999999999ll;
+    }
+    static void set_to_min(void* buf) {
+        CppType* data = reinterpret_cast<CppType*>(buf);
+        *data = -999999999999999999ll;
+    }
+};
+
+template <>
+struct FieldTypeTraits<OLAP_FIELD_TYPE_DECIMAL128>
+        : public BaseFieldtypeTraits<OLAP_FIELD_TYPE_DECIMAL128> {
+    static Status from_string(void* buf, const std::string& scan_key) {
+        int128_t value = 0;
+        const char* value_string = scan_key.c_str();
+        char* end = nullptr;
+        value = strtol(value_string, &end, 10);
+        if (*end != 0) {
+            value = 0;
+        } else if (value > LONG_MIN && value < LONG_MAX) {
+            // use strtol result directly
+        } else {
+            bool is_negative = false;
+            if (*value_string == '-' || *value_string == '+') {
+                if (*(value_string++) == '-') {
+                    is_negative = true;
+                }
+            }
+            uint128_t current = 0;
+            uint128_t max_int128 = ~((int128_t)(1) << 127);
+            while (*value_string != 0) {
+                if (current > max_int128 / 10) {
+                    break;
+                }
+                current = current * 10 + (*(value_string++) - '0');
+            }
+            if (*value_string != 0 || (!is_negative && current > max_int128) ||
+                (is_negative && current > max_int128 + 1)) {
+                current = 0;
+            }
+            value = is_negative ? -current : current;
+        }
+
+        *reinterpret_cast<PackedInt128*>(buf) = value;
+        return Status::OK();
+    }
+    static std::string to_string(const void* src) {
+        int128_t value = reinterpret_cast<const PackedInt128*>(src)->value;
+        fmt::memory_buffer buffer;
+        fmt::format_to(buffer, "{}", value);
+        return std::string(buffer.data(), buffer.size());
+    }
+    static void set_to_max(void* buf) {
+        *reinterpret_cast<PackedInt128*>(buf) =
+                static_cast<int128_t>(999999999999999999ll) * 100000000000000000ll * 1000ll +

Review Comment:
   Better to use hexadecimal numbers



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] liaoxin01 commented on a diff in pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
liaoxin01 commented on code in PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#discussion_r878672704


##########
be/src/vec/columns/column.h:
##########
@@ -183,6 +183,11 @@ class IColumn : public COW<IColumn> {
     /// All data will be inserted as single element
     virtual void insert_data(const char* pos, size_t length) = 0;
 
+    // just used for ColumnDecimal to be compatible with decimalv2
+    virtual void insert_many_decimalv2_data(const char* pos, size_t num) {

Review Comment:
   This function will be called in bitshuffle_page.h. ColumnDecimal\<T\> is template class. We don't know the real type of T when the function was called in bitshuffle_page.h, so we can't  use static_cast here.
   
   There are several solutions:
   1. use dynamic_cast
   2. change insert_many_decimalv2_data to a template function
   
   I don't think they are better than the current implementation.
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] liaoxin01 commented on pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
liaoxin01 commented on PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#issuecomment-1133588679

   > We should support schema change on decimal column between different precision and scale which is not done in this PR.
   
   Yes, we should support  it. I plan to write the schema change code after this PR.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] Gabriel39 commented on pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
Gabriel39 commented on PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#issuecomment-1138183867

   1. Add UTs in BE
   2. rebase master to resolve conflicts


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] liaoxin01 commented on a diff in pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
liaoxin01 commented on code in PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#discussion_r881117053


##########
be/src/vec/columns/column_decimal.cpp:
##########
@@ -130,6 +131,38 @@ void ColumnDecimal<T>::insert_data(const char* src, size_t /*length*/) {
     data.emplace_back(tmp);
 }
 
+template <typename T>
+void ColumnDecimal<T>::insert_many_decimalv2_data(const char* data_ptr, size_t num) {
+    for (int i = 0; i < num; i++) {
+        const char* cur_ptr = data_ptr + sizeof(decimal12_t) * i;
+        int64_t int_value = *(int64_t*)(cur_ptr);
+        int32_t frac_value = *(int32_t*)(cur_ptr + sizeof(int64_t));
+        if (config::enable_execution_decimalv3) {
+            bool is_negative = (int_value < 0 || frac_value < 0);
+            if (is_negative) {
+                int_value = std::abs(int_value);
+                frac_value = std::abs(frac_value);
+            }
+            frac_value /= (DecimalV2Value::ONE_BILLION / get_scale_multiplier());
+            T value = T(int_value) * get_scale_multiplier() + T(frac_value);
+            if (is_negative) {
+                value = -value;
+            }
+            this->insert_data(reinterpret_cast<char*>(&value), 0);
+        } else {
+            DecimalV2Value decimal_val(int_value, frac_value);
+            this->insert_data(reinterpret_cast<char*>(&decimal_val), 0);
+        }
+    }
+}
+
+template <typename T>
+void ColumnDecimal<T>::insert_many_fix_len_data(const char* data_ptr, size_t num) {

Review Comment:
   New decimalv3 type added on storage layer will use this method, and origin decimalv2 will use insert_many_decimalv2_data.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [incubator-doris] Gabriel39 commented on a diff in pull request #9582: [refactor](decimalv3) optimize decimal performance and precision

Posted by GitBox <gi...@apache.org>.
Gabriel39 commented on code in PR #9582:
URL: https://github.com/apache/incubator-doris/pull/9582#discussion_r881540625


##########
be/src/vec/functions/function_binary_arithmetic.h:
##########
@@ -449,6 +469,18 @@ struct DecimalBinaryOperation {
 private:
     /// there's implicit type convertion here
     static NativeResultType apply(NativeResultType a, NativeResultType b) {
+        if (config::enable_execution_decimalv3) {
+            if constexpr (OpTraits::can_overflow && check_overflow) {
+                NativeResultType res;
+                if (Op::template apply<NativeResultType>(a, b, res)) {
+                    LOG(FATAL) << "Decimal math overflow";

Review Comment:
   how about to return null when overflow?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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