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

yiguolei pushed a commit to branch branch-2.1
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/branch-2.1 by this push:
     new 8fdfbcb3c4d Revert "[Opt](func) opt the percentile func performance 
(#34373) (#34416)"
8fdfbcb3c4d is described below

commit 8fdfbcb3c4ddf2ad0f78f544d4c342f86aaff29c
Author: yiguolei <[email protected]>
AuthorDate: Tue May 7 07:23:48 2024 +0800

    Revert "[Opt](func) opt the percentile func performance (#34373) (#34416)"
    
    This reverts commit 509ae425e416b4779ae94eab9c2b21f9850e03c3.
---
 be/src/agent/be_exec_version_manager.h             |   5 +-
 be/src/util/counts.h                               | 200 +--------
 .../aggregate_function_percentile.cpp              |  64 ---
 .../aggregate_function_percentile.h                | 473 ---------------------
 .../aggregate_function_percentile_approx.cpp       |  48 ++-
 .../aggregate_function_percentile_approx.h         | 289 +++++++++++--
 .../aggregate_function_simple_factory.cpp          |   2 -
 be/src/vec/common/string_buffer.hpp                |  11 +-
 be/src/vec/functions/simple_function_factory.h     |  10 +-
 be/test/util/counts_test.cpp                       |  27 +-
 .../main/java/org/apache/doris/common/Config.java  |   2 +-
 11 files changed, 322 insertions(+), 809 deletions(-)

diff --git a/be/src/agent/be_exec_version_manager.h 
b/be/src/agent/be_exec_version_manager.h
index a06899d7a70..32a520cc4bd 100644
--- a/be/src/agent/be_exec_version_manager.h
+++ b/be/src/agent/be_exec_version_manager.h
@@ -70,11 +70,8 @@ private:
  *    f. shrink some function's nullable mode.
  *    g. do local merge of remote runtime filter
  *    h. "now": ALWAYS_NOT_NULLABLE -> DEPEND_ON_ARGUMENTS
- *
- * 5: start from doris 2.1.4
- *    a. change the impl of percentile
 */
-constexpr inline int BeExecVersionManager::max_be_exec_version = 5;
+constexpr inline int BeExecVersionManager::max_be_exec_version = 4;
 constexpr inline int BeExecVersionManager::min_be_exec_version = 0;
 
 /// functional
diff --git a/be/src/util/counts.h b/be/src/util/counts.h
index 70469d6fa72..fec18cedcd6 100644
--- a/be/src/util/counts.h
+++ b/be/src/util/counts.h
@@ -17,24 +17,20 @@
 
 #pragma once
 
-#include <pdqsort.h>
-
 #include <algorithm>
 #include <cmath>
-#include <queue>
+#include <unordered_map>
+#include <vector>
 
 #include "udf/udf.h"
-#include "vec/common/pod_array.h"
-#include "vec/common/string_buffer.hpp"
-#include "vec/io/io_helper.h"
 
 namespace doris {
 
-class OldCounts {
+class Counts {
 public:
-    OldCounts() = default;
+    Counts() = default;
 
-    inline void merge(const OldCounts* other) {
+    inline void merge(const Counts* other) {
         if (other == nullptr || other->_counts.empty()) {
             return;
         }
@@ -139,190 +135,4 @@ private:
     std::unordered_map<int64_t, uint32_t> _counts;
 };
 
-// #TODO use template to reduce the Counts memery. Eg: Int do not need use 
int64_t
-class Counts {
-public:
-    Counts() = default;
-
-    void merge(Counts* other) {
-        if (other != nullptr && !other->_nums.empty()) {
-            _sorted_nums_vec.emplace_back(std::move(other->_nums));
-        }
-    }
-
-    void increment(int64_t key, uint32_t i) {
-        auto old_size = _nums.size();
-        _nums.resize(_nums.size() + i);
-        for (uint32_t j = 0; j < i; ++j) {
-            _nums[old_size + j] = key;
-        }
-    }
-
-    void serialize(vectorized::BufferWritable& buf) {
-        if (!_nums.empty()) {
-            pdqsort(_nums.begin(), _nums.end());
-            size_t size = _nums.size();
-            write_binary(size, buf);
-            buf.write(reinterpret_cast<const char*>(_nums.data()), 
sizeof(int64_t) * size);
-        } else {
-            // convert _sorted_nums_vec to _nums and do seiralize again
-            _convert_sorted_num_vec_to_nums();
-            serialize(buf);
-        }
-    }
-
-    void unserialize(vectorized::BufferReadable& buf) {
-        size_t size;
-        read_binary(size, buf);
-        _nums.resize(size);
-        auto buff = buf.read(sizeof(int64_t) * size);
-        memcpy(_nums.data(), buff.data, buff.size);
-    }
-
-    double terminate(double quantile) {
-        if (_sorted_nums_vec.size() <= 1) {
-            if (_sorted_nums_vec.size() == 1) {
-                _nums = std::move(_sorted_nums_vec[0]);
-            }
-
-            if (_nums.empty()) {
-                // Although set null here, but the value is 0.0 and the call 
method just
-                // get val in aggregate_function_percentile_approx.h
-                return 0.0;
-            }
-            if (quantile == 1 || _nums.size() == 1) {
-                return _nums.back();
-            }
-            if (UNLIKELY(!std::is_sorted(_nums.begin(), _nums.end()))) {
-                pdqsort(_nums.begin(), _nums.end());
-            }
-
-            double u = (_nums.size() - 1) * quantile;
-            auto index = static_cast<uint32_t>(u);
-            return _nums[index] +
-                   (u - static_cast<double>(index)) * (_nums[index + 1] - 
_nums[index]);
-        } else {
-            DCHECK(_nums.empty());
-            size_t rows = 0;
-            for (const auto& i : _sorted_nums_vec) {
-                rows += i.size();
-            }
-            const bool reverse = quantile > 0.5 && rows > 2;
-            double u = (rows - 1) * quantile;
-            auto index = static_cast<uint32_t>(u);
-            // if reverse, the step of target should start 0 like not reverse
-            // so here rows need to minus index + 2
-            // eg: rows = 10, index = 5
-            // if not reverse, so the first number loc is 5, the second number 
loc is 6
-            // if reverse, so the second number is 3, the first number is 4
-            // 5 + 4 = 3 + 6 = 9 = rows - 1.
-            // the rows must GE 2 beacuse `_sorted_nums_vec` size GE 2
-            size_t target = reverse ? rows - index - 2 : index;
-            if (quantile == 1) {
-                target = 0;
-            }
-            auto [first_number, second_number] = 
_merge_sort_and_get_numbers(target, reverse);
-            if (quantile == 1) {
-                return second_number;
-            }
-            return first_number + (u - static_cast<double>(index)) * 
(second_number - first_number);
-        }
-    }
-
-private:
-    struct Node {
-        int64_t value;
-        int array_index;
-        int64_t element_index;
-
-        std::strong_ordering operator<=>(const Node& other) const { return 
value <=> other.value; }
-    };
-
-    void _convert_sorted_num_vec_to_nums() {
-        size_t rows = 0;
-        for (const auto& i : _sorted_nums_vec) {
-            rows += i.size();
-        }
-        _nums.resize(rows);
-        size_t count = 0;
-
-        std::priority_queue<Node, std::vector<Node>, std::greater<Node>> 
min_heap;
-        for (int i = 0; i < _sorted_nums_vec.size(); ++i) {
-            if (!_sorted_nums_vec[i].empty()) {
-                min_heap.emplace(_sorted_nums_vec[i][0], i, 0);
-            }
-        }
-
-        while (!min_heap.empty()) {
-            Node node = min_heap.top();
-            min_heap.pop();
-            _nums[count++] = node.value;
-            if (++node.element_index < 
_sorted_nums_vec[node.array_index].size()) {
-                node.value = 
_sorted_nums_vec[node.array_index][node.element_index];
-                min_heap.push(node);
-            }
-        }
-        _sorted_nums_vec.clear();
-    }
-
-    std::pair<int64_t, int64_t> _merge_sort_and_get_numbers(int64_t target, 
bool reverse) {
-        int64_t first_number = 0, second_number = 0;
-        size_t count = 0;
-        if (reverse) {
-            std::priority_queue<Node> max_heap;
-            for (int i = 0; i < _sorted_nums_vec.size(); ++i) {
-                if (!_sorted_nums_vec[i].empty()) {
-                    
max_heap.emplace(_sorted_nums_vec[i][_sorted_nums_vec[i].size() - 1], i,
-                                     _sorted_nums_vec[i].size() - 1);
-                }
-            }
-
-            while (!max_heap.empty()) {
-                Node node = max_heap.top();
-                max_heap.pop();
-                if (count == target) {
-                    second_number = node.value;
-                } else if (count == target + 1) {
-                    first_number = node.value;
-                    break;
-                }
-                ++count;
-                if (--node.element_index >= 0) {
-                    node.value = 
_sorted_nums_vec[node.array_index][node.element_index];
-                    max_heap.push(node);
-                }
-            }
-
-        } else {
-            std::priority_queue<Node, std::vector<Node>, std::greater<Node>> 
min_heap;
-            for (int i = 0; i < _sorted_nums_vec.size(); ++i) {
-                if (!_sorted_nums_vec[i].empty()) {
-                    min_heap.emplace(_sorted_nums_vec[i][0], i, 0);
-                }
-            }
-
-            while (!min_heap.empty()) {
-                Node node = min_heap.top();
-                min_heap.pop();
-                if (count == target) {
-                    first_number = node.value;
-                } else if (count == target + 1) {
-                    second_number = node.value;
-                    break;
-                }
-                ++count;
-                if (++node.element_index < 
_sorted_nums_vec[node.array_index].size()) {
-                    node.value = 
_sorted_nums_vec[node.array_index][node.element_index];
-                    min_heap.push(node);
-                }
-            }
-        }
-
-        return {first_number, second_number};
-    }
-
-    vectorized::PODArray<int64_t> _nums;
-    std::vector<vectorized::PODArray<int64_t>> _sorted_nums_vec;
-};
-
 } // namespace doris
diff --git a/be/src/vec/aggregate_functions/aggregate_function_percentile.cpp 
b/be/src/vec/aggregate_functions/aggregate_function_percentile.cpp
deleted file mode 100644
index 079b1da83ff..00000000000
--- a/be/src/vec/aggregate_functions/aggregate_function_percentile.cpp
+++ /dev/null
@@ -1,64 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-
-#include "vec/aggregate_functions/aggregate_function_percentile.h"
-
-#include "vec/aggregate_functions/aggregate_function_simple_factory.h"
-#include "vec/aggregate_functions/helpers.h"
-
-namespace doris::vectorized {
-
-template <bool is_nullable>
-AggregateFunctionPtr create_aggregate_function_percentile_approx(const 
std::string& name,
-                                                                 const 
DataTypes& argument_types,
-                                                                 const bool 
result_is_nullable) {
-    const DataTypePtr& argument_type = remove_nullable(argument_types[0]);
-    WhichDataType which(argument_type);
-    if (which.idx != TypeIndex::Float64) {
-        return nullptr;
-    }
-    if (argument_types.size() == 1) {
-        return 
creator_without_type::create<AggregateFunctionPercentileApproxMerge<is_nullable>>(
-                remove_nullable(argument_types), result_is_nullable);
-    }
-    if (argument_types.size() == 2) {
-        return creator_without_type::create<
-                AggregateFunctionPercentileApproxTwoParams<is_nullable>>(
-                remove_nullable(argument_types), result_is_nullable);
-    }
-    if (argument_types.size() == 3) {
-        return creator_without_type::create<
-                AggregateFunctionPercentileApproxThreeParams<is_nullable>>(
-                remove_nullable(argument_types), result_is_nullable);
-    }
-    return nullptr;
-}
-
-void register_aggregate_function_percentile(AggregateFunctionSimpleFactory& 
factory) {
-    factory.register_function_both("percentile",
-                                   
creator_without_type::creator<AggregateFunctionPercentile>);
-    factory.register_function_both("percentile_array",
-                                   
creator_without_type::creator<AggregateFunctionPercentileArray>);
-}
-
-void 
register_aggregate_function_percentile_approx(AggregateFunctionSimpleFactory& 
factory) {
-    factory.register_function("percentile_approx",
-                              
create_aggregate_function_percentile_approx<false>, false);
-    factory.register_function("percentile_approx",
-                              
create_aggregate_function_percentile_approx<true>, true);
-}
-} // namespace doris::vectorized
\ No newline at end of file
diff --git a/be/src/vec/aggregate_functions/aggregate_function_percentile.h 
b/be/src/vec/aggregate_functions/aggregate_function_percentile.h
deleted file mode 100644
index 6322a80c934..00000000000
--- a/be/src/vec/aggregate_functions/aggregate_function_percentile.h
+++ /dev/null
@@ -1,473 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-
-#pragma once
-
-#include <glog/logging.h>
-#include <stddef.h>
-#include <stdint.h>
-
-#include <algorithm>
-#include <boost/iterator/iterator_facade.hpp>
-#include <cmath>
-#include <memory>
-#include <ostream>
-#include <string>
-#include <vector>
-
-#include "util/counts.h"
-#include "util/tdigest.h"
-#include "vec/aggregate_functions/aggregate_function.h"
-#include "vec/columns/column.h"
-#include "vec/columns/column_array.h"
-#include "vec/columns/column_nullable.h"
-#include "vec/columns/column_vector.h"
-#include "vec/common/assert_cast.h"
-#include "vec/common/pod_array_fwd.h"
-#include "vec/common/string_ref.h"
-#include "vec/core/types.h"
-#include "vec/data_types/data_type_array.h"
-#include "vec/data_types/data_type_nullable.h"
-#include "vec/data_types/data_type_number.h"
-#include "vec/io/io_helper.h"
-
-namespace doris::vectorized {
-
-class Arena;
-class BufferReadable;
-
-struct PercentileApproxState {
-    static constexpr double INIT_QUANTILE = -1.0;
-    PercentileApproxState() = default;
-    ~PercentileApproxState() = default;
-
-    void init(double compression = 10000) {
-        if (!init_flag) {
-            
//https://doris.apache.org/zh-CN/sql-reference/sql-functions/aggregate-functions/percentile_approx.html#description
-            //The compression parameter setting range is [2048, 10000].
-            //If the value of compression parameter is not specified set, or 
is outside the range of [2048, 10000],
-            //will use the default value of 10000
-            if (compression < 2048 || compression > 10000) {
-                compression = 10000;
-            }
-            digest = TDigest::create_unique(compression);
-            compressions = compression;
-            init_flag = true;
-        }
-    }
-
-    void write(BufferWritable& buf) const {
-        write_binary(init_flag, buf);
-        if (!init_flag) {
-            return;
-        }
-
-        write_binary(target_quantile, buf);
-        write_binary(compressions, buf);
-        uint32_t serialize_size = digest->serialized_size();
-        std::string result(serialize_size, '0');
-        DCHECK(digest.get() != nullptr);
-        digest->serialize((uint8_t*)result.c_str());
-
-        write_binary(result, buf);
-    }
-
-    void read(BufferReadable& buf) {
-        read_binary(init_flag, buf);
-        if (!init_flag) {
-            return;
-        }
-
-        read_binary(target_quantile, buf);
-        read_binary(compressions, buf);
-        std::string str;
-        read_binary(str, buf);
-        digest = TDigest::create_unique(compressions);
-        digest->unserialize((uint8_t*)str.c_str());
-    }
-
-    double get() const {
-        if (init_flag) {
-            return digest->quantile(target_quantile);
-        } else {
-            return std::nan("");
-        }
-    }
-
-    void merge(const PercentileApproxState& rhs) {
-        if (!rhs.init_flag) {
-            return;
-        }
-        if (init_flag) {
-            DCHECK(digest.get() != nullptr);
-            digest->merge(rhs.digest.get());
-        } else {
-            digest = TDigest::create_unique(compressions);
-            digest->merge(rhs.digest.get());
-            init_flag = true;
-        }
-        if (target_quantile == PercentileApproxState::INIT_QUANTILE) {
-            target_quantile = rhs.target_quantile;
-        }
-    }
-
-    void add(double source, double quantile) {
-        digest->add(source);
-        target_quantile = quantile;
-    }
-
-    void reset() {
-        target_quantile = INIT_QUANTILE;
-        init_flag = false;
-        digest = TDigest::create_unique(compressions);
-    }
-
-    bool init_flag = false;
-    std::unique_ptr<TDigest> digest;
-    double target_quantile = INIT_QUANTILE;
-    double compressions = 10000;
-};
-
-class AggregateFunctionPercentileApprox
-        : public IAggregateFunctionDataHelper<PercentileApproxState,
-                                              
AggregateFunctionPercentileApprox> {
-public:
-    AggregateFunctionPercentileApprox(const DataTypes& argument_types_)
-            : IAggregateFunctionDataHelper<PercentileApproxState,
-                                           
AggregateFunctionPercentileApprox>(argument_types_) {}
-
-    String get_name() const override { return "percentile_approx"; }
-
-    DataTypePtr get_return_type() const override {
-        return make_nullable(std::make_shared<DataTypeFloat64>());
-    }
-
-    void reset(AggregateDataPtr __restrict place) const override {
-        AggregateFunctionPercentileApprox::data(place).reset();
-    }
-
-    void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs,
-               Arena*) const override {
-        AggregateFunctionPercentileApprox::data(place).merge(
-                AggregateFunctionPercentileApprox::data(rhs));
-    }
-
-    void serialize(ConstAggregateDataPtr __restrict place, BufferWritable& 
buf) const override {
-        AggregateFunctionPercentileApprox::data(place).write(buf);
-    }
-
-    void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf,
-                     Arena*) const override {
-        AggregateFunctionPercentileApprox::data(place).read(buf);
-    }
-
-    void insert_result_into(ConstAggregateDataPtr __restrict place, IColumn& 
to) const override {
-        ColumnNullable& nullable_column = assert_cast<ColumnNullable&>(to);
-        double result = AggregateFunctionPercentileApprox::data(place).get();
-
-        if (std::isnan(result)) {
-            nullable_column.insert_default();
-        } else {
-            auto& col = 
assert_cast<ColumnVector<Float64>&>(nullable_column.get_nested_column());
-            col.get_data().push_back(result);
-            nullable_column.get_null_map_data().push_back(0);
-        }
-    }
-};
-
-// only for merge
-template <bool is_nullable>
-class AggregateFunctionPercentileApproxMerge : public 
AggregateFunctionPercentileApprox {
-public:
-    AggregateFunctionPercentileApproxMerge(const DataTypes& argument_types_)
-            : AggregateFunctionPercentileApprox(argument_types_) {}
-    void add(AggregateDataPtr __restrict place, const IColumn** columns, 
ssize_t row_num,
-             Arena*) const override {
-        LOG(FATAL) << "AggregateFunctionPercentileApproxMerge do not support 
add()";
-        __builtin_unreachable();
-    }
-};
-
-template <bool is_nullable>
-class AggregateFunctionPercentileApproxTwoParams : public 
AggregateFunctionPercentileApprox {
-public:
-    AggregateFunctionPercentileApproxTwoParams(const DataTypes& 
argument_types_)
-            : AggregateFunctionPercentileApprox(argument_types_) {}
-    void add(AggregateDataPtr __restrict place, const IColumn** columns, 
ssize_t row_num,
-             Arena*) const override {
-        if constexpr (is_nullable) {
-            double column_data[2] = {0, 0};
-
-            for (int i = 0; i < 2; ++i) {
-                const auto* nullable_column = 
check_and_get_column<ColumnNullable>(columns[i]);
-                if (nullable_column == nullptr) { //Not Nullable column
-                    const auto& column = assert_cast<const 
ColumnVector<Float64>&>(*columns[i]);
-                    column_data[i] = column.get_float64(row_num);
-                } else if (!nullable_column->is_null_at(
-                                   row_num)) { // Nullable column && Not null 
data
-                    const auto& column = assert_cast<const 
ColumnVector<Float64>&>(
-                            nullable_column->get_nested_column());
-                    column_data[i] = column.get_float64(row_num);
-                } else { // Nullable column && null data
-                    if (i == 0) {
-                        return;
-                    }
-                }
-            }
-
-            this->data(place).init();
-            this->data(place).add(column_data[0], column_data[1]);
-
-        } else {
-            const auto& sources = assert_cast<const 
ColumnVector<Float64>&>(*columns[0]);
-            const auto& quantile = assert_cast<const 
ColumnVector<Float64>&>(*columns[1]);
-
-            this->data(place).init();
-            this->data(place).add(sources.get_float64(row_num), 
quantile.get_float64(row_num));
-        }
-    }
-};
-
-template <bool is_nullable>
-class AggregateFunctionPercentileApproxThreeParams : public 
AggregateFunctionPercentileApprox {
-public:
-    AggregateFunctionPercentileApproxThreeParams(const DataTypes& 
argument_types_)
-            : AggregateFunctionPercentileApprox(argument_types_) {}
-    void add(AggregateDataPtr __restrict place, const IColumn** columns, 
ssize_t row_num,
-             Arena*) const override {
-        if constexpr (is_nullable) {
-            double column_data[3] = {0, 0, 0};
-
-            for (int i = 0; i < 3; ++i) {
-                const auto* nullable_column = 
check_and_get_column<ColumnNullable>(columns[i]);
-                if (nullable_column == nullptr) { //Not Nullable column
-                    const auto& column = assert_cast<const 
ColumnVector<Float64>&>(*columns[i]);
-                    column_data[i] = column.get_float64(row_num);
-                } else if (!nullable_column->is_null_at(
-                                   row_num)) { // Nullable column && Not null 
data
-                    const auto& column = assert_cast<const 
ColumnVector<Float64>&>(
-                            nullable_column->get_nested_column());
-                    column_data[i] = column.get_float64(row_num);
-                } else { // Nullable column && null data
-                    if (i == 0) {
-                        return;
-                    }
-                }
-            }
-
-            this->data(place).init(column_data[2]);
-            this->data(place).add(column_data[0], column_data[1]);
-
-        } else {
-            const auto& sources = assert_cast<const 
ColumnVector<Float64>&>(*columns[0]);
-            const auto& quantile = assert_cast<const 
ColumnVector<Float64>&>(*columns[1]);
-            const auto& compression = assert_cast<const 
ColumnVector<Float64>&>(*columns[2]);
-
-            this->data(place).init(compression.get_float64(row_num));
-            this->data(place).add(sources.get_float64(row_num), 
quantile.get_float64(row_num));
-        }
-    }
-};
-
-struct PercentileState {
-    mutable std::vector<Counts> vec_counts;
-    std::vector<double> vec_quantile {-1};
-    bool inited_flag = false;
-
-    void write(BufferWritable& buf) const {
-        write_binary(inited_flag, buf);
-        int size_num = vec_quantile.size();
-        write_binary(size_num, buf);
-        for (const auto& quantile : vec_quantile) {
-            write_binary(quantile, buf);
-        }
-        for (auto& counts : vec_counts) {
-            counts.serialize(buf);
-        }
-    }
-
-    void read(BufferReadable& buf) {
-        read_binary(inited_flag, buf);
-        int size_num = 0;
-        read_binary(size_num, buf);
-        double data = 0.0;
-        vec_quantile.clear();
-        for (int i = 0; i < size_num; ++i) {
-            read_binary(data, buf);
-            vec_quantile.emplace_back(data);
-        }
-        vec_counts.clear();
-        vec_counts.resize(size_num);
-        for (int i = 0; i < size_num; ++i) {
-            vec_counts[i].unserialize(buf);
-        }
-    }
-
-    void add(int64_t source, const PaddedPODArray<Float64>& quantiles, int 
arg_size) {
-        if (!inited_flag) {
-            vec_counts.resize(arg_size);
-            vec_quantile.resize(arg_size, -1);
-            inited_flag = true;
-            for (int i = 0; i < arg_size; ++i) {
-                vec_quantile[i] = quantiles[i];
-            }
-        }
-        for (int i = 0; i < arg_size; ++i) {
-            vec_counts[i].increment(source, 1);
-        }
-    }
-
-    void merge(const PercentileState& rhs) {
-        if (!rhs.inited_flag) {
-            return;
-        }
-        int size_num = rhs.vec_quantile.size();
-        if (!inited_flag) {
-            vec_counts.resize(size_num);
-            vec_quantile.resize(size_num, -1);
-            inited_flag = true;
-        }
-
-        for (int i = 0; i < size_num; ++i) {
-            if (vec_quantile[i] == -1.0) {
-                vec_quantile[i] = rhs.vec_quantile[i];
-            }
-            vec_counts[i].merge(const_cast<Counts*>(&(rhs.vec_counts[i])));
-        }
-    }
-
-    void reset() {
-        vec_counts.clear();
-        vec_quantile.clear();
-        inited_flag = false;
-    }
-
-    double get() const { return vec_counts[0].terminate(vec_quantile[0]); }
-
-    void insert_result_into(IColumn& to) const {
-        auto& column_data = assert_cast<ColumnVector<Float64>&>(to).get_data();
-        for (int i = 0; i < vec_counts.size(); ++i) {
-            column_data.push_back(vec_counts[i].terminate(vec_quantile[i]));
-        }
-    }
-};
-
-class AggregateFunctionPercentile final
-        : public IAggregateFunctionDataHelper<PercentileState, 
AggregateFunctionPercentile> {
-public:
-    AggregateFunctionPercentile(const DataTypes& argument_types_)
-            : IAggregateFunctionDataHelper<PercentileState, 
AggregateFunctionPercentile>(
-                      argument_types_) {}
-
-    String get_name() const override { return "percentile"; }
-
-    DataTypePtr get_return_type() const override { return 
std::make_shared<DataTypeFloat64>(); }
-
-    void add(AggregateDataPtr __restrict place, const IColumn** columns, 
ssize_t row_num,
-             Arena*) const override {
-        const auto& sources = assert_cast<const 
ColumnVector<Int64>&>(*columns[0]);
-        const auto& quantile = assert_cast<const 
ColumnVector<Float64>&>(*columns[1]);
-        AggregateFunctionPercentile::data(place).add(sources.get_int(row_num), 
quantile.get_data(),
-                                                     1);
-    }
-
-    void reset(AggregateDataPtr __restrict place) const override {
-        AggregateFunctionPercentile::data(place).reset();
-    }
-
-    void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs,
-               Arena*) const override {
-        
AggregateFunctionPercentile::data(place).merge(AggregateFunctionPercentile::data(rhs));
-    }
-
-    void serialize(ConstAggregateDataPtr __restrict place, BufferWritable& 
buf) const override {
-        AggregateFunctionPercentile::data(place).write(buf);
-    }
-
-    void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf,
-                     Arena*) const override {
-        AggregateFunctionPercentile::data(place).read(buf);
-    }
-
-    void insert_result_into(ConstAggregateDataPtr __restrict place, IColumn& 
to) const override {
-        auto& col = assert_cast<ColumnVector<Float64>&>(to);
-        col.insert_value(AggregateFunctionPercentile::data(place).get());
-    }
-};
-
-class AggregateFunctionPercentileArray final
-        : public IAggregateFunctionDataHelper<PercentileState, 
AggregateFunctionPercentileArray> {
-public:
-    AggregateFunctionPercentileArray(const DataTypes& argument_types_)
-            : IAggregateFunctionDataHelper<PercentileState, 
AggregateFunctionPercentileArray>(
-                      argument_types_) {}
-
-    String get_name() const override { return "percentile_array"; }
-
-    DataTypePtr get_return_type() const override {
-        return 
std::make_shared<DataTypeArray>(make_nullable(std::make_shared<DataTypeFloat64>()));
-    }
-
-    void add(AggregateDataPtr __restrict place, const IColumn** columns, 
ssize_t row_num,
-             Arena*) const override {
-        const auto& sources = assert_cast<const 
ColumnVector<Int64>&>(*columns[0]);
-        const auto& quantile_array = assert_cast<const 
ColumnArray&>(*columns[1]);
-        const auto& offset_column_data = quantile_array.get_offsets();
-        const auto& nested_column =
-                assert_cast<const 
ColumnNullable&>(quantile_array.get_data()).get_nested_column();
-        const auto& nested_column_data = assert_cast<const 
ColumnVector<Float64>&>(nested_column);
-
-        AggregateFunctionPercentileArray::data(place).add(
-                sources.get_int(row_num), nested_column_data.get_data(),
-                offset_column_data.data()[row_num] - 
offset_column_data[(ssize_t)row_num - 1]);
-    }
-
-    void reset(AggregateDataPtr __restrict place) const override {
-        AggregateFunctionPercentileArray::data(place).reset();
-    }
-
-    void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs,
-               Arena*) const override {
-        AggregateFunctionPercentileArray::data(place).merge(
-                AggregateFunctionPercentileArray::data(rhs));
-    }
-
-    void serialize(ConstAggregateDataPtr __restrict place, BufferWritable& 
buf) const override {
-        AggregateFunctionPercentileArray::data(place).write(buf);
-    }
-
-    void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf,
-                     Arena*) const override {
-        AggregateFunctionPercentileArray::data(place).read(buf);
-    }
-
-    void insert_result_into(ConstAggregateDataPtr __restrict place, IColumn& 
to) const override {
-        auto& to_arr = assert_cast<ColumnArray&>(to);
-        auto& to_nested_col = to_arr.get_data();
-        if (to_nested_col.is_nullable()) {
-            auto col_null = reinterpret_cast<ColumnNullable*>(&to_nested_col);
-            AggregateFunctionPercentileArray::data(place).insert_result_into(
-                    col_null->get_nested_column());
-            
col_null->get_null_map_data().resize_fill(col_null->get_nested_column().size(), 
0);
-        } else {
-            
AggregateFunctionPercentileArray::data(place).insert_result_into(to_nested_col);
-        }
-        to_arr.get_offsets().push_back(to_nested_col.size());
-    }
-};
-
-} // namespace doris::vectorized
\ No newline at end of file
diff --git 
a/be/src/vec/aggregate_functions/aggregate_function_percentile_approx.cpp 
b/be/src/vec/aggregate_functions/aggregate_function_percentile_approx.cpp
index 01fdddf6074..5ffac66f8f2 100644
--- a/be/src/vec/aggregate_functions/aggregate_function_percentile_approx.cpp
+++ b/be/src/vec/aggregate_functions/aggregate_function_percentile_approx.cpp
@@ -22,15 +22,43 @@
 
 namespace doris::vectorized {
 
-void 
register_aggregate_function_percentile_old(AggregateFunctionSimpleFactory& 
factory) {
-    factory.register_alternative_function(
-            "percentile", 
creator_without_type::creator<AggregateFunctionPercentileOld>);
-    factory.register_alternative_function(
-            "percentile", 
creator_without_type::creator<AggregateFunctionPercentileOld>, true);
-    factory.register_alternative_function(
-            "percentile_array", 
creator_without_type::creator<AggregateFunctionPercentileArrayOld>);
-    factory.register_alternative_function(
-            "percentile_array", 
creator_without_type::creator<AggregateFunctionPercentileArrayOld>,
-            true);
+template <bool is_nullable>
+AggregateFunctionPtr create_aggregate_function_percentile_approx(const 
std::string& name,
+                                                                 const 
DataTypes& argument_types,
+                                                                 const bool 
result_is_nullable) {
+    const DataTypePtr& argument_type = remove_nullable(argument_types[0]);
+    WhichDataType which(argument_type);
+    if (which.idx != TypeIndex::Float64) {
+        return nullptr;
+    }
+    if (argument_types.size() == 1) {
+        return 
creator_without_type::create<AggregateFunctionPercentileApproxMerge<is_nullable>>(
+                remove_nullable(argument_types), result_is_nullable);
+    }
+    if (argument_types.size() == 2) {
+        return creator_without_type::create<
+                AggregateFunctionPercentileApproxTwoParams<is_nullable>>(
+                remove_nullable(argument_types), result_is_nullable);
+    }
+    if (argument_types.size() == 3) {
+        return creator_without_type::create<
+                AggregateFunctionPercentileApproxThreeParams<is_nullable>>(
+                remove_nullable(argument_types), result_is_nullable);
+    }
+    return nullptr;
+}
+
+void register_aggregate_function_percentile(AggregateFunctionSimpleFactory& 
factory) {
+    factory.register_function_both("percentile",
+                                   
creator_without_type::creator<AggregateFunctionPercentile>);
+    factory.register_function_both("percentile_array",
+                                   
creator_without_type::creator<AggregateFunctionPercentileArray>);
+}
+
+void 
register_aggregate_function_percentile_approx(AggregateFunctionSimpleFactory& 
factory) {
+    factory.register_function("percentile_approx",
+                              
create_aggregate_function_percentile_approx<false>, false);
+    factory.register_function("percentile_approx",
+                              
create_aggregate_function_percentile_approx<true>, true);
 }
 } // namespace doris::vectorized
\ No newline at end of file
diff --git 
a/be/src/vec/aggregate_functions/aggregate_function_percentile_approx.h 
b/be/src/vec/aggregate_functions/aggregate_function_percentile_approx.h
index 3f840124819..b89c273dd77 100644
--- a/be/src/vec/aggregate_functions/aggregate_function_percentile_approx.h
+++ b/be/src/vec/aggregate_functions/aggregate_function_percentile_approx.h
@@ -55,8 +55,241 @@ class BufferWritable;
 
 namespace doris::vectorized {
 
-struct OldPercentileState {
-    std::vector<OldCounts> vec_counts;
+struct PercentileApproxState {
+    static constexpr double INIT_QUANTILE = -1.0;
+    PercentileApproxState() = default;
+    ~PercentileApproxState() = default;
+
+    void init(double compression = 10000) {
+        if (!init_flag) {
+            
//https://doris.apache.org/zh-CN/sql-reference/sql-functions/aggregate-functions/percentile_approx.html#description
+            //The compression parameter setting range is [2048, 10000].
+            //If the value of compression parameter is not specified set, or 
is outside the range of [2048, 10000],
+            //will use the default value of 10000
+            if (compression < 2048 || compression > 10000) {
+                compression = 10000;
+            }
+            digest = TDigest::create_unique(compression);
+            compressions = compression;
+            init_flag = true;
+        }
+    }
+
+    void write(BufferWritable& buf) const {
+        write_binary(init_flag, buf);
+        if (!init_flag) {
+            return;
+        }
+
+        write_binary(target_quantile, buf);
+        write_binary(compressions, buf);
+        uint32_t serialize_size = digest->serialized_size();
+        std::string result(serialize_size, '0');
+        DCHECK(digest.get() != nullptr);
+        digest->serialize((uint8_t*)result.c_str());
+
+        write_binary(result, buf);
+    }
+
+    void read(BufferReadable& buf) {
+        read_binary(init_flag, buf);
+        if (!init_flag) {
+            return;
+        }
+
+        read_binary(target_quantile, buf);
+        read_binary(compressions, buf);
+        std::string str;
+        read_binary(str, buf);
+        digest = TDigest::create_unique(compressions);
+        digest->unserialize((uint8_t*)str.c_str());
+    }
+
+    double get() const {
+        if (init_flag) {
+            return digest->quantile(target_quantile);
+        } else {
+            return std::nan("");
+        }
+    }
+
+    void merge(const PercentileApproxState& rhs) {
+        if (!rhs.init_flag) {
+            return;
+        }
+        if (init_flag) {
+            DCHECK(digest.get() != nullptr);
+            digest->merge(rhs.digest.get());
+        } else {
+            digest = TDigest::create_unique(compressions);
+            digest->merge(rhs.digest.get());
+            init_flag = true;
+        }
+        if (target_quantile == PercentileApproxState::INIT_QUANTILE) {
+            target_quantile = rhs.target_quantile;
+        }
+    }
+
+    void add(double source, double quantile) {
+        digest->add(source);
+        target_quantile = quantile;
+    }
+
+    void reset() {
+        target_quantile = INIT_QUANTILE;
+        init_flag = false;
+        digest = TDigest::create_unique(compressions);
+    }
+
+    bool init_flag = false;
+    std::unique_ptr<TDigest> digest;
+    double target_quantile = INIT_QUANTILE;
+    double compressions = 10000;
+};
+
+class AggregateFunctionPercentileApprox
+        : public IAggregateFunctionDataHelper<PercentileApproxState,
+                                              
AggregateFunctionPercentileApprox> {
+public:
+    AggregateFunctionPercentileApprox(const DataTypes& argument_types_)
+            : IAggregateFunctionDataHelper<PercentileApproxState,
+                                           
AggregateFunctionPercentileApprox>(argument_types_) {}
+
+    String get_name() const override { return "percentile_approx"; }
+
+    DataTypePtr get_return_type() const override {
+        return make_nullable(std::make_shared<DataTypeFloat64>());
+    }
+
+    void reset(AggregateDataPtr __restrict place) const override {
+        AggregateFunctionPercentileApprox::data(place).reset();
+    }
+
+    void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs,
+               Arena*) const override {
+        AggregateFunctionPercentileApprox::data(place).merge(
+                AggregateFunctionPercentileApprox::data(rhs));
+    }
+
+    void serialize(ConstAggregateDataPtr __restrict place, BufferWritable& 
buf) const override {
+        AggregateFunctionPercentileApprox::data(place).write(buf);
+    }
+
+    void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf,
+                     Arena*) const override {
+        AggregateFunctionPercentileApprox::data(place).read(buf);
+    }
+
+    void insert_result_into(ConstAggregateDataPtr __restrict place, IColumn& 
to) const override {
+        ColumnNullable& nullable_column = assert_cast<ColumnNullable&>(to);
+        double result = AggregateFunctionPercentileApprox::data(place).get();
+
+        if (std::isnan(result)) {
+            nullable_column.insert_default();
+        } else {
+            auto& col = 
assert_cast<ColumnVector<Float64>&>(nullable_column.get_nested_column());
+            col.get_data().push_back(result);
+            nullable_column.get_null_map_data().push_back(0);
+        }
+    }
+};
+
+// only for merge
+template <bool is_nullable>
+class AggregateFunctionPercentileApproxMerge : public 
AggregateFunctionPercentileApprox {
+public:
+    AggregateFunctionPercentileApproxMerge(const DataTypes& argument_types_)
+            : AggregateFunctionPercentileApprox(argument_types_) {}
+    void add(AggregateDataPtr __restrict place, const IColumn** columns, 
ssize_t row_num,
+             Arena*) const override {
+        LOG(FATAL) << "AggregateFunctionPercentileApproxMerge do not support 
add()";
+        __builtin_unreachable();
+    }
+};
+
+template <bool is_nullable>
+class AggregateFunctionPercentileApproxTwoParams : public 
AggregateFunctionPercentileApprox {
+public:
+    AggregateFunctionPercentileApproxTwoParams(const DataTypes& 
argument_types_)
+            : AggregateFunctionPercentileApprox(argument_types_) {}
+    void add(AggregateDataPtr __restrict place, const IColumn** columns, 
ssize_t row_num,
+             Arena*) const override {
+        if constexpr (is_nullable) {
+            double column_data[2] = {0, 0};
+
+            for (int i = 0; i < 2; ++i) {
+                const auto* nullable_column = 
check_and_get_column<ColumnNullable>(columns[i]);
+                if (nullable_column == nullptr) { //Not Nullable column
+                    const auto& column = assert_cast<const 
ColumnVector<Float64>&>(*columns[i]);
+                    column_data[i] = column.get_float64(row_num);
+                } else if (!nullable_column->is_null_at(
+                                   row_num)) { // Nullable column && Not null 
data
+                    const auto& column = assert_cast<const 
ColumnVector<Float64>&>(
+                            nullable_column->get_nested_column());
+                    column_data[i] = column.get_float64(row_num);
+                } else { // Nullable column && null data
+                    if (i == 0) {
+                        return;
+                    }
+                }
+            }
+
+            this->data(place).init();
+            this->data(place).add(column_data[0], column_data[1]);
+
+        } else {
+            const auto& sources = assert_cast<const 
ColumnVector<Float64>&>(*columns[0]);
+            const auto& quantile = assert_cast<const 
ColumnVector<Float64>&>(*columns[1]);
+
+            this->data(place).init();
+            this->data(place).add(sources.get_float64(row_num), 
quantile.get_float64(row_num));
+        }
+    }
+};
+
+template <bool is_nullable>
+class AggregateFunctionPercentileApproxThreeParams : public 
AggregateFunctionPercentileApprox {
+public:
+    AggregateFunctionPercentileApproxThreeParams(const DataTypes& 
argument_types_)
+            : AggregateFunctionPercentileApprox(argument_types_) {}
+    void add(AggregateDataPtr __restrict place, const IColumn** columns, 
ssize_t row_num,
+             Arena*) const override {
+        if constexpr (is_nullable) {
+            double column_data[3] = {0, 0, 0};
+
+            for (int i = 0; i < 3; ++i) {
+                const auto* nullable_column = 
check_and_get_column<ColumnNullable>(columns[i]);
+                if (nullable_column == nullptr) { //Not Nullable column
+                    const auto& column = assert_cast<const 
ColumnVector<Float64>&>(*columns[i]);
+                    column_data[i] = column.get_float64(row_num);
+                } else if (!nullable_column->is_null_at(
+                                   row_num)) { // Nullable column && Not null 
data
+                    const auto& column = assert_cast<const 
ColumnVector<Float64>&>(
+                            nullable_column->get_nested_column());
+                    column_data[i] = column.get_float64(row_num);
+                } else { // Nullable column && null data
+                    if (i == 0) {
+                        return;
+                    }
+                }
+            }
+
+            this->data(place).init(column_data[2]);
+            this->data(place).add(column_data[0], column_data[1]);
+
+        } else {
+            const auto& sources = assert_cast<const 
ColumnVector<Float64>&>(*columns[0]);
+            const auto& quantile = assert_cast<const 
ColumnVector<Float64>&>(*columns[1]);
+            const auto& compression = assert_cast<const 
ColumnVector<Float64>&>(*columns[2]);
+
+            this->data(place).init(compression.get_float64(row_num));
+            this->data(place).add(sources.get_float64(row_num), 
quantile.get_float64(row_num));
+        }
+    }
+};
+
+struct PercentileState {
+    std::vector<Counts> vec_counts;
     std::vector<double> vec_quantile {-1};
     bool inited_flag = false;
 
@@ -108,7 +341,7 @@ struct OldPercentileState {
         }
     }
 
-    void merge(const OldPercentileState& rhs) {
+    void merge(const PercentileState& rhs) {
         if (!rhs.inited_flag) {
             return;
         }
@@ -143,11 +376,11 @@ struct OldPercentileState {
     }
 };
 
-class AggregateFunctionPercentileOld final
-        : public IAggregateFunctionDataHelper<OldPercentileState, 
AggregateFunctionPercentileOld> {
+class AggregateFunctionPercentile final
+        : public IAggregateFunctionDataHelper<PercentileState, 
AggregateFunctionPercentile> {
 public:
-    AggregateFunctionPercentileOld(const DataTypes& argument_types_)
-            : IAggregateFunctionDataHelper<OldPercentileState, 
AggregateFunctionPercentileOld>(
+    AggregateFunctionPercentile(const DataTypes& argument_types_)
+            : IAggregateFunctionDataHelper<PercentileState, 
AggregateFunctionPercentile>(
                       argument_types_) {}
 
     String get_name() const override { return "percentile"; }
@@ -158,41 +391,39 @@ public:
              Arena*) const override {
         const auto& sources = assert_cast<const 
ColumnVector<Int64>&>(*columns[0]);
         const auto& quantile = assert_cast<const 
ColumnVector<Float64>&>(*columns[1]);
-        
AggregateFunctionPercentileOld::data(place).add(sources.get_int(row_num),
-                                                        quantile.get_data(), 
1);
+        AggregateFunctionPercentile::data(place).add(sources.get_int(row_num), 
quantile.get_data(),
+                                                     1);
     }
 
     void reset(AggregateDataPtr __restrict place) const override {
-        AggregateFunctionPercentileOld::data(place).reset();
+        AggregateFunctionPercentile::data(place).reset();
     }
 
     void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs,
                Arena*) const override {
-        AggregateFunctionPercentileOld::data(place).merge(
-                AggregateFunctionPercentileOld::data(rhs));
+        
AggregateFunctionPercentile::data(place).merge(AggregateFunctionPercentile::data(rhs));
     }
 
     void serialize(ConstAggregateDataPtr __restrict place, BufferWritable& 
buf) const override {
-        AggregateFunctionPercentileOld::data(place).write(buf);
+        AggregateFunctionPercentile::data(place).write(buf);
     }
 
     void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf,
                      Arena*) const override {
-        AggregateFunctionPercentileOld::data(place).read(buf);
+        AggregateFunctionPercentile::data(place).read(buf);
     }
 
     void insert_result_into(ConstAggregateDataPtr __restrict place, IColumn& 
to) const override {
         auto& col = assert_cast<ColumnVector<Float64>&>(to);
-        col.insert_value(AggregateFunctionPercentileOld::data(place).get());
+        col.insert_value(AggregateFunctionPercentile::data(place).get());
     }
 };
 
-class AggregateFunctionPercentileArrayOld final
-        : public IAggregateFunctionDataHelper<OldPercentileState,
-                                              
AggregateFunctionPercentileArrayOld> {
+class AggregateFunctionPercentileArray final
+        : public IAggregateFunctionDataHelper<PercentileState, 
AggregateFunctionPercentileArray> {
 public:
-    AggregateFunctionPercentileArrayOld(const DataTypes& argument_types_)
-            : IAggregateFunctionDataHelper<OldPercentileState, 
AggregateFunctionPercentileArrayOld>(
+    AggregateFunctionPercentileArray(const DataTypes& argument_types_)
+            : IAggregateFunctionDataHelper<PercentileState, 
AggregateFunctionPercentileArray>(
                       argument_types_) {}
 
     String get_name() const override { return "percentile_array"; }
@@ -210,28 +441,28 @@ public:
                 assert_cast<const 
ColumnNullable&>(quantile_array.get_data()).get_nested_column();
         const auto& nested_column_data = assert_cast<const 
ColumnVector<Float64>&>(nested_column);
 
-        AggregateFunctionPercentileArrayOld::data(place).add(
+        AggregateFunctionPercentileArray::data(place).add(
                 sources.get_int(row_num), nested_column_data.get_data(),
                 offset_column_data.data()[row_num] - 
offset_column_data[(ssize_t)row_num - 1]);
     }
 
     void reset(AggregateDataPtr __restrict place) const override {
-        AggregateFunctionPercentileArrayOld::data(place).reset();
+        AggregateFunctionPercentileArray::data(place).reset();
     }
 
     void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs,
                Arena*) const override {
-        AggregateFunctionPercentileArrayOld::data(place).merge(
-                AggregateFunctionPercentileArrayOld::data(rhs));
+        AggregateFunctionPercentileArray::data(place).merge(
+                AggregateFunctionPercentileArray::data(rhs));
     }
 
     void serialize(ConstAggregateDataPtr __restrict place, BufferWritable& 
buf) const override {
-        AggregateFunctionPercentileArrayOld::data(place).write(buf);
+        AggregateFunctionPercentileArray::data(place).write(buf);
     }
 
     void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf,
                      Arena*) const override {
-        AggregateFunctionPercentileArrayOld::data(place).read(buf);
+        AggregateFunctionPercentileArray::data(place).read(buf);
     }
 
     void insert_result_into(ConstAggregateDataPtr __restrict place, IColumn& 
to) const override {
@@ -239,14 +470,14 @@ public:
         auto& to_nested_col = to_arr.get_data();
         if (to_nested_col.is_nullable()) {
             auto col_null = reinterpret_cast<ColumnNullable*>(&to_nested_col);
-            
AggregateFunctionPercentileArrayOld::data(place).insert_result_into(
+            AggregateFunctionPercentileArray::data(place).insert_result_into(
                     col_null->get_nested_column());
             
col_null->get_null_map_data().resize_fill(col_null->get_nested_column().size(), 
0);
         } else {
-            
AggregateFunctionPercentileArrayOld::data(place).insert_result_into(to_nested_col);
+            
AggregateFunctionPercentileArray::data(place).insert_result_into(to_nested_col);
         }
         to_arr.get_offsets().push_back(to_nested_col.size());
     }
 };
 
-} // namespace doris::vectorized
+} // namespace doris::vectorized
\ No newline at end of file
diff --git 
a/be/src/vec/aggregate_functions/aggregate_function_simple_factory.cpp 
b/be/src/vec/aggregate_functions/aggregate_function_simple_factory.cpp
index 80805dc83d5..d95d0ce6ccb 100644
--- a/be/src/vec/aggregate_functions/aggregate_function_simple_factory.cpp
+++ b/be/src/vec/aggregate_functions/aggregate_function_simple_factory.cpp
@@ -54,7 +54,6 @@ void 
register_aggregate_function_approx_count_distinct(AggregateFunctionSimpleFa
 void 
register_aggregate_function_group_array_intersect(AggregateFunctionSimpleFactory&
 factory);
 void register_aggregate_function_group_concat(AggregateFunctionSimpleFactory& 
factory);
 void register_aggregate_function_percentile(AggregateFunctionSimpleFactory& 
factory);
-void 
register_aggregate_function_percentile_old(AggregateFunctionSimpleFactory& 
factory);
 void register_aggregate_function_window_funnel(AggregateFunctionSimpleFactory& 
factory);
 void register_aggregate_function_retention(AggregateFunctionSimpleFactory& 
factory);
 void 
register_aggregate_function_percentile_approx(AggregateFunctionSimpleFactory& 
factory);
@@ -96,7 +95,6 @@ AggregateFunctionSimpleFactory& 
AggregateFunctionSimpleFactory::instance() {
         register_aggregate_function_topn(instance);
         register_aggregate_function_approx_count_distinct(instance);
         register_aggregate_function_percentile(instance);
-        register_aggregate_function_percentile_old(instance);
         register_aggregate_function_percentile_approx(instance);
         register_aggregate_function_window_funnel(instance);
         register_aggregate_function_retention(instance);
diff --git a/be/src/vec/common/string_buffer.hpp 
b/be/src/vec/common/string_buffer.hpp
index 22bfb67b140..84f0dffbff7 100644
--- a/be/src/vec/common/string_buffer.hpp
+++ b/be/src/vec/common/string_buffer.hpp
@@ -30,18 +30,17 @@ public:
     explicit BufferWritable(ColumnString& vector)
             : _data(vector.get_chars()), _offsets(vector.get_offsets()) {}
 
-    void write(const char* data, size_t len) {
+    inline void write(const char* data, int len) {
         _data.insert(data, data + len);
         _now_offset += len;
     }
-
-    void write(char c) {
+    inline void write(char c) {
         const char* p = &c;
         _data.insert(p, p + 1);
         _now_offset += 1;
     }
 
-    void commit() {
+    inline void commit() {
         ColumnString::check_chars_length(_offsets.back() + _now_offset, 0);
         _offsets.push_back(_offsets.back() + _now_offset);
         _now_offset = 0;
@@ -71,13 +70,13 @@ public:
     explicit BufferReadable(StringRef&& ref) : _data(ref.data) {}
     ~BufferReadable() = default;
 
-    StringRef read(size_t len) {
+    inline StringRef read(int len) {
         StringRef ref(_data, len);
         _data += len;
         return ref;
     }
 
-    void read(char* data, int len) {
+    inline void read(char* data, int len) {
         memcpy(data, _data, len);
         _data += len;
     }
diff --git a/be/src/vec/functions/simple_function_factory.h 
b/be/src/vec/functions/simple_function_factory.h
index 889a9743635..68e2f85e01c 100644
--- a/be/src/vec/functions/simple_function_factory.h
+++ b/be/src/vec/functions/simple_function_factory.h
@@ -109,8 +109,8 @@ class SimpleFunctionFactory {
     using Creator = std::function<FunctionBuilderPtr()>;
     using FunctionCreators = phmap::flat_hash_map<std::string, Creator>;
     using FunctionIsVariadic = phmap::flat_hash_set<std::string>;
-    /// @TEMPORARY: for be_exec_version=5
-    constexpr static int NEWEST_VERSION_FUNCTION_SUBSTITUTE = 5;
+    /// @TEMPORARY: for be_exec_version=4
+    constexpr static int NEWEST_VERSION_FUNCTION_SUBSTITUTE = 4;
 
 public:
     void register_function(const std::string& name, const Creator& ptr) {
@@ -149,7 +149,7 @@ public:
     /// @TEMPORARY: for be_exec_version=3
     template <class Function>
     void register_alternative_function() {
-        static std::string suffix {"_old_for_version_before_5_0"};
+        static std::string suffix {"_old_for_version_before_4_0"};
         function_to_replace[Function::name] = Function::name + suffix;
         register_function(Function::name + suffix, 
&createDefaultFunction<Function>);
     }
@@ -193,7 +193,7 @@ private:
     FunctionCreators function_creators;
     FunctionIsVariadic function_variadic_set;
     std::unordered_map<std::string, std::string> function_alias;
-    /// @TEMPORARY: for be_exec_version=4. replace function to old version.
+    /// @TEMPORARY: for be_exec_version=3. replace function to old version.
     std::unordered_map<std::string, std::string> function_to_replace;
 
     template <typename Function>
@@ -201,7 +201,7 @@ private:
         return std::make_shared<DefaultFunctionBuilder>(Function::create());
     }
 
-    /// @TEMPORARY: for be_exec_version=4
+    /// @TEMPORARY: for be_exec_version=3
     void temporary_function_update(int fe_version_now, std::string& name) {
         // replace if fe is old version.
         if (fe_version_now < NEWEST_VERSION_FUNCTION_SUBSTITUTE &&
diff --git a/be/test/util/counts_test.cpp b/be/test/util/counts_test.cpp
index 20d9ea54c97..908bbcefd58 100644
--- a/be/test/util/counts_test.cpp
+++ b/be/test/util/counts_test.cpp
@@ -42,16 +42,12 @@ TEST_F(TCountsTest, TotalTest) {
 
     double result = counts.terminate(0.2);
     EXPECT_EQ(1, result);
-
-    auto cs = vectorized::ColumnString::create();
-    vectorized::BufferWritable bw(*cs);
-    counts.serialize(bw);
-    bw.commit();
+    uint8_t* writer = new uint8_t[counts.serialized_size()];
+    uint8_t* type_reader = writer;
+    counts.serialize(writer);
 
     Counts other;
-    StringRef res(cs->get_chars().data(), cs->get_chars().size());
-    vectorized::BufferReadable br(res);
-    other.unserialize(br);
+    other.unserialize(type_reader);
     double result1 = other.terminate(0.2);
     EXPECT_EQ(result, result1);
 
@@ -62,19 +58,10 @@ TEST_F(TCountsTest, TotalTest) {
     other1.increment(10, 1);
     other1.increment(99, 2);
 
-    // deserialize other1
-    cs->clear();
-    other1.serialize(bw);
-    bw.commit();
-    Counts other1_deserialized;
-    vectorized::BufferReadable br1(res);
-    other1_deserialized.unserialize(br1);
-
-    Counts merge_res;
-    merge_res.merge(&other);
-    merge_res.merge(&other1_deserialized);
+    counts.merge(&other1);
     // 1 1 1 1 2 5 7 7 9 9 10 19 50 50 50 99 99 100 100 100
-    EXPECT_EQ(merge_res.terminate(0.3), 6.4);
+    EXPECT_EQ(counts.terminate(0.3), 6.4);
+    delete[] writer;
 }
 
 } // namespace doris
diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java 
b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java
index a6bcadadfab..94016b55833 100644
--- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java
+++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java
@@ -1765,7 +1765,7 @@ public class Config extends ConfigBase {
      * Max data version of backends serialize block.
      */
     @ConfField(mutable = false)
-    public static int max_be_exec_version = 5;
+    public static int max_be_exec_version = 4;
 
     /**
      * Min data version of backends serialize block.


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to