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]