You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by yi...@apache.org on 2024/04/16 12:03:57 UTC

(doris) 34/39: [feature](function) support hll functions hll_from_base64, hll_to_base64 (#32089)

This is an automated email from the ASF dual-hosted git repository.

yiguolei pushed a commit to branch branch-2.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 86402460d7b5b039ed136f3b3eb31f86e4a71e3d
Author: Chester <42...@users.noreply.github.com>
AuthorDate: Tue Apr 16 16:27:02 2024 +0800

    [feature](function) support hll functions hll_from_base64, hll_to_base64 (#32089)
    
    Issue Number: #31320
    
    Support two hll functions:
    
    - hll_from_base64
    Convert a base64 string(result of function hll_to_base64) into a hll.
    - hll_to_base64
    Convert an input hll to a base64 string.
---
 be/src/vec/functions/hll_from_base64.cpp           | 111 ++++++++++++++++++++
 be/src/vec/functions/hll_to_base64.cpp             |  89 ++++++++++++++++
 be/src/vec/functions/simple_function_factory.h     |   4 +
 .../doris/catalog/BuiltinScalarFunctions.java      |   4 +
 .../functions/scalar/HllFromBase64.java            |  71 +++++++++++++
 .../expressions/functions/scalar/HllToBase64.java  |  69 ++++++++++++
 .../expressions/visitor/ScalarFunctionVisitor.java |  10 ++
 gensrc/script/doris_builtins_functions.py          |   7 +-
 .../data/nereids_function_p0/scalar_function/H.out | 116 +++++++++++++++++++++
 .../hll_functions/test_hll_functions.out           |  35 +++++++
 .../nereids_function_p0/scalar_function/H.groovy   |   8 ++
 .../hll_functions/test_hll_functions.groovy        |  72 +++++++++++++
 12 files changed, 594 insertions(+), 2 deletions(-)

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 00000000000..4cfd9c7817c
--- /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 <cstddef>
+#include <cstdint>
+
+#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<FunctionHllFromBase64>(); }
+
+    DataTypePtr get_return_type_impl(const DataTypes& arguments) const override {
+        return make_nullable(std::make_shared<DataTypeHLL>());
+    }
+
+    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<const ColumnString&>(*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<const char*>(&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<FunctionHllFromBase64>();
+}
+
+} // 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 00000000000..30020361e32
--- /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 <cstddef>
+#include <cstdint>
+
+#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<HyperLogLog>& 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<HyperLogLog&>(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<HyperLogLog&>(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<uint8_t*>(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<HllToBase64, NameHllToBase64>;
+
+void register_function_hll_to_base64(SimpleFunctionFactory& factory) {
+    factory.register_function<FunctionHllToBase64>();
+}
+
+} // 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 649db732093..793632e8fef 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 @@ public:
             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 1654a2098db..eb7a2c6050c 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.GroupingId;
 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 00000000000..23d682e8ea7
--- /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<FunctionSignature> 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<Expression> children) {
+        Preconditions.checkArgument(children.size() == 1);
+        return new HllFromBase64(children.get(0));
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> 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 00000000000..14b58b9a6e6
--- /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<FunctionSignature> 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<Expression> children) {
+        Preconditions.checkArgument(children.size() == 1);
+        return new HllToBase64(children.get(0));
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> 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 cb26a8bf4dc..b8d2ccfbf4f 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.Greatest;
 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 @@ public interface ScalarFunctionVisitor<R, C> {
         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 fdc08755307..841799f0a13 100644
--- a/gensrc/script/doris_builtins_functions.py
+++ b/gensrc/script/doris_builtins_functions.py
@@ -1823,9 +1823,12 @@ visible_functions = {
     #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 08425584180..fa6e2caa7e4 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 00000000000..c9cebac5079
--- /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 2c9ee4bc2ea..08d6f9f92ba 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 00000000000..87a5f5360d0
--- /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(''));"
+}


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