github-actions[bot] commented on code in PR #25386:
URL: https://github.com/apache/doris/pull/25386#discussion_r1362537872


##########
be/src/olap/rowset/segment_v2/bitshuffle_page.h:
##########
@@ -267,6 +267,7 @@ inline Status parse_bit_shuffle_header(const Slice& data, 
size_t& num_elements,
     case 8:
     case 12:
     case 16:
+    case 32:

Review Comment:
   warning: 32 is a magic number; consider replacing it with a named constant 
[readability-magic-numbers]
   ```cpp
       case 32:
            ^
   ```
   



##########
be/src/runtime/raw_value.h:
##########
@@ -103,6 +103,8 @@ inline uint32_t RawValue::zlib_crc32(const void* v, size_t 
len, const PrimitiveT
         return HashUtil::zlib_crc_hash(v, 8, seed);
     case TYPE_DECIMAL128I:
         return HashUtil::zlib_crc_hash(v, 16, seed);
+    case TYPE_DECIMAL256:
+        return HashUtil::zlib_crc_hash(v, 32, seed);

Review Comment:
   warning: 32 is a magic number; consider replacing it with a named constant 
[readability-magic-numbers]
   ```cpp
           return HashUtil::zlib_crc_hash(v, 32, seed);
                                             ^
   ```
   



##########
be/src/olap/types.h:
##########
@@ -1083,6 +1089,32 @@ struct 
FieldTypeTraits<FieldType::OLAP_FIELD_TYPE_DECIMAL128I>
     }
 };
 
+template <>
+struct FieldTypeTraits<FieldType::OLAP_FIELD_TYPE_DECIMAL256>
+        : public BaseFieldtypeTraits<FieldType::OLAP_FIELD_TYPE_DECIMAL256> {
+    static Status from_string(void* buf, const std::string& scan_key, const 
int precision,
+                              const int scale) {
+        StringParser::ParseResult result = StringParser::PARSE_SUCCESS;
+        auto value = StringParser::string_to_decimal<TYPE_DECIMAL256>(
+                scan_key.c_str(), scan_key.size(), 76, scale, &result);

Review Comment:
   warning: 76 is a magic number; consider replacing it with a named constant 
[readability-magic-numbers]
   ```cpp
                   scan_key.c_str(), scan_key.size(), 76, scale, &result);
                                                      ^
   ```
   



##########
be/src/runtime/types.h:
##########
@@ -244,13 +239,16 @@ struct TypeDescriptor {
 
     static inline int get_decimal_byte_size(int precision) {
         DCHECK_GT(precision, 0);
-        if (precision <= MAX_DECIMAL4_PRECISION) {
+        if (precision <= BeConsts::MAX_DECIMAL32_PRECISION) {
             return 4;
         }
-        if (precision <= MAX_DECIMAL8_PRECISION) {
+        if (precision <= BeConsts::MAX_DECIMAL64_PRECISION) {
             return 8;
         }
-        return 16;
+        if (precision <= BeConsts::MAX_DECIMAL128_PRECISION) {
+            return 16;

Review Comment:
   warning: 16 is a magic number; consider replacing it with a named constant 
[readability-magic-numbers]
   ```cpp
               return 16;
                      ^
   ```
   



##########
be/src/vec/aggregate_functions/aggregate_function_sum_old.h:
##########
@@ -0,0 +1,201 @@
+// 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.
+// This file is copied from
+// 
https://github.com/ClickHouse/ClickHouse/blob/master/src/AggregateFunctions/AggregateFunctionOldSum.h
+// and modified by Doris
+
+#pragma once
+
+#include <stddef.h>
+
+#include <memory>
+#include <type_traits>
+#include <vector>
+
+#include "vec/aggregate_functions/aggregate_function.h"
+#include "vec/aggregate_functions/aggregate_function_sum.h"
+#include "vec/columns/column.h"
+#include "vec/common/assert_cast.h"
+#include "vec/core/field.h"
+#include "vec/core/types.h"
+#include "vec/data_types/data_type.h"
+#include "vec/data_types/data_type_decimal.h"
+#include "vec/io/io_helper.h"
+namespace doris {
+namespace vectorized {
+class Arena;
+class BufferReadable;
+class BufferWritable;
+template <typename T>
+class ColumnDecimal;
+template <typename T>
+class DataTypeNumber;
+template <typename>
+class ColumnVector;
+} // namespace vectorized
+} // namespace doris
+
+namespace doris::vectorized {
+
+/*
+ * this function is used to solve agg of sum/count is not compatibility during 
the upgrade process
+ * in PR #20370 have changed the serialize type and serialize column
+ * before is ColumnVector, now sum/count change to use ColumnFixedLengthObject
+ * so during the upgrade process, will be not compatible if exist old BE and 
Newer BE
+ */
+
+template <typename T, typename TResult, typename Data>
+class AggregateFunctionOldSum final
+        : public IAggregateFunctionDataHelper<Data, AggregateFunctionOldSum<T, 
TResult, Data>> {
+public:
+    using ResultDataType = std::conditional_t<IsDecimalNumber<T>, 
DataTypeDecimal<TResult>,
+                                              DataTypeNumber<TResult>>;
+    using ColVecType = std::conditional_t<IsDecimalNumber<T>, 
ColumnDecimal<T>, ColumnVector<T>>;
+    using ColVecResult =
+            std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<TResult>, 
ColumnVector<TResult>>;
+
+    String get_name() const override { return "sum"; }
+
+    AggregateFunctionOldSum(const DataTypes& argument_types_)
+            : IAggregateFunctionDataHelper<Data, AggregateFunctionOldSum<T, 
TResult, Data>>(
+                      argument_types_),
+              scale(get_decimal_scale(*argument_types_[0])) {}
+
+    DataTypePtr get_return_type() const override {
+        if constexpr (IsDecimalNumber<T>) {
+            return 
std::make_shared<ResultDataType>(ResultDataType::max_precision(), scale);
+        } else {
+            return std::make_shared<ResultDataType>();
+        }
+    }
+
+    void add(AggregateDataPtr __restrict place, const IColumn** columns, 
size_t row_num,
+             Arena*) const override {
+        const auto& column = assert_cast<const ColVecType&>(*columns[0]);
+        this->data(place).add(TResult(column.get_data()[row_num]));
+    }
+
+    void reset(AggregateDataPtr place) const override { this->data(place).sum 
= {}; }
+
+    void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs,
+               Arena*) const override {
+        this->data(place).merge(this->data(rhs));
+    }
+
+    void serialize(ConstAggregateDataPtr __restrict place, BufferWritable& 
buf) const override {
+        this->data(place).write(buf);
+    }
+
+    void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf,
+                     Arena*) const override {

Review Comment:
   warning: all parameters should be named in a function 
[readability-named-parameter]
   
   ```suggestion
                        Arena* /*unused*/) const override {
   ```
   



##########
be/src/vec/aggregate_functions/aggregate_function_sum_old.h:
##########
@@ -0,0 +1,201 @@
+// 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.
+// This file is copied from
+// 
https://github.com/ClickHouse/ClickHouse/blob/master/src/AggregateFunctions/AggregateFunctionOldSum.h
+// and modified by Doris
+
+#pragma once
+
+#include <stddef.h>

Review Comment:
   warning: inclusion of deprecated C++ header 'stddef.h'; consider using 
'cstddef' instead [modernize-deprecated-headers]
   
   ```suggestion
   #include <cstddef>
   ```
   



##########
be/src/vec/aggregate_functions/aggregate_function_sum_old.h:
##########
@@ -0,0 +1,201 @@
+// 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.
+// This file is copied from
+// 
https://github.com/ClickHouse/ClickHouse/blob/master/src/AggregateFunctions/AggregateFunctionOldSum.h
+// and modified by Doris
+
+#pragma once
+
+#include <stddef.h>
+
+#include <memory>
+#include <type_traits>
+#include <vector>
+
+#include "vec/aggregate_functions/aggregate_function.h"
+#include "vec/aggregate_functions/aggregate_function_sum.h"
+#include "vec/columns/column.h"
+#include "vec/common/assert_cast.h"
+#include "vec/core/field.h"
+#include "vec/core/types.h"
+#include "vec/data_types/data_type.h"
+#include "vec/data_types/data_type_decimal.h"
+#include "vec/io/io_helper.h"
+namespace doris {
+namespace vectorized {
+class Arena;
+class BufferReadable;
+class BufferWritable;
+template <typename T>
+class ColumnDecimal;
+template <typename T>
+class DataTypeNumber;
+template <typename>
+class ColumnVector;
+} // namespace vectorized
+} // namespace doris
+
+namespace doris::vectorized {
+
+/*
+ * this function is used to solve agg of sum/count is not compatibility during 
the upgrade process
+ * in PR #20370 have changed the serialize type and serialize column
+ * before is ColumnVector, now sum/count change to use ColumnFixedLengthObject
+ * so during the upgrade process, will be not compatible if exist old BE and 
Newer BE
+ */
+
+template <typename T, typename TResult, typename Data>
+class AggregateFunctionOldSum final
+        : public IAggregateFunctionDataHelper<Data, AggregateFunctionOldSum<T, 
TResult, Data>> {
+public:
+    using ResultDataType = std::conditional_t<IsDecimalNumber<T>, 
DataTypeDecimal<TResult>,
+                                              DataTypeNumber<TResult>>;
+    using ColVecType = std::conditional_t<IsDecimalNumber<T>, 
ColumnDecimal<T>, ColumnVector<T>>;
+    using ColVecResult =
+            std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<TResult>, 
ColumnVector<TResult>>;
+
+    String get_name() const override { return "sum"; }
+
+    AggregateFunctionOldSum(const DataTypes& argument_types_)
+            : IAggregateFunctionDataHelper<Data, AggregateFunctionOldSum<T, 
TResult, Data>>(
+                      argument_types_),
+              scale(get_decimal_scale(*argument_types_[0])) {}
+
+    DataTypePtr get_return_type() const override {
+        if constexpr (IsDecimalNumber<T>) {
+            return 
std::make_shared<ResultDataType>(ResultDataType::max_precision(), scale);
+        } else {
+            return std::make_shared<ResultDataType>();
+        }
+    }
+
+    void add(AggregateDataPtr __restrict place, const IColumn** columns, 
size_t row_num,
+             Arena*) const override {

Review Comment:
   warning: all parameters should be named in a function 
[readability-named-parameter]
   
   ```suggestion
                Arena* /*unused*/) const override {
   ```
   



##########
be/src/runtime/types.h:
##########
@@ -244,13 +239,16 @@ struct TypeDescriptor {
 
     static inline int get_decimal_byte_size(int precision) {
         DCHECK_GT(precision, 0);
-        if (precision <= MAX_DECIMAL4_PRECISION) {
+        if (precision <= BeConsts::MAX_DECIMAL32_PRECISION) {
             return 4;
         }
-        if (precision <= MAX_DECIMAL8_PRECISION) {
+        if (precision <= BeConsts::MAX_DECIMAL64_PRECISION) {
             return 8;
         }
-        return 16;
+        if (precision <= BeConsts::MAX_DECIMAL128_PRECISION) {
+            return 16;
+        }
+        return 32;

Review Comment:
   warning: 32 is a magic number; consider replacing it with a named constant 
[readability-magic-numbers]
   ```cpp
           return 32;
                  ^
   ```
   



##########
be/src/vec/aggregate_functions/aggregate_function_sum_old.h:
##########
@@ -0,0 +1,201 @@
+// 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.
+// This file is copied from
+// 
https://github.com/ClickHouse/ClickHouse/blob/master/src/AggregateFunctions/AggregateFunctionOldSum.h
+// and modified by Doris
+
+#pragma once
+
+#include <stddef.h>
+
+#include <memory>
+#include <type_traits>
+#include <vector>
+
+#include "vec/aggregate_functions/aggregate_function.h"
+#include "vec/aggregate_functions/aggregate_function_sum.h"
+#include "vec/columns/column.h"
+#include "vec/common/assert_cast.h"
+#include "vec/core/field.h"
+#include "vec/core/types.h"
+#include "vec/data_types/data_type.h"
+#include "vec/data_types/data_type_decimal.h"
+#include "vec/io/io_helper.h"
+namespace doris {
+namespace vectorized {
+class Arena;
+class BufferReadable;
+class BufferWritable;
+template <typename T>
+class ColumnDecimal;
+template <typename T>
+class DataTypeNumber;
+template <typename>
+class ColumnVector;
+} // namespace vectorized
+} // namespace doris
+
+namespace doris::vectorized {
+
+/*
+ * this function is used to solve agg of sum/count is not compatibility during 
the upgrade process
+ * in PR #20370 have changed the serialize type and serialize column
+ * before is ColumnVector, now sum/count change to use ColumnFixedLengthObject
+ * so during the upgrade process, will be not compatible if exist old BE and 
Newer BE
+ */
+
+template <typename T, typename TResult, typename Data>
+class AggregateFunctionOldSum final
+        : public IAggregateFunctionDataHelper<Data, AggregateFunctionOldSum<T, 
TResult, Data>> {
+public:
+    using ResultDataType = std::conditional_t<IsDecimalNumber<T>, 
DataTypeDecimal<TResult>,
+                                              DataTypeNumber<TResult>>;
+    using ColVecType = std::conditional_t<IsDecimalNumber<T>, 
ColumnDecimal<T>, ColumnVector<T>>;
+    using ColVecResult =
+            std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<TResult>, 
ColumnVector<TResult>>;
+
+    String get_name() const override { return "sum"; }
+
+    AggregateFunctionOldSum(const DataTypes& argument_types_)
+            : IAggregateFunctionDataHelper<Data, AggregateFunctionOldSum<T, 
TResult, Data>>(
+                      argument_types_),
+              scale(get_decimal_scale(*argument_types_[0])) {}
+
+    DataTypePtr get_return_type() const override {
+        if constexpr (IsDecimalNumber<T>) {
+            return 
std::make_shared<ResultDataType>(ResultDataType::max_precision(), scale);
+        } else {
+            return std::make_shared<ResultDataType>();
+        }
+    }
+
+    void add(AggregateDataPtr __restrict place, const IColumn** columns, 
size_t row_num,
+             Arena*) const override {
+        const auto& column = assert_cast<const ColVecType&>(*columns[0]);
+        this->data(place).add(TResult(column.get_data()[row_num]));
+    }
+
+    void reset(AggregateDataPtr place) const override { this->data(place).sum 
= {}; }
+
+    void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs,
+               Arena*) const override {
+        this->data(place).merge(this->data(rhs));
+    }
+
+    void serialize(ConstAggregateDataPtr __restrict place, BufferWritable& 
buf) const override {
+        this->data(place).write(buf);
+    }
+
+    void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf,
+                     Arena*) const override {
+        this->data(place).read(buf);
+    }
+
+    void insert_result_into(ConstAggregateDataPtr __restrict place, IColumn& 
to) const override {
+        auto& column = assert_cast<ColVecResult&>(to);
+        column.get_data().push_back(this->data(place).get());
+    }
+
+    void deserialize_from_column(AggregateDataPtr places, const IColumn& 
column, Arena* arena,

Review Comment:
   warning: pointer parameter 'places' can be pointer to const 
[readability-non-const-parameter]
   
   ```suggestion
       void deserialize_from_column(const AggregateDataPtr places, const 
IColumn& column, Arena* arena,
   ```
   



##########
be/src/vec/common/arithmetic_overflow.h:
##########
@@ -109,4 +125,13 @@ inline bool mul_overflow(__int128 x, __int128 y, __int128& 
res) {
     unsigned __int128 b = (y > 0) ? y : -y;
     return (a * b) / b != a;
 }
+
+template <>
+inline bool mul_overflow(wide::Int256 x, wide::Int256 y, wide::Int256& res) {
+    res = x * y;
+    if (!x || !y) return false;

Review Comment:
   warning: statement should be inside braces 
[readability-braces-around-statements]
   
   ```suggestion
       if (!x || !y) { return false;
   }
   ```
   



##########
be/src/vec/core/accurate_comparison.h:
##########
@@ -464,6 +467,114 @@ template <typename A, typename B>
 inline bool_if_safe_conversion<A, B> greaterOrEqualsOp(A a, B b) {
     return a >= b;
 }
+*/
+
+template <typename A, typename B>
+bool lessOp(A a, B b) {
+    if constexpr (std::is_same_v<A, B>) return a < b;

Review Comment:
   warning: statement should be inside braces 
[readability-braces-around-statements]
   
   ```suggestion
       if constexpr (std::is_same_v<A, B>) { return a < b;
   }
   ```
   



##########
be/src/vec/core/accurate_comparison.h:
##########
@@ -464,6 +467,114 @@ template <typename A, typename B>
 inline bool_if_safe_conversion<A, B> greaterOrEqualsOp(A a, B b) {
     return a >= b;
 }
+*/
+
+template <typename A, typename B>
+bool lessOp(A a, B b) {
+    if constexpr (std::is_same_v<A, B>) return a < b;
+
+    /// float vs float
+    if constexpr (std::is_floating_point_v<A> && std::is_floating_point_v<B>) 
return a < b;
+
+    /// anything vs NaN
+    if (is_nan(a) || is_nan(b)) return false;

Review Comment:
   warning: statement should be inside braces 
[readability-braces-around-statements]
   
   ```suggestion
       if (is_nan(a) || is_nan(b)) { return false;
   }
   ```
   



##########
be/src/vec/core/accurate_comparison.h:
##########
@@ -464,6 +467,114 @@ template <typename A, typename B>
 inline bool_if_safe_conversion<A, B> greaterOrEqualsOp(A a, B b) {
     return a >= b;
 }
+*/
+
+template <typename A, typename B>
+bool lessOp(A a, B b) {
+    if constexpr (std::is_same_v<A, B>) return a < b;
+
+    /// float vs float
+    if constexpr (std::is_floating_point_v<A> && std::is_floating_point_v<B>) 
return a < b;

Review Comment:
   warning: statement should be inside braces 
[readability-braces-around-statements]
   
   ```suggestion
       if constexpr (std::is_floating_point_v<A> && 
std::is_floating_point_v<B>) { return a < b;
   }
   ```
   



##########
be/src/vec/core/accurate_comparison.h:
##########
@@ -464,6 +467,114 @@ template <typename A, typename B>
 inline bool_if_safe_conversion<A, B> greaterOrEqualsOp(A a, B b) {
     return a >= b;
 }
+*/
+
+template <typename A, typename B>
+bool lessOp(A a, B b) {
+    if constexpr (std::is_same_v<A, B>) return a < b;
+
+    /// float vs float
+    if constexpr (std::is_floating_point_v<A> && std::is_floating_point_v<B>) 
return a < b;
+
+    /// anything vs NaN
+    if (is_nan(a) || is_nan(b)) return false;
+
+    /// int vs int
+    if constexpr (is_integer<A> && is_integer<B>) {
+        /// same signedness
+        if constexpr (is_signed_v<A> == is_signed_v<B>) return a < b;
+
+        /// different signedness
+
+        if constexpr (is_signed_v<A> && !is_signed_v<B>)
+            return a < 0 || static_cast<make_unsigned_t<A>>(a) < b;

Review Comment:
   warning: statement should be inside braces 
[readability-braces-around-statements]
   
   ```suggestion
           if constexpr (is_signed_v<A> && !is_signed_v<B>) {
               return a < 0 || static_cast<make_unsigned_t<A>>(a) < b;
   }
   ```
   



##########
be/src/vec/core/accurate_comparison.h:
##########
@@ -464,6 +467,114 @@ template <typename A, typename B>
 inline bool_if_safe_conversion<A, B> greaterOrEqualsOp(A a, B b) {
     return a >= b;
 }
+*/
+
+template <typename A, typename B>
+bool lessOp(A a, B b) {
+    if constexpr (std::is_same_v<A, B>) return a < b;
+
+    /// float vs float
+    if constexpr (std::is_floating_point_v<A> && std::is_floating_point_v<B>) 
return a < b;
+
+    /// anything vs NaN
+    if (is_nan(a) || is_nan(b)) return false;
+
+    /// int vs int
+    if constexpr (is_integer<A> && is_integer<B>) {
+        /// same signedness
+        if constexpr (is_signed_v<A> == is_signed_v<B>) return a < b;
+
+        /// different signedness
+
+        if constexpr (is_signed_v<A> && !is_signed_v<B>)
+            return a < 0 || static_cast<make_unsigned_t<A>>(a) < b;
+
+        if constexpr (!is_signed_v<A> && is_signed_v<B>)

Review Comment:
   warning: statement should be inside braces 
[readability-braces-around-statements]
   ```cpp
           if constexpr (!is_signed_v<A> && is_signed_v<B>)
                                                           ^
   ```
   



##########
be/src/vec/core/accurate_comparison.h:
##########
@@ -464,6 +467,114 @@ template <typename A, typename B>
 inline bool_if_safe_conversion<A, B> greaterOrEqualsOp(A a, B b) {
     return a >= b;
 }
+*/
+
+template <typename A, typename B>
+bool lessOp(A a, B b) {
+    if constexpr (std::is_same_v<A, B>) return a < b;
+
+    /// float vs float
+    if constexpr (std::is_floating_point_v<A> && std::is_floating_point_v<B>) 
return a < b;
+
+    /// anything vs NaN
+    if (is_nan(a) || is_nan(b)) return false;
+
+    /// int vs int
+    if constexpr (is_integer<A> && is_integer<B>) {
+        /// same signedness
+        if constexpr (is_signed_v<A> == is_signed_v<B>) return a < b;
+
+        /// different signedness
+
+        if constexpr (is_signed_v<A> && !is_signed_v<B>)
+            return a < 0 || static_cast<make_unsigned_t<A>>(a) < b;
+
+        if constexpr (!is_signed_v<A> && is_signed_v<B>)
+            return b >= 0 && a < static_cast<make_unsigned_t<B>>(b);
+    }
+
+    /// int vs float
+    if constexpr (is_integer<A> && std::is_floating_point_v<B>) {
+        if constexpr (sizeof(A) <= 4) return static_cast<double>(a) < 
static_cast<double>(b);
+
+        return DecomposedFloat<B>(b).greater(a);
+    }
+
+    if constexpr (std::is_floating_point_v<A> && is_integer<B>) {
+        if constexpr (sizeof(B) <= 4) return static_cast<double>(a) < 
static_cast<double>(b);
+
+        return DecomposedFloat<A>(a).less(b);
+    }
+
+    static_assert(is_integer<A> || std::is_floating_point_v<A>);
+    static_assert(is_integer<B> || std::is_floating_point_v<B>);
+    __builtin_unreachable();
+}
+
+template <typename A, typename B>
+bool greaterOp(A a, B b) {
+    return lessOp(b, a);
+}
+
+template <typename A, typename B>
+bool greaterOrEqualsOp(A a, B b) {
+    if (is_nan(a) || is_nan(b)) return false;

Review Comment:
   warning: statement should be inside braces 
[readability-braces-around-statements]
   
   ```suggestion
       if (is_nan(a) || is_nan(b)) { return false;
   }
   ```
   



##########
be/src/vec/core/accurate_comparison.h:
##########
@@ -464,6 +467,114 @@ template <typename A, typename B>
 inline bool_if_safe_conversion<A, B> greaterOrEqualsOp(A a, B b) {
     return a >= b;
 }
+*/
+
+template <typename A, typename B>
+bool lessOp(A a, B b) {
+    if constexpr (std::is_same_v<A, B>) return a < b;
+
+    /// float vs float
+    if constexpr (std::is_floating_point_v<A> && std::is_floating_point_v<B>) 
return a < b;
+
+    /// anything vs NaN
+    if (is_nan(a) || is_nan(b)) return false;
+
+    /// int vs int
+    if constexpr (is_integer<A> && is_integer<B>) {
+        /// same signedness
+        if constexpr (is_signed_v<A> == is_signed_v<B>) return a < b;
+
+        /// different signedness
+
+        if constexpr (is_signed_v<A> && !is_signed_v<B>)
+            return a < 0 || static_cast<make_unsigned_t<A>>(a) < b;
+
+        if constexpr (!is_signed_v<A> && is_signed_v<B>)
+            return b >= 0 && a < static_cast<make_unsigned_t<B>>(b);
+    }
+
+    /// int vs float
+    if constexpr (is_integer<A> && std::is_floating_point_v<B>) {
+        if constexpr (sizeof(A) <= 4) return static_cast<double>(a) < 
static_cast<double>(b);
+
+        return DecomposedFloat<B>(b).greater(a);
+    }
+
+    if constexpr (std::is_floating_point_v<A> && is_integer<B>) {
+        if constexpr (sizeof(B) <= 4) return static_cast<double>(a) < 
static_cast<double>(b);

Review Comment:
   warning: statement should be inside braces 
[readability-braces-around-statements]
   
   ```suggestion
           if constexpr (sizeof(B) <= 4) { return static_cast<double>(a) < 
static_cast<double>(b);
   }
   ```
   



##########
be/src/vec/core/accurate_comparison.h:
##########
@@ -464,6 +467,114 @@ template <typename A, typename B>
 inline bool_if_safe_conversion<A, B> greaterOrEqualsOp(A a, B b) {
     return a >= b;
 }
+*/
+
+template <typename A, typename B>
+bool lessOp(A a, B b) {
+    if constexpr (std::is_same_v<A, B>) return a < b;
+
+    /// float vs float
+    if constexpr (std::is_floating_point_v<A> && std::is_floating_point_v<B>) 
return a < b;
+
+    /// anything vs NaN
+    if (is_nan(a) || is_nan(b)) return false;
+
+    /// int vs int
+    if constexpr (is_integer<A> && is_integer<B>) {
+        /// same signedness
+        if constexpr (is_signed_v<A> == is_signed_v<B>) return a < b;
+
+        /// different signedness
+
+        if constexpr (is_signed_v<A> && !is_signed_v<B>)
+            return a < 0 || static_cast<make_unsigned_t<A>>(a) < b;
+
+        if constexpr (!is_signed_v<A> && is_signed_v<B>)
+            return b >= 0 && a < static_cast<make_unsigned_t<B>>(b);
+    }
+
+    /// int vs float
+    if constexpr (is_integer<A> && std::is_floating_point_v<B>) {
+        if constexpr (sizeof(A) <= 4) return static_cast<double>(a) < 
static_cast<double>(b);
+
+        return DecomposedFloat<B>(b).greater(a);
+    }
+
+    if constexpr (std::is_floating_point_v<A> && is_integer<B>) {
+        if constexpr (sizeof(B) <= 4) return static_cast<double>(a) < 
static_cast<double>(b);
+
+        return DecomposedFloat<A>(a).less(b);
+    }
+
+    static_assert(is_integer<A> || std::is_floating_point_v<A>);
+    static_assert(is_integer<B> || std::is_floating_point_v<B>);
+    __builtin_unreachable();
+}
+
+template <typename A, typename B>
+bool greaterOp(A a, B b) {
+    return lessOp(b, a);
+}
+
+template <typename A, typename B>
+bool greaterOrEqualsOp(A a, B b) {
+    if (is_nan(a) || is_nan(b)) return false;
+
+    return !lessOp(a, b);
+}
+
+template <typename A, typename B>
+bool lessOrEqualsOp(A a, B b) {
+    if (is_nan(a) || is_nan(b)) return false;
+
+    return !lessOp(b, a);
+}
+
+template <typename A, typename B>
+bool equalsOp(A a, B b) {
+    if constexpr (std::is_same_v<A, B>) return a == b;
+
+    /// float vs float
+    if constexpr (std::is_floating_point_v<A> && std::is_floating_point_v<B>) 
return a == b;

Review Comment:
   warning: statement should be inside braces 
[readability-braces-around-statements]
   
   ```suggestion
       if constexpr (std::is_floating_point_v<A> && 
std::is_floating_point_v<B>) { return a == b;
   }
   ```
   



##########
be/src/vec/core/accurate_comparison.h:
##########
@@ -464,6 +467,114 @@ template <typename A, typename B>
 inline bool_if_safe_conversion<A, B> greaterOrEqualsOp(A a, B b) {
     return a >= b;
 }
+*/
+
+template <typename A, typename B>
+bool lessOp(A a, B b) {
+    if constexpr (std::is_same_v<A, B>) return a < b;
+
+    /// float vs float
+    if constexpr (std::is_floating_point_v<A> && std::is_floating_point_v<B>) 
return a < b;
+
+    /// anything vs NaN
+    if (is_nan(a) || is_nan(b)) return false;
+
+    /// int vs int
+    if constexpr (is_integer<A> && is_integer<B>) {
+        /// same signedness
+        if constexpr (is_signed_v<A> == is_signed_v<B>) return a < b;
+
+        /// different signedness
+
+        if constexpr (is_signed_v<A> && !is_signed_v<B>)
+            return a < 0 || static_cast<make_unsigned_t<A>>(a) < b;
+
+        if constexpr (!is_signed_v<A> && is_signed_v<B>)
+            return b >= 0 && a < static_cast<make_unsigned_t<B>>(b);
+    }
+
+    /// int vs float
+    if constexpr (is_integer<A> && std::is_floating_point_v<B>) {
+        if constexpr (sizeof(A) <= 4) return static_cast<double>(a) < 
static_cast<double>(b);
+
+        return DecomposedFloat<B>(b).greater(a);
+    }
+
+    if constexpr (std::is_floating_point_v<A> && is_integer<B>) {
+        if constexpr (sizeof(B) <= 4) return static_cast<double>(a) < 
static_cast<double>(b);
+
+        return DecomposedFloat<A>(a).less(b);
+    }
+
+    static_assert(is_integer<A> || std::is_floating_point_v<A>);
+    static_assert(is_integer<B> || std::is_floating_point_v<B>);
+    __builtin_unreachable();
+}
+
+template <typename A, typename B>
+bool greaterOp(A a, B b) {
+    return lessOp(b, a);
+}
+
+template <typename A, typename B>
+bool greaterOrEqualsOp(A a, B b) {
+    if (is_nan(a) || is_nan(b)) return false;
+
+    return !lessOp(a, b);
+}
+
+template <typename A, typename B>
+bool lessOrEqualsOp(A a, B b) {
+    if (is_nan(a) || is_nan(b)) return false;
+
+    return !lessOp(b, a);
+}
+
+template <typename A, typename B>
+bool equalsOp(A a, B b) {
+    if constexpr (std::is_same_v<A, B>) return a == b;

Review Comment:
   warning: statement should be inside braces 
[readability-braces-around-statements]
   
   ```suggestion
       if constexpr (std::is_same_v<A, B>) { return a == b;
   }
   ```
   



##########
be/src/vec/aggregate_functions/aggregate_function_sum_old.h:
##########
@@ -0,0 +1,201 @@
+// 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.
+// This file is copied from
+// 
https://github.com/ClickHouse/ClickHouse/blob/master/src/AggregateFunctions/AggregateFunctionOldSum.h
+// and modified by Doris
+
+#pragma once
+
+#include <stddef.h>
+
+#include <memory>
+#include <type_traits>
+#include <vector>
+
+#include "vec/aggregate_functions/aggregate_function.h"
+#include "vec/aggregate_functions/aggregate_function_sum.h"
+#include "vec/columns/column.h"
+#include "vec/common/assert_cast.h"
+#include "vec/core/field.h"
+#include "vec/core/types.h"
+#include "vec/data_types/data_type.h"
+#include "vec/data_types/data_type_decimal.h"
+#include "vec/io/io_helper.h"
+namespace doris {
+namespace vectorized {

Review Comment:
   warning: nested namespaces can be concatenated 
[modernize-concat-nested-namespaces]
   
   ```suggestion
   namespace doris::vectorized {
   ```
   
   be/src/vec/aggregate_functions/aggregate_function_sum_old.h:48:
   ```diff
   - } // namespace vectorized
   - } // namespace doris
   + } // namespace doris
   ```
   



##########
be/src/vec/aggregate_functions/aggregate_function_sum_old.h:
##########
@@ -0,0 +1,201 @@
+// 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.
+// This file is copied from
+// 
https://github.com/ClickHouse/ClickHouse/blob/master/src/AggregateFunctions/AggregateFunctionOldSum.h
+// and modified by Doris
+
+#pragma once
+
+#include <stddef.h>
+
+#include <memory>
+#include <type_traits>
+#include <vector>
+
+#include "vec/aggregate_functions/aggregate_function.h"
+#include "vec/aggregate_functions/aggregate_function_sum.h"
+#include "vec/columns/column.h"
+#include "vec/common/assert_cast.h"
+#include "vec/core/field.h"
+#include "vec/core/types.h"
+#include "vec/data_types/data_type.h"
+#include "vec/data_types/data_type_decimal.h"
+#include "vec/io/io_helper.h"
+namespace doris {
+namespace vectorized {
+class Arena;
+class BufferReadable;
+class BufferWritable;
+template <typename T>
+class ColumnDecimal;
+template <typename T>
+class DataTypeNumber;
+template <typename>
+class ColumnVector;
+} // namespace vectorized
+} // namespace doris
+
+namespace doris::vectorized {
+
+/*
+ * this function is used to solve agg of sum/count is not compatibility during 
the upgrade process
+ * in PR #20370 have changed the serialize type and serialize column
+ * before is ColumnVector, now sum/count change to use ColumnFixedLengthObject
+ * so during the upgrade process, will be not compatible if exist old BE and 
Newer BE
+ */
+
+template <typename T, typename TResult, typename Data>
+class AggregateFunctionOldSum final
+        : public IAggregateFunctionDataHelper<Data, AggregateFunctionOldSum<T, 
TResult, Data>> {
+public:
+    using ResultDataType = std::conditional_t<IsDecimalNumber<T>, 
DataTypeDecimal<TResult>,
+                                              DataTypeNumber<TResult>>;
+    using ColVecType = std::conditional_t<IsDecimalNumber<T>, 
ColumnDecimal<T>, ColumnVector<T>>;
+    using ColVecResult =
+            std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<TResult>, 
ColumnVector<TResult>>;
+
+    String get_name() const override { return "sum"; }
+
+    AggregateFunctionOldSum(const DataTypes& argument_types_)
+            : IAggregateFunctionDataHelper<Data, AggregateFunctionOldSum<T, 
TResult, Data>>(
+                      argument_types_),
+              scale(get_decimal_scale(*argument_types_[0])) {}
+
+    DataTypePtr get_return_type() const override {
+        if constexpr (IsDecimalNumber<T>) {
+            return 
std::make_shared<ResultDataType>(ResultDataType::max_precision(), scale);
+        } else {
+            return std::make_shared<ResultDataType>();
+        }
+    }
+
+    void add(AggregateDataPtr __restrict place, const IColumn** columns, 
size_t row_num,
+             Arena*) const override {
+        const auto& column = assert_cast<const ColVecType&>(*columns[0]);
+        this->data(place).add(TResult(column.get_data()[row_num]));
+    }
+
+    void reset(AggregateDataPtr place) const override { this->data(place).sum 
= {}; }
+
+    void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs,
+               Arena*) const override {

Review Comment:
   warning: all parameters should be named in a function 
[readability-named-parameter]
   
   ```suggestion
                  Arena* /*unused*/) const override {
   ```
   



##########
be/src/vec/common/int_exp.h:
##########
@@ -78,4 +80,94 @@ inline constexpr __int128 exp10_i128(int x) {
     return exp_details::get_exp<__int128, 10, 39>(x);
 }
 
+using wide::Int256;
+inline Int256 exp10_i256(int x) {
+    if (x < 0) return 0;

Review Comment:
   warning: statement should be inside braces 
[readability-braces-around-statements]
   
   ```suggestion
       if (x < 0) { return 0;
   }
   ```
   



##########
be/src/vec/core/accurate_comparison.h:
##########
@@ -464,6 +467,114 @@ template <typename A, typename B>
 inline bool_if_safe_conversion<A, B> greaterOrEqualsOp(A a, B b) {
     return a >= b;
 }
+*/
+
+template <typename A, typename B>
+bool lessOp(A a, B b) {
+    if constexpr (std::is_same_v<A, B>) return a < b;
+
+    /// float vs float
+    if constexpr (std::is_floating_point_v<A> && std::is_floating_point_v<B>) 
return a < b;
+
+    /// anything vs NaN
+    if (is_nan(a) || is_nan(b)) return false;
+
+    /// int vs int
+    if constexpr (is_integer<A> && is_integer<B>) {
+        /// same signedness
+        if constexpr (is_signed_v<A> == is_signed_v<B>) return a < b;

Review Comment:
   warning: statement should be inside braces 
[readability-braces-around-statements]
   
   ```suggestion
           if constexpr (is_signed_v<A> == is_signed_v<B>) { return a < b;
   }
   ```
   



##########
be/src/vec/core/accurate_comparison.h:
##########
@@ -464,6 +467,114 @@ template <typename A, typename B>
 inline bool_if_safe_conversion<A, B> greaterOrEqualsOp(A a, B b) {
     return a >= b;
 }
+*/
+
+template <typename A, typename B>
+bool lessOp(A a, B b) {
+    if constexpr (std::is_same_v<A, B>) return a < b;
+
+    /// float vs float
+    if constexpr (std::is_floating_point_v<A> && std::is_floating_point_v<B>) 
return a < b;
+
+    /// anything vs NaN
+    if (is_nan(a) || is_nan(b)) return false;
+
+    /// int vs int
+    if constexpr (is_integer<A> && is_integer<B>) {
+        /// same signedness
+        if constexpr (is_signed_v<A> == is_signed_v<B>) return a < b;
+
+        /// different signedness
+
+        if constexpr (is_signed_v<A> && !is_signed_v<B>)
+            return a < 0 || static_cast<make_unsigned_t<A>>(a) < b;
+
+        if constexpr (!is_signed_v<A> && is_signed_v<B>)
+            return b >= 0 && a < static_cast<make_unsigned_t<B>>(b);
+    }
+
+    /// int vs float
+    if constexpr (is_integer<A> && std::is_floating_point_v<B>) {
+        if constexpr (sizeof(A) <= 4) return static_cast<double>(a) < 
static_cast<double>(b);

Review Comment:
   warning: statement should be inside braces 
[readability-braces-around-statements]
   
   ```suggestion
           if constexpr (sizeof(A) <= 4) { return static_cast<double>(a) < 
static_cast<double>(b);
   }
   ```
   



##########
be/src/vec/core/accurate_comparison.h:
##########
@@ -464,6 +467,114 @@ template <typename A, typename B>
 inline bool_if_safe_conversion<A, B> greaterOrEqualsOp(A a, B b) {
     return a >= b;
 }
+*/
+
+template <typename A, typename B>
+bool lessOp(A a, B b) {
+    if constexpr (std::is_same_v<A, B>) return a < b;
+
+    /// float vs float
+    if constexpr (std::is_floating_point_v<A> && std::is_floating_point_v<B>) 
return a < b;
+
+    /// anything vs NaN
+    if (is_nan(a) || is_nan(b)) return false;
+
+    /// int vs int
+    if constexpr (is_integer<A> && is_integer<B>) {
+        /// same signedness
+        if constexpr (is_signed_v<A> == is_signed_v<B>) return a < b;
+
+        /// different signedness
+
+        if constexpr (is_signed_v<A> && !is_signed_v<B>)
+            return a < 0 || static_cast<make_unsigned_t<A>>(a) < b;
+
+        if constexpr (!is_signed_v<A> && is_signed_v<B>)
+            return b >= 0 && a < static_cast<make_unsigned_t<B>>(b);
+    }
+
+    /// int vs float
+    if constexpr (is_integer<A> && std::is_floating_point_v<B>) {
+        if constexpr (sizeof(A) <= 4) return static_cast<double>(a) < 
static_cast<double>(b);
+
+        return DecomposedFloat<B>(b).greater(a);
+    }
+
+    if constexpr (std::is_floating_point_v<A> && is_integer<B>) {
+        if constexpr (sizeof(B) <= 4) return static_cast<double>(a) < 
static_cast<double>(b);
+
+        return DecomposedFloat<A>(a).less(b);
+    }
+
+    static_assert(is_integer<A> || std::is_floating_point_v<A>);
+    static_assert(is_integer<B> || std::is_floating_point_v<B>);
+    __builtin_unreachable();
+}
+
+template <typename A, typename B>
+bool greaterOp(A a, B b) {
+    return lessOp(b, a);
+}
+
+template <typename A, typename B>
+bool greaterOrEqualsOp(A a, B b) {
+    if (is_nan(a) || is_nan(b)) return false;
+
+    return !lessOp(a, b);
+}
+
+template <typename A, typename B>
+bool lessOrEqualsOp(A a, B b) {
+    if (is_nan(a) || is_nan(b)) return false;

Review Comment:
   warning: statement should be inside braces 
[readability-braces-around-statements]
   
   ```suggestion
       if (is_nan(a) || is_nan(b)) { return false;
   }
   ```
   



-- 
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-unsubscr...@doris.apache.org

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


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


Reply via email to