You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by li...@apache.org on 2022/07/23 09:58:37 UTC

[doris] branch master updated: [Vectorized] Support sort combinator (#10469)

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

lihaopeng 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 6e98ebba27 [Vectorized] Support sort combinator (#10469)
6e98ebba27 is described below

commit 6e98ebba27b985937a8845add49b42e24a2fa0ba
Author: Pxl <95...@qq.com>
AuthorDate: Sat Jul 23 17:58:31 2022 +0800

    [Vectorized] Support sort combinator (#10469)
---
 be/src/vec/CMakeLists.txt                          |   1 +
 .../aggregate_function_distinct.cpp                |   6 -
 .../aggregate_function_simple_factory.cpp          |   8 +-
 .../aggregate_function_simple_factory.h            |   8 +-
 .../aggregate_function_sort.cpp                    | 102 +++++++++++
 .../aggregate_functions/aggregate_function_sort.h  | 193 +++++++++++++++++++++
 be/src/vec/columns/column_const.h                  |   2 +
 be/src/vec/core/sort_description.h                 |  19 +-
 .../java/org/apache/doris/analysis/CastExpr.java   |  17 +-
 .../java/org/apache/doris/catalog/FunctionSet.java |  14 +-
 .../main/java/org/apache/doris/catalog/Type.java   |  52 +++---
 11 files changed, 342 insertions(+), 80 deletions(-)

diff --git a/be/src/vec/CMakeLists.txt b/be/src/vec/CMakeLists.txt
index 33c1caccc3..01804896be 100644
--- a/be/src/vec/CMakeLists.txt
+++ b/be/src/vec/CMakeLists.txt
@@ -26,6 +26,7 @@ set(VEC_FILES
   aggregate_functions/aggregate_function_count.cpp
   aggregate_functions/aggregate_function_distinct.cpp
   aggregate_functions/aggregate_function_sum.cpp
+  aggregate_functions/aggregate_function_sort.cpp
   aggregate_functions/aggregate_function_min_max.cpp
   aggregate_functions/aggregate_function_min_max_by.cpp
   aggregate_functions/aggregate_function_null.cpp
diff --git a/be/src/vec/aggregate_functions/aggregate_function_distinct.cpp b/be/src/vec/aggregate_functions/aggregate_function_distinct.cpp
index 26e9e1520c..eb54d8070c 100644
--- a/be/src/vec/aggregate_functions/aggregate_function_distinct.cpp
+++ b/be/src/vec/aggregate_functions/aggregate_function_distinct.cpp
@@ -20,9 +20,6 @@
 
 #include "vec/aggregate_functions/aggregate_function_distinct.h"
 
-#include <algorithm>
-#include <boost/algorithm/string.hpp>
-
 #include "vec/aggregate_functions/aggregate_function_combinator.h"
 #include "vec/aggregate_functions/aggregate_function_simple_factory.h"
 #include "vec/aggregate_functions/helpers.h"
@@ -89,9 +86,6 @@ void register_aggregate_function_combinator_distinct(AggregateFunctionSimpleFact
                        [](const auto& e) { return remove_nullable(e); });
         auto function_combinator = std::make_shared<AggregateFunctionCombinatorDistinct>();
         auto transform_arguments = function_combinator->transform_arguments(nested_types);
-        if (!boost::algorithm::starts_with(name, DISTINCT_FUNCTION_PREFIX)) {
-            return AggregateFunctionPtr();
-        }
         auto nested_function_name = name.substr(DISTINCT_FUNCTION_PREFIX.size());
         auto nested_function = factory.get(nested_function_name, transform_arguments, params);
         return function_combinator->transform_aggregate_function(nested_function, types, params,
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 58dc0a4c9b..c2169eb223 100644
--- a/be/src/vec/aggregate_functions/aggregate_function_simple_factory.cpp
+++ b/be/src/vec/aggregate_functions/aggregate_function_simple_factory.cpp
@@ -26,15 +26,17 @@ namespace doris::vectorized {
 
 class AggregateFunctionSimpleFactory;
 
-void register_aggregate_function_sum(AggregateFunctionSimpleFactory& factory);
+void register_aggregate_function_combinator_sort(AggregateFunctionSimpleFactory& factory);
+void register_aggregate_function_combinator_distinct(AggregateFunctionSimpleFactory& factory);
 void register_aggregate_function_combinator_null(AggregateFunctionSimpleFactory& factory);
+
+void register_aggregate_function_sum(AggregateFunctionSimpleFactory& factory);
 void register_aggregate_function_minmax(AggregateFunctionSimpleFactory& factory);
 void register_aggregate_function_min_max_by(AggregateFunctionSimpleFactory& factory);
 void register_aggregate_function_avg(AggregateFunctionSimpleFactory& factory);
 void register_aggregate_function_count(AggregateFunctionSimpleFactory& factory);
 void register_aggregate_function_HLL_union_agg(AggregateFunctionSimpleFactory& factory);
 void register_aggregate_function_uniq(AggregateFunctionSimpleFactory& factory);
-void register_aggregate_function_combinator_distinct(AggregateFunctionSimpleFactory& factory);
 void register_aggregate_function_bitmap(AggregateFunctionSimpleFactory& factory);
 void register_aggregate_function_window_rank(AggregateFunctionSimpleFactory& factory);
 void register_aggregate_function_window_lead_lag(AggregateFunctionSimpleFactory& factory);
@@ -82,6 +84,8 @@ AggregateFunctionSimpleFactory& AggregateFunctionSimpleFactory::instance() {
         register_aggregate_function_window_lead_lag(instance);
         register_aggregate_function_HLL_union_agg(instance);
         register_aggregate_function_percentile_approx(instance);
+
+        register_aggregate_function_combinator_sort(instance);
     });
     return instance;
 }
diff --git a/be/src/vec/aggregate_functions/aggregate_function_simple_factory.h b/be/src/vec/aggregate_functions/aggregate_function_simple_factory.h
index 833e52d1bf..3751c6b762 100644
--- a/be/src/vec/aggregate_functions/aggregate_function_simple_factory.h
+++ b/be/src/vec/aggregate_functions/aggregate_function_simple_factory.h
@@ -57,16 +57,18 @@ public:
         }
     }
 
-    void register_distinct_function_combinator(const Creator& creator, const std::string& prefix) {
+    void register_distinct_function_combinator(const Creator& creator, const std::string& prefix,
+                                               bool nullable = false) {
+        auto& functions = nullable ? nullable_aggregate_functions : aggregate_functions;
         std::vector<std::string> need_insert;
         for (const auto& entity : aggregate_functions) {
             std::string target_value = prefix + entity.first;
-            if (aggregate_functions.find(target_value) == aggregate_functions.end()) {
+            if (functions.find(target_value) == functions.end()) {
                 need_insert.emplace_back(std::move(target_value));
             }
         }
         for (const auto& function_name : need_insert) {
-            aggregate_functions[function_name] = creator;
+            register_function(function_name, creator, nullable);
         }
     }
 
diff --git a/be/src/vec/aggregate_functions/aggregate_function_sort.cpp b/be/src/vec/aggregate_functions/aggregate_function_sort.cpp
new file mode 100644
index 0000000000..fbdb16df4f
--- /dev/null
+++ b/be/src/vec/aggregate_functions/aggregate_function_sort.cpp
@@ -0,0 +1,102 @@
+// 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_sort.h"
+
+#include "vec/aggregate_functions/aggregate_function_combinator.h"
+#include "vec/aggregate_functions/aggregate_function_simple_factory.h"
+#include "vec/aggregate_functions/helpers.h"
+#include "vec/common/typeid_cast.h"
+#include "vec/data_types/data_type_nullable.h"
+#include "vec/utils/template_helpers.hpp"
+
+namespace doris::vectorized {
+
+class AggregateFunctionCombinatorSort final : public IAggregateFunctionCombinator {
+private:
+    int _sort_column_number;
+
+public:
+    AggregateFunctionCombinatorSort(int sort_column_number)
+            : _sort_column_number(sort_column_number) {}
+
+    String get_name() const override { return "Sort"; }
+
+    DataTypes transform_arguments(const DataTypes& arguments) const override {
+        if (arguments.size() < _sort_column_number + 2) {
+            LOG(FATAL) << "Incorrect number of arguments for aggregate function with Sort, "
+                       << arguments.size() << " less than " << _sort_column_number + 2;
+        }
+
+        DataTypes nested_types;
+        nested_types.assign(arguments.begin(), arguments.end() - 1 - _sort_column_number);
+        return nested_types;
+    }
+
+    template <int sort_column_number>
+    struct Reducer {
+        static void run(AggregateFunctionPtr& function, const AggregateFunctionPtr& nested_function,
+                        const DataTypes& arguments) {
+            function = std::make_shared<
+                    AggregateFunctionSort<sort_column_number, AggregateFunctionSortData>>(
+                    nested_function, arguments);
+        }
+    };
+
+    AggregateFunctionPtr transform_aggregate_function(
+            const AggregateFunctionPtr& nested_function, const DataTypes& arguments,
+            const Array& params, const bool result_is_nullable) const override {
+        DCHECK(nested_function != nullptr);
+        if (nested_function == nullptr) {
+            return nullptr;
+        }
+
+        AggregateFunctionPtr function = nullptr;
+        constexpr_int_match<1, 3, Reducer>::run(_sort_column_number, function, nested_function,
+                                                arguments);
+
+        return function;
+    }
+};
+
+const std::string SORT_FUNCTION_PREFIX = "sort_";
+
+void register_aggregate_function_combinator_sort(AggregateFunctionSimpleFactory& factory) {
+    AggregateFunctionCreator creator = [&](const std::string& name, const DataTypes& types,
+                                           const Array& params, const bool result_is_nullable) {
+        int sort_column_number = std::stoi(name.substr(SORT_FUNCTION_PREFIX.size(), 2));
+        auto nested_function_name = name.substr(SORT_FUNCTION_PREFIX.size() + 2);
+
+        auto function_combinator =
+                std::make_shared<AggregateFunctionCombinatorSort>(sort_column_number);
+
+        auto transform_arguments = function_combinator->transform_arguments(types);
+
+        auto nested_function =
+                factory.get(nested_function_name, transform_arguments, params, result_is_nullable);
+        return function_combinator->transform_aggregate_function(nested_function, types, params,
+                                                                 result_is_nullable);
+    };
+
+    for (char c = '1'; c <= '3'; c++) {
+        factory.register_distinct_function_combinator(creator, SORT_FUNCTION_PREFIX + c + "_",
+                                                      false);
+        factory.register_distinct_function_combinator(creator, SORT_FUNCTION_PREFIX + c + "_",
+                                                      true);
+    }
+}
+} // namespace doris::vectorized
diff --git a/be/src/vec/aggregate_functions/aggregate_function_sort.h b/be/src/vec/aggregate_functions/aggregate_function_sort.h
new file mode 100644
index 0000000000..5cad555c37
--- /dev/null
+++ b/be/src/vec/aggregate_functions/aggregate_function_sort.h
@@ -0,0 +1,193 @@
+// 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 <string>
+
+#include "vec/aggregate_functions/aggregate_function.h"
+#include "vec/aggregate_functions/key_holder_helpers.h"
+#include "vec/columns/column.h"
+#include "vec/columns/column_const.h"
+#include "vec/common/aggregation_common.h"
+#include "vec/common/assert_cast.h"
+#include "vec/common/field_visitors.h"
+#include "vec/common/hash_table/hash_set.h"
+#include "vec/common/hash_table/hash_table.h"
+#include "vec/common/sip_hash.h"
+#include "vec/core/sort_block.h"
+#include "vec/core/sort_description.h"
+#include "vec/io/io_helper.h"
+
+namespace doris::vectorized {
+
+template <int sort_column_size>
+struct AggregateFunctionSortData {
+    void merge(const AggregateFunctionSortData& rhs) {
+        if (block.is_empty_column()) {
+            block = rhs.block;
+        } else {
+            for (size_t i = 0; i < block.columns(); i++) {
+                auto column = block.get_by_position(i).column->assume_mutable();
+                auto column_rhs = rhs.block.get_by_position(i).column;
+                column->insert_many_from(*column_rhs, 0, rhs.block.rows());
+            }
+        }
+    }
+
+    void serialize(BufferWritable& buf) const {
+        PBlock pblock;
+        size_t uncompressed_bytes = 0;
+        size_t compressed_bytes = 0;
+        block.serialize(&pblock, &uncompressed_bytes, &compressed_bytes);
+
+        write_string_binary(pblock.SerializeAsString(), buf);
+    }
+
+    void deserialize(BufferReadable& buf) {
+        std::string data;
+        read_binary(data, buf);
+
+        PBlock pblock;
+        pblock.ParseFromString(data);
+        new (&block) Block(pblock);
+    }
+
+    void add(const IColumn** columns, size_t columns_num, size_t row_num) {
+        DCHECK(block.columns() == columns_num)
+                << fmt::format("block.columns()!=columns_num, block.columns()={}, columns_num={}",
+                               block.columns(), columns_num);
+
+        for (size_t i = 0; i < columns_num; ++i) {
+            auto column = block.get_by_position(i).column->assume_mutable();
+            column->insert_from(*columns[i], row_num);
+        }
+    }
+
+    void sort() {
+        size_t sort_desc_idx = block.columns() - sort_column_size - 1;
+        StringRef desc_str =
+                block.get_by_position(sort_desc_idx).column->assume_mutable()->get_data_at(0);
+        DCHECK(sort_column_size == desc_str.size);
+
+        SortDescription sort_description(sort_column_size);
+        for (size_t i = 0; i < sort_column_size; i++) {
+            sort_description[i].column_number = sort_desc_idx + 1 + i;
+            sort_description[i].direction = (desc_str.data[i] == '0') ? 1 : -1;
+            sort_description[i].nulls_direction = sort_description[i].direction;
+        }
+
+        sort_block(block, sort_description, block.rows());
+    }
+
+    void try_init(const DataTypes& _arguments) {
+        if (!block.is_empty_column()) {
+            return;
+        }
+
+        for (auto type : _arguments) {
+            block.insert({type, ""});
+        }
+    }
+
+    Block block;
+};
+
+template <int sort_column_size, template <int> typename Data>
+class AggregateFunctionSort
+        : public IAggregateFunctionDataHelper<Data<sort_column_size>,
+                                              AggregateFunctionSort<sort_column_size, Data>> {
+    using DataReal = Data<sort_column_size>;
+
+private:
+    static constexpr auto prefix_size = sizeof(DataReal);
+    AggregateFunctionPtr _nested_func;
+    DataTypes _arguments;
+
+    AggregateDataPtr get_nested_place(AggregateDataPtr __restrict place) const noexcept {
+        return place + prefix_size;
+    }
+
+    ConstAggregateDataPtr get_nested_place(ConstAggregateDataPtr __restrict place) const noexcept {
+        return place + prefix_size;
+    }
+
+public:
+    AggregateFunctionSort(AggregateFunctionPtr nested_func, const DataTypes& arguments)
+            : IAggregateFunctionDataHelper<DataReal, AggregateFunctionSort>(
+                      arguments, nested_func->get_parameters()),
+              _nested_func(nested_func),
+              _arguments(arguments) {}
+
+    void add(AggregateDataPtr __restrict place, const IColumn** columns, size_t row_num,
+             Arena* arena) const override {
+        this->data(place).try_init(_arguments);
+        this->data(place).add(columns, _arguments.size(), row_num);
+    }
+
+    void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs,
+               Arena* arena) const override {
+        this->data(place).merge(this->data(rhs));
+    }
+
+    void serialize(ConstAggregateDataPtr __restrict place, BufferWritable& buf) const override {
+        this->data(place).serialize(buf);
+    }
+
+    void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf,
+                     Arena* arena) const override {
+        this->data(place).deserialize(buf);
+    }
+
+    void insert_result_into(ConstAggregateDataPtr targetplace, IColumn& to) const override {
+        auto place = const_cast<AggregateDataPtr>(targetplace);
+        if (!this->data(place).block.is_empty_column()) {
+            this->data(place).sort();
+
+            ColumnRawPtrs arguments_nested;
+            for (int i = 0; i < _arguments.size() - 1 - sort_column_size; i++) {
+                arguments_nested.emplace_back(
+                        this->data(place).block.get_by_position(i).column.get());
+            }
+            _nested_func->add_batch_single_place(arguments_nested[0]->size(),
+                                                 get_nested_place(place), arguments_nested.data(),
+                                                 nullptr);
+        }
+
+        _nested_func->insert_result_into(get_nested_place(place), to);
+    }
+
+    size_t size_of_data() const override { return prefix_size + _nested_func->size_of_data(); }
+
+    size_t align_of_data() const override { return _nested_func->align_of_data(); }
+
+    void create(AggregateDataPtr __restrict place) const override {
+        new (place) DataReal;
+        _nested_func->create(get_nested_place(place));
+    }
+
+    void destroy(AggregateDataPtr __restrict place) const noexcept override {
+        this->data(place).~DataReal();
+        _nested_func->destroy(get_nested_place(place));
+    }
+
+    String get_name() const override { return _nested_func->get_name() + "Sort"; }
+
+    DataTypePtr get_return_type() const override { return _nested_func->get_return_type(); }
+};
+
+} // namespace doris::vectorized
diff --git a/be/src/vec/columns/column_const.h b/be/src/vec/columns/column_const.h
index b29d91ddf3..c63da6e29b 100644
--- a/be/src/vec/columns/column_const.h
+++ b/be/src/vec/columns/column_const.h
@@ -52,6 +52,8 @@ public:
 
     const char* get_family_name() const override { return "Const"; }
 
+    void resize(size_t new_size) override { s = new_size; }
+
     MutableColumnPtr clone_resized(size_t new_size) const override {
         return ColumnConst::create(data, new_size);
     }
diff --git a/be/src/vec/core/sort_description.h b/be/src/vec/core/sort_description.h
index 6a29e95a59..ee57897965 100644
--- a/be/src/vec/core/sort_description.h
+++ b/be/src/vec/core/sort_description.h
@@ -28,32 +28,17 @@
 
 namespace doris::vectorized {
 
-struct FillColumnDescription {
-    /// All missed values in range [FROM, TO) will be filled
-    /// Range [FROM, TO) respects sorting direction
-    Field fill_from; /// Fill value >= FILL_FROM
-    Field fill_to;   /// Fill value + STEP < FILL_TO
-    Field fill_step; /// Default = 1 or -1 according to direction
-};
-
 /// Description of the sorting rule by one column.
 struct SortColumnDescription {
     std::string column_name; /// The name of the column.
     int column_number;       /// Column number (used if no name is given).
     int direction;           /// 1 - ascending, -1 - descending.
     int nulls_direction;     /// 1 - NULLs and NaNs are greater, -1 - less.
-            /// To achieve NULLS LAST, set it equal to direction, to achieve NULLS FIRST, set it opposite.
-    bool with_fill = false;
-    FillColumnDescription fill_description = {};
 
-    SortColumnDescription(int column_number_, int direction_, int nulls_direction_,
-                          bool with_fill_ = false,
-                          const FillColumnDescription& fill_description_ = {})
+    SortColumnDescription(int column_number_, int direction_, int nulls_direction_)
             : column_number(column_number_),
               direction(direction_),
-              nulls_direction(nulls_direction_),
-              with_fill(with_fill_),
-              fill_description(fill_description_) {}
+              nulls_direction(nulls_direction_) {}
 
     SortColumnDescription() = default;
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CastExpr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CastExpr.java
index 46fe04d7e5..e1895e1bc1 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CastExpr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CastExpr.java
@@ -98,16 +98,15 @@ public class CastExpr extends Expr {
         isImplicit = true;
 
         children.add(e);
-        if (isImplicit) {
-            try {
-                analyze();
-            } catch (AnalysisException ex) {
-                LOG.warn("Implicit casts fail", ex);
-                Preconditions.checkState(false,
-                        "Implicit casts should never throw analysis exception.");
-            }
-            analysisDone();
+
+        try {
+            analyze();
+        } catch (AnalysisException ex) {
+            LOG.warn("Implicit casts fail", ex);
+            Preconditions.checkState(false,
+                    "Implicit casts should never throw analysis exception.");
         }
+        analysisDone();
     }
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionSet.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionSet.java
index cf1bce79a7..007cc470f3 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionSet.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionSet.java
@@ -2488,20 +2488,10 @@ public class FunctionSet<T> {
                         false, false));
         // Group_concat(string) vectorized
         addBuiltin(AggregateFunction.createBuiltin("group_concat", Lists.<Type>newArrayList(Type.VARCHAR), Type.VARCHAR,
-                        Type.VARCHAR, initNullString,
-                        prefix + "20string_concat_updateEPN9doris_udf15FunctionContextERKNS1_9StringValEPS4_",
-                        prefix + "19string_concat_mergeEPN9doris_udf15FunctionContextERKNS1_9StringValEPS4_",
-                        stringValSerializeOrFinalize,
-                        prefix + "22string_concat_finalizeEPN9doris_udf15FunctionContextERKNS1_9StringValE", false,
-                        true, false, true));
+                Type.VARCHAR, initNullString, "", "", "", "", false, true, false, true));
         // Group_concat(string, string) vectorized
         addBuiltin(AggregateFunction.createBuiltin("group_concat", Lists.<Type>newArrayList(Type.VARCHAR, Type.VARCHAR),
-                        Type.VARCHAR, Type.VARCHAR, initNullString,
-                        prefix + "20string_concat_updateEPN9doris_udf15FunctionContextERKNS1_9StringValES6_PS4_",
-                        prefix + "19string_concat_mergeEPN9doris_udf15FunctionContextERKNS1_9StringValEPS4_",
-                        stringValSerializeOrFinalize,
-                        prefix + "22string_concat_finalizeEPN9doris_udf15FunctionContextERKNS1_9StringValE", false,
-                        true, false, true));
+                Type.VARCHAR, Type.VARCHAR, initNullString, "", "", "", "", false, true, false, true));
 
         // analytic functions
         // Rank
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Type.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Type.java
index 8be3d110d0..f5a3e9a1cd 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Type.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Type.java
@@ -109,6 +109,7 @@ public abstract class Type {
     private static final ArrayList<ScalarType> numericTypes;
     private static final ArrayList<ScalarType> supportedTypes;
     private static final ArrayList<Type> arraySubTypes;
+    private static final ArrayList<ScalarType> trivialTypes;
 
     static {
         integerTypes = Lists.newArrayList();
@@ -119,11 +120,7 @@ public abstract class Type {
         integerTypes.add(LARGEINT);
 
         numericTypes = Lists.newArrayList();
-        numericTypes.add(TINYINT);
-        numericTypes.add(SMALLINT);
-        numericTypes.add(INT);
-        numericTypes.add(BIGINT);
-        numericTypes.add(LARGEINT);
+        numericTypes.addAll(integerTypes);
         numericTypes.add(FLOAT);
         numericTypes.add(DOUBLE);
         numericTypes.add(DECIMALV2);
@@ -131,40 +128,29 @@ public abstract class Type {
         numericTypes.add(DECIMAL64);
         numericTypes.add(DECIMAL128);
 
+        trivialTypes = Lists.newArrayList();
+        trivialTypes.addAll(numericTypes);
+        trivialTypes.add(BOOLEAN);
+        trivialTypes.add(VARCHAR);
+        trivialTypes.add(STRING);
+        trivialTypes.add(CHAR);
+        trivialTypes.add(DATE);
+        trivialTypes.add(DATETIME);
+        trivialTypes.add(DATEV2);
+        trivialTypes.add(DATETIMEV2);
+        trivialTypes.add(TIME);
+        trivialTypes.add(TIMEV2);
+
         supportedTypes = Lists.newArrayList();
+        supportedTypes.addAll(trivialTypes);
         supportedTypes.add(NULL);
-        supportedTypes.add(BOOLEAN);
-        supportedTypes.add(TINYINT);
-        supportedTypes.add(SMALLINT);
-        supportedTypes.add(INT);
-        supportedTypes.add(BIGINT);
-        supportedTypes.add(LARGEINT);
-        supportedTypes.add(FLOAT);
-        supportedTypes.add(DOUBLE);
-        supportedTypes.add(VARCHAR);
         supportedTypes.add(HLL);
         supportedTypes.add(BITMAP);
         supportedTypes.add(QUANTILE_STATE);
-        supportedTypes.add(CHAR);
-        supportedTypes.add(DATE);
-        supportedTypes.add(DATETIME);
-        supportedTypes.add(DATEV2);
-        supportedTypes.add(DATETIMEV2);
-        supportedTypes.add(DECIMALV2);
-        supportedTypes.add(DECIMAL32);
-        supportedTypes.add(DECIMAL64);
-        supportedTypes.add(DECIMAL128);
-        supportedTypes.add(TIME);
-        supportedTypes.add(TIMEV2);
-        supportedTypes.add(STRING);
 
         arraySubTypes = Lists.newArrayList();
+        arraySubTypes.addAll(integerTypes);
         arraySubTypes.add(BOOLEAN);
-        arraySubTypes.add(TINYINT);
-        arraySubTypes.add(SMALLINT);
-        arraySubTypes.add(INT);
-        arraySubTypes.add(BIGINT);
-        arraySubTypes.add(LARGEINT);
         arraySubTypes.add(FLOAT);
         arraySubTypes.add(DOUBLE);
         arraySubTypes.add(DECIMALV2);
@@ -183,6 +169,10 @@ public abstract class Type {
         return numericTypes;
     }
 
+    public static ArrayList<ScalarType> getTrivialTypes() {
+        return trivialTypes;
+    }
+
     public static ArrayList<ScalarType> getSupportedTypes() {
         return supportedTypes;
     }


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