You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by mo...@apache.org on 2022/07/18 02:52:47 UTC

[doris] branch master updated: [feature-wip](array) add the array_sort function (#10598)

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

morningman 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 2d5aca18fb [feature-wip](array) add the array_sort function (#10598)
2d5aca18fb is described below

commit 2d5aca18fba1d57881d6a558fa17e9714cf7a2a5
Author: carlvinhust2012 <hu...@126.com>
AuthorDate: Mon Jul 18 10:52:42 2022 +0800

    [feature-wip](array) add the array_sort function (#10598)
    
    Co-authored-by: hucheng01 <hu...@baidu.com>
---
 be/src/vec/CMakeLists.txt                          |   1 +
 .../functions/array/function_array_register.cpp    |   6 +-
 be/src/vec/functions/array/function_array_sort.cpp |  28 ++
 be/src/vec/functions/array/function_array_sort.h   | 288 +++++++++++++++++++++
 .../sql-functions/array-functions/array_sort.md    |  78 ++++++
 .../sql-functions/array-functions/array_sort.md    |  78 ++++++
 gensrc/script/doris_builtins_functions.py          |  13 +
 .../array_functions/test_array_functions.out       |  15 ++
 .../array_functions/test_array_functions.groovy    |   2 +
 9 files changed, 507 insertions(+), 2 deletions(-)

diff --git a/be/src/vec/CMakeLists.txt b/be/src/vec/CMakeLists.txt
index e9646b2729..6ac009cc73 100644
--- a/be/src/vec/CMakeLists.txt
+++ b/be/src/vec/CMakeLists.txt
@@ -134,6 +134,7 @@ set(VEC_FILES
   functions/array/function_array_register.cpp
   functions/array/function_array_size.cpp
   functions/array/function_array_aggregation.cpp
+  functions/array/function_array_sort.cpp
   functions/array/function_array_utils.cpp
   functions/array/function_arrays_overlap.cpp
   functions/array/function_array_distinct.cpp
diff --git a/be/src/vec/functions/array/function_array_register.cpp b/be/src/vec/functions/array/function_array_register.cpp
index 573870144b..cc8092114f 100644
--- a/be/src/vec/functions/array/function_array_register.cpp
+++ b/be/src/vec/functions/array/function_array_register.cpp
@@ -26,18 +26,20 @@ void register_function_array_element(SimpleFunctionFactory&);
 void register_function_array_index(SimpleFunctionFactory&);
 void register_function_array_size(SimpleFunctionFactory&);
 void register_function_array_aggregation(SimpleFunctionFactory&);
-void register_function_arrays_overlap(SimpleFunctionFactory&);
 void register_function_array_distinct(SimpleFunctionFactory&);
 void register_function_array_remove(SimpleFunctionFactory&);
+void register_function_array_sort(SimpleFunctionFactory&);
+void register_function_arrays_overlap(SimpleFunctionFactory&);
 
 void register_function_array(SimpleFunctionFactory& factory) {
     register_function_array_element(factory);
     register_function_array_index(factory);
     register_function_array_size(factory);
     register_function_array_aggregation(factory);
-    register_function_arrays_overlap(factory);
     register_function_array_distinct(factory);
     register_function_array_remove(factory);
+    register_function_array_sort(factory);
+    register_function_arrays_overlap(factory);
 }
 
 } // namespace doris::vectorized
diff --git a/be/src/vec/functions/array/function_array_sort.cpp b/be/src/vec/functions/array/function_array_sort.cpp
new file mode 100644
index 0000000000..932cc2a1f0
--- /dev/null
+++ b/be/src/vec/functions/array/function_array_sort.cpp
@@ -0,0 +1,28 @@
+// 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/functions/array/function_array_sort.h"
+
+#include "vec/functions/simple_function_factory.h"
+
+namespace doris::vectorized {
+
+void register_function_array_sort(SimpleFunctionFactory& factory) {
+    factory.register_function<FunctionArraySort>();
+}
+
+} // namespace doris::vectorized
\ No newline at end of file
diff --git a/be/src/vec/functions/array/function_array_sort.h b/be/src/vec/functions/array/function_array_sort.h
new file mode 100644
index 0000000000..e8dee4a9bc
--- /dev/null
+++ b/be/src/vec/functions/array/function_array_sort.h
@@ -0,0 +1,288 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+// This file is copied from
+// https://github.com/ClickHouse/ClickHouse/blob/master/src/Functions/array/arraySort.cpp
+// and modified by Doris
+#pragma once
+
+#include "vec/columns/column_array.h"
+#include "vec/columns/column_const.h"
+#include "vec/data_types/data_type_array.h"
+#include "vec/data_types/data_type_number.h"
+#include "vec/functions/function.h"
+#include "vec/functions/function_helpers.h"
+
+namespace doris::vectorized {
+
+class FunctionArraySort : public IFunction {
+public:
+    static constexpr auto name = "array_sort";
+    static FunctionPtr create() { return std::make_shared<FunctionArraySort>(); }
+    using NullMapType = PaddedPODArray<UInt8>;
+
+    /// Get function name.
+    String get_name() const override { return name; }
+
+    bool is_variadic() const override { return false; }
+
+    size_t get_number_of_arguments() const override { return 1; }
+
+    DataTypePtr get_return_type_impl(const DataTypes& arguments) const override {
+        DCHECK(is_array(arguments[0]))
+                << "first argument for function: " << name << " should be DataTypeArray"
+                << " and arguments[0] is " << arguments[0]->get_name();
+        return arguments[0];
+    }
+
+    Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments,
+                        size_t result, size_t input_rows_count) override {
+        ColumnPtr src_column =
+                block.get_by_position(arguments[0]).column->convert_to_full_column_if_const();
+        const auto& src_column_array = check_and_get_column<ColumnArray>(*src_column);
+        if (!src_column_array) {
+            return Status::RuntimeError(
+                    fmt::format("unsupported types for function {}({})", get_name(),
+                                block.get_by_position(arguments[0]).type->get_name()));
+        }
+        const auto& src_offsets = src_column_array->get_offsets();
+        const auto* src_nested_column = &src_column_array->get_data();
+        DCHECK(src_nested_column != nullptr);
+
+        DataTypePtr src_column_type = block.get_by_position(arguments[0]).type;
+        auto nested_type = assert_cast<const DataTypeArray&>(*src_column_type).get_nested_type();
+        auto dest_column_ptr = ColumnArray::create(nested_type->create_column(),
+                                                   ColumnArray::ColumnOffsets::create());
+        IColumn* dest_nested_column = &dest_column_ptr->get_data();
+        ColumnArray::Offsets& dest_offsets = dest_column_ptr->get_offsets();
+        DCHECK(dest_nested_column != nullptr);
+        dest_nested_column->reserve(src_nested_column->size());
+        dest_offsets.reserve(input_rows_count);
+
+        const NullMapType* src_null_map = nullptr;
+        if (src_nested_column->is_nullable()) {
+            const ColumnNullable* src_nested_nullable_col =
+                    check_and_get_column<ColumnNullable>(*src_nested_column);
+            src_nested_column = src_nested_nullable_col->get_nested_column_ptr();
+            src_null_map = &src_nested_nullable_col->get_null_map_column().get_data();
+        }
+
+        NullMapType* dest_null_map = nullptr;
+        if (dest_nested_column->is_nullable()) {
+            ColumnNullable* dest_nested_nullable_col =
+                    reinterpret_cast<ColumnNullable*>(dest_nested_column);
+            dest_nested_column = dest_nested_nullable_col->get_nested_column_ptr();
+            dest_null_map = &dest_nested_nullable_col->get_null_map_column().get_data();
+        }
+
+        auto res_val = _execute_by_type(*src_nested_column, src_offsets, *dest_nested_column,
+                                        dest_offsets, src_null_map, dest_null_map, nested_type);
+        if (!res_val) {
+            return Status::RuntimeError(
+                    fmt::format("execute failed or unsupported types for function {}({})",
+                                get_name(), block.get_by_position(arguments[0]).type->get_name()));
+        }
+
+        block.replace_by_position(result, std::move(dest_column_ptr));
+        return Status::OK();
+    }
+
+private:
+    // sort the non-null element according to the permutation
+    template <typename SrcDataType>
+    void _sort_by_permutation(ColumnArray::Offset& prev_offset,
+                              const ColumnArray::Offset& curr_offset,
+                              const SrcDataType* src_data_concrete, const IColumn& src_column,
+                              const NullMapType* src_null_map, IColumn::Permutation& permutation) {
+        for (ColumnArray::Offset j = prev_offset; j < curr_offset - 1; ++j) {
+            if (src_null_map && (*src_null_map)[j]) {
+                continue;
+            }
+            for (ColumnArray::Offset k = j + 1; k < curr_offset; ++k) {
+                if (src_null_map && (*src_null_map)[k]) {
+                    continue;
+                }
+                int result = src_data_concrete->compare_at(permutation[j], permutation[k],
+                                                           src_column, 1);
+                if (result > 0) {
+                    auto temp = permutation[j];
+                    permutation[j] = permutation[k];
+                    permutation[k] = temp;
+                }
+            }
+        }
+        return;
+    }
+
+    template <typename ColumnType>
+    bool _execute_number(const IColumn& src_column, const ColumnArray::Offsets& src_offsets,
+                         IColumn& dest_column, ColumnArray::Offsets& dest_offsets,
+                         const NullMapType* src_null_map, NullMapType* dest_null_map) {
+        using NestType = typename ColumnType::value_type;
+        const ColumnType* src_data_concrete = reinterpret_cast<const ColumnType*>(&src_column);
+        if (!src_data_concrete) {
+            return false;
+        }
+        const PaddedPODArray<NestType>& src_datas = src_data_concrete->get_data();
+
+        ColumnType& dest_data_concrete = reinterpret_cast<ColumnType&>(dest_column);
+        PaddedPODArray<NestType>& dest_datas = dest_data_concrete.get_data();
+
+        ColumnArray::Offset prev_src_offset = 0;
+        IColumn::Permutation permutation(src_column.size());
+        for (size_t i = 0; i < src_column.size(); ++i) {
+            permutation[i] = i;
+        }
+
+        for (auto curr_src_offset : src_offsets) {
+            // filter and insert null element first
+            for (ColumnArray::Offset j = prev_src_offset; j < curr_src_offset; ++j) {
+                if (src_null_map && (*src_null_map)[j]) {
+                    DCHECK(dest_null_map != nullptr);
+                    (*dest_null_map).push_back(true);
+                    dest_datas.push_back(NestType());
+                }
+            }
+
+            _sort_by_permutation<ColumnType>(prev_src_offset, curr_src_offset, src_data_concrete,
+                                             src_column, src_null_map, permutation);
+
+            // insert non-null element after sort by permutation
+            for (ColumnArray::Offset j = prev_src_offset; j < curr_src_offset; ++j) {
+                if (src_null_map && (*src_null_map)[j]) {
+                    continue;
+                }
+
+                dest_datas.push_back(src_datas[permutation[j]]);
+                if (dest_null_map) {
+                    (*dest_null_map).push_back(false);
+                }
+            }
+            dest_offsets.push_back(curr_src_offset);
+            prev_src_offset = curr_src_offset;
+        }
+
+        return true;
+    }
+
+    bool _execute_string(const IColumn& src_column, const ColumnArray::Offsets& src_offsets,
+                         IColumn& dest_column, ColumnArray::Offsets& dest_offsets,
+                         const NullMapType* src_null_map, NullMapType* dest_null_map) {
+        const ColumnString* src_data_concrete = reinterpret_cast<const ColumnString*>(&src_column);
+        if (!src_data_concrete) {
+            return false;
+        }
+
+        ColumnString& dest_column_string = reinterpret_cast<ColumnString&>(dest_column);
+        ColumnString::Chars& column_string_chars = dest_column_string.get_chars();
+        ColumnString::Offsets& column_string_offsets = dest_column_string.get_offsets();
+        column_string_chars.reserve(src_column.size());
+
+        ColumnArray::Offset prev_src_offset = 0;
+        IColumn::Permutation permutation(src_column.size());
+        for (size_t i = 0; i < src_column.size(); ++i) {
+            permutation[i] = i;
+        }
+
+        for (auto curr_src_offset : src_offsets) {
+            // filter and insert null element first
+            for (ColumnArray::Offset j = prev_src_offset; j < curr_src_offset; ++j) {
+                if (src_null_map && (*src_null_map)[j]) {
+                    DCHECK(dest_null_map != nullptr);
+                    column_string_offsets.push_back(column_string_offsets.back());
+                    (*dest_null_map).push_back(true);
+                }
+            }
+
+            _sort_by_permutation<ColumnString>(prev_src_offset, curr_src_offset, src_data_concrete,
+                                               src_column, src_null_map, permutation);
+
+            // insert non-null element after sort by permutation
+            for (ColumnArray::Offset j = prev_src_offset; j < curr_src_offset; ++j) {
+                if (src_null_map && (*src_null_map)[j]) {
+                    continue;
+                }
+
+                StringRef src_str_ref = src_data_concrete->get_data_at(permutation[j]);
+                // copy the src data to column_string_chars
+                const size_t old_size = column_string_chars.size();
+                const size_t new_size = old_size + src_str_ref.size + 1;
+                column_string_chars.resize(new_size);
+                if (src_str_ref.size > 0) {
+                    memcpy(column_string_chars.data() + old_size, src_str_ref.data,
+                           src_str_ref.size);
+                }
+                column_string_chars[old_size + src_str_ref.size] = 0;
+                column_string_offsets.push_back(new_size);
+
+                if (dest_null_map) {
+                    (*dest_null_map).push_back(false);
+                }
+            }
+            dest_offsets.push_back(curr_src_offset);
+            prev_src_offset = curr_src_offset;
+        }
+        return true;
+    }
+
+    bool _execute_by_type(const IColumn& src_column, const ColumnArray::Offsets& src_offsets,
+                          IColumn& dest_column, ColumnArray::Offsets& dest_offsets,
+                          const NullMapType* src_null_map, NullMapType* dest_null_map,
+                          DataTypePtr& nested_type) {
+        bool res = false;
+        WhichDataType which(remove_nullable(nested_type));
+        if (which.is_uint8()) {
+            res = _execute_number<ColumnUInt8>(src_column, src_offsets, dest_column, dest_offsets,
+                                               src_null_map, dest_null_map);
+        } else if (which.is_int8()) {
+            res = _execute_number<ColumnInt8>(src_column, src_offsets, dest_column, dest_offsets,
+                                              src_null_map, dest_null_map);
+        } else if (which.is_int16()) {
+            res = _execute_number<ColumnInt16>(src_column, src_offsets, dest_column, dest_offsets,
+                                               src_null_map, dest_null_map);
+        } else if (which.is_int32()) {
+            res = _execute_number<ColumnInt32>(src_column, src_offsets, dest_column, dest_offsets,
+                                               src_null_map, dest_null_map);
+        } else if (which.is_int64()) {
+            res = _execute_number<ColumnInt64>(src_column, src_offsets, dest_column, dest_offsets,
+                                               src_null_map, dest_null_map);
+        } else if (which.is_int128()) {
+            res = _execute_number<ColumnInt128>(src_column, src_offsets, dest_column, dest_offsets,
+                                                src_null_map, dest_null_map);
+        } else if (which.is_float32()) {
+            res = _execute_number<ColumnFloat32>(src_column, src_offsets, dest_column, dest_offsets,
+                                                 src_null_map, dest_null_map);
+        } else if (which.is_float64()) {
+            res = _execute_number<ColumnFloat64>(src_column, src_offsets, dest_column, dest_offsets,
+                                                 src_null_map, dest_null_map);
+        } else if (which.is_date()) {
+            res = _execute_number<ColumnDate>(src_column, src_offsets, dest_column, dest_offsets,
+                                              src_null_map, dest_null_map);
+        } else if (which.is_date_time()) {
+            res = _execute_number<ColumnDateTime>(src_column, src_offsets, dest_column,
+                                                  dest_offsets, src_null_map, dest_null_map);
+        } else if (which.is_decimal128()) {
+            res = _execute_number<ColumnDecimal128>(src_column, src_offsets, dest_column,
+                                                    dest_offsets, src_null_map, dest_null_map);
+        } else if (which.is_string()) {
+            res = _execute_string(src_column, src_offsets, dest_column, dest_offsets, src_null_map,
+                                  dest_null_map);
+        }
+        return res;
+    }
+};
+
+} // namespace doris::vectorized
\ No newline at end of file
diff --git a/docs/en/docs/sql-manual/sql-functions/array-functions/array_sort.md b/docs/en/docs/sql-manual/sql-functions/array-functions/array_sort.md
new file mode 100644
index 0000000000..1ef5a08000
--- /dev/null
+++ b/docs/en/docs/sql-manual/sql-functions/array-functions/array_sort.md
@@ -0,0 +1,78 @@
+---
+{
+    "title": "array_sort",
+    "language": "en"
+}
+---
+
+<!-- 
+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.
+-->
+
+## array_sort
+
+### description
+
+#### Syntax
+
+```
+ARRAY<T> array_sort(ARRAY<T> arr)
+```
+
+Return the array which has been sorted in ascending order. Return NULL for NULL input.
+If the element of array is NULL, it will be placed in the front of the sorted array.
+
+### notice
+
+`Only supported in vectorized engine`
+
+### example
+
+```
+mysql> set enable_vectorized_engine=true;
+mysql> select k1, k2, array_sort(k2) array_test;
++------+-----------------------------+-----------------------------+
+| k1   | k2                          | array_sort(`k2`)            |
++------+-----------------------------+-----------------------------+
+|  1   | [1, 2, 3, 4, 5]             | [1, 2, 3, 4, 5]             |
+|  2   | [6, 7, 8]                   | [6, 7, 8]                   |
+|  3   | []                          | []                          |
+|  4   | NULL                        | NULL                        |
+|  5   | [1, 2, 3, 4, 5, 4, 3, 2, 1] | [1, 1, 2, 2, 3, 3, 4, 4, 5] |
+|  6   | [1, 2, 3, NULL]             | [NULL, 1, 2, 3]             |
+|  7   | [1, 2, 3, NULL, NULL]       | [NULL, NULL, 1, 2, 3]       |
+|  8   | [1, 1, 2, NULL, NULL]       | [NULL, NULL, 1, 1, 2]       |
+|  9   | [1, NULL, 1, 2, NULL, NULL] | [NULL, NULL, NULL, 1, 1, 2] |
++------+-----------------------------+-----------------------------+
+
+mysql> select k1, k2, array_sort(k2) from array_test01;
++------+------------------------------------------+------------------------------------------+
+| k1   | k2                                       | array_sort(`k2`)                         |
++------+------------------------------------------+------------------------------------------+
+|  1   | ['a', 'b', 'c', 'd', 'e']                | ['a', 'b', 'c', 'd', 'e']                |
+|  2   | ['f', 'g', 'h']                          | ['f', 'g', 'h']                          |
+|  3   | ['']                                     | ['']                                     |
+|  3   | [NULL]                                   | [NULL]                                   |
+|  5   | ['a', 'b', 'c', 'd', 'e', 'a', 'b', 'c'] | ['a', 'a', 'b', 'b', 'c', 'c', 'd', 'e'] |
+|  6   | NULL                                     | NULL                                     |
+|  7   | ['a', 'b', NULL]                         | [NULL, 'a', 'b']                         |
+|  8   | ['a', 'b', NULL, NULL]                   | [NULL, NULL, 'a', 'b']                   |
++------+------------------------------------------+------------------------------------------+
+```
+
+### keywords
+
+ARRAY, SORT, ARRAY_SORT
\ No newline at end of file
diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/array-functions/array_sort.md b/docs/zh-CN/docs/sql-manual/sql-functions/array-functions/array_sort.md
new file mode 100644
index 0000000000..dbfbc78652
--- /dev/null
+++ b/docs/zh-CN/docs/sql-manual/sql-functions/array-functions/array_sort.md
@@ -0,0 +1,78 @@
+---
+{
+    "title": "array_sort",
+    "language": "zh-CN"
+}
+---
+
+<!-- 
+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.
+-->
+
+## array_sort
+
+### description
+
+#### Syntax
+
+```
+ARRAY<T> array_sort(ARRAY<T> arr)
+```
+
+返回按升序排列后的数组,如果输入数组为NULL,则返回NULL。
+如果数组元素包含NULL, 则输出的排序数组会将NULL放在最前面。
+
+### notice
+
+`仅支持向量化引擎中使用`
+
+### example
+
+```
+mysql> set enable_vectorized_engine=true;
+mysql> select k1, k2, array_sort(k2) array_test;
++------+-----------------------------+-----------------------------+
+| k1   | k2                          | array_sort(`k2`)            |
++------+-----------------------------+-----------------------------+
+|  1   | [1, 2, 3, 4, 5]             | [1, 2, 3, 4, 5]             |
+|  2   | [6, 7, 8]                   | [6, 7, 8]                   |
+|  3   | []                          | []                          |
+|  4   | NULL                        | NULL                        |
+|  5   | [1, 2, 3, 4, 5, 4, 3, 2, 1] | [1, 1, 2, 2, 3, 3, 4, 4, 5] |
+|  6   | [1, 2, 3, NULL]             | [NULL, 1, 2, 3]             |
+|  7   | [1, 2, 3, NULL, NULL]       | [NULL, NULL, 1, 2, 3]       |
+|  8   | [1, 1, 2, NULL, NULL]       | [NULL, NULL, 1, 1, 2]       |
+|  9   | [1, NULL, 1, 2, NULL, NULL] | [NULL, NULL, NULL, 1, 1, 2] |
++------+-----------------------------+-----------------------------+
+
+mysql> select k1, k2, array_sort(k2) from array_test01;
++------+------------------------------------------+------------------------------------------+
+| k1   | k2                                       | array_sort(`k2`)                         |
++------+------------------------------------------+------------------------------------------+
+|  1   | ['a', 'b', 'c', 'd', 'e']                | ['a', 'b', 'c', 'd', 'e']                |
+|  2   | ['f', 'g', 'h']                          | ['f', 'g', 'h']                          |
+|  3   | ['']                                     | ['']                                     |
+|  3   | [NULL]                                   | [NULL]                                   |
+|  5   | ['a', 'b', 'c', 'd', 'e', 'a', 'b', 'c'] | ['a', 'a', 'b', 'b', 'c', 'c', 'd', 'e'] |
+|  6   | NULL                                     | NULL                                     |
+|  7   | ['a', 'b', NULL]                         | [NULL, 'a', 'b']                         |
+|  8   | ['a', 'b', NULL, NULL]                   | [NULL, NULL, 'a', 'b']                   |
++------+------------------------------------------+------------------------------------------+
+```
+
+### keywords
+
+ARRAY, SORT, ARRAY_SORT
\ No newline at end of file
diff --git a/gensrc/script/doris_builtins_functions.py b/gensrc/script/doris_builtins_functions.py
index 249638831b..8a4aad385c 100755
--- a/gensrc/script/doris_builtins_functions.py
+++ b/gensrc/script/doris_builtins_functions.py
@@ -185,6 +185,19 @@ visible_functions = [
     [['array_distinct'], 'ARRAY_DECIMALV2', ['ARRAY_DECIMALV2'], '', '', '', 'vec', ''],
     [['array_distinct'], 'ARRAY_VARCHAR',   ['ARRAY_VARCHAR'], '', '', '', 'vec', ''],
     [['array_distinct'], 'ARRAY_STRING',    ['ARRAY_STRING'], '', '', '', 'vec', ''],
+    
+    [['array_sort'], 'ARRAY_TINYINT',   ['ARRAY_TINYINT'], '', '', '', 'vec', ''],
+    [['array_sort'], 'ARRAY_SMALLINT',  ['ARRAY_SMALLINT'], '', '', '', 'vec', ''],
+    [['array_sort'], 'ARRAY_INT',       ['ARRAY_INT'], '', '', '', 'vec', ''],
+    [['array_sort'], 'ARRAY_BIGINT',    ['ARRAY_BIGINT'], '', '', '', 'vec', ''],
+    [['array_sort'], 'ARRAY_LARGEINT',  ['ARRAY_LARGEINT'], '', '', '', 'vec', ''],
+    [['array_sort'], 'ARRAY_DATETIME',  ['ARRAY_DATETIME'], '', '', '', 'vec', ''],
+    [['array_sort'], 'ARRAY_DATE',      ['ARRAY_DATE'], '', '', '', 'vec', ''],
+    [['array_sort'], 'ARRAY_FLOAT',     ['ARRAY_FLOAT'], '', '', '', 'vec', ''],
+    [['array_sort'], 'ARRAY_DOUBLE',    ['ARRAY_DOUBLE'], '', '', '', 'vec', ''],
+    [['array_sort'], 'ARRAY_DECIMALV2', ['ARRAY_DECIMALV2'], '', '', '', 'vec', ''],
+    [['array_sort'], 'ARRAY_VARCHAR',   ['ARRAY_VARCHAR'], '', '', '', 'vec', ''],
+    [['array_sort'], 'ARRAY_STRING',    ['ARRAY_STRING'], '', '', '', 'vec', ''],
 
     [['array_min'],     'TINYINT',  ['ARRAY_TINYINT'],  '', '', '', 'vec', 'ALWAYS_NULLABLE'],
     [['array_min'],     'SMALLINT', ['ARRAY_SMALLINT'], '', '', '', 'vec', 'ALWAYS_NULLABLE'],
diff --git a/regression-test/data/query/sql_functions/array_functions/test_array_functions.out b/regression-test/data/query/sql_functions/array_functions/test_array_functions.out
index e0830f69b1..02dd5d80e4 100644
--- a/regression-test/data/query/sql_functions/array_functions/test_array_functions.out
+++ b/regression-test/data/query/sql_functions/array_functions/test_array_functions.out
@@ -6,6 +6,7 @@
 4	9	0
 5	0	7
 6	9	7
+7	5	5
 
 -- !select --
 1	3	3
@@ -14,6 +15,7 @@
 4	9	0
 5	0	7
 6	9	7
+7	5	5
 
 -- !select --
 1	true
@@ -22,6 +24,7 @@
 4	false
 5	\N
 6	\N
+7	\N
 
 -- !select --
 1	[1, 2, 3]	['a', 'b', '']
@@ -30,6 +33,7 @@
 4	[1, 2, 3, 4, 5]	[]
 5	[]	['a', 'b', 'c', 'd']
 6	[1, 2, 3, 4, 5]	['a', 'b', 'c', 'd']
+7	[8, 9, NULL, 10, NULL]	['f', NULL, 'g', NULL, 'h']
 
 -- !select --
 [2, 3]	1
@@ -38,3 +42,14 @@
 [1, 2, 3, 5, 3, 2, 1]	4
 []	5
 [1, 2, 3, 4, 5, 4, 3, 2, 1]	6
+[8, 9, NULL, 10, NULL]	7
+
+-- !select --
+1	[1, 2, 3]	['', 'a', 'b']	[1, 2]
+2	[4]	\N	[5]
+3	[]	[]	\N
+4	[1, 1, 2, 2, 3, 3, 4, 4, 5]	[]	[]
+5	[]	['a', 'a', 'b', 'b', 'c', 'c', 'd']	\N
+6	[1, 1, 2, 2, 3, 3, 4, 4, 5]	['a', 'a', 'b', 'b', 'c', 'c', 'd']	\N
+7	[NULL, NULL, 8, 9, 10]	[NULL, NULL, 'f', 'g', 'h']	\N
+
diff --git a/regression-test/suites/query/sql_functions/array_functions/test_array_functions.groovy b/regression-test/suites/query/sql_functions/array_functions/test_array_functions.groovy
index 75e16a5c2f..20ae65a6ed 100644
--- a/regression-test/suites/query/sql_functions/array_functions/test_array_functions.groovy
+++ b/regression-test/suites/query/sql_functions/array_functions/test_array_functions.groovy
@@ -43,10 +43,12 @@ suite("test_array_functions", "query") {
     sql """ INSERT INTO ${tableName} VALUES(4, [1, 2, 3, 4, 5, 4, 3, 2, 1], [], []) """
     sql """ INSERT INTO ${tableName} VALUES(5, [], ["a", "b", "c", "d", "c", "b", "a"], NULL) """
     sql """ INSERT INTO ${tableName} VALUES(6, [1, 2, 3, 4, 5, 4, 3, 2, 1], ["a", "b", "c", "d", "c", "b", "a"], NULL) """
+    sql """ INSERT INTO ${tableName} VALUES(7, [8, 9, NULL, 10, NULL], ["f", NULL, "g", NULL, "h"], NULL) """
 
     qt_select "SELECT k1, size(k2), size(k3) FROM ${tableName} ORDER BY k1"
     qt_select "SELECT k1, cardinality(k2), cardinality(k3) FROM ${tableName} ORDER BY k1"
     qt_select "SELECT k1, arrays_overlap(k2, k4) FROM ${tableName} ORDER BY k1"
     qt_select "SELECT k1, array_distinct(k2), array_distinct(k3) FROM ${tableName} ORDER BY k1"
     qt_select "SELECT array_remove(k2, k1), k1 FROM ${tableName} ORDER BY k1"
+    qt_select "SELECT k1, array_sort(k2), array_sort(k3), array_sort(k4) FROM ${tableName} ORDER BY k1"
 }


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