[function](nereids) support array_apply/array_repeat/group_uniq_array/ipv4numtostring (#25249)

nereids support functions: array_apply/array_repeat/group_uniq_array/ipv4numtostring
This commit is contained in:
minghong
2023-10-12 11:08:42 +08:00
committed by GitHub
parent a0d2b1ec56
commit 42f8b253aa
13 changed files with 440 additions and 3 deletions

View File

@ -87,7 +87,7 @@ public class BuiltinAggregateFunctions implements FunctionHelper {
agg(BitmapUnionCount.class, "bitmap_union_count"),
agg(BitmapUnionInt.class, "bitmap_union_int"),
agg(CollectList.class, "collect_list"),
agg(CollectSet.class, "collect_set"),
agg(CollectSet.class, "collect_set", "group_uniq_array"),
agg(Count.class, "count"),
agg(CountByEnum.class, "count_by_enum"),
agg(GroupBitAnd.class, "group_bit_and"),

View File

@ -27,6 +27,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.AesEncrypt;
import org.apache.doris.nereids.trees.expressions.functions.scalar.AesEncryptV2;
import org.apache.doris.nereids.trees.expressions.functions.scalar.AppendTrailingCharIfAbsent;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Array;
import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayApply;
import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayAvg;
import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayCompact;
import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayContains;
@ -53,6 +54,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayPosition
import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayProduct;
import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayRange;
import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayRemove;
import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayRepeat;
import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayReverseSort;
import org.apache.doris.nereids.trees.expressions.functions.scalar.ArraySlice;
import org.apache.doris.nereids.trees.expressions.functions.scalar.ArraySort;
@ -183,6 +185,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.If;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Initcap;
import org.apache.doris.nereids.trees.expressions.functions.scalar.InnerProduct;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Instr;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv4NumToString;
import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonArray;
import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonContains;
import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonExtract;
@ -414,6 +417,7 @@ public class BuiltinScalarFunctions implements FunctionHelper {
scalar(AesEncryptV2.class, "aes_encrypt_v2"),
scalar(AppendTrailingCharIfAbsent.class, "append_trailing_char_if_absent"),
scalar(Array.class, "array"),
scalar(ArrayApply.class, "array_apply"),
scalar(ArrayAvg.class, "array_avg"),
scalar(ArrayCompact.class, "array_compact"),
scalar(ArrayContains.class, "array_contains"),
@ -440,6 +444,7 @@ public class BuiltinScalarFunctions implements FunctionHelper {
scalar(ArrayProduct.class, "array_product"),
scalar(ArrayRange.class, "array_range"),
scalar(ArrayRemove.class, "array_remove"),
scalar(ArrayRepeat.class, "array_repeat"),
scalar(ArrayReverseSort.class, "array_reverse_sort"),
scalar(ArraySlice.class, "array_slice"),
scalar(ArraySort.class, "array_sort"),
@ -567,6 +572,7 @@ public class BuiltinScalarFunctions implements FunctionHelper {
scalar(Initcap.class, "initcap"),
scalar(InnerProduct.class, "inner_product"),
scalar(Instr.class, "instr"),
scalar(Ipv4NumToString.class, "ipv4numtostring", "inet_ntoa"),
scalar(JsonArray.class, "json_array"),
scalar(JsonObject.class, "json_object"),
scalar(JsonQuote.class, "json_quote"),

View File

@ -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.
package org.apache.doris.nereids.trees.expressions.functions.scalar;
import org.apache.doris.catalog.FunctionSignature;
import org.apache.doris.nereids.exceptions.AnalysisException;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable;
import org.apache.doris.nereids.trees.expressions.literal.StringLikeLiteral;
import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.ArrayType;
import org.apache.doris.nereids.types.VarcharType;
import org.apache.doris.nereids.types.coercion.AnyDataType;
import org.apache.doris.nereids.types.coercion.FollowToAnyDataType;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import java.util.List;
/**
* scalar function array_apply
*/
public class ArrayApply extends ScalarFunction
implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullable {
public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
FunctionSignature.retArgType(0)
.args(ArrayType.of(new AnyDataType(0)), VarcharType.SYSTEM_DEFAULT,
new FollowToAnyDataType(0)));
/**
* constructor
*/
public ArrayApply(Expression arg0, Expression arg1, Expression arg2) {
super("array_apply", arg0, arg1, arg2);
if (!(arg1 instanceof StringLikeLiteral)) {
throw new AnalysisException(
"array_apply(arr, op, val): op support const value only.");
} else {
String op = ((StringLikeLiteral) arg1).getStringValue();
if (! "=".equals(op) && !">".equals(op) && !"<".equals(op)
&& !">=".equals(op) && !"<=".equals(op) && !"!=".equals(op)) {
throw new AnalysisException(
"array_apply(arr, op, val): op support =, >=, <=, >, <, !=, but we get " + op);
}
}
if (!(arg2.isConstant())) {
throw new AnalysisException(
"array_apply(arr, op, val): val support const value only.");
}
}
@Override
public ArrayApply withChildren(List<Expression> children) {
Preconditions.checkArgument(children.size() == 3,
"array_apply accept 3 args, but got %s (%s)",
children.size(),
children);
return new ArrayApply(children.get(0), children.get(1), children.get(2));
}
@Override
public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
return visitor.visitArrayApply(this, context);
}
@Override
public List<FunctionSignature> getSignatures() {
return SIGNATURES;
}
}

View File

@ -0,0 +1,68 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.trees.expressions.functions.scalar;
import org.apache.doris.catalog.FunctionSignature;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable;
import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.ArrayType;
import org.apache.doris.nereids.types.BigIntType;
import org.apache.doris.nereids.types.coercion.AnyDataType;
import org.apache.doris.nereids.types.coercion.FollowToAnyDataType;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import java.util.List;
/**
* ScalarFunction ArrayRepeat
*/
public class ArrayRepeat extends ScalarFunction
implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable {
public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
FunctionSignature.ret(ArrayType.of(new FollowToAnyDataType(0)))
.args(new AnyDataType(0), BigIntType.INSTANCE));
public ArrayRepeat(Expression arg0, Expression arg1) {
super("array_repeat", arg0, arg1);
}
@Override
public ArrayRepeat withChildren(List<Expression> children) {
Preconditions.checkArgument(children.size() == 2,
"array_repeat accept 2 args, but got %s (%s)",
children.size(),
children);
return new ArrayRepeat(children.get(0), children.get(1));
}
@Override
public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
return visitor.visitArrayRepeat(this, context);
}
@Override
public List<FunctionSignature> getSignatures() {
return SIGNATURES;
}
}

View File

@ -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.
package org.apache.doris.nereids.trees.expressions.functions.scalar;
import org.apache.doris.catalog.FunctionSignature;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable;
import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.BigIntType;
import org.apache.doris.nereids.types.IntegerType;
import org.apache.doris.nereids.types.SmallIntType;
import org.apache.doris.nereids.types.TinyIntType;
import org.apache.doris.nereids.types.VarcharType;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import java.util.List;
/**
* scalar function Ipv4NumToString
*/
public class Ipv4NumToString extends ScalarFunction
implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNullable {
public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(TinyIntType.INSTANCE),
FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(SmallIntType.INSTANCE),
FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(IntegerType.INSTANCE),
FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(BigIntType.INSTANCE));
public Ipv4NumToString(Expression arg0) {
super("ipv4numtostring", arg0);
}
@Override
public Ipv4NumToString withChildren(List<Expression> children) {
Preconditions.checkArgument(children.size() == 1,
"ipv4numtostring accept 1 args, but got %s (%s)",
children.size(),
children);
return new Ipv4NumToString(children.get(0));
}
@Override
public List<FunctionSignature> getSignatures() {
return SIGNATURES;
}
@Override
public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
return visitor.visitIpv4NumToString(this, context);
}
}

View File

@ -29,6 +29,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.AesEncrypt;
import org.apache.doris.nereids.trees.expressions.functions.scalar.AesEncryptV2;
import org.apache.doris.nereids.trees.expressions.functions.scalar.AppendTrailingCharIfAbsent;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Array;
import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayApply;
import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayAvg;
import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayCompact;
import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayContains;
@ -53,6 +54,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayPosition
import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayProduct;
import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayRange;
import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayRemove;
import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayRepeat;
import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayReverseSort;
import org.apache.doris.nereids.trees.expressions.functions.scalar.ArraySlice;
import org.apache.doris.nereids.trees.expressions.functions.scalar.ArraySort;
@ -179,6 +181,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.If;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Initcap;
import org.apache.doris.nereids.trees.expressions.functions.scalar.InnerProduct;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Instr;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv4NumToString;
import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonArray;
import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonContains;
import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonExtract;
@ -424,6 +427,10 @@ public interface ScalarFunctionVisitor<R, C> {
return visitScalarFunction(array, context);
}
default R visitArrayApply(ArrayApply arrayApply, C context) {
return visitScalarFunction(arrayApply, context);
}
default R visitArrayAvg(ArrayAvg arrayAvg, C context) {
return visitScalarFunction(arrayAvg, context);
}
@ -532,6 +539,10 @@ public interface ScalarFunctionVisitor<R, C> {
return visitScalarFunction(arraySort, context);
}
default R visitArrayRepeat(ArrayRepeat arrayRepeat, C context) {
return visitScalarFunction(arrayRepeat, context);
}
default R visitArraySum(ArraySum arraySum, C context) {
return visitScalarFunction(arraySum, context);
}
@ -1060,6 +1071,10 @@ public interface ScalarFunctionVisitor<R, C> {
return visitScalarFunction(instr, context);
}
default R visitIpv4NumToString(Ipv4NumToString ipv4NumToString, C context) {
return visitScalarFunction(ipv4NumToString, context);
}
default R visitJsonArray(JsonArray jsonArray, C context) {
return visitScalarFunction(jsonArray, context);
}

View File

@ -493,7 +493,7 @@ visible_functions = {
[['array_apply'], 'ARRAY_SMALLINT', ['ARRAY_SMALLINT', 'VARCHAR', 'SMALLINT'], ''],
[['array_apply'], 'ARRAY_INT', ['ARRAY_INT', 'VARCHAR', 'INT'], ''],
[['array_apply'], 'ARRAY_BIGINT', ['ARRAY_BIGINT', 'VARCHAR', 'BIGINT'], ''],
[['array_apply'], 'ARRAY_LARGEINT', ['ARRAY_LARGEINT', 'VARCHAR', 'ARRAY_LARGEINT'], ''],
[['array_apply'], 'ARRAY_LARGEINT', ['ARRAY_LARGEINT', 'VARCHAR', 'LARGEINT'], ''],
[['array_apply'], 'ARRAY_FLOAT', ['ARRAY_FLOAT', 'VARCHAR', 'FLOAT'], ''],
[['array_apply'], 'ARRAY_DOUBLE', ['ARRAY_DOUBLE', 'VARCHAR', 'DOUBLE'], ''],
[['array_apply'], 'ARRAY_DECIMALV2', ['ARRAY_DECIMALV2', 'VARCHAR', 'DECIMALV2'], ''],
@ -501,7 +501,7 @@ visible_functions = {
[['array_apply'], 'ARRAY_DECIMAL64', ['ARRAY_DECIMAL64', 'VARCHAR', 'DECIMAL64'], ''],
[['array_apply'], 'ARRAY_DECIMAL128', ['ARRAY_DECIMAL128', 'VARCHAR', 'DECIMAL128'], ''],
[['array_apply'], 'ARRAY_DATETIME', ['ARRAY_DATETIME', 'VARCHAR', 'DATETIME'], ''],
[['array_apply'], 'ARRAY_DATE', ['ARRAY_DATE', 'DATE', 'DATE'], ''],
[['array_apply'], 'ARRAY_DATE', ['ARRAY_DATE', 'VARCHAR', 'DATE'], ''],
[['array_apply'], 'ARRAY_DATETIMEV2', ['ARRAY_DATETIMEV2', 'VARCHAR', 'DATETIMEV2'], ''],
[['array_apply'], 'ARRAY_DATEV2', ['ARRAY_DATEV2', 'VARCHAR', 'DATEV2'], ''],

View File

@ -0,0 +1,10 @@
-- This file is automatically generated. You should know what you did if you want to edit this
-- !1 --
[4, 2, 1, 3] [2, 1]
-- !2 --
1 [2023-01-01] ["hello"]
2 [2023-01-01, 2023-01-02] ["hello"]
3 [2023-01-02] ["hello"]
4 [2023-01-02, 2023-01-03] ["sql"]

View File

@ -0,0 +1,25 @@
-- This file is automatically generated. You should know what you did if you want to edit this
-- !ip1 --
\N
-- !ip2 --
127.0.0.1
-- !ip3 --
\N
-- !ip4 --
192.168.0.1
-- !ip5 --
\N
-- !ip6 --
127.0.0.1
-- !ip7 --
\N
-- !ip8 --
192.168.0.1

View File

@ -12981,3 +12981,39 @@ true
[1]
[1]
-- !array_apply1 --
[]
[]
-- !array_apply2 --
[2022-12-01 22:23:25.000, 2022-12-01 23:23:25.000]
[2022-12-02 22:23:25.000, 2022-12-02 23:23:25.000]
-- !array_apply3 --
\N
\N
-- !array_apply4 --
[22.679]
[]
-- !array_apply5 --
[22.679, 33.679]
[23.679, 34.679]
-- !array_apply6 --
\N
\N
-- !array_repeat1 --
["hello", "hello"]
-- !array_repeat1 --
[123, 123]
-- !array_repeat1 --
[NULL, NULL]
-- !array_repeat1 --
[]

View File

@ -0,0 +1,51 @@
// 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("group_unique_array") {
sql 'set enable_nereids_planner=true'
sql 'set enable_fallback_to_original_planner=false'
sql "drop table if exists test_group_unique_array_table"
sql """
CREATE TABLE IF NOT EXISTS `test_group_unique_array_table` (
k1 int,
k2 date,
k3 varchar
) ENGINE=OLAP
DUPLICATE KEY(k1)
DISTRIBUTED BY HASH(k1) BUCKETS 1
PROPERTIES (
"replication_allocation" = "tag.location.default: 1",
"storage_format" = "V2"
)
"""
sql """
insert into test_group_unique_array_table values
(1, "2023-01-01", "hello"),
(2, "2023-01-01", null),
(2, "2023-01-02", "hello"),
(3, null, "world"),
(3, "2023-01-02", "hello"),
(4, "2023-01-02", "doris"),
(4, "2023-01-03", "sql")
"""
qt_1 """
select collect_set(k1),collect_set(k1,2) from test_group_unique_array_table;
"""
qt_2 """
select k1,collect_set(k2),collect_set(k3,1) from test_group_unique_array_table group by k1 order by k1;
"""
}

View File

@ -0,0 +1,30 @@
// 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("ip_functions") {
sql "set batch_size = 4096;"
qt_ip1 "SELECT ipv4numtostring(-1);"
qt_ip2 "SELECT ipv4numtostring(2130706433);"
qt_ip3 "SELECT ipv4numtostring(4294967298);"
qt_ip4 "SELECT ipv4numtostring(3232235521);"
qt_ip5 "SELECT inet_ntoa(-1);"
qt_ip6 "SELECT inet_ntoa(2130706433);"
qt_ip7 "SELECT inet_ntoa(4294967298);"
qt_ip8 "SELECT inet_ntoa(3232235521);"
}

View File

@ -1120,5 +1120,42 @@ suite("nereids_scalar_fn_Array") {
}
}
sql "DROP TABLE IF EXISTS test_array_with_scale_type_table"
sql """
CREATE TABLE IF NOT EXISTS `test_array_with_scale_type_table` (
`uid` int(11) NULL COMMENT "",
`c_datetimev2` datetimev2(3) NULL COMMENT "",
`c_decimal` decimal(8,3) NULL COMMENT "",
`c_decimalv3` decimalv3(8,3) NULL COMMENT "",
`c_array_datetimev2` ARRAY<datetimev2(3)> NULL COMMENT "",
`c_array_decimal` ARRAY<decimal(8,3)> NULL COMMENT "",
`c_array_decimalv3` ARRAY<decimalv3(8,3)> NULL COMMENT ""
) ENGINE=OLAP
DUPLICATE KEY(`uid`)
DISTRIBUTED BY HASH(`uid`) BUCKETS 1
PROPERTIES (
"replication_allocation" = "tag.location.default: 1",
"storage_format" = "V2"
)
"""
sql """INSERT INTO test_array_with_scale_type_table values
(1,"2022-12-01 22:23:24.999999",22.6789,33.6789,["2022-12-01 22:23:24.999999","2022-12-01 23:23:24.999999"],[22.6789,33.6789],[22.6789,33.6789]),
(2,"2022-12-02 22:23:24.999999",23.6789,34.6789,["2022-12-02 22:23:24.999999","2022-12-02 23:23:24.999999"],[23.6789,34.6789],[22.6789,34.6789])
"""
// array_apply
qt_array_apply1 """select array_apply(c_array_datetimev2, "=", '2022-12-02 22:23:24.999999') from test_array_with_scale_type_table"""
qt_array_apply2 """select array_apply(c_array_datetimev2, ">", '2022-12-01 22:23:24.999999') from test_array_with_scale_type_table"""
qt_array_apply3 """select array_apply(c_array_datetimev2, ">", null) from test_array_with_scale_type_table"""
qt_array_apply4 """select array_apply(c_array_decimal, "=", 22.679) from test_array_with_scale_type_table"""
qt_array_apply5 """select array_apply(c_array_decimal, ">=", 22.1) from test_array_with_scale_type_table"""
qt_array_apply6 """select array_apply(c_array_decimal, ">=", null) from test_array_with_scale_type_table"""
// array_repeat
qt_array_repeat1 """select array_repeat("hello", 2)"""
qt_array_repeat1 """select array_repeat(123, 2)"""
qt_array_repeat1 """select array_repeat(null, 2)"""
qt_array_repeat1 """select array_repeat(3, null)"""
}