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/06/09 07:03:13 UTC

[incubator-doris] branch master updated: [feature-wip](array-type) Add array functions size and cardinality (#9921)

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/incubator-doris.git


The following commit(s) were added to refs/heads/master by this push:
     new 6fab1cbf3c [feature-wip](array-type) Add array functions size and cardinality (#9921)
6fab1cbf3c is described below

commit 6fab1cbf3c38f827da151186c167295fdedf3439
Author: camby <10...@qq.com>
AuthorDate: Thu Jun 9 15:03:03 2022 +0800

    [feature-wip](array-type) Add array functions size and cardinality (#9921)
    
    Co-authored-by: cambyzju <zh...@baidu.com>
---
 be/src/vec/CMakeLists.txt                          |  1 +
 .../functions/array/function_array_register.cpp    |  2 +
 ..._array_register.cpp => function_array_size.cpp} | 14 ++--
 be/src/vec/functions/array/function_array_size.h   | 71 +++++++++++++++++
 be/test/CMakeLists.txt                             |  1 +
 be/test/vec/function/function_array_size_test.cpp  | 88 ++++++++++++++++++++++
 .../sql-manual/sql-reference/Data-Types/ARRAY.md   |  5 ++
 .../sql-functions/array-functions/size.md          | 70 +++++++++++++++++
 .../sql-manual/sql-reference/Data-Types/ARRAY.md   |  6 ++
 .../sql-functions/array-functions/size.md          | 70 +++++++++++++++++
 .../java/org/apache/doris/analysis/TypeDef.java    |  3 -
 .../java/org/apache/doris/catalog/ArrayType.java   |  4 +-
 .../main/java/org/apache/doris/common/Config.java  |  5 --
 .../java/org/apache/doris/qe/SessionVariable.java  | 13 ++++
 .../org/apache/doris/analysis/ColumnDefTest.java   | 15 +++-
 .../apache/doris/analysis/InsertArrayStmtTest.java |  2 +-
 .../org/apache/doris/catalog/CreateTableTest.java  |  2 +-
 gensrc/script/doris_builtins_functions.py          |  2 +
 .../array_functions/test_array_functions.out       | 11 +++
 .../array_functions/test_array_functions.groovy    | 45 +++++++++++
 20 files changed, 407 insertions(+), 23 deletions(-)

diff --git a/be/src/vec/CMakeLists.txt b/be/src/vec/CMakeLists.txt
index 49f92ddf79..d42b12e5bb 100644
--- a/be/src/vec/CMakeLists.txt
+++ b/be/src/vec/CMakeLists.txt
@@ -125,6 +125,7 @@ set(VEC_FILES
   functions/array/function_array_index.cpp
   functions/array/function_array_element.cpp
   functions/array/function_array_register.cpp
+  functions/array/function_array_size.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 90bd5dcb5f..3c5dc4fd19 100644
--- a/be/src/vec/functions/array/function_array_register.cpp
+++ b/be/src/vec/functions/array/function_array_register.cpp
@@ -24,10 +24,12 @@ namespace doris::vectorized {
 
 void register_function_array_element(SimpleFunctionFactory&);
 void register_function_array_index(SimpleFunctionFactory&);
+void register_function_array_size(SimpleFunctionFactory&);
 
 void register_function_array(SimpleFunctionFactory& factory) {
     register_function_array_element(factory);
     register_function_array_index(factory);
+    register_function_array_size(factory);
 }
 
 } // namespace doris::vectorized
diff --git a/be/src/vec/functions/array/function_array_register.cpp b/be/src/vec/functions/array/function_array_size.cpp
similarity index 68%
copy from be/src/vec/functions/array/function_array_register.cpp
copy to be/src/vec/functions/array/function_array_size.cpp
index 90bd5dcb5f..4b66570ad6 100644
--- a/be/src/vec/functions/array/function_array_register.cpp
+++ b/be/src/vec/functions/array/function_array_size.cpp
@@ -14,20 +14,16 @@
 // 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/registerFunctionsArray.cpp
-// and modified by Doris
+
+#include "vec/functions/array/function_array_size.h"
 
 #include "vec/functions/simple_function_factory.h"
 
 namespace doris::vectorized {
 
-void register_function_array_element(SimpleFunctionFactory&);
-void register_function_array_index(SimpleFunctionFactory&);
-
-void register_function_array(SimpleFunctionFactory& factory) {
-    register_function_array_element(factory);
-    register_function_array_index(factory);
+void register_function_array_size(SimpleFunctionFactory& factory) {
+    factory.register_function<FunctionArraySize>();
+    factory.register_alias(FunctionArraySize::name, "cardinality");
 }
 
 } // namespace doris::vectorized
diff --git a/be/src/vec/functions/array/function_array_size.h b/be/src/vec/functions/array/function_array_size.h
new file mode 100644
index 0000000000..bffed4460d
--- /dev/null
+++ b/be/src/vec/functions/array/function_array_size.h
@@ -0,0 +1,71 @@
+// 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/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 FunctionArraySize : public IFunction {
+public:
+    static constexpr auto name = "size";
+    static FunctionPtr create() { return std::make_shared<FunctionArraySize>(); }
+
+    /// 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";
+        return std::make_shared<DataTypeInt64>();
+    }
+
+    Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments,
+                        size_t result, size_t input_rows_count) override {
+        auto left_column =
+                block.get_by_position(arguments[0]).column->convert_to_full_column_if_const();
+        const auto array_column = check_and_get_column<ColumnArray>(*left_column);
+        if (!array_column) {
+            return Status::RuntimeError(
+                    fmt::format("unsupported types for function {}({})", get_name(),
+                                block.get_by_position(arguments[0]).type->get_name()));
+        }
+        const auto& offsets = array_column->get_offsets();
+
+        auto dst_column = ColumnInt64::create(input_rows_count);
+        auto& dst_data = dst_column->get_data();
+
+        for (size_t i = 0; i < offsets.size(); ++i) {
+            dst_data[i] = offsets[i] - offsets[i - 1];
+        }
+
+        block.replace_by_position(result, std::move(dst_column));
+        return Status::OK();
+    }
+};
+
+} // namespace doris::vectorized
diff --git a/be/test/CMakeLists.txt b/be/test/CMakeLists.txt
index 9c7a9190f0..b18a9a659b 100644
--- a/be/test/CMakeLists.txt
+++ b/be/test/CMakeLists.txt
@@ -342,6 +342,7 @@ set(VEC_TEST_FILES
     vec/exprs/vexpr_test.cpp
     vec/function/function_array_element_test.cpp
     vec/function/function_array_index_test.cpp
+    vec/function/function_array_size_test.cpp
     vec/function/function_bitmap_test.cpp
     vec/function/function_comparison_test.cpp
     vec/function/function_hash_test.cpp
diff --git a/be/test/vec/function/function_array_size_test.cpp b/be/test/vec/function/function_array_size_test.cpp
new file mode 100644
index 0000000000..3db14a28c1
--- /dev/null
+++ b/be/test/vec/function/function_array_size_test.cpp
@@ -0,0 +1,88 @@
+// 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 <gtest/gtest.h>
+#include <time.h>
+
+#include <string>
+
+#include "function_test_util.h"
+#include "runtime/tuple_row.h"
+#include "util/url_coding.h"
+#include "vec/core/field.h"
+
+namespace doris::vectorized {
+
+TEST(function_array_size_test, size) {
+    std::string func_name = "size";
+    Array empty_arr;
+
+    // size(Array<Int32>)
+    {
+        InputTypeSet input_types = {TypeIndex::Array, TypeIndex::Int32};
+
+        Array vec = {Int32(1), Int32(2), Int32(3)};
+        DataSet data_set = {{{vec}, Int64(3)}, {{Null()}, Null()}, {{empty_arr}, Int64(0)}};
+
+        check_function<DataTypeInt64, true>(func_name, input_types, data_set);
+    }
+
+    // size(Array<String>)
+    {
+        InputTypeSet input_types = {TypeIndex::Array, TypeIndex::String};
+
+        Array vec1 = {Field("abc", 3), Field("", 0), Field("def", 3)};
+        Array vec2 = {Field("abc", 3), Field("123", 0), Field("def", 3)};
+        DataSet data_set = {{{vec1}, Int64(3)},
+                            {{vec2}, Int64(3)},
+                            {{Null()}, Null()},
+                            {{empty_arr}, Int64(0)}};
+
+        check_function<DataTypeInt64, true>(func_name, input_types, data_set);
+    }
+}
+
+TEST(function_array_size_test, cardinality) {
+    std::string func_name = "cardinality";
+    Array empty_arr;
+
+    // cardinality(Array<Int32>)
+    {
+        InputTypeSet input_types = {TypeIndex::Array, TypeIndex::Int32};
+
+        Array vec = {Int32(1), Int32(2), Int32(3)};
+        DataSet data_set = {{{vec}, Int64(3)}, {{Null()}, Null()}, {{empty_arr}, Int64(0)}};
+
+        check_function<DataTypeInt64, true>(func_name, input_types, data_set);
+    }
+
+    // cardinality(Array<String>)
+    {
+        InputTypeSet input_types = {TypeIndex::Array, TypeIndex::String};
+
+        Array vec1 = {Field("abc", 3), Field("", 0), Field("def", 3)};
+        Array vec2 = {Field("abc", 3), Field("123", 0), Field("def", 3)};
+        DataSet data_set = {{{vec1}, Int64(3)},
+                            {{vec2}, Int64(3)},
+                            {{Null()}, Null()},
+                            {{empty_arr}, Int64(0)}};
+
+        check_function<DataTypeInt64, true>(func_name, input_types, data_set);
+    }
+}
+
+} // namespace doris::vectorized
diff --git a/docs/en/docs/sql-manual/sql-reference/Data-Types/ARRAY.md b/docs/en/docs/sql-manual/sql-reference/Data-Types/ARRAY.md
index a157503bc5..8a0ddb88ea 100644
--- a/docs/en/docs/sql-manual/sql-reference/Data-Types/ARRAY.md
+++ b/docs/en/docs/sql-manual/sql-reference/Data-Types/ARRAY.md
@@ -38,12 +38,17 @@ T-type could be any of:
 BOOLEAN, TINYINT, SMALLINT, INT, BIGINT, LARGEINT, FLOAT, DOUBLE, DECIMAL, DATE,
 DATETIME, CHAR, VARCHAR, STRING
 ```
+### notice
+
+please open `enable_array_type` before use ARRAY
 
 ### example
 
 Create table example:
 
 ```
+mysql> set enable_array_type=true;
+
 mysql> CREATE TABLE `array_test` (
   `id` int(11) NULL COMMENT "",
   `c_array` ARRAY<int(11)> NULL COMMENT ""
diff --git a/docs/en/sql-manual/sql-functions/array-functions/size.md b/docs/en/sql-manual/sql-functions/array-functions/size.md
new file mode 100644
index 0000000000..8642edb9ff
--- /dev/null
+++ b/docs/en/sql-manual/sql-functions/array-functions/size.md
@@ -0,0 +1,70 @@
+---
+{
+    "title": "size",
+    "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.
+-->
+
+## size (cardinality)
+
+### description
+
+#### Syntax
+
+```
+BIGINT size(ARRAY<T> arr) 
+BIGINT cardinality(ARRAY<T> arr)
+```
+
+Returns the size of the array, returns NULL for NULL input.
+
+### notice
+
+`Only supported in vectorized engine`
+
+### example
+
+```
+mysql> set enable_vectorized_engine=true;
+
+mysql> select k1,k2,size(k2) from array_test;
++------+-----------+------------+
+| k1   | k2        | size(`k2`) |
++------+-----------+------------+
+|    1 | [1, 2, 3] |          3 |
+|    2 | []        |          0 |
+|    3 | NULL      |       NULL |
++------+-----------+------------+
+
+mysql> select k1,k2,cardinality(k2) from array_test;
++------+-----------+-------------------+
+| k1   | k2        | cardinality(`k2`) |
++------+-----------+-------------------+
+|    1 | [1, 2, 3] |                 3 |
+|    2 | []        |                 0 |
+|    3 | NULL      |              NULL |
++------+-----------+-------------------+
+```
+
+### keywords
+
+SIZE, CARDINALITY
diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Types/ARRAY.md b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Types/ARRAY.md
index cffe63f3be..3a6277b6cc 100644
--- a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Types/ARRAY.md
+++ b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Types/ARRAY.md
@@ -39,11 +39,17 @@ BOOLEAN, TINYINT, SMALLINT, INT, BIGINT, LARGEINT, FLOAT, DOUBLE, DECIMAL, DATE,
 DATETIME, CHAR, VARCHAR, STRING
 ```
 
+### notice
+
+使用时需要先打开`enable_array_type`开关
+
 ### example
 
 建表示例如下:
 
 ```
+mysql> set enable_array_type=true;
+
 mysql> CREATE TABLE `array_test` (
   `id` int(11) NULL COMMENT "",
   `c_array` ARRAY<int(11)> NULL COMMENT ""
diff --git a/docs/zh-CN/sql-manual/sql-functions/array-functions/size.md b/docs/zh-CN/sql-manual/sql-functions/array-functions/size.md
new file mode 100644
index 0000000000..bf4e319fd3
--- /dev/null
+++ b/docs/zh-CN/sql-manual/sql-functions/array-functions/size.md
@@ -0,0 +1,70 @@
+---
+{
+    "title": "size",
+    "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.
+-->
+
+## size (cardinality)
+
+### description
+
+#### Syntax
+
+```
+BIGINT size(ARRAY<T> arr) 
+BIGINT cardinality(ARRAY<T> arr)
+```
+
+返回数组中元素数量,如果输入数组为NULL,则返回NULL
+
+### notice
+
+`仅支持向量化引擎中使用`
+
+### example
+
+```
+mysql> set enable_vectorized_engine=true;
+
+mysql> select k1,k2,size(k2) from array_test;
++------+-----------+------------+
+| k1   | k2        | size(`k2`) |
++------+-----------+------------+
+|    1 | [1, 2, 3] |          3 |
+|    2 | []        |          0 |
+|    3 | NULL      |       NULL |
++------+-----------+------------+
+
+mysql> select k1,k2,cardinality(k2) from array_test;
++------+-----------+-------------------+
+| k1   | k2        | cardinality(`k2`) |
++------+-----------+-------------------+
+|    1 | [1, 2, 3] |                 3 |
+|    2 | []        |                 0 |
+|    3 | NULL      |              NULL |
++------+-----------+-------------------+
+```
+
+### keywords
+
+SIZE, CARDINALITY
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/TypeDef.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/TypeDef.java
index 9f82878d76..e8ed068a43 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/TypeDef.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/TypeDef.java
@@ -86,9 +86,6 @@ public class TypeDef implements ParseNode {
         }
 
         if (type.isComplexType()) {
-            if (!Config.enable_complex_type_support) {
-                throw new AnalysisException("Unsupported data type: " + type.toSql());
-            }
             if (type.isArrayType()) {
                 Type itemType = ((ArrayType) type).getItemType();
                 if (itemType instanceof ScalarType) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/ArrayType.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/ArrayType.java
index b86da0a275..028cb760fd 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/ArrayType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/ArrayType.java
@@ -17,7 +17,7 @@
 
 package org.apache.doris.catalog;
 
-import org.apache.doris.common.Config;
+import org.apache.doris.qe.ConnectContext;
 import org.apache.doris.thrift.TColumnType;
 import org.apache.doris.thrift.TTypeDesc;
 import org.apache.doris.thrift.TTypeNode;
@@ -137,7 +137,7 @@ public class ArrayType extends Type {
 
     @Override
     public boolean isSupported() {
-        if (!Config.enable_complex_type_support) {
+        if (!ConnectContext.get().getSessionVariable().isEnableArrayType()) {
             return false;
         }
         return !itemType.isNull();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/Config.java b/fe/fe-core/src/main/java/org/apache/doris/common/Config.java
index 1657e03b1b..65e6598d15 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/Config.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/Config.java
@@ -1294,11 +1294,6 @@ public class Config extends ConfigBase {
     @ConfField(mutable = true, masterOnly = true)
     public static int period_of_auto_resume_min = 5;
 
-    /*
-     * If set to true, Doris will support complex type
-     */
-    @ConfField
-    public static boolean enable_complex_type_support = false;
     /**
      * If set to true, the backend will be automatically dropped after finishing decommission.
      * If set to false, the backend will not be dropped and remaining in DECOMMISSION state.
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java
index 53392e9901..c78172306c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java
@@ -182,6 +182,8 @@ public class SessionVariable implements Serializable, Writable {
 
     public static final String TRIM_TAILING_SPACES_FOR_EXTERNAL_TABLE_QUERY = "trim_tailing_spaces_for_external_table_query";
 
+    static final String ENABLE_ARRAY_TYPE = "enable_array_type";
+
     // session origin value
     public Map<Field, String> sessionOriginValue = new HashMap<Field, String>();
     // check stmt is or not [select /*+ SET_VAR(...)*/ ...]
@@ -444,6 +446,9 @@ public class SessionVariable implements Serializable, Writable {
     @VariableMgr.VarAttr(name = TRIM_TAILING_SPACES_FOR_EXTERNAL_TABLE_QUERY, needForward = true)
     public boolean trimTailingSpacesForExternalTableQuery = false;
 
+    @VariableMgr.VarAttr(name = ENABLE_ARRAY_TYPE)
+    boolean enableArrayType = false;
+
     public String getBlockEncryptionMode() {
         return blockEncryptionMode;
     }
@@ -926,6 +931,14 @@ public class SessionVariable implements Serializable, Writable {
         this.disableJoinReorder = disableJoinReorder;
     }
 
+    public boolean isEnableArrayType() {
+        return enableArrayType;
+    }
+
+    public void setEnableArrayType(boolean enableArrayType) {
+        this.enableArrayType = enableArrayType;
+    }
+
     // Serialize to thrift object
     // used for rest api
     public TQueryOptions toThrift() {
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/ColumnDefTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/ColumnDefTest.java
index 968ff716b5..f18e11b222 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/ColumnDefTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/ColumnDefTest.java
@@ -25,8 +25,10 @@ import org.apache.doris.catalog.PrimitiveType;
 import org.apache.doris.catalog.ScalarType;
 import org.apache.doris.catalog.Type;
 import org.apache.doris.common.AnalysisException;
-import org.apache.doris.common.Config;
+import org.apache.doris.qe.ConnectContext;
 
+import mockit.Mock;
+import mockit.MockUp;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -36,6 +38,7 @@ public class ColumnDefTest {
     private TypeDef stringCol;
     private TypeDef floatCol;
     private TypeDef booleanCol;
+    private ConnectContext ctx;
 
     @Before
     public void setUp() {
@@ -43,6 +46,14 @@ public class ColumnDefTest {
         stringCol = new TypeDef(ScalarType.createChar(10));
         floatCol = new TypeDef(ScalarType.createType(PrimitiveType.FLOAT));
         booleanCol = new TypeDef(ScalarType.createType(PrimitiveType.BOOLEAN));
+
+        ctx = new ConnectContext(null);
+        new MockUp<ConnectContext>() {
+            @Mock
+            public ConnectContext get() {
+                return ctx;
+            }
+        };
     }
 
     @Test
@@ -122,7 +133,7 @@ public class ColumnDefTest {
 
     @Test
     public void testArray() throws AnalysisException {
-        Config.enable_complex_type_support = true;
+        ctx.getSessionVariable().setEnableArrayType(true);
         TypeDef typeDef = new TypeDef(new ArrayType(Type.INT));
         ColumnDef columnDef = new ColumnDef("array", typeDef, false, null, true, DefaultValue.NOT_SET, "");
         Column column = columnDef.toColumn();
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/InsertArrayStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/InsertArrayStmtTest.java
index 3dca3638d2..04d9968c46 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/InsertArrayStmtTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/InsertArrayStmtTest.java
@@ -42,9 +42,9 @@ public class InsertArrayStmtTest {
 
     @BeforeClass
     public static void setUp() throws Exception {
-        Config.enable_complex_type_support = true;
         UtFrameUtils.createDorisCluster(RUNNING_DIR);
         connectContext = UtFrameUtils.createDefaultCtx();
+        connectContext.getSessionVariable().setEnableArrayType(true);
         createDatabase("create database test;");
     }
 
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableTest.java
index 513b3cef1c..71dd3e399d 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableTest.java
@@ -530,7 +530,7 @@ public class CreateTableTest {
 
     @Test
     public void testCreateTableWithArrayType() throws Exception {
-        Config.enable_complex_type_support = true;
+        ConnectContext.get().getSessionVariable().setEnableArrayType(true);
         ExceptionChecker.expectThrowsNoException(() -> {
             createTable("create table test.table1(k1 INT, k2 Array<int>) duplicate key (k1) "
                     + "distributed by hash(k1) buckets 1 properties('replication_num' = '1');");
diff --git a/gensrc/script/doris_builtins_functions.py b/gensrc/script/doris_builtins_functions.py
index 2bf9802a83..af88200166 100755
--- a/gensrc/script/doris_builtins_functions.py
+++ b/gensrc/script/doris_builtins_functions.py
@@ -159,6 +159,8 @@ visible_functions = [
     [['array_position'], 'BIGINT', ['ARRAY_VARCHAR', 'VARCHAR'], '', '', '', 'vec', ''],
     [['array_position'], 'BIGINT', ['ARRAY_STRING', 'STRING'], '', '', '', 'vec', ''],
 
+    [['cardinality', 'size'], 'BIGINT', ['ARRAY'], '', '', '', '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
new file mode 100644
index 0000000000..18ed9ec24d
--- /dev/null
+++ b/regression-test/data/query/sql_functions/array_functions/test_array_functions.out
@@ -0,0 +1,11 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !select --
+1	3	3
+2	1	\N
+3	0	0
+
+-- !select --
+1	3	3
+2	1	\N
+3	0	0
+
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
new file mode 100644
index 0000000000..2897da0ae9
--- /dev/null
+++ b/regression-test/suites/query/sql_functions/array_functions/test_array_functions.groovy
@@ -0,0 +1,45 @@
+// 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_array_functions", "query") {
+    def tableName = "tbl_test_array_functions"
+    // open enable_array_type
+    sql """ set enable_array_type = true """
+    // array functions only supported in vectorized engine
+    sql """ set enable_vectorized_engine = true """
+
+    sql """DROP TABLE IF EXISTS ${tableName}"""
+    sql """ 
+            CREATE TABLE IF NOT EXISTS ${tableName} (
+              `k1` int(11) NULL COMMENT "",
+              `k2` ARRAY<int(11)> NOT NULL COMMENT "",
+              `k3` ARRAY<VARCHAR(20)> NULL COMMENT ""
+            ) ENGINE=OLAP
+            DUPLICATE KEY(`k1`)
+            DISTRIBUTED BY HASH(`k1`) BUCKETS 1
+            PROPERTIES (
+            "replication_allocation" = "tag.location.default: 1",
+            "storage_format" = "V2"
+            )
+        """
+    sql """ INSERT INTO ${tableName} VALUES(1, [1, 2, 3], ["a", "b", ""]) """
+    sql """ INSERT INTO ${tableName} VALUES(2, [4], NULL) """
+    sql """ INSERT INTO ${tableName} VALUES(3, [], []) """
+
+    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"
+}


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