From d6fcb38eb2f10a627712ab424be2e26de954cca1 Mon Sep 17 00:00:00 2001 From: chesterxu Date: Sun, 10 Mar 2024 20:04:14 +0800 Subject: [PATCH 1/6] add hll_from_base64 fix --- be/src/vec/functions/hll_from_base64.cpp | 164 ++++++++++++++++++ .../doris/catalog/BuiltinScalarFunctions.java | 2 + .../functions/scalar/HllFromBase64.java | 71 ++++++++ .../visitor/ScalarFunctionVisitor.java | 5 + gensrc/script/doris_builtins_functions.py | 2 + 5 files changed, 244 insertions(+) create mode 100644 be/src/vec/functions/hll_from_base64.cpp create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HllFromBase64.java 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..3ec348d006d135 --- /dev/null +++ b/be/src/vec/functions/hll_from_base64.cpp @@ -0,0 +1,164 @@ +// 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_array.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_array.h" +#include "vec/data_types/data_type_hll.h" +#include "vec/data_types/data_type_string.h" +#include "vec/functions/simple_function_factory.h" + +namespace doris::vectorized { + +struct HllFromBase64 { + using ArgumentType = DataTypeString; + + static constexpr auto name = "hll_from_base64"; + + static Status vector(const ColumnString::Chars& data, const ColumnString::Offsets& offsets, + std::vector& res, NullMap& null_map, + size_t input_rows_count) { + res.reserve(input_rows_count); + if (offsets.size() == 0 && input_rows_count == 1) { + // For NULL constant + res.emplace_back(); + null_map[0] = 1; + return Status::OK(); + } + 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]; + if (0 != src_size % 4) { + res.emplace_back(); + null_map[i] = 1; + continue; + } + 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; + } + int 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)); + } + } + } + return Status::OK(); + } +}; + +template +class FunctionHllAlwaysNull : public IFunction { +public: + static constexpr auto name = Impl::name; + + 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(); + + ColumnPtr& argument_column = block.get_by_position(arguments[0]).column; + if constexpr (std::is_same_v) { + const auto& str_column = static_cast(*argument_column); + const ColumnString::Chars& data = str_column.get_chars(); + const ColumnString::Offsets& offsets = str_column.get_offsets(); + RETURN_IF_ERROR(Impl::vector(data, offsets, res, null_map, input_rows_count)); + } else if constexpr (std::is_same_v) { + auto argument_type = remove_nullable( + assert_cast(*block.get_by_position(arguments[0]).type) + .get_nested_type()); + const auto& array_column = static_cast(*argument_column); + const auto& offset_column_data = array_column.get_offsets(); + const auto& nested_nullable_column = + static_cast(array_column.get_data()); + const auto& nested_column = nested_nullable_column.get_nested_column(); + const auto& nested_null_map = nested_nullable_column.get_null_map_column().get_data(); + + WhichDataType which_type(argument_type); + if (which_type.is_int8()) { + RETURN_IF_ERROR(Impl::template vector(offset_column_data, nested_column, + nested_null_map, res, null_map)); + } else if (which_type.is_uint8()) { + RETURN_IF_ERROR(Impl::template vector( + offset_column_data, nested_column, nested_null_map, res, null_map)); + } else if (which_type.is_int16()) { + RETURN_IF_ERROR(Impl::template vector( + offset_column_data, nested_column, nested_null_map, res, null_map)); + } else if (which_type.is_int32()) { + RETURN_IF_ERROR(Impl::template vector( + offset_column_data, nested_column, nested_null_map, res, null_map)); + } else if (which_type.is_int64()) { + RETURN_IF_ERROR(Impl::template vector( + offset_column_data, nested_column, nested_null_map, res, null_map)); + } else { + return Status::RuntimeError("Illegal column {} of argument of function {}", + block.get_by_position(arguments[0]).column->get_name(), + get_name()); + } + } else { + return Status::RuntimeError("Illegal column {} of argument of function {}", + block.get_by_position(arguments[0]).column->get_name(), + get_name()); + } + block.get_by_position(result).column = + ColumnNullable::create(std::move(res_data_column), std::move(res_null_map)); + return Status::OK(); + } +}; + +using FunctionHllFromBase64 = FunctionHllAlwaysNull; + +void register_function_hll_from_base64(SimpleFunctionFactory& factory) { + factory.register_function(); +} + +} // namespace doris::vectorized \ No newline at end of file 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 9d19ea9e2e9923..3975df22d94c97 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,6 +186,7 @@ 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.Hour; import org.apache.doris.nereids.trees.expressions.functions.scalar.HourCeil; @@ -613,6 +614,7 @@ 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(Hour.class, "hour"), scalar(HourCeil.class, "hour_ceil"), 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/visitor/ScalarFunctionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java index 7cef47557cc525..1094b9e7eee607 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,6 +189,7 @@ 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.Hour; import org.apache.doris.nereids.trees.expressions.functions.scalar.HourCeil; @@ -1157,6 +1158,10 @@ 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); } diff --git a/gensrc/script/doris_builtins_functions.py b/gensrc/script/doris_builtins_functions.py index 629f83f6a13cef..c0d29320fa5fbb 100644 --- a/gensrc/script/doris_builtins_functions.py +++ b/gensrc/script/doris_builtins_functions.py @@ -1821,6 +1821,8 @@ [['hll_cardinality'], 'BIGINT', ['HLL'], 'ALWAYS_NOT_NULLABLE'], [['hll_hash'], 'HLL', ['VARCHAR'], 'ALWAYS_NOT_NULLABLE'], [['hll_empty'], 'HLL', [], 'ALWAYS_NOT_NULLABLE'], + [['hll_from_base64'], 'HLL', ['VARCHAR'], 'ALWAYS_NULLABLE'], + [['hll_from_base64'], 'HLL', ['STRING'], 'ALWAYS_NULLABLE'], [['hll_hash'], 'HLL', ['STRING'], 'ALWAYS_NOT_NULLABLE'] ], From ad49443c83e32fbbe5903d436de3cb1885dd690d Mon Sep 17 00:00:00 2001 From: chesterxu Date: Mon, 11 Mar 2024 11:26:24 +0800 Subject: [PATCH 2/6] add hll_to_base64 --- be/src/vec/functions/hll_to_base64.cpp | 89 +++++++++++++++++++ .../vec/functions/simple_function_factory.h | 4 + docs/sidebars.json | 2 + .../doris/catalog/BuiltinScalarFunctions.java | 2 + .../functions/scalar/HllToBase64.java | 69 ++++++++++++++ .../visitor/ScalarFunctionVisitor.java | 5 ++ gensrc/script/doris_builtins_functions.py | 5 +- 7 files changed, 174 insertions(+), 2 deletions(-) create mode 100644 be/src/vec/functions/hll_to_base64.cpp create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HllToBase64.java 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..f80eedbbe6b2a7 --- /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) { + HyperLogLog& 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) { + HyperLogLog& 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())); + int outlen = base64_encode((const unsigned char*)ser_buff.data(), cur_ser_size, + chars_data + encoded_offset); + DCHECK(outlen > 0); + + encoded_offset += (int)(4.0 * ceil((double)cur_ser_size / 3.0)); + 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 b1c1b394bffa41..d737ec09d4de58 100644 --- a/be/src/vec/functions/simple_function_factory.h +++ b/be/src/vec/functions/simple_function_factory.h @@ -37,7 +37,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); @@ -223,7 +225,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/docs/sidebars.json b/docs/sidebars.json index 8a9baf9efeb1d4..1099863f9dc323 100644 --- a/docs/sidebars.json +++ b/docs/sidebars.json @@ -737,7 +737,9 @@ "items": [ "sql-manual/sql-functions/hll-functions/hll-cardinality", "sql-manual/sql-functions/hll-functions/hll-empty", + "sql-manual/sql-functions/hll-functions/hll-from_base64", "sql-manual/sql-functions/hll-functions/hll-hash" + "sql-manual/sql-functions/hll-functions/hll-to_base64" ] }, { 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 3975df22d94c97..fcd436b23f4ef1 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 @@ -188,6 +188,7 @@ 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; @@ -616,6 +617,7 @@ public class BuiltinScalarFunctions implements FunctionHelper { 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/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 1094b9e7eee607..dbae72f66cbbc1 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 @@ -191,6 +191,7 @@ 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; @@ -1166,6 +1167,10 @@ 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 c0d29320fa5fbb..5f01c52f693024 100644 --- a/gensrc/script/doris_builtins_functions.py +++ b/gensrc/script/doris_builtins_functions.py @@ -1819,11 +1819,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_from_base64'], 'HLL', ['VARCHAR'], 'ALWAYS_NULLABLE'], [['hll_from_base64'], 'HLL', ['STRING'], 'ALWAYS_NULLABLE'], - [['hll_hash'], 'HLL', ['STRING'], 'ALWAYS_NOT_NULLABLE'] + [['hll_hash'], 'HLL', ['VARCHAR'], 'ALWAYS_NOT_NULLABLE'], + [['hll_hash'], 'HLL', ['STRING'], 'ALWAYS_NOT_NULLABLE'], + [['hll_to_base64'], 'STRING', ['HLL'], ''] ], From f0ff7da6656625fd77d0ee700ac0f4d48a53469e Mon Sep 17 00:00:00 2001 From: chesterxu Date: Tue, 12 Mar 2024 11:28:47 +0800 Subject: [PATCH 3/6] add docs and tests --- .../hll-functions/hll-from-base64.md | 72 +++++++++++ .../hll-functions/hll-to-base64.md | 80 ++++++++++++ .../hll-functions/hll-from-base64.md | 72 +++++++++++ .../hll-functions/hll-to-base64.md | 80 ++++++++++++ .../nereids_function_p0/scalar_function/H.out | 116 ++++++++++++++++++ .../hll_functions/test_hll_functions.out | 35 ++++++ .../scalar_function/H.groovy | 8 ++ .../hll_functions/test_hll_functions.groovy | 72 +++++++++++ 8 files changed, 535 insertions(+) create mode 100644 docs/en/docs/sql-manual/sql-functions/hll-functions/hll-from-base64.md create mode 100644 docs/en/docs/sql-manual/sql-functions/hll-functions/hll-to-base64.md create mode 100644 docs/zh-CN/docs/sql-manual/sql-functions/hll-functions/hll-from-base64.md create mode 100644 docs/zh-CN/docs/sql-manual/sql-functions/hll-functions/hll-to-base64.md create mode 100644 regression-test/data/query_p0/sql_functions/hll_functions/test_hll_functions.out create mode 100644 regression-test/suites/query_p0/sql_functions/hll_functions/test_hll_functions.groovy diff --git a/docs/en/docs/sql-manual/sql-functions/hll-functions/hll-from-base64.md b/docs/en/docs/sql-manual/sql-functions/hll-functions/hll-from-base64.md new file mode 100644 index 00000000000000..b04c6c4531786e --- /dev/null +++ b/docs/en/docs/sql-manual/sql-functions/hll-functions/hll-from-base64.md @@ -0,0 +1,72 @@ +--- +{ + "title": "HLL_FROM_BASE64", + "language": "zh-CN" +} +--- + + + +## hll_from_base64 + +### description +#### Syntax + +`HLL HLL_FROM_BASE64(VARCHAR input)` + +Convert a base64 string(result of function `hll_to_base64`) into a hll. If input string is invalid, return NULL. + +### example + +``` +mysql> select hll_union_agg(hll_from_base64(hll_to_base64(pv))), hll_union_agg(pv) from test_hll; ++---------------------------------------------------+-------------------+ +| hll_union_agg(hll_from_base64(hll_to_base64(pv))) | hll_union_agg(pv) | ++---------------------------------------------------+-------------------+ +| 3 | 3 | ++---------------------------------------------------+-------------------+ +1 row in set (0.04 sec) + +mysql> select hll_cardinality(hll_from_base64(hll_to_base64(hll_hash('abc')))); ++------------------------------------------------------------------+ +| hll_cardinality(hll_from_base64(hll_to_base64(hll_hash('abc')))) | ++------------------------------------------------------------------+ +| 1 | ++------------------------------------------------------------------+ +1 row in set (0.04 sec) + +mysql> select hll_cardinality(hll_from_base64(hll_to_base64(hll_hash('')))); ++---------------------------------------------------------------+ +| hll_cardinality(hll_from_base64(hll_to_base64(hll_hash('')))) | ++---------------------------------------------------------------+ +| 1 | ++---------------------------------------------------------------+ +1 row in set (0.02 sec) + +mysql> select hll_cardinality(hll_from_base64(hll_to_base64(hll_hash(NULL)))); ++-----------------------------------------------------------------+ +| hll_cardinality(hll_from_base64(hll_to_base64(hll_hash(NULL)))) | ++-----------------------------------------------------------------+ +| 0 | ++-----------------------------------------------------------------+ +1 row in set (0.02 sec) +``` + +### keywords + + HLL_FROM_BASE64,HLL diff --git a/docs/en/docs/sql-manual/sql-functions/hll-functions/hll-to-base64.md b/docs/en/docs/sql-manual/sql-functions/hll-functions/hll-to-base64.md new file mode 100644 index 00000000000000..7e5701558fea22 --- /dev/null +++ b/docs/en/docs/sql-manual/sql-functions/hll-functions/hll-to-base64.md @@ -0,0 +1,80 @@ +--- +{ + "title": "HLL_TO_BASE64", + "language": "en" +} +--- + + + +## hll_to_base64 + +### description +#### Syntax + +`VARCHAR HLL_TO_BASE64(HLL input)` + +Convert an input hll to a base64 string. If input is NULL, return NULL. + +### example + +``` +mysql> select hll_to_base64(NULL); ++---------------------+ +| hll_to_base64(NULL) | ++---------------------+ +| NULL | ++---------------------+ +1 row in set (0.00 sec) + +mysql> select hll_to_base64(hll_empty()); ++----------------------------+ +| hll_to_base64(hll_empty()) | ++----------------------------+ +| AA== | ++----------------------------+ +1 row in set (0.02 sec) + +mysql> select hll_to_base64(hll_hash('abc')); ++--------------------------------+ +| hll_to_base64(hll_hash('abc')) | ++--------------------------------+ +| AQEC5XSzrpDsdw== | ++--------------------------------+ +1 row in set (0.03 sec) + +mysql> select hll_union_agg(hll_from_base64(hll_to_base64(pv))), hll_union_agg(pv) from test_hll; ++---------------------------------------------------+-------------------+ +| hll_union_agg(hll_from_base64(hll_to_base64(pv))) | hll_union_agg(pv) | ++---------------------------------------------------+-------------------+ +| 3 | 3 | ++---------------------------------------------------+-------------------+ +1 row in set (0.04 sec) + +mysql> select hll_cardinality(hll_from_base64(hll_to_base64(hll_hash('abc')))); ++------------------------------------------------------------------+ +| hll_cardinality(hll_from_base64(hll_to_base64(hll_hash('abc')))) | ++------------------------------------------------------------------+ +| 1 | ++------------------------------------------------------------------+ +1 row in set (0.04 sec) +``` + +### keywords + + HLL_TO_BASE64,HLL diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/hll-functions/hll-from-base64.md b/docs/zh-CN/docs/sql-manual/sql-functions/hll-functions/hll-from-base64.md new file mode 100644 index 00000000000000..4dc66cf4118d66 --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-functions/hll-functions/hll-from-base64.md @@ -0,0 +1,72 @@ +--- +{ + "title": "HLL_FROM_BASE64", + "language": "zh-CN" +} +--- + + + +## hll_from_base64 + +### description +#### Syntax + +`HLL HLL_FROM_BASE64(VARCHAR input)` + +将一个base64字符串(`hll_to_base64`函数的结果)转化为一个HLL。当输入字符串不合法时,返回NULL。 + +### example + +``` +mysql> select hll_union_agg(hll_from_base64(hll_to_base64(pv))), hll_union_agg(pv) from test_hll; ++---------------------------------------------------+-------------------+ +| hll_union_agg(hll_from_base64(hll_to_base64(pv))) | hll_union_agg(pv) | ++---------------------------------------------------+-------------------+ +| 3 | 3 | ++---------------------------------------------------+-------------------+ +1 row in set (0.04 sec) + +mysql> select hll_cardinality(hll_from_base64(hll_to_base64(hll_hash('abc')))); ++------------------------------------------------------------------+ +| hll_cardinality(hll_from_base64(hll_to_base64(hll_hash('abc')))) | ++------------------------------------------------------------------+ +| 1 | ++------------------------------------------------------------------+ +1 row in set (0.04 sec) + +mysql> select hll_cardinality(hll_from_base64(hll_to_base64(hll_hash('')))); ++---------------------------------------------------------------+ +| hll_cardinality(hll_from_base64(hll_to_base64(hll_hash('')))) | ++---------------------------------------------------------------+ +| 1 | ++---------------------------------------------------------------+ +1 row in set (0.02 sec) + +mysql> select hll_cardinality(hll_from_base64(hll_to_base64(hll_hash(NULL)))); ++-----------------------------------------------------------------+ +| hll_cardinality(hll_from_base64(hll_to_base64(hll_hash(NULL)))) | ++-----------------------------------------------------------------+ +| 0 | ++-----------------------------------------------------------------+ +1 row in set (0.02 sec) +``` + +### keywords + + HLL_FROM_BASE64,HLL diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/hll-functions/hll-to-base64.md b/docs/zh-CN/docs/sql-manual/sql-functions/hll-functions/hll-to-base64.md new file mode 100644 index 00000000000000..0fe33ad6732478 --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-functions/hll-functions/hll-to-base64.md @@ -0,0 +1,80 @@ +--- +{ + "title": "HLL_TO_BASE64", + "language": "zh-CN" +} +--- + + + +## hll_to_base64 + +### description +#### Syntax + +`VARCHAR HLL_TO_BASE64(HLL input)` + +将一个hll转化成一个base64字符串。输入是NULL的话返回NULL。 + +### example + +``` +mysql> select hll_to_base64(NULL); ++---------------------+ +| hll_to_base64(NULL) | ++---------------------+ +| NULL | ++---------------------+ +1 row in set (0.00 sec) + +mysql> select hll_to_base64(hll_empty()); ++----------------------------+ +| hll_to_base64(hll_empty()) | ++----------------------------+ +| AA== | ++----------------------------+ +1 row in set (0.02 sec) + +mysql> select hll_to_base64(hll_hash('abc')); ++--------------------------------+ +| hll_to_base64(hll_hash('abc')) | ++--------------------------------+ +| AQEC5XSzrpDsdw== | ++--------------------------------+ +1 row in set (0.03 sec) + +mysql> select hll_union_agg(hll_from_base64(hll_to_base64(pv))), hll_union_agg(pv) from test_hll; ++---------------------------------------------------+-------------------+ +| hll_union_agg(hll_from_base64(hll_to_base64(pv))) | hll_union_agg(pv) | ++---------------------------------------------------+-------------------+ +| 3 | 3 | ++---------------------------------------------------+-------------------+ +1 row in set (0.04 sec) + +mysql> select hll_cardinality(hll_from_base64(hll_to_base64(hll_hash('abc')))); ++------------------------------------------------------------------+ +| hll_cardinality(hll_from_base64(hll_to_base64(hll_hash('abc')))) | ++------------------------------------------------------------------+ +| 1 | ++------------------------------------------------------------------+ +1 row in set (0.04 sec) +``` + +### keywords + + HLL_TO_BASE64,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(''));" +} From 65af1aebe9a52757660af01b13f152fc76999c06 Mon Sep 17 00:00:00 2001 From: chesterxu Date: Wed, 13 Mar 2024 12:39:05 +0800 Subject: [PATCH 4/6] fix and opt --- be/src/vec/functions/hll_from_base64.cpp | 116 +++++------------- be/src/vec/functions/hll_to_base64.cpp | 6 +- .../hll-functions/hll-from-base64.md | 2 +- docs/sidebars.json | 2 +- 4 files changed, 33 insertions(+), 93 deletions(-) diff --git a/be/src/vec/functions/hll_from_base64.cpp b/be/src/vec/functions/hll_from_base64.cpp index 3ec348d006d135..db918ce50f9e86 100644 --- a/be/src/vec/functions/hll_from_base64.cpp +++ b/be/src/vec/functions/hll_from_base64.cpp @@ -20,33 +20,45 @@ #include "olap/hll.h" #include "util/url_coding.h" -#include "vec/columns/column_array.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_array.h" #include "vec/data_types/data_type_hll.h" -#include "vec/data_types/data_type_string.h" #include "vec/functions/simple_function_factory.h" namespace doris::vectorized { -struct HllFromBase64 { - using ArgumentType = DataTypeString; - +class FunctionHllFromBase64 : public IFunction { +public: static constexpr auto name = "hll_from_base64"; - static Status vector(const ColumnString::Chars& data, const ColumnString::Offsets& offsets, - std::vector& res, NullMap& null_map, - size_t input_rows_count) { + 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(); + + ColumnPtr& 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); - if (offsets.size() == 0 && input_rows_count == 1) { - // For NULL constant - res.emplace_back(); - null_map[0] = 1; - return Status::OK(); - } + std::string decode_buff; int last_decode_buff_len = 0; int curr_decode_buff_len = 0; @@ -63,7 +75,7 @@ struct HllFromBase64 { decode_buff.resize(curr_decode_buff_len); last_decode_buff_len = curr_decode_buff_len; } - int outlen = base64_decode(src_str, src_size, decode_buff.data()); + auto outlen = base64_decode(src_str, src_size, decode_buff.data()); if (outlen < 0) { res.emplace_back(); null_map[i] = 1; @@ -78,85 +90,13 @@ struct HllFromBase64 { } } } - return Status::OK(); - } -}; - -template -class FunctionHllAlwaysNull : public IFunction { -public: - static constexpr auto name = Impl::name; - - 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(); - - ColumnPtr& argument_column = block.get_by_position(arguments[0]).column; - if constexpr (std::is_same_v) { - const auto& str_column = static_cast(*argument_column); - const ColumnString::Chars& data = str_column.get_chars(); - const ColumnString::Offsets& offsets = str_column.get_offsets(); - RETURN_IF_ERROR(Impl::vector(data, offsets, res, null_map, input_rows_count)); - } else if constexpr (std::is_same_v) { - auto argument_type = remove_nullable( - assert_cast(*block.get_by_position(arguments[0]).type) - .get_nested_type()); - const auto& array_column = static_cast(*argument_column); - const auto& offset_column_data = array_column.get_offsets(); - const auto& nested_nullable_column = - static_cast(array_column.get_data()); - const auto& nested_column = nested_nullable_column.get_nested_column(); - const auto& nested_null_map = nested_nullable_column.get_null_map_column().get_data(); - - WhichDataType which_type(argument_type); - if (which_type.is_int8()) { - RETURN_IF_ERROR(Impl::template vector(offset_column_data, nested_column, - nested_null_map, res, null_map)); - } else if (which_type.is_uint8()) { - RETURN_IF_ERROR(Impl::template vector( - offset_column_data, nested_column, nested_null_map, res, null_map)); - } else if (which_type.is_int16()) { - RETURN_IF_ERROR(Impl::template vector( - offset_column_data, nested_column, nested_null_map, res, null_map)); - } else if (which_type.is_int32()) { - RETURN_IF_ERROR(Impl::template vector( - offset_column_data, nested_column, nested_null_map, res, null_map)); - } else if (which_type.is_int64()) { - RETURN_IF_ERROR(Impl::template vector( - offset_column_data, nested_column, nested_null_map, res, null_map)); - } else { - return Status::RuntimeError("Illegal column {} of argument of function {}", - block.get_by_position(arguments[0]).column->get_name(), - get_name()); - } - } else { - return Status::RuntimeError("Illegal column {} of argument of function {}", - block.get_by_position(arguments[0]).column->get_name(), - get_name()); - } block.get_by_position(result).column = ColumnNullable::create(std::move(res_data_column), std::move(res_null_map)); return Status::OK(); } }; -using FunctionHllFromBase64 = FunctionHllAlwaysNull; - void register_function_hll_from_base64(SimpleFunctionFactory& factory) { factory.register_function(); } diff --git a/be/src/vec/functions/hll_to_base64.cpp b/be/src/vec/functions/hll_to_base64.cpp index f80eedbbe6b2a7..adc51437f86a43 100644 --- a/be/src/vec/functions/hll_to_base64.cpp +++ b/be/src/vec/functions/hll_to_base64.cpp @@ -69,11 +69,11 @@ struct HllToBase64 { ser_buff.resize(cur_ser_size); } hll_val.serialize(reinterpret_cast(ser_buff.data())); - int outlen = base64_encode((const unsigned char*)ser_buff.data(), cur_ser_size, - chars_data + encoded_offset); + auto outlen = base64_encode((const unsigned char*)ser_buff.data(), cur_ser_size, + chars_data + encoded_offset); DCHECK(outlen > 0); - encoded_offset += (int)(4.0 * ceil((double)cur_ser_size / 3.0)); + encoded_offset += outlen; offsets[i] = encoded_offset; } return Status::OK(); diff --git a/docs/en/docs/sql-manual/sql-functions/hll-functions/hll-from-base64.md b/docs/en/docs/sql-manual/sql-functions/hll-functions/hll-from-base64.md index b04c6c4531786e..38ab609c2b7d8a 100644 --- a/docs/en/docs/sql-manual/sql-functions/hll-functions/hll-from-base64.md +++ b/docs/en/docs/sql-manual/sql-functions/hll-functions/hll-from-base64.md @@ -1,7 +1,7 @@ --- { "title": "HLL_FROM_BASE64", - "language": "zh-CN" + "language": "en" } --- diff --git a/docs/sidebars.json b/docs/sidebars.json index 02c47a223ac496..5830579866272a 100644 --- a/docs/sidebars.json +++ b/docs/sidebars.json @@ -738,7 +738,7 @@ "sql-manual/sql-functions/hll-functions/hll-cardinality", "sql-manual/sql-functions/hll-functions/hll-empty", "sql-manual/sql-functions/hll-functions/hll-from_base64", - "sql-manual/sql-functions/hll-functions/hll-hash" + "sql-manual/sql-functions/hll-functions/hll-hash", "sql-manual/sql-functions/hll-functions/hll-to_base64" ] }, From 3964185a2b4625428f706eb9227e65c502f192f9 Mon Sep 17 00:00:00 2001 From: chesterxu Date: Thu, 11 Apr 2024 20:30:03 +0800 Subject: [PATCH 5/6] remove md files, fmt --- be/src/vec/functions/hll_from_base64.cpp | 6 +- be/src/vec/functions/hll_to_base64.cpp | 8 +- .../hll-functions/hll-from-base64.md | 72 ----------------- .../hll-functions/hll-to-base64.md | 80 ------------------- 4 files changed, 7 insertions(+), 159 deletions(-) delete mode 100644 docs/zh-CN/docs/sql-manual/sql-functions/hll-functions/hll-from-base64.md delete mode 100644 docs/zh-CN/docs/sql-manual/sql-functions/hll-functions/hll-to-base64.md diff --git a/be/src/vec/functions/hll_from_base64.cpp b/be/src/vec/functions/hll_from_base64.cpp index db918ce50f9e86..c62316fc7e80dc 100644 --- a/be/src/vec/functions/hll_from_base64.cpp +++ b/be/src/vec/functions/hll_from_base64.cpp @@ -15,8 +15,8 @@ // specific language governing permissions and limitations // under the License. -#include -#include +#include +#include #include "olap/hll.h" #include "util/url_coding.h" @@ -52,7 +52,7 @@ class FunctionHllFromBase64 : public IFunction { auto& null_map = res_null_map->get_data(); auto& res = res_data_column->get_data(); - ColumnPtr& argument_column = block.get_by_position(arguments[0]).column; + 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(); diff --git a/be/src/vec/functions/hll_to_base64.cpp b/be/src/vec/functions/hll_to_base64.cpp index adc51437f86a43..30020361e32475 100644 --- a/be/src/vec/functions/hll_to_base64.cpp +++ b/be/src/vec/functions/hll_to_base64.cpp @@ -15,8 +15,8 @@ // specific language governing permissions and limitations // under the License. -#include -#include +#include +#include #include "olap/hll.h" #include "util/url_coding.h" @@ -48,7 +48,7 @@ struct HllToBase64 { offsets.resize(size); size_t output_char_size = 0; for (size_t i = 0; i < size; ++i) { - HyperLogLog& hll_val = const_cast(data[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)); } @@ -61,7 +61,7 @@ struct HllToBase64 { std::string ser_buff; size_t encoded_offset = 0; for (size_t i = 0; i < size; ++i) { - HyperLogLog& hll_val = const_cast(data[i]); + auto& hll_val = const_cast(data[i]); cur_ser_size = hll_val.max_serialized_size(); if (cur_ser_size > last_ser_size) { diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/hll-functions/hll-from-base64.md b/docs/zh-CN/docs/sql-manual/sql-functions/hll-functions/hll-from-base64.md deleted file mode 100644 index 4dc66cf4118d66..00000000000000 --- a/docs/zh-CN/docs/sql-manual/sql-functions/hll-functions/hll-from-base64.md +++ /dev/null @@ -1,72 +0,0 @@ ---- -{ - "title": "HLL_FROM_BASE64", - "language": "zh-CN" -} ---- - - - -## hll_from_base64 - -### description -#### Syntax - -`HLL HLL_FROM_BASE64(VARCHAR input)` - -将一个base64字符串(`hll_to_base64`函数的结果)转化为一个HLL。当输入字符串不合法时,返回NULL。 - -### example - -``` -mysql> select hll_union_agg(hll_from_base64(hll_to_base64(pv))), hll_union_agg(pv) from test_hll; -+---------------------------------------------------+-------------------+ -| hll_union_agg(hll_from_base64(hll_to_base64(pv))) | hll_union_agg(pv) | -+---------------------------------------------------+-------------------+ -| 3 | 3 | -+---------------------------------------------------+-------------------+ -1 row in set (0.04 sec) - -mysql> select hll_cardinality(hll_from_base64(hll_to_base64(hll_hash('abc')))); -+------------------------------------------------------------------+ -| hll_cardinality(hll_from_base64(hll_to_base64(hll_hash('abc')))) | -+------------------------------------------------------------------+ -| 1 | -+------------------------------------------------------------------+ -1 row in set (0.04 sec) - -mysql> select hll_cardinality(hll_from_base64(hll_to_base64(hll_hash('')))); -+---------------------------------------------------------------+ -| hll_cardinality(hll_from_base64(hll_to_base64(hll_hash('')))) | -+---------------------------------------------------------------+ -| 1 | -+---------------------------------------------------------------+ -1 row in set (0.02 sec) - -mysql> select hll_cardinality(hll_from_base64(hll_to_base64(hll_hash(NULL)))); -+-----------------------------------------------------------------+ -| hll_cardinality(hll_from_base64(hll_to_base64(hll_hash(NULL)))) | -+-----------------------------------------------------------------+ -| 0 | -+-----------------------------------------------------------------+ -1 row in set (0.02 sec) -``` - -### keywords - - HLL_FROM_BASE64,HLL diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/hll-functions/hll-to-base64.md b/docs/zh-CN/docs/sql-manual/sql-functions/hll-functions/hll-to-base64.md deleted file mode 100644 index 0fe33ad6732478..00000000000000 --- a/docs/zh-CN/docs/sql-manual/sql-functions/hll-functions/hll-to-base64.md +++ /dev/null @@ -1,80 +0,0 @@ ---- -{ - "title": "HLL_TO_BASE64", - "language": "zh-CN" -} ---- - - - -## hll_to_base64 - -### description -#### Syntax - -`VARCHAR HLL_TO_BASE64(HLL input)` - -将一个hll转化成一个base64字符串。输入是NULL的话返回NULL。 - -### example - -``` -mysql> select hll_to_base64(NULL); -+---------------------+ -| hll_to_base64(NULL) | -+---------------------+ -| NULL | -+---------------------+ -1 row in set (0.00 sec) - -mysql> select hll_to_base64(hll_empty()); -+----------------------------+ -| hll_to_base64(hll_empty()) | -+----------------------------+ -| AA== | -+----------------------------+ -1 row in set (0.02 sec) - -mysql> select hll_to_base64(hll_hash('abc')); -+--------------------------------+ -| hll_to_base64(hll_hash('abc')) | -+--------------------------------+ -| AQEC5XSzrpDsdw== | -+--------------------------------+ -1 row in set (0.03 sec) - -mysql> select hll_union_agg(hll_from_base64(hll_to_base64(pv))), hll_union_agg(pv) from test_hll; -+---------------------------------------------------+-------------------+ -| hll_union_agg(hll_from_base64(hll_to_base64(pv))) | hll_union_agg(pv) | -+---------------------------------------------------+-------------------+ -| 3 | 3 | -+---------------------------------------------------+-------------------+ -1 row in set (0.04 sec) - -mysql> select hll_cardinality(hll_from_base64(hll_to_base64(hll_hash('abc')))); -+------------------------------------------------------------------+ -| hll_cardinality(hll_from_base64(hll_to_base64(hll_hash('abc')))) | -+------------------------------------------------------------------+ -| 1 | -+------------------------------------------------------------------+ -1 row in set (0.04 sec) -``` - -### keywords - - HLL_TO_BASE64,HLL From efae565c75100668269029886e08c19c77a0d8e1 Mon Sep 17 00:00:00 2001 From: chesterxu Date: Fri, 12 Apr 2024 16:21:29 +0800 Subject: [PATCH 6/6] add comments --- be/src/vec/functions/hll_from_base64.cpp | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/be/src/vec/functions/hll_from_base64.cpp b/be/src/vec/functions/hll_from_base64.cpp index c62316fc7e80dc..4cfd9c7817c2cc 100644 --- a/be/src/vec/functions/hll_from_base64.cpp +++ b/be/src/vec/functions/hll_from_base64.cpp @@ -65,11 +65,18 @@ class FunctionHllFromBase64 : public IFunction { 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);