You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by xu...@apache.org on 2022/07/27 10:43:59 UTC

[doris] branch master updated: [feature-wip](array-type) add function array_slice (#11054)

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

xuyang 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 5913c7c52c [feature-wip](array-type) add function array_slice (#11054)
5913c7c52c is described below

commit 5913c7c52c7006e2e0075bd06dbde68fe46e7fd2
Author: xy720 <22...@users.noreply.github.com>
AuthorDate: Wed Jul 27 18:43:52 2022 +0800

    [feature-wip](array-type) add function array_slice (#11054)
    
    array_slice function returns a slice of the array.
---
 be/src/vec/CMakeLists.txt                          |   1 +
 .../functions/array/function_array_register.cpp    |   2 +
 .../vec/functions/array/function_array_slice.cpp   |  29 ++++
 be/src/vec/functions/array/function_array_slice.h  | 134 ++++++++++++++++++
 .../sql-functions/array-functions/array_slice.md   | 157 +++++++++++++++++++++
 .../sql-functions/array-functions/array_slice.md   | 157 +++++++++++++++++++++
 fe/fe-core/src/main/cup/sql_parser.cup             |  19 +++
 fe/fe-core/src/main/jflex/sql_scanner.flex         |   2 +
 gensrc/script/doris_builtins_functions.py          |  28 ++++
 .../array_functions/test_array_functions.out       |  18 +++
 .../array_functions/test_array_functions.groovy    |   2 +
 11 files changed, 549 insertions(+)

diff --git a/be/src/vec/CMakeLists.txt b/be/src/vec/CMakeLists.txt
index 01804896be..9ee5a5066f 100644
--- a/be/src/vec/CMakeLists.txt
+++ b/be/src/vec/CMakeLists.txt
@@ -143,6 +143,7 @@ set(VEC_FILES
   functions/array/function_array_union.cpp
   functions/array/function_array_except.cpp
   functions/array/function_array_intersect.cpp
+  functions/array/function_array_slice.cpp
   exprs/table_function/vexplode_json_array.cpp
   functions/math.cpp
   functions/function_bitmap.cpp
diff --git a/be/src/vec/functions/array/function_array_register.cpp b/be/src/vec/functions/array/function_array_register.cpp
index f87f28c4c1..39d4902e83 100644
--- a/be/src/vec/functions/array/function_array_register.cpp
+++ b/be/src/vec/functions/array/function_array_register.cpp
@@ -33,6 +33,7 @@ void register_function_arrays_overlap(SimpleFunctionFactory&);
 void register_function_array_union(SimpleFunctionFactory&);
 void register_function_array_except(SimpleFunctionFactory&);
 void register_function_array_intersect(SimpleFunctionFactory&);
+void register_function_array_slice(SimpleFunctionFactory&);
 
 void register_function_array(SimpleFunctionFactory& factory) {
     register_function_array_element(factory);
@@ -46,6 +47,7 @@ void register_function_array(SimpleFunctionFactory& factory) {
     register_function_array_union(factory);
     register_function_array_except(factory);
     register_function_array_intersect(factory);
+    register_function_array_slice(factory);
 }
 
 } // namespace doris::vectorized
diff --git a/be/src/vec/functions/array/function_array_slice.cpp b/be/src/vec/functions/array/function_array_slice.cpp
new file mode 100644
index 0000000000..5828351773
--- /dev/null
+++ b/be/src/vec/functions/array/function_array_slice.cpp
@@ -0,0 +1,29 @@
+// 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_slice.h"
+
+#include "vec/functions/simple_function_factory.h"
+
+namespace doris::vectorized {
+
+void register_function_array_slice(SimpleFunctionFactory& factory) {
+    factory.register_function<FunctionArraySlice>();
+    factory.register_alias(FunctionArraySlice::name, "%element_slice%");
+}
+
+} // namespace doris::vectorized
diff --git a/be/src/vec/functions/array/function_array_slice.h b/be/src/vec/functions/array/function_array_slice.h
new file mode 100644
index 0000000000..0a60135637
--- /dev/null
+++ b/be/src/vec/functions/array/function_array_slice.h
@@ -0,0 +1,134 @@
+// 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_view>
+
+#include "vec/columns/column_array.h"
+#include "vec/columns/column_string.h"
+#include "vec/common/string_ref.h"
+#include "vec/data_types/data_type_array.h"
+#include "vec/data_types/data_type_number.h"
+#include "vec/functions/array/function_array_utils.h"
+#include "vec/functions/function.h"
+
+namespace doris::vectorized {
+
+class FunctionArraySlice : public IFunction {
+public:
+    static constexpr auto name = "array_slice";
+    static FunctionPtr create() { return std::make_shared<FunctionArraySlice>(); }
+
+    /// Get function name.
+    String get_name() const override { return name; }
+
+    bool is_variadic() const override { return true; }
+
+    size_t get_number_of_arguments() const override { return 0; }
+
+    DataTypePtr get_return_type_impl(const DataTypes& arguments) const override {
+        DCHECK(is_array(arguments[0]))
+                << "First argument for function: " << name
+                << " should be DataTypeArray but it has type " << arguments[0]->get_name() << ".";
+        DCHECK(is_integer(arguments[1]))
+                << "Second argument for function: " << name << " should be Integer but it has type "
+                << arguments[1]->get_name() << ".";
+        if (arguments.size() > 2) {
+            DCHECK(is_integer(arguments[2]))
+                    << "Third argument for function: " << name
+                    << " should be Integer but it has type " << arguments[2]->get_name() << ".";
+        }
+        return arguments[0];
+    }
+
+    Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments,
+                        size_t result, size_t input_rows_count) override {
+        auto array_column =
+                block.get_by_position(arguments[0]).column->convert_to_full_column_if_const();
+        auto offset_column =
+                block.get_by_position(arguments[1]).column->convert_to_full_column_if_const();
+        ColumnPtr length_column = nullptr;
+        if (arguments.size() > 2) {
+            length_column =
+                    block.get_by_position(arguments[2]).column->convert_to_full_column_if_const();
+        }
+        // extract src array column
+        ColumnArrayExecutionData src;
+        if (!extract_column_array_info(*array_column, src)) {
+            return Status::RuntimeError(
+                    fmt::format("execute failed, unsupported types for function {}({}, {})",
+                                get_name(), block.get_by_position(arguments[0]).type->get_name(),
+                                block.get_by_position(arguments[1]).type->get_name()));
+        }
+        // prepare dst array column
+        bool is_nullable = src.nested_nullmap_data ? true : false;
+        ColumnArrayMutableData dst = create_mutable_data(src.nested_col, is_nullable);
+        dst.offsets_ptr->reserve(input_rows_count);
+        // execute
+        _execute_internal(dst, src, *offset_column, length_column.get());
+        ColumnPtr res_column = assemble_column_array(dst);
+        block.replace_by_position(result, std::move(res_column));
+        return Status::OK();
+    }
+
+private:
+    void _execute_internal(ColumnArrayMutableData& dst, ColumnArrayExecutionData& src,
+                           const IColumn& offset_column, const IColumn* length_column) {
+        size_t cur = 0;
+        for (size_t row = 0; row < src.offsets_ptr->size(); ++row) {
+            size_t off = (*src.offsets_ptr)[row - 1];
+            size_t len = (*src.offsets_ptr)[row] - off;
+            Int64 start = offset_column.get_int(row);
+            if (len == 0 || start == 0) {
+                dst.offsets_ptr->push_back(cur);
+                continue;
+            }
+            if (start > 0 && start <= len) {
+                start += off - 1;
+            } else if (start < 0 && -start <= len) {
+                start += off + len;
+            } else {
+                dst.offsets_ptr->push_back(cur);
+                continue;
+            }
+            Int64 end;
+            if (length_column) {
+                Int64 size = length_column->get_int(row);
+                end = std::max((Int64)off, std::min((Int64)(off + len), start + size));
+            } else {
+                end = off + len;
+            }
+            for (size_t pos = start; pos < end; ++pos) {
+                if (src.nested_nullmap_data && src.nested_nullmap_data[pos]) {
+                    dst.nested_col->insert_default();
+                    dst.nested_nullmap_data->push_back(1);
+                } else {
+                    dst.nested_col->insert_from(*src.nested_col, pos);
+                    if (dst.nested_nullmap_data) {
+                        dst.nested_nullmap_data->push_back(0);
+                    }
+                }
+            }
+            if (start < end) {
+                cur += end - start;
+            }
+            dst.offsets_ptr->push_back(cur);
+        }
+    }
+};
+
+} // namespace doris::vectorized
diff --git a/docs/en/docs/sql-manual/sql-functions/array-functions/array_slice.md b/docs/en/docs/sql-manual/sql-functions/array-functions/array_slice.md
new file mode 100644
index 0000000000..96d99633e4
--- /dev/null
+++ b/docs/en/docs/sql-manual/sql-functions/array-functions/array_slice.md
@@ -0,0 +1,157 @@
+---
+{
+    "title": "array_slice",
+    "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_slice
+
+### description
+
+#### Syntax
+
+```
+ARRAY<T> array_slice(ARRAY<T> arr, BIGINT off, BIGINT len)
+```
+
+Returns a slice of the array.
+
+```
+A positive off indicates an indent on the left
+A negative off indicates an indent on the right.
+An empty array is returned when the off is not within the actual range of the array.
+A negative len will be treated as 0.
+```
+
+### notice
+
+`Only supported in vectorized engine`
+
+### example
+
+
+```
+mysql> set enable_vectorized_engine=true;
+
+mysql> select k2, k2[2:2] from array_type_table_nullable;
++-----------------+-------------------------+
+| k2              | array_slice(`k2`, 2, 2) |
++-----------------+-------------------------+
+| [1, 2, 3]       | [2, 3]                  |
+| [1, NULL, 3]    | [NULL, 3]               |
+| [2, 3]          | [3]                     |
+| NULL            | NULL                    |
++-----------------+-------------------------+
+
+mysql> select k2, array_slice(k2, 2, 2) from array_type_table_nullable;
++-----------------+-------------------------+
+| k2              | array_slice(`k2`, 2, 2) |
++-----------------+-------------------------+
+| [1, 2, 3]       | [2, 3]                  |
+| [1, NULL, 3]    | [NULL, 3]               |
+| [2, 3]          | [3]                     |
+| NULL            | NULL                    |
++-----------------+-------------------------+
+
+mysql> select k2, k2[2:2] from array_type_table_nullable_varchar;
++----------------------------+-------------------------+
+| k2                         | array_slice(`k2`, 2, 2) |
++----------------------------+-------------------------+
+| ['hello', 'world', 'c++']  | ['world', 'c++']        |
+| ['a1', 'equals', 'b1']     | ['equals', 'b1']        |
+| ['hasnull', NULL, 'value'] | [NULL, 'value']         |
+| ['hasnull', NULL, 'value'] | [NULL, 'value']         |
++----------------------------+-------------------------+
+
+mysql> select k2, array_slice(k2, 2, 2) from array_type_table_nullable_varchar;
++----------------------------+-------------------------+
+| k2                         | array_slice(`k2`, 2, 2) |
++----------------------------+-------------------------+
+| ['hello', 'world', 'c++']  | ['world', 'c++']        |
+| ['a1', 'equals', 'b1']     | ['equals', 'b1']        |
+| ['hasnull', NULL, 'value'] | [NULL, 'value']         |
+| ['hasnull', NULL, 'value'] | [NULL, 'value']         |
++----------------------------+-------------------------+
+```
+
+Negative off:
+
+```
+mysql> select k2, k2[-2:1] from array_type_table_nullable;
++-----------+--------------------------+
+| k2        | array_slice(`k2`, -2, 1) |
++-----------+--------------------------+
+| [1, 2, 3] | [2]                      |
+| [1, 2, 3] | [2]                      |
+| [2, 3]    | [2]                      |
+| [2, 3]    | [2]                      |
++-----------+--------------------------+
+
+mysql> select k2, array_slice(k2, -2, 1) from array_type_table_nullable;
++-----------+--------------------------+
+| k2        | array_slice(`k2`, -2, 1) |
++-----------+--------------------------+
+| [1, 2, 3] | [2]                      |
+| [1, 2, 3] | [2]                      |
+| [2, 3]    | [2]                      |
+| [2, 3]    | [2]                      |
++-----------+--------------------------+
+
+mysql> select k2, k2[-2:2] from array_type_table_nullable_varchar;
++----------------------------+--------------------------+
+| k2                         | array_slice(`k2`, -2, 2) |
++----------------------------+--------------------------+
+| ['hello', 'world', 'c++']  | ['world', 'c++']         |
+| ['a1', 'equals', 'b1']     | ['equals', 'b1']         |
+| ['hasnull', NULL, 'value'] | [NULL, 'value']          |
+| ['hasnull', NULL, 'value'] | [NULL, 'value']          |
++----------------------------+--------------------------+
+
+mysql> select k2, array_slice(k2, -2, 2) from array_type_table_nullable_varchar;
++----------------------------+--------------------------+
+| k2                         | array_slice(`k2`, -2, 2) |
++----------------------------+--------------------------+
+| ['hello', 'world', 'c++']  | ['world', 'c++']         |
+| ['a1', 'equals', 'b1']     | ['equals', 'b1']         |
+| ['hasnull', NULL, 'value'] | [NULL, 'value']          |
+| ['hasnull', NULL, 'value'] | [NULL, 'value']          |
++----------------------------+--------------------------+
+```
+
+```
+mysql> select k2, array_slice(k2, 0) from array_type_table;
++-----------+-------------------------+
+| k2        | array_slice(`k2`, 0) |
++-----------+-------------------------+
+| [1, 2, 3] | []                      |
++-----------+-------------------------+
+
+mysql> select k2, array_slice(k2, -5) from array_type_table;
++-----------+----------------------+
+| k2        | array_slice(`k2`, -5) |
++-----------+----------------------+
+| [1, 2, 3] | []                   |
++-----------+----------------------+
+```
+
+### keywords
+
+ARRAY,SLICE,ARRAY_SLICE
\ No newline at end of file
diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/array-functions/array_slice.md b/docs/zh-CN/docs/sql-manual/sql-functions/array-functions/array_slice.md
new file mode 100644
index 0000000000..6f617a2c84
--- /dev/null
+++ b/docs/zh-CN/docs/sql-manual/sql-functions/array-functions/array_slice.md
@@ -0,0 +1,157 @@
+---
+{
+    "title": "array_slice",
+    "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_slice
+
+### description
+
+#### Syntax
+
+```
+ARRAY<T> array_slice(ARRAY<T> arr, BIGINT off, BIGINT len)
+```
+
+返回一个子数组,包含所有从指定位置开始的指定长度的元素,如果输入参数为NULL,则返回NULL
+
+```
+如果off是正数,则表示从左侧开始的偏移量
+如果off是负数,则表示从右侧开始的偏移量
+当指定的off不在数组的实际范围内,返回空数组
+如果len是负数,则表示长度为0
+```
+
+### notice
+
+`仅支持向量化引擎中使用`
+
+### example
+
+
+```
+mysql> set enable_vectorized_engine=true;
+
+mysql> select k2, k2[2:2] from array_type_table_nullable;
++-----------------+-------------------------+
+| k2              | array_slice(`k2`, 2, 2) |
++-----------------+-------------------------+
+| [1, 2, 3]       | [2, 3]                  |
+| [1, NULL, 3]    | [NULL, 3]               |
+| [2, 3]          | [3]                     |
+| NULL            | NULL                    |
++-----------------+-------------------------+
+
+mysql> select k2, array_slice(k2, 2, 2) from array_type_table_nullable;
++-----------------+-------------------------+
+| k2              | array_slice(`k2`, 2, 2) |
++-----------------+-------------------------+
+| [1, 2, 3]       | [2, 3]                  |
+| [1, NULL, 3]    | [NULL, 3]               |
+| [2, 3]          | [3]                     |
+| NULL            | NULL                    |
++-----------------+-------------------------+
+
+mysql> select k2, k2[2:2] from array_type_table_nullable_varchar;
++----------------------------+-------------------------+
+| k2                         | array_slice(`k2`, 2, 2) |
++----------------------------+-------------------------+
+| ['hello', 'world', 'c++']  | ['world', 'c++']        |
+| ['a1', 'equals', 'b1']     | ['equals', 'b1']        |
+| ['hasnull', NULL, 'value'] | [NULL, 'value']         |
+| ['hasnull', NULL, 'value'] | [NULL, 'value']         |
++----------------------------+-------------------------+
+
+mysql> select k2, array_slice(k2, 2, 2) from array_type_table_nullable_varchar;
++----------------------------+-------------------------+
+| k2                         | array_slice(`k2`, 2, 2) |
++----------------------------+-------------------------+
+| ['hello', 'world', 'c++']  | ['world', 'c++']        |
+| ['a1', 'equals', 'b1']     | ['equals', 'b1']        |
+| ['hasnull', NULL, 'value'] | [NULL, 'value']         |
+| ['hasnull', NULL, 'value'] | [NULL, 'value']         |
++----------------------------+-------------------------+
+```
+
+当指定off为负数:
+
+```
+mysql> select k2, k2[-2:1] from array_type_table_nullable;
++-----------+--------------------------+
+| k2        | array_slice(`k2`, -2, 1) |
++-----------+--------------------------+
+| [1, 2, 3] | [2]                      |
+| [1, 2, 3] | [2]                      |
+| [2, 3]    | [2]                      |
+| [2, 3]    | [2]                      |
++-----------+--------------------------+
+
+mysql> select k2, array_slice(k2, -2, 1) from array_type_table_nullable;
++-----------+--------------------------+
+| k2        | array_slice(`k2`, -2, 1) |
++-----------+--------------------------+
+| [1, 2, 3] | [2]                      |
+| [1, 2, 3] | [2]                      |
+| [2, 3]    | [2]                      |
+| [2, 3]    | [2]                      |
++-----------+--------------------------+
+
+mysql> select k2, k2[-2:2] from array_type_table_nullable_varchar;
++----------------------------+--------------------------+
+| k2                         | array_slice(`k2`, -2, 2) |
++----------------------------+--------------------------+
+| ['hello', 'world', 'c++']  | ['world', 'c++']         |
+| ['a1', 'equals', 'b1']     | ['equals', 'b1']         |
+| ['hasnull', NULL, 'value'] | [NULL, 'value']          |
+| ['hasnull', NULL, 'value'] | [NULL, 'value']          |
++----------------------------+--------------------------+
+
+mysql> select k2, array_slice(k2, -2, 2) from array_type_table_nullable_varchar;
++----------------------------+--------------------------+
+| k2                         | array_slice(`k2`, -2, 2) |
++----------------------------+--------------------------+
+| ['hello', 'world', 'c++']  | ['world', 'c++']         |
+| ['a1', 'equals', 'b1']     | ['equals', 'b1']         |
+| ['hasnull', NULL, 'value'] | [NULL, 'value']          |
+| ['hasnull', NULL, 'value'] | [NULL, 'value']          |
++----------------------------+--------------------------+
+```
+
+```
+mysql> select k2, array_slice(k2, 0) from array_type_table;
++-----------+-------------------------+
+| k2        | array_slice(`k2`, 0) |
++-----------+-------------------------+
+| [1, 2, 3] | []                      |
++-----------+-------------------------+
+
+mysql> select k2, array_slice(k2, -5) from array_type_table;
++-----------+----------------------+
+| k2        | array_slice(`k2`, -5) |
++-----------+----------------------+
+| [1, 2, 3] | []                   |
++-----------+----------------------+
+```
+
+### keywords
+
+ARRAY,SLICE,ARRAY_SLICE
\ No newline at end of file
diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup
index 6104019eaa..556ab0785f 100644
--- a/fe/fe-core/src/main/cup/sql_parser.cup
+++ b/fe/fe-core/src/main/cup/sql_parser.cup
@@ -394,6 +394,7 @@ nonterminal AnalyticWindow.Type window_type;
 nonterminal AnalyticWindow.Boundary window_boundary;
 nonterminal SlotRef column_ref;
 nonterminal FunctionCallExpr column_subscript;
+nonterminal FunctionCallExpr column_slice;
 nonterminal ArrayList<TableRef> table_ref_list, base_table_ref_list;
 nonterminal ArrayList<LateralViewRef> opt_lateral_view_ref_list, lateral_view_ref_list;
 nonterminal FromClause from_clause;
@@ -4969,6 +4970,8 @@ non_pred_expr ::=
   {: RESULT = c; :}
   | column_subscript:c
   {: RESULT = c; :}
+  | column_slice:c
+  {: RESULT = c; :}
   | timestamp_arithmetic_expr:e
   {: RESULT = e; :}
   | arithmetic_expr:e
@@ -5331,6 +5334,22 @@ column_subscript ::=
   :}
   ;
 
+column_slice ::=
+  expr:e LBRACKET expr:offset COLON RBRACKET
+  {: ArrayList<Expr> list = new ArrayList<Expr>();
+       list.add(e);
+       list.add(offset);
+       RESULT = new FunctionCallExpr("%element_slice%", list);
+  :}
+  | expr:e LBRACKET expr:offset COLON expr:length RBRACKET
+  {: ArrayList<Expr> list = new ArrayList<Expr>();
+     list.add(e);
+     list.add(offset);
+     list.add(length);
+     RESULT = new FunctionCallExpr("%element_slice%", list);
+  :}
+  ;
+
 privilege_type ::=
     ident:name
     {:
diff --git a/fe/fe-core/src/main/jflex/sql_scanner.flex b/fe/fe-core/src/main/jflex/sql_scanner.flex
index f52f8e3c52..40c2a59622 100644
--- a/fe/fe-core/src/main/jflex/sql_scanner.flex
+++ b/fe/fe-core/src/main/jflex/sql_scanner.flex
@@ -457,6 +457,7 @@ import org.apache.doris.qe.SqlModeHelper;
     tokenIdMap.put(new Integer(SqlParserSymbols.RPAREN), ")");
     tokenIdMap.put(new Integer(SqlParserSymbols.LBRACKET), "[");
     tokenIdMap.put(new Integer(SqlParserSymbols.RBRACKET), "]");
+    tokenIdMap.put(new Integer(SqlParserSymbols.COLON), ":");
     tokenIdMap.put(new Integer(SqlParserSymbols.SEMICOLON), ";");
     tokenIdMap.put(new Integer(SqlParserSymbols.FLOATINGPOINT_LITERAL),
         "FLOATING POINT LITERAL");
@@ -600,6 +601,7 @@ EndOfLineComment = "--" !({HintContent}|{ContainsLineTerminator}) {LineTerminato
 "@" { return newToken(SqlParserSymbols.AT, null); }
 "(" { return newToken(SqlParserSymbols.LPAREN, null); }
 ")" { return newToken(SqlParserSymbols.RPAREN, null); }
+":" { return newToken(SqlParserSymbols.COLON, null); }
 ";" { return newToken(SqlParserSymbols.SEMICOLON, null); }
 "[" { return newToken(SqlParserSymbols.LBRACKET, null); }
 "]" { return newToken(SqlParserSymbols.RBRACKET, null); }
diff --git a/gensrc/script/doris_builtins_functions.py b/gensrc/script/doris_builtins_functions.py
index 8753edd7f1..63a4cb4a6f 100755
--- a/gensrc/script/doris_builtins_functions.py
+++ b/gensrc/script/doris_builtins_functions.py
@@ -297,6 +297,34 @@ visible_functions = [
     [['array_intersect'], 'ARRAY_VARCHAR',   ['ARRAY_VARCHAR', 'ARRAY_VARCHAR'], '', '', '', 'vec', ''],
     [['array_intersect'], 'ARRAY_STRING',    ['ARRAY_STRING', 'ARRAY_STRING'], '', '', '', 'vec', ''],
 
+    [['array_slice', '%element_slice%'], 'ARRAY_BOOLEAN', ['ARRAY_BOOLEAN', 'BIGINT'], '', '', '', 'vec', ''],
+    [['array_slice', '%element_slice%'], 'ARRAY_TINYINT', ['ARRAY_TINYINT', 'BIGINT'], '', '', '', 'vec', ''],
+    [['array_slice', '%element_slice%'], 'ARRAY_SMALLINT', ['ARRAY_SMALLINT', 'BIGINT'], '', '', '', 'vec', ''],
+    [['array_slice', '%element_slice%'], 'ARRAY_INT', ['ARRAY_INT', 'BIGINT'], '', '', '', 'vec', ''],
+    [['array_slice', '%element_slice%'], 'ARRAY_BIGINT', ['ARRAY_BIGINT', 'BIGINT'], '', '', '', 'vec', ''],
+    [['array_slice', '%element_slice%'], 'ARRAY_LARGEINT', ['ARRAY_LARGEINT', 'BIGINT'], '', '', '', 'vec', ''],
+    [['array_slice', '%element_slice%'], 'ARRAY_DATETIME', ['ARRAY_DATETIME', 'BIGINT'], '', '', '', 'vec', ''],
+    [['array_slice', '%element_slice%'], 'ARRAY_DATE', ['ARRAY_DATE', 'BIGINT'], '', '', '', 'vec', ''],
+    [['array_slice', '%element_slice%'], 'ARRAY_FLOAT', ['ARRAY_FLOAT', 'BIGINT'], '', '', '', 'vec', ''],
+    [['array_slice', '%element_slice%'], 'ARRAY_DOUBLE', ['ARRAY_DOUBLE', 'BIGINT'], '', '', '', 'vec', ''],
+    [['array_slice', '%element_slice%'], 'ARRAY_DECIMALV2', ['ARRAY_DECIMALV2', 'BIGINT'], '', '', '', 'vec', ''],
+    [['array_slice', '%element_slice%'], 'ARRAY_VARCHAR', ['ARRAY_VARCHAR', 'BIGINT'], '', '', '', 'vec', ''],
+    [['array_slice', '%element_slice%'], 'ARRAY_STRING', ['ARRAY_STRING', 'BIGINT'], '', '', '', 'vec', ''],
+
+    [['array_slice', '%element_slice%'], 'ARRAY_BOOLEAN', ['ARRAY_BOOLEAN', 'BIGINT', 'BIGINT'], '', '', '', 'vec', ''],
+    [['array_slice', '%element_slice%'], 'ARRAY_TINYINT', ['ARRAY_TINYINT', 'BIGINT', 'BIGINT'], '', '', '', 'vec', ''],
+    [['array_slice', '%element_slice%'], 'ARRAY_SMALLINT', ['ARRAY_SMALLINT', 'BIGINT', 'BIGINT'], '', '', '', 'vec', ''],
+    [['array_slice', '%element_slice%'], 'ARRAY_INT', ['ARRAY_INT', 'BIGINT', 'BIGINT'], '', '', '', 'vec', ''],
+    [['array_slice', '%element_slice%'], 'ARRAY_BIGINT', ['ARRAY_BIGINT', 'BIGINT', 'BIGINT'], '', '', '', 'vec', ''],
+    [['array_slice', '%element_slice%'], 'ARRAY_LARGEINT', ['ARRAY_LARGEINT', 'BIGINT', 'BIGINT'], '', '', '', 'vec', ''],
+    [['array_slice', '%element_slice%'], 'ARRAY_DATETIME', ['ARRAY_DATETIME', 'BIGINT', 'BIGINT'], '', '', '', 'vec', ''],
+    [['array_slice', '%element_slice%'], 'ARRAY_DATE', ['ARRAY_DATE', 'BIGINT', 'BIGINT'], '', '', '', 'vec', ''],
+    [['array_slice', '%element_slice%'], 'ARRAY_FLOAT', ['ARRAY_FLOAT', 'BIGINT', 'BIGINT'], '', '', '', 'vec', ''],
+    [['array_slice', '%element_slice%'], 'ARRAY_DOUBLE', ['ARRAY_DOUBLE', 'BIGINT', 'BIGINT'], '', '', '', 'vec', ''],
+    [['array_slice', '%element_slice%'], 'ARRAY_DECIMALV2', ['ARRAY_DECIMALV2', 'BIGINT', 'BIGINT'], '', '', '', 'vec', ''],
+    [['array_slice', '%element_slice%'], 'ARRAY_VARCHAR', ['ARRAY_VARCHAR', 'BIGINT', 'BIGINT'], '', '', '', 'vec', ''],
+    [['array_slice', '%element_slice%'], 'ARRAY_STRING', ['ARRAY_STRING', 'BIGINT', 'BIGINT'], '', '', '', 'vec', ''],
+
     # Timestamp functions
     [['unix_timestamp'], 'INT', [],
         '_ZN5doris18TimestampFunctions7to_unixEPN9doris_udf15FunctionContextE',
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 9b6cc814fe..634ca8d9a8 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
@@ -79,3 +79,21 @@
 5	\N
 6	\N
 7	\N
+
+-- !select --
+1	[2, 3]
+2	[]
+3	[]
+4	[2, 3, 4, 5, 4, 3, 2, 1]
+5	[]
+6	[2, 3, 4, 5, 4, 3, 2, 1]
+7	[9, NULL, 10, NULL]
+
+-- !select --
+1	[1, 2]
+2	[4]
+3	[]
+4	[1, 2]
+5	[]
+6	[1, 2]
+7	[8, 9]
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 a0292418b3..5852bf83e8 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
@@ -54,4 +54,6 @@ suite("test_array_functions", "query") {
     qt_select "select k1, array_union(k2, k4) FROM ${tableName} ORDER BY k1"
     qt_select "select k1, array_except(k2, k4) FROM ${tableName} ORDER BY k1"
     qt_select "select k1, array_intersect(k2, k4) FROM ${tableName} ORDER BY k1"
+    qt_select "select k1, array_slice(k2, 2) FROM ${tableName} ORDER BY k1"
+    qt_select "select k1, array_slice(k2, 1, 2) FROM ${tableName} ORDER BY k1"
 }


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