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

airborne pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/master by this push:
     new a15c88ce4aa [opt](agg) remove experimental functions approx_top_k and 
approx_top_sum (#55845)
a15c88ce4aa is described below

commit a15c88ce4aa7a188275651a2f361427a51fea76b
Author: zzzxl <[email protected]>
AuthorDate: Thu Sep 11 18:14:43 2025 +0800

    [opt](agg) remove experimental functions approx_top_k and approx_top_sum 
(#55845)
---
 .../aggregate_function_approx_top.h                | 116 ----------
 .../aggregate_function_approx_top_k.cpp            |  42 ----
 .../aggregate_function_approx_top_k.h              | 232 -------------------
 .../aggregate_function_approx_top_sum.cpp          |  69 ------
 .../aggregate_function_approx_top_sum.h            | 248 ---------------------
 .../aggregate_function_simple_factory.cpp          |   4 -
 .../doris/catalog/BuiltinAggregateFunctions.java   |   4 -
 .../expressions/functions/agg/ApproxTopK.java      |  99 --------
 .../expressions/functions/agg/ApproxTopSum.java    |  99 --------
 .../visitor/AggregateFunctionVisitor.java          |  10 -
 .../inverted_index_p0/test_index_approx_top_k.out  | Bin 9037 -> 0 bytes
 .../test_index_approx_top_sum.out                  | Bin 9145 -> 0 bytes
 .../test_index_approx_top_k.groovy                 | 153 -------------
 .../test_index_approx_top_sum.groovy               | 153 -------------
 14 files changed, 1229 deletions(-)

diff --git a/be/src/vec/aggregate_functions/aggregate_function_approx_top.h 
b/be/src/vec/aggregate_functions/aggregate_function_approx_top.h
deleted file mode 100644
index 0df1eb1d05e..00000000000
--- a/be/src/vec/aggregate_functions/aggregate_function_approx_top.h
+++ /dev/null
@@ -1,116 +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 "vec/core/types.h"
-#include "vec/data_types/data_type.h"
-#include "vec/data_types/data_type_nullable.h"
-
-namespace doris::vectorized {
-#include "common/compile_check_begin.h"
-
-class AggregateFunctionApproxTop {
-public:
-    AggregateFunctionApproxTop(const std::vector<std::string>& column_names)
-            : _column_names(column_names) {}
-
-    static int32_t is_valid_const_columns(const std::vector<bool>& 
is_const_columns) {
-        int32_t true_count = 0;
-        bool found_false_after_true = false;
-        for (int64_t i = is_const_columns.size() - 1; i >= 0; --i) {
-            if (is_const_columns[i]) {
-                true_count++;
-                if (found_false_after_true) {
-                    return false;
-                }
-            } else {
-                if (true_count > 2) {
-                    return false;
-                }
-                found_false_after_true = true;
-            }
-        }
-        if (true_count > 2) {
-            throw Exception(ErrorCode::INVALID_ARGUMENT, "Invalid 
is_const_columns configuration");
-        }
-        return true_count;
-    }
-
-protected:
-    void lazy_init(const IColumn** columns, ssize_t row_num,
-                   const DataTypes& argument_types) const {
-        auto get_param = [](size_t idx, const DataTypes& data_types,
-                            const IColumn** columns) -> uint64_t {
-            const auto& data_type = data_types.at(idx);
-            const IColumn* column = columns[idx];
-
-            const auto* type = data_type.get();
-            if (type->is_nullable()) {
-                type = assert_cast<const DataTypeNullable*, 
TypeCheckOnRelease::DISABLE>(type)
-                               ->get_nested_type()
-                               .get();
-            }
-            int64_t value = 0;
-            switch (type->get_primitive_type()) {
-            case PrimitiveType::TYPE_TINYINT:
-                value = assert_cast<const ColumnInt8*, 
TypeCheckOnRelease::DISABLE>(column)
-                                ->get_element(0);
-                break;
-            case PrimitiveType::TYPE_SMALLINT:
-                value = assert_cast<const ColumnInt16*, 
TypeCheckOnRelease::DISABLE>(column)
-                                ->get_element(0);
-                break;
-            case PrimitiveType::TYPE_INT:
-                value = assert_cast<const ColumnInt32*, 
TypeCheckOnRelease::DISABLE>(column)
-                                ->get_element(0);
-                break;
-            default:
-                break;
-            }
-            if (value <= 0) {
-                throw Exception(ErrorCode::INVALID_ARGUMENT,
-                                "The parameter cannot be less than or equal to 
0.");
-            }
-            return value;
-        };
-
-        _threshold =
-                std::min(get_param(_column_names.size(), argument_types, 
columns), (uint64_t)4096);
-        _reserved = std::min(
-                std::max(get_param(_column_names.size() + 1, argument_types, 
columns), _threshold),
-                (uint64_t)4096);
-
-        if (_threshold == 0 || _reserved == 0 || _threshold > 4096 || 
_reserved > 4096) {
-            throw Exception(ErrorCode::INTERNAL_ERROR,
-                            "approx_top_sum param error, _threshold: {}, 
_reserved: {}", _threshold,
-                            _reserved);
-        }
-
-        _init_flag = true;
-    }
-
-    static inline constexpr UInt64 TOP_K_MAX_SIZE = 0xFFFFFF;
-
-    mutable std::vector<std::string> _column_names;
-    mutable bool _init_flag = false;
-    mutable uint64_t _threshold = 10;
-    mutable uint64_t _reserved = 30;
-};
-
-#include "common/compile_check_end.h"
-} // namespace doris::vectorized
\ No newline at end of file
diff --git a/be/src/vec/aggregate_functions/aggregate_function_approx_top_k.cpp 
b/be/src/vec/aggregate_functions/aggregate_function_approx_top_k.cpp
deleted file mode 100644
index 0227767f7f5..00000000000
--- a/be/src/vec/aggregate_functions/aggregate_function_approx_top_k.cpp
+++ /dev/null
@@ -1,42 +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_approx_top_k.h"
-
-#include "vec/aggregate_functions/aggregate_function_simple_factory.h"
-#include "vec/aggregate_functions/factory_helpers.h"
-#include "vec/aggregate_functions/helpers.h"
-#include "vec/data_types/data_type.h"
-
-namespace doris::vectorized {
-#include "common/compile_check_begin.h"
-
-AggregateFunctionPtr create_aggregate_function_approx_top_k(const std::string& 
name,
-                                                            const DataTypes& 
argument_types,
-                                                            const bool 
result_is_nullable,
-                                                            const 
AggregateFunctionAttr& attr) {
-    assert_arity_range(name, argument_types, 3, 128);
-    return creator_without_type::create<AggregateFunctionApproxTopK>(
-            argument_types, result_is_nullable, attr, attr.column_names);
-}
-
-void register_aggregate_function_approx_top_k(AggregateFunctionSimpleFactory& 
factory) {
-    factory.register_function_both("approx_top_k", 
create_aggregate_function_approx_top_k);
-}
-
-#include "common/compile_check_end.h"
-} // namespace doris::vectorized
\ No newline at end of file
diff --git a/be/src/vec/aggregate_functions/aggregate_function_approx_top_k.h 
b/be/src/vec/aggregate_functions/aggregate_function_approx_top_k.h
deleted file mode 100644
index 34c431b2975..00000000000
--- a/be/src/vec/aggregate_functions/aggregate_function_approx_top_k.h
+++ /dev/null
@@ -1,232 +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 <rapidjson/encodings.h>
-#include <rapidjson/prettywriter.h>
-#include <rapidjson/stringbuffer.h>
-#include <rapidjson/writer.h>
-
-#include <cstdint>
-#include <string>
-
-#include "vec/aggregate_functions/aggregate_function.h"
-#include "vec/aggregate_functions/aggregate_function_approx_top.h"
-#include "vec/columns/column.h"
-#include "vec/columns/column_array.h"
-#include "vec/columns/column_string.h"
-#include "vec/columns/column_struct.h"
-#include "vec/columns/column_vector.h"
-#include "vec/common/assert_cast.h"
-#include "vec/common/space_saving.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_ipv4.h"
-#include "vec/data_types/data_type_nullable.h"
-#include "vec/data_types/data_type_struct.h"
-
-namespace doris::vectorized {
-#include "common/compile_check_begin.h"
-
-struct AggregateFunctionTopKGenericData {
-    using Set = SpaceSaving<StringRef, StringRefHash>;
-
-    Set value;
-};
-
-class AggregateFunctionApproxTopK final
-        : public IAggregateFunctionDataHelper<AggregateFunctionTopKGenericData,
-                                              AggregateFunctionApproxTopK>,
-          AggregateFunctionApproxTop,
-          VarargsExpression,
-          NullableAggregateFunction {
-private:
-    using State = AggregateFunctionTopKGenericData;
-
-public:
-    AggregateFunctionApproxTopK(const std::vector<std::string>& column_names,
-                                const DataTypes& argument_types_)
-            : IAggregateFunctionDataHelper<AggregateFunctionTopKGenericData,
-                                           
AggregateFunctionApproxTopK>(argument_types_),
-              AggregateFunctionApproxTop(column_names) {}
-
-    String get_name() const override { return "approx_top_k"; }
-
-    DataTypePtr get_return_type() const override { return 
std::make_shared<DataTypeString>(); }
-
-    // Serializes the aggregate function's state (including the SpaceSaving 
structure and threshold) into a buffer.
-    void serialize(ConstAggregateDataPtr __restrict place, BufferWritable& 
buf) const override {
-        this->data(place).value.write(buf);
-
-        buf.write_var_uint(_column_names.size());
-        for (const auto& column_name : _column_names) {
-            buf.write_binary(column_name);
-        }
-        buf.write_var_uint(_threshold);
-        buf.write_var_uint(_reserved);
-    }
-
-    // Deserializes the aggregate function's state from a buffer (including 
the SpaceSaving structure and threshold).
-    void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf,
-                     Arena& arena) const override {
-        auto readStringBinaryInto = [](Arena& arena, BufferReadable& buf) {
-            uint64_t size = 0;
-            buf.read_var_uint(size);
-
-            if (UNLIKELY(size > DEFAULT_MAX_STRING_SIZE)) {
-                throw Exception(ErrorCode::INTERNAL_ERROR, "Too large string 
size.");
-            }
-
-            char* data = arena.alloc(size);
-            buf.read(data, size);
-
-            return StringRef(data, size);
-        };
-
-        auto& set = this->data(place).value;
-        set.clear();
-
-        uint64_t size = 0;
-        buf.read_var_uint(size);
-        if (UNLIKELY(size > TOP_K_MAX_SIZE)) {
-            throw Exception(ErrorCode::INTERNAL_ERROR,
-                            "Too large size ({}) for aggregate function '{}' 
state (maximum is {})",
-                            size, get_name(), TOP_K_MAX_SIZE);
-        }
-
-        set.resize(size);
-        for (size_t i = 0; i < size; ++i) {
-            auto ref = readStringBinaryInto(arena, buf);
-            uint64_t count = 0;
-            uint64_t error = 0;
-            buf.read_var_uint(count);
-            buf.read_var_uint(error);
-            set.insert(ref, count, error);
-            arena.rollback(ref.size);
-        }
-
-        set.read_alpha_map(buf);
-
-        uint64_t column_size = 0;
-        buf.read_var_uint(column_size);
-        _column_names.clear();
-        for (uint64_t i = 0; i < column_size; i++) {
-            std::string column_name;
-            buf.read_binary(column_name);
-            _column_names.emplace_back(std::move(column_name));
-        }
-        buf.read_var_uint(_threshold);
-        buf.read_var_uint(_reserved);
-    }
-
-    // Adds a new row of data to the aggregate function (inserts a new value 
into the SpaceSaving structure).
-    void add(AggregateDataPtr __restrict place, const IColumn** columns, 
ssize_t row_num,
-             Arena& arena) const override {
-        if (!_init_flag) {
-            lazy_init(columns, row_num, this->get_argument_types());
-        }
-
-        auto& set = this->data(place).value;
-        if (set.capacity() != _reserved) {
-            set.resize(_reserved);
-        }
-
-        auto all_serialize_value_into_arena =
-                [](size_t i, size_t keys_size, const IColumn** columns, Arena& 
arena) -> StringRef {
-            const char* begin = nullptr;
-
-            size_t sum_size = 0;
-            for (size_t j = 0; j < keys_size; ++j) {
-                sum_size += columns[j]->serialize_value_into_arena(i, arena, 
begin).size;
-            }
-
-            return {begin, sum_size};
-        };
-
-        StringRef str_serialized =
-                all_serialize_value_into_arena(row_num, _column_names.size(), 
columns, arena);
-        set.insert(str_serialized);
-        arena.rollback(str_serialized.size);
-    }
-
-    void add_many(AggregateDataPtr __restrict place, const IColumn** columns,
-                  std::vector<int>& rows, Arena& arena) const override {
-        for (auto row : rows) {
-            add(place, columns, row, arena);
-        }
-    }
-
-    void reset(AggregateDataPtr __restrict place) const override {
-        this->data(place).value.clear();
-    }
-
-    // Merges the state of another aggregate function into the current one 
(merges two SpaceSaving sets).
-    void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs,
-               Arena&) const override {
-        auto& rhs_set = this->data(rhs).value;
-        if (!rhs_set.size()) {
-            return;
-        }
-
-        auto& set = this->data(place).value;
-        if (set.capacity() != _reserved) {
-            set.resize(_reserved);
-        }
-        set.merge(rhs_set);
-    }
-
-    void insert_result_into(ConstAggregateDataPtr __restrict place, IColumn& 
to) const override {
-        auto& data_to = assert_cast<ColumnString&, 
TypeCheckOnRelease::DISABLE>(to);
-
-        const typename State::Set& set = this->data(place).value;
-        auto result_vec = set.top_k(_threshold);
-
-        rapidjson::StringBuffer buffer;
-        rapidjson::PrettyWriter<rapidjson::StringBuffer> writer(buffer);
-        writer.StartArray();
-        for (auto& result : result_vec) {
-            auto argument_types = this->get_argument_types();
-            MutableColumns argument_columns(_column_names.size());
-            for (size_t i = 0; i < _column_names.size(); ++i) {
-                argument_columns[i] = argument_types[i]->create_column();
-            }
-            rapidjson::StringBuffer sub_buffer;
-            rapidjson::Writer<rapidjson::StringBuffer> sub_writer(sub_buffer);
-            sub_writer.StartObject();
-            const char* begin = result.key.data;
-            for (size_t i = 0; i < _column_names.size(); i++) {
-                begin = 
argument_columns[i]->deserialize_and_insert_from_arena(begin);
-                std::string row_str = 
argument_types[i]->to_string(*argument_columns[i], 0);
-                sub_writer.Key(_column_names[i].data(),
-                               cast_set<uint32_t>(_column_names[i].size()));
-                sub_writer.String(row_str.data(), 
cast_set<uint32_t>(row_str.size()));
-            }
-            sub_writer.Key("count");
-            sub_writer.String(std::to_string(result.count).c_str());
-            sub_writer.EndObject();
-            writer.RawValue(sub_buffer.GetString(), sub_buffer.GetSize(), 
rapidjson::kObjectType);
-        }
-        writer.EndArray();
-        std::string res = buffer.GetString();
-        data_to.insert_data(res.data(), res.size());
-    }
-};
-
-#include "common/compile_check_end.h"
-} // namespace doris::vectorized
\ No newline at end of file
diff --git 
a/be/src/vec/aggregate_functions/aggregate_function_approx_top_sum.cpp 
b/be/src/vec/aggregate_functions/aggregate_function_approx_top_sum.cpp
deleted file mode 100644
index 178de3b2bfe..00000000000
--- a/be/src/vec/aggregate_functions/aggregate_function_approx_top_sum.cpp
+++ /dev/null
@@ -1,69 +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_approx_top_sum.h"
-
-#include "common/exception.h"
-#include "vec/aggregate_functions/aggregate_function_simple_factory.h"
-#include "vec/aggregate_functions/factory_helpers.h"
-#include "vec/aggregate_functions/helpers.h"
-#include "vec/data_types/data_type.h"
-
-namespace doris::vectorized {
-#include "common/compile_check_begin.h"
-
-template <int define_index>
-using creator = creator_with_type_list_base<define_index, TYPE_TINYINT, 
TYPE_SMALLINT, TYPE_INT,
-                                            TYPE_BIGINT, TYPE_LARGEINT>;
-
-template <size_t N>
-AggregateFunctionPtr create_aggregate_function_multi_top_sum_impl(
-        const DataTypes& argument_types, const bool result_is_nullable,
-        const AggregateFunctionAttr& attr) {
-    if (N == argument_types.size() - 3) {
-        return creator<N>::template 
create<AggregateFunctionApproxTopSumSimple>(
-                argument_types, result_is_nullable, attr, attr.column_names);
-    } else {
-        return create_aggregate_function_multi_top_sum_impl<N - 
1>(argument_types,
-                                                                   
result_is_nullable, attr);
-    }
-}
-
-template <>
-AggregateFunctionPtr create_aggregate_function_multi_top_sum_impl<0>(
-        const DataTypes& argument_types, const bool result_is_nullable,
-        const AggregateFunctionAttr& attr) {
-    return creator<0>::create<AggregateFunctionApproxTopSumSimple>(
-            argument_types, result_is_nullable, attr, attr.column_names);
-}
-
-AggregateFunctionPtr create_aggregate_function_approx_top_sum(const 
std::string& name,
-                                                              const DataTypes& 
argument_types,
-                                                              const bool 
result_is_nullable,
-                                                              const 
AggregateFunctionAttr& attr) {
-    constexpr size_t max_param_value = 10;
-    assert_arity_range(name, argument_types, 3, max_param_value);
-    return 
create_aggregate_function_multi_top_sum_impl<max_param_value>(argument_types,
-                                                                         
result_is_nullable, attr);
-}
-
-void 
register_aggregate_function_approx_top_sum(AggregateFunctionSimpleFactory& 
factory) {
-    factory.register_function_both("approx_top_sum", 
create_aggregate_function_approx_top_sum);
-}
-
-#include "common/compile_check_end.h"
-} // namespace doris::vectorized
\ No newline at end of file
diff --git a/be/src/vec/aggregate_functions/aggregate_function_approx_top_sum.h 
b/be/src/vec/aggregate_functions/aggregate_function_approx_top_sum.h
deleted file mode 100644
index d09371375d0..00000000000
--- a/be/src/vec/aggregate_functions/aggregate_function_approx_top_sum.h
+++ /dev/null
@@ -1,248 +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 <rapidjson/encodings.h>
-#include <rapidjson/prettywriter.h>
-#include <rapidjson/stringbuffer.h>
-#include <rapidjson/writer.h>
-
-#include <cstdint>
-#include <string>
-
-#include "common/cast_set.h"
-#include "vec/aggregate_functions/aggregate_function.h"
-#include "vec/aggregate_functions/aggregate_function_approx_top.h"
-#include "vec/columns/column.h"
-#include "vec/columns/column_array.h"
-#include "vec/columns/column_string.h"
-#include "vec/columns/column_struct.h"
-#include "vec/columns/column_vector.h"
-#include "vec/common/assert_cast.h"
-#include "vec/common/space_saving.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_ipv4.h"
-#include "vec/data_types/data_type_struct.h"
-
-namespace doris::vectorized {
-#include "common/compile_check_begin.h"
-
-struct AggregateFunctionTopKGenericData {
-    using Set = SpaceSaving<StringRef, StringRefHash>;
-
-    Set value;
-};
-
-template <PrimitiveType T, typename TResult, typename Data>
-class AggregateFunctionApproxTopSum final
-        : public IAggregateFunctionDataHelper<Data,
-                                              AggregateFunctionApproxTopSum<T, 
TResult, Data>>,
-          AggregateFunctionApproxTop,
-          VarargsExpression,
-          NullableAggregateFunction {
-private:
-    using State = AggregateFunctionTopKGenericData;
-
-    using ResultDataType = DataTypeNumber<T>;
-    using ColVecType = typename PrimitiveTypeTraits<T>::ColumnType;
-    using ColVecResult = typename PrimitiveTypeTraits<T>::ColumnType;
-
-public:
-    AggregateFunctionApproxTopSum(const std::vector<std::string>& column_names,
-                                  const DataTypes& argument_types_)
-            : IAggregateFunctionDataHelper<Data, 
AggregateFunctionApproxTopSum<T, TResult, Data>>(
-                      argument_types_),
-              AggregateFunctionApproxTop(column_names) {}
-
-    String get_name() const override { return "approx_top_sum"; }
-
-    DataTypePtr get_return_type() const override { return 
std::make_shared<DataTypeString>(); }
-
-    // Serializes the aggregate function's state (including the SpaceSaving 
structure and threshold) into a buffer.
-    void serialize(ConstAggregateDataPtr __restrict place, BufferWritable& 
buf) const override {
-        this->data(place).value.write(buf);
-
-        buf.write_var_uint(_column_names.size());
-        for (const auto& column_name : _column_names) {
-            buf.write_binary(column_name);
-        }
-        buf.write_var_uint(_threshold);
-        buf.write_var_uint(_reserved);
-    }
-
-    // Deserializes the aggregate function's state from a buffer (including 
the SpaceSaving structure and threshold).
-    void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf,
-                     Arena& arena) const override {
-        auto readStringBinaryInto = [](Arena& arena, BufferReadable& buf) {
-            uint64_t size = 0;
-            buf.read_var_uint(size);
-
-            if (UNLIKELY(size > DEFAULT_MAX_STRING_SIZE)) {
-                throw Exception(ErrorCode::INTERNAL_ERROR, "Too large string 
size.");
-            }
-
-            char* data = arena.alloc(size);
-            buf.read(data, size);
-
-            return StringRef(data, size);
-        };
-
-        auto& set = this->data(place).value;
-        set.clear();
-
-        uint64_t size = 0;
-        buf.read_var_uint(size);
-        if (UNLIKELY(size > TOP_K_MAX_SIZE)) {
-            throw Exception(ErrorCode::INTERNAL_ERROR,
-                            "Too large size ({}) for aggregate function '{}' 
state (maximum is {})",
-                            size, get_name(), TOP_K_MAX_SIZE);
-        }
-
-        set.resize(size);
-        for (size_t i = 0; i < size; ++i) {
-            auto ref = readStringBinaryInto(arena, buf);
-            uint64_t count = 0;
-            uint64_t error = 0;
-            buf.read_var_uint(count);
-            buf.read_var_uint(error);
-            set.insert(ref, count, error);
-            arena.rollback(ref.size);
-        }
-
-        set.read_alpha_map(buf);
-
-        uint64_t column_size = 0;
-        buf.read_var_uint(column_size);
-        _column_names.clear();
-        for (uint64_t i = 0; i < column_size; i++) {
-            std::string column_name;
-            buf.read_binary(column_name);
-            _column_names.emplace_back(std::move(column_name));
-        }
-        buf.read_var_uint(_threshold);
-        buf.read_var_uint(_reserved);
-    }
-
-    // Adds a new row of data to the aggregate function (inserts a new value 
into the SpaceSaving structure).
-    void add(AggregateDataPtr __restrict place, const IColumn** columns, 
ssize_t row_num,
-             Arena& arena) const override {
-        if (!_init_flag) {
-            lazy_init(columns, row_num, this->get_argument_types());
-        }
-
-        auto& set = this->data(place).value;
-        if (set.capacity() != _reserved) {
-            set.resize(_reserved);
-        }
-
-        auto all_serialize_value_into_arena =
-                [](size_t i, size_t keys_size, const IColumn** columns, Arena& 
arena) -> StringRef {
-            const char* begin = nullptr;
-
-            size_t sum_size = 0;
-            for (size_t j = 0; j < keys_size; ++j) {
-                sum_size += columns[j]->serialize_value_into_arena(i, arena, 
begin).size;
-            }
-
-            return {begin, sum_size};
-        };
-
-        StringRef str_serialized =
-                all_serialize_value_into_arena(row_num, _column_names.size(), 
columns, arena);
-        const auto& column = assert_cast<const ColVecType&, 
TypeCheckOnRelease::DISABLE>(
-                *columns[_column_names.size() - 1]);
-        set.insert(str_serialized, 
static_cast<uint64_t>(TResult(column.get_data()[row_num])));
-        arena.rollback(str_serialized.size);
-    }
-
-    void add_many(AggregateDataPtr __restrict place, const IColumn** columns,
-                  std::vector<int>& rows, Arena& arena) const override {
-        for (auto row : rows) {
-            add(place, columns, row, arena);
-        }
-    }
-
-    void reset(AggregateDataPtr __restrict place) const override {
-        this->data(place).value.clear();
-    }
-
-    // Merges the state of another aggregate function into the current one 
(merges two SpaceSaving sets).
-    void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs,
-               Arena&) const override {
-        auto& rhs_set = this->data(rhs).value;
-        if (!rhs_set.size()) {
-            return;
-        }
-
-        auto& set = this->data(place).value;
-        if (set.capacity() != _reserved) {
-            set.resize(_reserved);
-        }
-        set.merge(rhs_set);
-    }
-
-    void insert_result_into(ConstAggregateDataPtr __restrict place, IColumn& 
to) const override {
-        auto& data_to = assert_cast<ColumnString&, 
TypeCheckOnRelease::DISABLE>(to);
-
-        const typename State::Set& set = this->data(place).value;
-        auto result_vec = set.top_k(_threshold);
-
-        rapidjson::StringBuffer buffer;
-        rapidjson::PrettyWriter<rapidjson::StringBuffer> writer(buffer);
-        writer.StartArray();
-        for (auto& result : result_vec) {
-            auto argument_types = this->get_argument_types();
-            MutableColumns argument_columns(_column_names.size());
-            for (size_t i = 0; i < _column_names.size(); ++i) {
-                argument_columns[i] = argument_types[i]->create_column();
-            }
-            rapidjson::StringBuffer sub_buffer;
-            rapidjson::Writer<rapidjson::StringBuffer> sub_writer(sub_buffer);
-            sub_writer.StartObject();
-            const char* begin = result.key.data;
-            for (size_t i = 0; i < _column_names.size(); i++) {
-                begin = 
argument_columns[i]->deserialize_and_insert_from_arena(begin);
-                std::string row_str = 
argument_types[i]->to_string(*argument_columns[i], 0);
-                sub_writer.Key(_column_names[i].data(),
-                               cast_set<uint32_t>(_column_names[i].size()));
-                sub_writer.String(row_str.data(), 
cast_set<uint32_t>(row_str.size()));
-            }
-            sub_writer.Key("sum");
-            sub_writer.String(std::to_string(result.count).c_str());
-            sub_writer.EndObject();
-            writer.RawValue(sub_buffer.GetString(), sub_buffer.GetSize(), 
rapidjson::kObjectType);
-        }
-        writer.EndArray();
-        std::string res = buffer.GetString();
-        data_to.insert_data(res.data(), res.size());
-    }
-};
-
-template <PrimitiveType T>
-struct TopSumSimple {
-    using ResultType = typename PrimitiveTypeTraits<T>::CppType;
-    using AggregateDataType = AggregateFunctionTopKGenericData;
-    using Function = AggregateFunctionApproxTopSum<T, ResultType, 
AggregateDataType>;
-};
-
-template <PrimitiveType T>
-using AggregateFunctionApproxTopSumSimple = typename TopSumSimple<T>::Function;
-#include "common/compile_check_end.h"
-} // 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 50123ad2111..91d6fe83c21 100644
--- a/be/src/vec/aggregate_functions/aggregate_function_simple_factory.cpp
+++ b/be/src/vec/aggregate_functions/aggregate_function_simple_factory.cpp
@@ -76,8 +76,6 @@ void 
register_aggregate_function_covar_pop(AggregateFunctionSimpleFactory& facto
 void register_aggregate_function_covar_samp(AggregateFunctionSimpleFactory& 
factory);
 void register_aggregate_function_skewness(AggregateFunctionSimpleFactory& 
factory);
 void register_aggregate_function_kurtosis(AggregateFunctionSimpleFactory& 
factory);
-void register_aggregate_function_approx_top_k(AggregateFunctionSimpleFactory& 
factory);
-void 
register_aggregate_function_approx_top_sum(AggregateFunctionSimpleFactory& 
factory);
 void 
register_aggregate_function_percentile_reservoir(AggregateFunctionSimpleFactory&
 factory);
 void register_aggregate_function_ai_agg(AggregateFunctionSimpleFactory& 
factory);
 void register_aggregate_function_bool_union(AggregateFunctionSimpleFactory& 
factory);
@@ -134,8 +132,6 @@ AggregateFunctionSimpleFactory& 
AggregateFunctionSimpleFactory::instance() {
         register_aggregate_function_covar_samp(instance);
         register_aggregate_function_skewness(instance);
         register_aggregate_function_kurtosis(instance);
-        register_aggregate_function_approx_top_k(instance);
-        register_aggregate_function_approx_top_sum(instance);
         register_aggregate_function_percentile_reservoir(instance);
         register_aggregate_function_ai_agg(instance);
         register_aggregate_function_bool_union(instance);
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinAggregateFunctions.java
 
b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinAggregateFunctions.java
index 831776731ae..bf68560e5c3 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinAggregateFunctions.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinAggregateFunctions.java
@@ -19,8 +19,6 @@ package org.apache.doris.catalog;
 
 import org.apache.doris.nereids.trees.expressions.functions.agg.AIAgg;
 import org.apache.doris.nereids.trees.expressions.functions.agg.AnyValue;
-import org.apache.doris.nereids.trees.expressions.functions.agg.ApproxTopK;
-import org.apache.doris.nereids.trees.expressions.functions.agg.ApproxTopSum;
 import org.apache.doris.nereids.trees.expressions.functions.agg.ArrayAgg;
 import org.apache.doris.nereids.trees.expressions.functions.agg.Avg;
 import org.apache.doris.nereids.trees.expressions.functions.agg.AvgWeighted;
@@ -110,8 +108,6 @@ public class BuiltinAggregateFunctions implements 
FunctionHelper {
     public final List<AggregateFunc> aggregateFunctions = ImmutableList.of(
             agg(AIAgg.class, "ai_agg"),
             agg(AnyValue.class, "any", "any_value"),
-            agg(ApproxTopK.class, "approx_top_k"),
-            agg(ApproxTopSum.class, "approx_top_sum"),
             agg(ArrayAgg.class, "array_agg"),
             agg(Avg.class, "avg"),
             agg(AvgWeighted.class, "avg_weighted"),
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/ApproxTopK.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/ApproxTopK.java
deleted file mode 100644
index ae051f55efb..00000000000
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/ApproxTopK.java
+++ /dev/null
@@ -1,99 +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.
-
-package org.apache.doris.nereids.trees.expressions.functions.agg;
-
-import org.apache.doris.catalog.FunctionSignature;
-import org.apache.doris.nereids.exceptions.AnalysisException;
-import org.apache.doris.nereids.trees.expressions.Expression;
-import 
org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
-import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
-import org.apache.doris.nereids.types.StringType;
-import org.apache.doris.nereids.types.coercion.AnyDataType;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableList;
-
-import java.util.List;
-
-/**
- * AggregateFunction 'approx_top_k'. This class is generated by 
GenerateFunction.
- */
-public class ApproxTopK extends NullableAggregateFunction
-        implements ExplicitlyCastableSignature {
-
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(StringType.INSTANCE)
-                    .varArgs(AnyDataType.INSTANCE_WITHOUT_INDEX)
-    );
-
-    public ApproxTopK(Expression... varArgs) {
-        this(false, varArgs);
-    }
-
-    public ApproxTopK(boolean distinct, Expression... varArgs) {
-        this(distinct, false, varArgs);
-    }
-
-    public ApproxTopK(boolean distinct, boolean alwaysNullable, Expression... 
varArgs) {
-        super("approx_top_k", distinct, alwaysNullable, varArgs);
-    }
-
-    /** constructor for withChildren and reuse signature */
-    private ApproxTopK(NullableAggregateFunctionParams functionParams) {
-        super(functionParams);
-    }
-
-    @Override
-    public void checkLegalityBeforeTypeCoercion() {
-        if (arity() < 3) {
-            throw new AnalysisException(
-                    "Function requires at least 3 parameters: " + 
this.toSql());
-        }
-
-        if (!getArgument(arity() - 2).isConstant() || !getArgumentType(arity() 
- 2).isIntegerLikeType()) {
-            throw new AnalysisException(
-                    "The second to last parameter must be a constant Integer 
Type: " + this.toSql());
-        }
-
-        if (!getArgument(arity() - 1).isConstant() || !getArgumentType(arity() 
- 1).isIntegerLikeType()) {
-            throw new AnalysisException(
-                    "The last parameter must be a constant Integer Type: " + 
this.toSql());
-        }
-    }
-
-    @Override
-    public ApproxTopK withDistinctAndChildren(boolean distinct, 
List<Expression> children) {
-        Preconditions.checkArgument(!children.isEmpty());
-        return new ApproxTopK(getFunctionParams(distinct, children));
-    }
-
-    @Override
-    public NullableAggregateFunction withAlwaysNullable(boolean 
alwaysNullable) {
-        return new ApproxTopK(getAlwaysNullableFunctionParams(alwaysNullable));
-    }
-
-    @Override
-    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
-        return visitor.visitApproxTopK(this, context);
-    }
-
-    @Override
-    public List<FunctionSignature> getSignatures() {
-        return SIGNATURES;
-    }
-}
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/ApproxTopSum.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/ApproxTopSum.java
deleted file mode 100644
index b9f98ade7e0..00000000000
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/ApproxTopSum.java
+++ /dev/null
@@ -1,99 +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.
-
-package org.apache.doris.nereids.trees.expressions.functions.agg;
-
-import org.apache.doris.catalog.FunctionSignature;
-import org.apache.doris.nereids.exceptions.AnalysisException;
-import org.apache.doris.nereids.trees.expressions.Expression;
-import 
org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
-import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
-import org.apache.doris.nereids.types.StringType;
-import org.apache.doris.nereids.types.coercion.AnyDataType;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableList;
-
-import java.util.List;
-
-/**
- * AggregateFunction 'approx_top_sum'. This class is generated by 
GenerateFunction.
- */
-public class ApproxTopSum extends NullableAggregateFunction
-        implements ExplicitlyCastableSignature {
-
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(StringType.INSTANCE)
-                    .varArgs(AnyDataType.INSTANCE_WITHOUT_INDEX)
-    );
-
-    public ApproxTopSum(Expression... varArgs) {
-        this(false, varArgs);
-    }
-
-    public ApproxTopSum(boolean distinct, Expression... varArgs) {
-        this(distinct, false, varArgs);
-    }
-
-    public ApproxTopSum(boolean distinct, boolean alwaysNullable, 
Expression... varArgs) {
-        super("approx_top_sum", distinct, alwaysNullable, varArgs);
-    }
-
-    /** constructor for withChildren and reuse signature */
-    private ApproxTopSum(NullableAggregateFunctionParams functionParams) {
-        super(functionParams);
-    }
-
-    @Override
-    public void checkLegalityBeforeTypeCoercion() {
-        if (arity() < 3) {
-            throw new AnalysisException(
-                    "Function requires at least 3 parameters: " + 
this.toSql());
-        }
-
-        if (!getArgument(arity() - 2).isConstant() || !getArgumentType(arity() 
- 2).isIntegerLikeType()) {
-            throw new AnalysisException(
-                    "The second to last parameter must be a constant Integer 
Type: " + this.toSql());
-        }
-
-        if (!getArgument(arity() - 1).isConstant() || !getArgumentType(arity() 
- 1).isIntegerLikeType()) {
-            throw new AnalysisException(
-                    "The last parameter must be a constant Integer Type: " + 
this.toSql());
-        }
-    }
-
-    @Override
-    public ApproxTopSum withDistinctAndChildren(boolean distinct, 
List<Expression> children) {
-        Preconditions.checkArgument(!children.isEmpty());
-        return new ApproxTopSum(getFunctionParams(distinct, children));
-    }
-
-    @Override
-    public NullableAggregateFunction withAlwaysNullable(boolean 
alwaysNullable) {
-        return new 
ApproxTopSum(getAlwaysNullableFunctionParams(alwaysNullable));
-    }
-
-    @Override
-    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
-        return visitor.visitApproxTopSum(this, context);
-    }
-
-    @Override
-    public List<FunctionSignature> getSignatures() {
-        return SIGNATURES;
-    }
-}
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/AggregateFunctionVisitor.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/AggregateFunctionVisitor.java
index 6afa20ec29a..fa98c577428 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/AggregateFunctionVisitor.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/AggregateFunctionVisitor.java
@@ -20,8 +20,6 @@ package org.apache.doris.nereids.trees.expressions.visitor;
 import org.apache.doris.nereids.trees.expressions.functions.agg.AIAgg;
 import 
org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction;
 import org.apache.doris.nereids.trees.expressions.functions.agg.AnyValue;
-import org.apache.doris.nereids.trees.expressions.functions.agg.ApproxTopK;
-import org.apache.doris.nereids.trees.expressions.functions.agg.ApproxTopSum;
 import org.apache.doris.nereids.trees.expressions.functions.agg.ArrayAgg;
 import org.apache.doris.nereids.trees.expressions.functions.agg.Avg;
 import org.apache.doris.nereids.trees.expressions.functions.agg.AvgWeighted;
@@ -396,12 +394,4 @@ public interface AggregateFunctionVisitor<R, C> {
         return visitAggregateFunction(javaUdaf, context);
     }
 
-    default R visitApproxTopK(ApproxTopK approxTopK, C context) {
-        return visitNullableAggregateFunction(approxTopK, context);
-    }
-
-    default R visitApproxTopSum(ApproxTopSum approxTopSum, C context) {
-        return visitNullableAggregateFunction(approxTopSum, context);
-    }
-
 }
diff --git a/regression-test/data/inverted_index_p0/test_index_approx_top_k.out 
b/regression-test/data/inverted_index_p0/test_index_approx_top_k.out
deleted file mode 100644
index 4e7ac3cd07c..00000000000
Binary files 
a/regression-test/data/inverted_index_p0/test_index_approx_top_k.out and 
/dev/null differ
diff --git 
a/regression-test/data/inverted_index_p0/test_index_approx_top_sum.out 
b/regression-test/data/inverted_index_p0/test_index_approx_top_sum.out
deleted file mode 100644
index 2711e78af2c..00000000000
Binary files 
a/regression-test/data/inverted_index_p0/test_index_approx_top_sum.out and 
/dev/null differ
diff --git 
a/regression-test/suites/inverted_index_p0/test_index_approx_top_k.groovy 
b/regression-test/suites/inverted_index_p0/test_index_approx_top_k.groovy
deleted file mode 100644
index 4dffd80be86..00000000000
--- a/regression-test/suites/inverted_index_p0/test_index_approx_top_k.groovy
+++ /dev/null
@@ -1,153 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-
-
-suite("test_index_approx_top_k", "p0"){
-    def tableName = "test_index_approx_top_k"
-
-    sql "DROP TABLE IF EXISTS ${tableName}"
-
-    def create_table = {table_name ->
-      sql """
-        CREATE TABLE ${table_name} (
-          `@timestamp` int(11) NULL COMMENT "",
-          `clientip` text NULL COMMENT "",
-          `request` text NULL COMMENT "",
-          `status` int NULL COMMENT "",
-          `size` int NULL COMMENT ""
-        ) ENGINE=OLAP
-        DUPLICATE KEY(`@timestamp`)
-        COMMENT "OLAP"
-        DISTRIBUTED BY RANDOM BUCKETS 1
-        PROPERTIES (
-          "replication_allocation" = "tag.location.default: 1",
-          "disable_auto_compaction" = "true"
-        );
-      """
-    }
-
-    def load_httplogs_data = {table_name, label, read_flag, format_flag, 
file_name, ignore_failure=false,
-                        expected_succ_rows = -1, load_to_single_tablet = 
'true' ->
-        
-        // load the json data
-        streamLoad {
-            table "${table_name}"
-            
-            // set http request header params
-            set 'label', label + "_" + UUID.randomUUID().toString()
-            set 'read_json_by_line', read_flag
-            set 'format', format_flag
-            file file_name // import json file
-            time 10000 // limit inflight 10s
-            if (expected_succ_rows >= 0) {
-                set 'max_filter_ratio', '1'
-            }
-
-            // if declared a check callback, the default check condition will 
ignore.
-            // So you must check all condition
-            check { result, exception, startTime, endTime ->
-                       if (ignore_failure && expected_succ_rows < 0) { return }
-                    if (exception != null) {
-                        throw exception
-                    }
-                    log.info("Stream load result: ${result}".toString())
-                    def json = parseJson(result)
-                    assertEquals("success", json.Status.toLowerCase())
-                    if (expected_succ_rows >= 0) {
-                        assertEquals(json.NumberLoadedRows, expected_succ_rows)
-                    } else {
-                        assertEquals(json.NumberTotalRows, 
json.NumberLoadedRows + json.NumberUnselectedRows)
-                        assertTrue(json.NumberLoadedRows > 0 && json.LoadBytes 
> 0)
-                }
-            }
-        }
-    }
-
-    try {
-        create_table(tableName)
-
-        load_httplogs_data.call(tableName, 'test_index_approx_top_k', 'true', 
'json', 'documents-1000.json')
-
-        sql "sync"
-
-        sql """ set enable_common_expr_pushdown = true """
-
-        sql """ set debug_skip_fold_constant = true; """
-        qt_sql """ select clientip, count(*) as count from ${tableName} group 
by clientip order by count desc, clientip asc limit 10; """
-        qt_sql """ select approx_top_k(clientip, 10, 300) from ${tableName}; 
"""
-        qt_sql """ select approx_top_k(clientip, 5 + 5, 300) from 
${tableName}; """
-        qt_sql """ select approx_top_k(clientip, abs(-10), 300) from 
${tableName}; """
-
-        qt_sql """ select clientip, status, size, count(*) as count from 
${tableName} group by clientip, status, size order by count desc, clientip asc 
limit 10; """
-        qt_sql """ select approx_top_k(clientip, status, size, 10, 300) from 
${tableName}; """
-        qt_sql """ select approx_top_k(clientip, status, size, 5 + 5, 300) 
from ${tableName}; """
-        qt_sql """ select approx_top_k(clientip, status, size, abs(-10), 300) 
from ${tableName}; """
-
-        def result1 = "fail"
-        try {
-            drop_result = sql " select approx_top_k(clientip, -10, 300) from 
${tableName}; "
-            result1 = 'success'
-        } catch(Exception ex) {
-            logger.info("error msg: " + ex)
-        }
-        assertEquals(result1, 'fail')
-
-        qt_sql """
-            WITH tmp AS (
-                SELECT approx_top_k(clientip, status, size, 10, 300) AS 
json_output FROM ${tableName}
-            )
-            SELECT 
-                e1
-            FROM 
-                tmp 
-            LATERAL VIEW explode_json_array_json(json_output) tmp1 AS e1;
-        """
-
-        sql """ set debug_skip_fold_constant = true; """
-        qt_sql """ select clientip, count(*) as count from ${tableName} group 
by clientip order by count desc, clientip asc limit 10; """
-        qt_sql """ select approx_top_k(clientip, 10, 300) from ${tableName}; 
"""
-        qt_sql """ select approx_top_k(clientip, 5 + 5, 300) from 
${tableName}; """
-        qt_sql """ select approx_top_k(clientip, abs(-10), 300) from 
${tableName}; """
-
-        qt_sql """ select clientip, status, size, count(*) as count from 
${tableName} group by clientip, status, size order by count desc, clientip asc 
limit 10; """
-        qt_sql """ select approx_top_k(clientip, status, size, 10, 300) from 
${tableName}; """
-        qt_sql """ select approx_top_k(clientip, status, size, 5 + 5, 300) 
from ${tableName}; """
-        qt_sql """ select approx_top_k(clientip, status, size, abs(-10), 300) 
from ${tableName}; """
-
-        def result2 = "fail"
-        try {
-            drop_result = sql " select approx_top_k(clientip, -10, 300) from 
${tableName}; "
-            result2 = 'success'
-        } catch(Exception ex) {
-            logger.info("error msg: " + ex)
-        }
-        assertEquals(result2, 'fail')
-
-        qt_sql """
-            WITH tmp AS (
-                SELECT approx_top_k(clientip, status, size, 10, 300) AS 
json_output FROM ${tableName}
-            )
-            SELECT 
-                e1
-            FROM 
-                tmp 
-            LATERAL VIEW explode_json_array_json(json_output) tmp1 AS e1;
-        """
-    } finally {
-        //try_sql("DROP TABLE IF EXISTS ${testTable}")
-    }
-}
diff --git 
a/regression-test/suites/inverted_index_p0/test_index_approx_top_sum.groovy 
b/regression-test/suites/inverted_index_p0/test_index_approx_top_sum.groovy
deleted file mode 100644
index 5d764f95b1a..00000000000
--- a/regression-test/suites/inverted_index_p0/test_index_approx_top_sum.groovy
+++ /dev/null
@@ -1,153 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-
-
-suite("test_index_approx_top_sum", "p0"){
-    def tableName = "test_index_approx_top_sum"
-
-    sql "DROP TABLE IF EXISTS ${tableName}"
-
-    def create_table = {table_name ->
-      sql """
-        CREATE TABLE ${table_name} (
-          `@timestamp` int(11) NULL COMMENT "",
-          `clientip` text NULL COMMENT "",
-          `request` text NULL COMMENT "",
-          `status` int NULL COMMENT "",
-          `size` int NULL COMMENT ""
-        ) ENGINE=OLAP
-        DUPLICATE KEY(`@timestamp`)
-        COMMENT "OLAP"
-        DISTRIBUTED BY RANDOM BUCKETS 1
-        PROPERTIES (
-          "replication_allocation" = "tag.location.default: 1",
-          "disable_auto_compaction" = "true"
-        );
-      """
-    }
-
-    def load_httplogs_data = {table_name, label, read_flag, format_flag, 
file_name, ignore_failure=false,
-                        expected_succ_rows = -1, load_to_single_tablet = 
'true' ->
-        
-        // load the json data
-        streamLoad {
-            table "${table_name}"
-            
-            // set http request header params
-            set 'label', label + "_" + UUID.randomUUID().toString()
-            set 'read_json_by_line', read_flag
-            set 'format', format_flag
-            file file_name // import json file
-            time 10000 // limit inflight 10s
-            if (expected_succ_rows >= 0) {
-                set 'max_filter_ratio', '1'
-            }
-
-            // if declared a check callback, the default check condition will 
ignore.
-            // So you must check all condition
-            check { result, exception, startTime, endTime ->
-                       if (ignore_failure && expected_succ_rows < 0) { return }
-                    if (exception != null) {
-                        throw exception
-                    }
-                    log.info("Stream load result: ${result}".toString())
-                    def json = parseJson(result)
-                    assertEquals("success", json.Status.toLowerCase())
-                    if (expected_succ_rows >= 0) {
-                        assertEquals(json.NumberLoadedRows, expected_succ_rows)
-                    } else {
-                        assertEquals(json.NumberTotalRows, 
json.NumberLoadedRows + json.NumberUnselectedRows)
-                        assertTrue(json.NumberLoadedRows > 0 && json.LoadBytes 
> 0)
-                }
-            }
-        }
-    }
-
-    try {
-        create_table(tableName)
-
-        load_httplogs_data.call(tableName, 'test_index_approx_top_sum', 
'true', 'json', 'documents-1000.json')
-
-        sql "sync"
-
-        sql """ set enable_common_expr_pushdown = true """
-
-        sql """ set debug_skip_fold_constant = true; """
-        qt_sql """ select size, sum(size) as sum from ${tableName} group by 
size order by sum desc, size asc limit 10; """
-        qt_sql """ select approx_top_sum(size, 10, 300) from ${tableName}; """
-        qt_sql """ select approx_top_sum(size, 5 + 5, 300) from ${tableName}; 
"""
-        qt_sql """ select approx_top_sum(size, abs(-10), 300) from 
${tableName}; """
-
-        qt_sql """ select clientip, status, size, sum(size) as sum from 
${tableName} group by clientip, status, size order by sum desc, clientip asc 
limit 10; """
-        qt_sql """ select approx_top_sum(clientip, status, size, 10, 300) from 
${tableName}; """
-        qt_sql """ select approx_top_sum(clientip, status, size, 5 + 5, 300) 
from ${tableName}; """
-        qt_sql """ select approx_top_sum(clientip, status, size, abs(-10), 
300) from ${tableName}; """
-
-        def result1 = "fail"
-        try {
-            drop_result = sql " select approx_top_sum(size, -10, 300) from 
${tableName}; "
-            result1 = 'success'
-        } catch(Exception ex) {
-            logger.info("error msg: " + ex)
-        }
-        assertEquals(result1, 'fail')
-
-        qt_sql """
-            WITH tmp AS (
-                SELECT approx_top_sum(clientip, status, size, 10, 300) AS 
json_output FROM ${tableName}
-            )
-            SELECT 
-                e1
-            FROM 
-                tmp 
-            LATERAL VIEW explode_json_array_json(json_output) tmp1 AS e1;
-        """
-
-        sql """ set debug_skip_fold_constant = true; """
-        qt_sql """ select size, sum(size) as sum from ${tableName} group by 
size order by sum desc, size asc limit 10; """
-        qt_sql """ select approx_top_sum(size, 10, 300) from ${tableName}; """
-        qt_sql """ select approx_top_sum(size, 5 + 5, 300) from ${tableName}; 
"""
-        qt_sql """ select approx_top_sum(size, abs(-10), 300) from 
${tableName}; """
-
-        qt_sql """ select clientip, status, size, sum(size) as sum from 
${tableName} group by clientip, status, size order by sum desc, clientip asc 
limit 10; """
-        qt_sql """ select approx_top_sum(clientip, status, size, 10, 300) from 
${tableName}; """
-        qt_sql """ select approx_top_sum(clientip, status, size, 5 + 5, 300) 
from ${tableName}; """
-        qt_sql """ select approx_top_sum(clientip, status, size, abs(-10), 
300) from ${tableName}; """
-
-        def result2 = "fail"
-        try {
-            drop_result = sql " select approx_top_sum(size, -10, 300) from 
${tableName}; "
-            result2 = 'success'
-        } catch(Exception ex) {
-            logger.info("error msg: " + ex)
-        }
-        assertEquals(result2, 'fail')
-
-        qt_sql """
-            WITH tmp AS (
-                SELECT approx_top_sum(clientip, status, size, 10, 300) AS 
json_output FROM ${tableName}
-            )
-            SELECT 
-                e1
-            FROM 
-                tmp 
-            LATERAL VIEW explode_json_array_json(json_output) tmp1 AS e1;
-        """
-    } finally {
-        //try_sql("DROP TABLE IF EXISTS ${testTable}")
-    }
-}


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

Reply via email to