diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinAggregateFunctions.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinAggregateFunctions.java index 0e361afb790f3b..0b813fcd89b917 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinAggregateFunctions.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinAggregateFunctions.java @@ -84,7 +84,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"), diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java index 789b7f760ee7d4..0c7bfa97c308b5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java @@ -27,6 +27,7 @@ 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.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; @@ -181,6 +183,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.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; @@ -410,6 +413,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"), @@ -436,6 +440,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"), @@ -561,6 +566,7 @@ public class BuiltinScalarFunctions implements FunctionHelper { scalar(If.class, "if"), scalar(Initcap.class, "initcap"), 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"), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ArrayApply.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ArrayApply.java new file mode 100644 index 00000000000000..82bad4e4868cc1 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ArrayApply.java @@ -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 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 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 accept(ExpressionVisitor visitor, C context) { + return visitor.visitArrayApply(this, context); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ArrayRepeat.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ArrayRepeat.java new file mode 100644 index 00000000000000..7596251a60fa97 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ArrayRepeat.java @@ -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 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 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 accept(ExpressionVisitor visitor, C context) { + return visitor.visitArrayRepeat(this, context); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Ipv4NumToString.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Ipv4NumToString.java new file mode 100644 index 00000000000000..afca51cdc95707 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Ipv4NumToString.java @@ -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 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 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 getSignatures() { + return SIGNATURES; + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitIpv4NumToString(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java index 111116af0eb4e5..6fe72a60a6ea52 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java @@ -29,6 +29,7 @@ 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.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; @@ -177,6 +179,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.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; @@ -420,6 +423,10 @@ default R visitArray(Array array, C context) { 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); } @@ -528,6 +535,10 @@ default R visitArrayMap(ArrayMap arraySort, C context) { 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); } @@ -1052,6 +1063,10 @@ default R visitInstr(Instr instr, C context) { 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); } diff --git a/gensrc/script/doris_builtins_functions.py b/gensrc/script/doris_builtins_functions.py index 8cbb924c358329..41c2e50d586cfe 100644 --- a/gensrc/script/doris_builtins_functions.py +++ b/gensrc/script/doris_builtins_functions.py @@ -493,7 +493,7 @@ [['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 @@ [['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'], ''], diff --git a/regression-test/data/nereids_function_p0/agg_function/group_unique_array.out b/regression-test/data/nereids_function_p0/agg_function/group_unique_array.out new file mode 100644 index 00000000000000..f4c9f38959b312 --- /dev/null +++ b/regression-test/data/nereids_function_p0/agg_function/group_unique_array.out @@ -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"] + diff --git a/regression-test/data/nereids_function_p0/ip_functions.out b/regression-test/data/nereids_function_p0/ip_functions.out new file mode 100644 index 00000000000000..fbe1c8cc6dbb86 --- /dev/null +++ b/regression-test/data/nereids_function_p0/ip_functions.out @@ -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 + diff --git a/regression-test/data/nereids_function_p0/scalar_function/Array.out b/regression-test/data/nereids_function_p0/scalar_function/Array.out index df91f9f1770521..b473c8cee9769f 100644 --- a/regression-test/data/nereids_function_p0/scalar_function/Array.out +++ b/regression-test/data/nereids_function_p0/scalar_function/Array.out @@ -12169,3 +12169,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 -- +[] + diff --git a/regression-test/suites/nereids_function_p0/agg_function/group_unique_array.groovy b/regression-test/suites/nereids_function_p0/agg_function/group_unique_array.groovy new file mode 100644 index 00000000000000..f17eadf73a55f3 --- /dev/null +++ b/regression-test/suites/nereids_function_p0/agg_function/group_unique_array.groovy @@ -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; + """ +} \ No newline at end of file diff --git a/regression-test/suites/nereids_function_p0/ip_functions.groovy b/regression-test/suites/nereids_function_p0/ip_functions.groovy new file mode 100644 index 00000000000000..8ec88569ce68b7 --- /dev/null +++ b/regression-test/suites/nereids_function_p0/ip_functions.groovy @@ -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);" + +} \ No newline at end of file diff --git a/regression-test/suites/nereids_function_p0/scalar_function/Array.groovy b/regression-test/suites/nereids_function_p0/scalar_function/Array.groovy index 8bd097bf56dd3c..3550ef1942e251 100644 --- a/regression-test/suites/nereids_function_p0/scalar_function/Array.groovy +++ b/regression-test/suites/nereids_function_p0/scalar_function/Array.groovy @@ -1056,5 +1056,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 NULL COMMENT "", + `c_array_decimal` ARRAY NULL COMMENT "", + `c_array_decimalv3` ARRAY 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)""" } \ No newline at end of file