diff --git a/be/src/vec/functions/hll_from_base64.cpp b/be/src/vec/functions/hll_from_base64.cpp new file mode 100644 index 00000000000000..4cfd9c7817c2cc --- /dev/null +++ b/be/src/vec/functions/hll_from_base64.cpp @@ -0,0 +1,111 @@ +// 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 +#include + +#include "olap/hll.h" +#include "util/url_coding.h" +#include "vec/columns/column_complex.h" +#include "vec/columns/column_nullable.h" +#include "vec/columns/column_string.h" +#include "vec/data_types/data_type.h" +#include "vec/data_types/data_type_hll.h" +#include "vec/functions/simple_function_factory.h" + +namespace doris::vectorized { + +class FunctionHllFromBase64 : public IFunction { +public: + static constexpr auto name = "hll_from_base64"; + + String get_name() const override { return name; } + + static FunctionPtr create() { return std::make_shared(); } + + DataTypePtr get_return_type_impl(const DataTypes& arguments) const override { + return make_nullable(std::make_shared()); + } + + size_t get_number_of_arguments() const override { return 1; } + + bool use_default_implementation_for_nulls() const override { return true; } + + Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments, + size_t result, size_t input_rows_count) const override { + auto res_null_map = ColumnUInt8::create(input_rows_count, 0); + auto res_data_column = ColumnHLL::create(); + auto& null_map = res_null_map->get_data(); + auto& res = res_data_column->get_data(); + + auto& argument_column = block.get_by_position(arguments[0]).column; + const auto& str_column = static_cast(*argument_column); + const ColumnString::Chars& data = str_column.get_chars(); + const ColumnString::Offsets& offsets = str_column.get_offsets(); + + res.reserve(input_rows_count); + + std::string decode_buff; + int last_decode_buff_len = 0; + int curr_decode_buff_len = 0; + for (size_t i = 0; i < input_rows_count; ++i) { + const char* src_str = reinterpret_cast(&data[offsets[i - 1]]); + int64_t src_size = offsets[i] - offsets[i - 1]; + + // Base64 encoding has a characteristic where every 4 characters represent 3 bytes of data. + // Here, we check if the length of the input string is a multiple of 4 to ensure it's a valid base64 encoded string. + if (0 != src_size % 4) { + res.emplace_back(); + null_map[i] = 1; + continue; + } + + // Allocate sufficient space for the decoded data. + // The number 3 here represents the number of bytes in the decoded data for each group of 4 base64 characters. + // We set the size of the decoding buffer to be 'src_size + 3' to ensure there is enough space to store the decoded data. + curr_decode_buff_len = src_size + 3; + if (curr_decode_buff_len > last_decode_buff_len) { + decode_buff.resize(curr_decode_buff_len); + last_decode_buff_len = curr_decode_buff_len; + } + auto outlen = base64_decode(src_str, src_size, decode_buff.data()); + if (outlen < 0) { + res.emplace_back(); + null_map[i] = 1; + } else { + doris::Slice decoded_slice(decode_buff.data(), outlen); + doris::HyperLogLog hll; + if (!hll.deserialize(decoded_slice)) { + return Status::RuntimeError( + fmt::format("hll_from_base64 decode failed: base64: {}", src_str)); + } else { + res.emplace_back(std::move(hll)); + } + } + } + + block.get_by_position(result).column = + ColumnNullable::create(std::move(res_data_column), std::move(res_null_map)); + return Status::OK(); + } +}; + +void register_function_hll_from_base64(SimpleFunctionFactory& factory) { + factory.register_function(); +} + +} // namespace doris::vectorized \ No newline at end of file diff --git a/be/src/vec/functions/hll_to_base64.cpp b/be/src/vec/functions/hll_to_base64.cpp new file mode 100644 index 00000000000000..30020361e32475 --- /dev/null +++ b/be/src/vec/functions/hll_to_base64.cpp @@ -0,0 +1,89 @@ +// 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 +#include + +#include "olap/hll.h" +#include "util/url_coding.h" +#include "vec/columns/column_complex.h" +#include "vec/columns/column_nullable.h" +#include "vec/columns/column_string.h" +#include "vec/data_types/data_type.h" +#include "vec/data_types/data_type_hll.h" +#include "vec/data_types/data_type_string.h" +#include "vec/functions/function_totype.h" +#include "vec/functions/simple_function_factory.h" + +namespace doris::vectorized { + +struct NameHllToBase64 { + static constexpr auto name = "hll_to_base64"; +}; + +struct HllToBase64 { + using ReturnType = DataTypeString; + static constexpr auto TYPE_INDEX = TypeIndex::HLL; + using Type = DataTypeHLL::FieldType; + using ReturnColumnType = ColumnString; + using Chars = ColumnString::Chars; + using Offsets = ColumnString::Offsets; + + static Status vector(const std::vector& data, Chars& chars, Offsets& offsets) { + size_t size = data.size(); + offsets.resize(size); + size_t output_char_size = 0; + for (size_t i = 0; i < size; ++i) { + auto& hll_val = const_cast(data[i]); + auto ser_size = hll_val.max_serialized_size(); + output_char_size += ser_size * (int)(4.0 * ceil((double)ser_size / 3.0)); + } + ColumnString::check_chars_length(output_char_size, size); + chars.resize(output_char_size); + auto chars_data = chars.data(); + + size_t cur_ser_size = 0; + size_t last_ser_size = 0; + std::string ser_buff; + size_t encoded_offset = 0; + for (size_t i = 0; i < size; ++i) { + auto& hll_val = const_cast(data[i]); + + cur_ser_size = hll_val.max_serialized_size(); + if (cur_ser_size > last_ser_size) { + last_ser_size = cur_ser_size; + ser_buff.resize(cur_ser_size); + } + hll_val.serialize(reinterpret_cast(ser_buff.data())); + auto outlen = base64_encode((const unsigned char*)ser_buff.data(), cur_ser_size, + chars_data + encoded_offset); + DCHECK(outlen > 0); + + encoded_offset += outlen; + offsets[i] = encoded_offset; + } + return Status::OK(); + } +}; + +using FunctionHllToBase64 = FunctionUnaryToType; + +void register_function_hll_to_base64(SimpleFunctionFactory& factory) { + factory.register_function(); +} + +} // namespace doris::vectorized \ No newline at end of file diff --git a/be/src/vec/functions/simple_function_factory.h b/be/src/vec/functions/simple_function_factory.h index 649db732093934..793632e8fefcd0 100644 --- a/be/src/vec/functions/simple_function_factory.h +++ b/be/src/vec/functions/simple_function_factory.h @@ -35,7 +35,9 @@ void register_function_comparison(SimpleFunctionFactory& factory); void register_function_comparison_eq_for_null(SimpleFunctionFactory& factory); void register_function_hll_cardinality(SimpleFunctionFactory& factory); void register_function_hll_empty(SimpleFunctionFactory& factory); +void register_function_hll_from_base64(SimpleFunctionFactory& factory); void register_function_hll_hash(SimpleFunctionFactory& factory); +void register_function_hll_to_base64(SimpleFunctionFactory& factory); void register_function_logical(SimpleFunctionFactory& factory); void register_function_case(SimpleFunctionFactory& factory); void register_function_cast(SimpleFunctionFactory& factory); @@ -222,7 +224,9 @@ class SimpleFunctionFactory { register_function_bitmap_variadic(instance); register_function_hll_cardinality(instance); register_function_hll_empty(instance); + register_function_hll_from_base64(instance); register_function_hll_hash(instance); + register_function_hll_to_base64(instance); register_function_comparison(instance); register_function_logical(instance); register_function_case(instance); 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 1654a2098db012..eb7a2c6050ce6c 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 @@ -186,7 +186,9 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.Hex; import org.apache.doris.nereids.trees.expressions.functions.scalar.HllCardinality; import org.apache.doris.nereids.trees.expressions.functions.scalar.HllEmpty; +import org.apache.doris.nereids.trees.expressions.functions.scalar.HllFromBase64; import org.apache.doris.nereids.trees.expressions.functions.scalar.HllHash; +import org.apache.doris.nereids.trees.expressions.functions.scalar.HllToBase64; import org.apache.doris.nereids.trees.expressions.functions.scalar.Hour; import org.apache.doris.nereids.trees.expressions.functions.scalar.HourCeil; import org.apache.doris.nereids.trees.expressions.functions.scalar.HourFloor; @@ -617,7 +619,9 @@ public class BuiltinScalarFunctions implements FunctionHelper { scalar(Hex.class, "hex"), scalar(HllCardinality.class, "hll_cardinality"), scalar(HllEmpty.class, "hll_empty"), + scalar(HllFromBase64.class, "hll_from_base64"), scalar(HllHash.class, "hll_hash"), + scalar(HllToBase64.class, "hll_to_base64"), scalar(Hour.class, "hour"), scalar(HourCeil.class, "hour_ceil"), scalar(HourFloor.class, "hour_floor"), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HllFromBase64.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HllFromBase64.java new file mode 100644 index 00000000000000..23d682e8ea78c0 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HllFromBase64.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.UnaryExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.HllType; +import org.apache.doris.nereids.types.StringType; +import org.apache.doris.nereids.types.VarcharType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * ScalarFunction 'hll_from_string'. + */ +public class HllFromBase64 extends ScalarFunction + implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNullable { + + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(HllType.INSTANCE).args(VarcharType.SYSTEM_DEFAULT), + FunctionSignature.ret(HllType.INSTANCE).args(StringType.INSTANCE) + ); + + /** + * constructor with 1 argument. + */ + public HllFromBase64(Expression arg) { + super("hll_from_base64", arg); + } + + /** + * withChildren. + */ + @Override + public HllFromBase64 withChildren(List children) { + Preconditions.checkArgument(children.size() == 1); + return new HllFromBase64(children.get(0)); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitHllFromBase64(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HllToBase64.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HllToBase64.java new file mode 100644 index 00000000000000..14b58b9a6e69ef --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HllToBase64.java @@ -0,0 +1,69 @@ +// 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.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable; +import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.HllType; +import org.apache.doris.nereids.types.StringType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * ScalarFunction 'hll_to_base64'. + */ +public class HllToBase64 extends ScalarFunction + implements UnaryExpression, ExplicitlyCastableSignature, PropagateNullable { + + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(StringType.INSTANCE).args(HllType.INSTANCE) + ); + + /** + * constructor with 1 argument. + */ + public HllToBase64(Expression arg) { + super("hll_to_base64", arg); + } + + /** + * withChildren. + */ + @Override + public HllToBase64 withChildren(List children) { + Preconditions.checkArgument(children.size() == 1); + return new HllToBase64(children.get(0)); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitHllToBase64(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 cb26a8bf4dc26b..b8d2ccfbf4f3d0 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 @@ -189,7 +189,9 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.Hex; import org.apache.doris.nereids.trees.expressions.functions.scalar.HllCardinality; import org.apache.doris.nereids.trees.expressions.functions.scalar.HllEmpty; +import org.apache.doris.nereids.trees.expressions.functions.scalar.HllFromBase64; import org.apache.doris.nereids.trees.expressions.functions.scalar.HllHash; +import org.apache.doris.nereids.trees.expressions.functions.scalar.HllToBase64; import org.apache.doris.nereids.trees.expressions.functions.scalar.Hour; import org.apache.doris.nereids.trees.expressions.functions.scalar.HourCeil; import org.apache.doris.nereids.trees.expressions.functions.scalar.HourFloor; @@ -1161,10 +1163,18 @@ default R visitHllEmpty(HllEmpty hllEmpty, C context) { return visitScalarFunction(hllEmpty, context); } + default R visitHllFromBase64(HllFromBase64 hllFromBase64, C context) { + return visitScalarFunction(hllFromBase64, context); + } + default R visitHllHash(HllHash hllHash, C context) { return visitScalarFunction(hllHash, context); } + default R visitHllToBase64(HllToBase64 hllToBase64, C context) { + return visitScalarFunction(hllToBase64, context); + } + default R visitHour(Hour hour, C context) { return visitScalarFunction(hour, context); } diff --git a/gensrc/script/doris_builtins_functions.py b/gensrc/script/doris_builtins_functions.py index fdc087553076ea..841799f0a13b85 100644 --- a/gensrc/script/doris_builtins_functions.py +++ b/gensrc/script/doris_builtins_functions.py @@ -1823,9 +1823,12 @@ #hll function "Hll": [ [['hll_cardinality'], 'BIGINT', ['HLL'], 'ALWAYS_NOT_NULLABLE'], - [['hll_hash'], 'HLL', ['VARCHAR'], 'ALWAYS_NOT_NULLABLE'], [['hll_empty'], 'HLL', [], 'ALWAYS_NOT_NULLABLE'], - [['hll_hash'], 'HLL', ['STRING'], 'ALWAYS_NOT_NULLABLE'] + [['hll_from_base64'], 'HLL', ['VARCHAR'], 'ALWAYS_NULLABLE'], + [['hll_from_base64'], 'HLL', ['STRING'], 'ALWAYS_NULLABLE'], + [['hll_hash'], 'HLL', ['VARCHAR'], 'ALWAYS_NOT_NULLABLE'], + [['hll_hash'], 'HLL', ['STRING'], 'ALWAYS_NOT_NULLABLE'], + [['hll_to_base64'], 'STRING', ['HLL'], ''] ], diff --git a/regression-test/data/nereids_function_p0/scalar_function/H.out b/regression-test/data/nereids_function_p0/scalar_function/H.out index 08425584180782..fa6e2caa7e4478 100644 --- a/regression-test/data/nereids_function_p0/scalar_function/H.out +++ b/regression-test/data/nereids_function_p0/scalar_function/H.out @@ -86,6 +86,64 @@ C 737472696E6733 737472696E6733 +-- !sql_hll_from_base64_1 -- +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 + +-- !sql_hll_from_base64_notnull_1 -- +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 + +-- !sql_hll_from_base64_2 -- +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 + +-- !sql_hll_from_base64_notnull_2 -- +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 + -- !sql_hll_hash_Varchar -- \N \N @@ -144,6 +202,64 @@ C \N \N +-- !sql_hll_to_base64_1 -- +AQEsDw6y1DKvWA== +AQEBHCni6iZ4IA== +AQEBHCni6iZ4IA== +AQEBHCni6iZ4IA== +AQEBHCni6iZ4IA== +AQHDvC2tP+3Ztg== +AQHDvC2tP+3Ztg== +AQHDvC2tP+3Ztg== +AQHDvC2tP+3Ztg== +AQGGjGuwPQuJSg== +AQGGjGuwPQuJSg== +AQGGjGuwPQuJSg== +AQGGjGuwPQuJSg== + +-- !sql_hll_to_base64_notnull_1 -- +AQEBHCni6iZ4IA== +AQEBHCni6iZ4IA== +AQEBHCni6iZ4IA== +AQEBHCni6iZ4IA== +AQHDvC2tP+3Ztg== +AQHDvC2tP+3Ztg== +AQHDvC2tP+3Ztg== +AQHDvC2tP+3Ztg== +AQGGjGuwPQuJSg== +AQGGjGuwPQuJSg== +AQGGjGuwPQuJSg== +AQGGjGuwPQuJSg== + +-- !sql_hll_to_base64_2 -- +AQEVuTIP+2DCFQ== +AQEVuTIP+2DCFQ== +AQEVuTIP+2DCFQ== +AQEVuTIP+2DCFQ== +AQFrObf9ClSMJg== +AQFrObf9ClSMJg== +AQFrObf9ClSMJg== +AQFrObf9ClSMJg== +AQGO5rMmsc7hDA== +AQGO5rMmsc7hDA== +AQGO5rMmsc7hDA== +AQGO5rMmsc7hDA== + +-- !sql_hll_to_base64_notnull_2 -- +AQEsDw6y1DKvWA== +AQEVuTIP+2DCFQ== +AQEVuTIP+2DCFQ== +AQEVuTIP+2DCFQ== +AQEVuTIP+2DCFQ== +AQFrObf9ClSMJg== +AQFrObf9ClSMJg== +AQFrObf9ClSMJg== +AQFrObf9ClSMJg== +AQGO5rMmsc7hDA== +AQGO5rMmsc7hDA== +AQGO5rMmsc7hDA== +AQGO5rMmsc7hDA== + -- !sql_hour_DateTime -- \N 1 diff --git a/regression-test/data/query_p0/sql_functions/hll_functions/test_hll_functions.out b/regression-test/data/query_p0/sql_functions/hll_functions/test_hll_functions.out new file mode 100644 index 00000000000000..c9cebac5079798 --- /dev/null +++ b/regression-test/data/query_p0/sql_functions/hll_functions/test_hll_functions.out @@ -0,0 +1,35 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !table_select -- +3 + +-- !table_select -- +beijing 3 +hebei 1 +jiangsu 1 +shanghai 2 +shanxi 1 + +-- !table_select -- +1 + +-- !const_select -- +1 + +-- !const_select -- +1 + +-- !const_select -- +0 + +-- !const_select -- +\N + +-- !const_select -- +AA== + +-- !const_select -- +AQEC5XSzrpDsdw== + +-- !const_select -- +AQEyl7yFZerf2A== + diff --git a/regression-test/suites/nereids_function_p0/scalar_function/H.groovy b/regression-test/suites/nereids_function_p0/scalar_function/H.groovy index 2c9ee4bc2eaab6..08d6f9f92baf59 100644 --- a/regression-test/suites/nereids_function_p0/scalar_function/H.groovy +++ b/regression-test/suites/nereids_function_p0/scalar_function/H.groovy @@ -25,10 +25,18 @@ suite("nereids_scalar_fn_H") { qt_sql_hex_Varchar_notnull "select hex(kvchrs1) from fn_test_not_nullable order by kvchrs1" qt_sql_hex_String "select hex(kstr) from fn_test order by kstr" qt_sql_hex_String_notnull "select hex(kstr) from fn_test_not_nullable order by kstr" + qt_sql_hll_from_base64_1 "select hll_cardinality(hll_from_base64(hll_to_base64(hll_hash(kvchrs1)))) from fn_test order by kvchrs1" + qt_sql_hll_from_base64_notnull_1 "select hll_cardinality(hll_from_base64(hll_to_base64(hll_hash(kvchrs1)))) from fn_test_not_nullable order by kvchrs1" + qt_sql_hll_from_base64_2 "select hll_cardinality(hll_from_base64(hll_to_base64(hll_hash(kstr)))) from fn_test_not_nullable order by kstr" + qt_sql_hll_from_base64_notnull_2 "select hll_cardinality(hll_from_base64(hll_to_base64(hll_hash(kstr)))) from fn_test order by kstr" qt_sql_hll_hash_Varchar "select hll_hash(kvchrs1) from fn_test order by kvchrs1" qt_sql_hll_hash_Varchar_notnull "select hll_hash(kvchrs1) from fn_test_not_nullable order by kvchrs1" qt_sql_hll_hash_String "select hll_hash(kstr) from fn_test order by kstr" qt_sql_hll_hash_String_notnull "select hll_hash(kstr) from fn_test_not_nullable order by kstr" + qt_sql_hll_to_base64_1 "select hll_to_base64(hll_hash(kvchrs1)) from fn_test order by kvchrs1" + qt_sql_hll_to_base64_notnull_1 "select hll_to_base64(hll_hash(kvchrs1)) from fn_test_not_nullable order by kvchrs1" + qt_sql_hll_to_base64_2 "select hll_to_base64(hll_hash(kstr)) from fn_test_not_nullable order by kstr" + qt_sql_hll_to_base64_notnull_2 "select hll_to_base64(hll_hash(kstr)) from fn_test order by kstr" qt_sql_hour_DateTime "select hour(kdtm) from fn_test order by kdtm" qt_sql_hour_DateTime_notnull "select hour(kdtm) from fn_test_not_nullable order by kdtm" qt_sql_hour_DateTimeV2 "select hour(kdtmv2s1) from fn_test order by kdtmv2s1" diff --git a/regression-test/suites/query_p0/sql_functions/hll_functions/test_hll_functions.groovy b/regression-test/suites/query_p0/sql_functions/hll_functions/test_hll_functions.groovy new file mode 100644 index 00000000000000..87a5f5360d0e2a --- /dev/null +++ b/regression-test/suites/query_p0/sql_functions/hll_functions/test_hll_functions.groovy @@ -0,0 +1,72 @@ +// 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_hll_functions") { + sql """drop TABLE if EXISTS test_hll_func;""" + sql """ + create table test_hll_func( + dt date, + id int, + name char(10), + province char(10), + os char(10), + pv hll hll_union + ) + Aggregate KEY (dt,id,name,province,os) + distributed by hash(id) buckets 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "storage_format" = "V2" + ) + """ + sql """ + insert into test_hll_func + SELECT + dt,id,name,province,os,pv + from ( + SELECT '2022-05-05' as dt,'10001' as id,'test01' as name,'beijing' as province,'windows' as os,hll_hash('windows') as pv + union all + SELECT '2022-05-05' as dt,'10002' as id,'test01' as name,'beijing' as province,'linux' as os,hll_hash('linux') as pv + union all + SELECT '2022-05-05' as dt,'10003' as id,'test01' as name,'beijing' as province,'macos' as os,hll_hash('macos') as pv + union all + SELECT '2022-05-05' as dt,'10004' as id,'test01' as name,'hebei' as province,'windows' as os,hll_hash('windows') as pv + union all + SELECT '2022-05-06' as dt,'10001' as id,'test01' as name,'shanghai' as province,'windows' as os,hll_hash('windows') as pv + union all + SELECT '2022-05-06' as dt,'10002' as id,'test01' as name,'shanghai' as province,'linux' as os,hll_hash('linux') as pv + union all + SELECT '2022-05-06' as dt,'10003' as id,'test01' as name,'jiangsu' as province,'macos' as os,hll_hash('macos') as pv + union all + SELECT '2022-05-06' as dt,'10004' as id,'test01' as name,'shanxi' as province,'windows' as os,hll_hash('windows') as pv + union all + SELECT '2022-05-07' as dt,'10005' as id,'test01' as name,'shanxi' as province,'windows' as os,hll_empty() as pv + ) as a + """ + + qt_table_select "select hll_union_agg(hll_from_base64(hll_to_base64(pv))) from test_hll_func;" + qt_table_select "select province, hll_union_agg(hll_from_base64(hll_to_base64(pv))) from test_hll_func group by province order by province;" + qt_table_select "select hll_cardinality(hll_from_base64(hll_to_base64(pv))) from test_hll_func limit 1;" + + qt_const_select "select hll_cardinality(hll_from_base64(hll_to_base64(hll_hash('abc'))));" + qt_const_select "select hll_cardinality(hll_from_base64(hll_to_base64(hll_hash(''))));" + qt_const_select "select hll_cardinality(hll_from_base64(hll_to_base64(hll_hash(NULL))));" + qt_const_select "select hll_to_base64(NULL);" + qt_const_select "select hll_to_base64(hll_empty());" + qt_const_select "select hll_to_base64(hll_hash('abc'));" + qt_const_select "select hll_to_base64(hll_hash(''));" +}