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

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


The following commit(s) were added to refs/heads/branch-4.0 by this push:
     new 396c09362fb branch-4.0: [Feature](func) Support function 
QUANTILE_STATE_TO/FROM_BASE64 #59664 (#59746)
396c09362fb is described below

commit 396c09362fb39a7488aec6cfb1dfce4b6af8ba54
Author: github-actions[bot] 
<41898282+github-actions[bot]@users.noreply.github.com>
AuthorDate: Tue Jan 13 10:12:38 2026 +0800

    branch-4.0: [Feature](func) Support function QUANTILE_STATE_TO/FROM_BASE64 
#59664 (#59746)
    
    Cherry-picked from #59664
    
    Co-authored-by: linrrarity <[email protected]>
---
 be/src/vec/functions/function_quantile_state.cpp   | 127 ++++++++++++
 .../vec/function/function_quantile_state_test.cpp  | 216 +++++++++++++++++++++
 be/test/vec/function/function_test_util.cpp        |  11 ++
 .../doris/catalog/BuiltinScalarFunctions.java      |   4 +
 .../functions/scalar/QuantileStateFromBase64.java  |  71 +++++++
 .../functions/scalar/QuantileStateToBase64.java    |  69 +++++++
 .../expressions/visitor/ScalarFunctionVisitor.java |  10 +
 .../test_quantile_state_function.out               |  37 ++++
 .../test_quantile_state_function.groovy            |  82 ++++++++
 9 files changed, 627 insertions(+)

diff --git a/be/src/vec/functions/function_quantile_state.cpp 
b/be/src/vec/functions/function_quantile_state.cpp
index 63460255fdc..3ba5541ae30 100644
--- a/be/src/vec/functions/function_quantile_state.cpp
+++ b/be/src/vec/functions/function_quantile_state.cpp
@@ -29,6 +29,7 @@
 #include "common/compiler_util.h" // IWYU pragma: keep
 #include "common/status.h"
 #include "util/quantile_state.h"
+#include "util/url_coding.h"
 #include "vec/aggregate_functions/aggregate_function.h"
 #include "vec/columns/column.h"
 #include "vec/columns/column_complex.h"
@@ -45,9 +46,11 @@
 #include "vec/data_types/data_type_nullable.h"
 #include "vec/data_types/data_type_number.h"
 #include "vec/data_types/data_type_quantilestate.h" // IWYU pragma: keep
+#include "vec/data_types/data_type_string.h"
 #include "vec/functions/function.h"
 #include "vec/functions/function_const.h"
 #include "vec/functions/function_helpers.h"
+#include "vec/functions/function_totype.h"
 #include "vec/functions/simple_function_factory.h"
 #include "vec/utils/util.hpp"
 
@@ -210,10 +213,134 @@ public:
     }
 };
 
+class FunctionQuantileStateFromBase64 : public IFunction {
+public:
+    static constexpr auto name = "quantile_state_from_base64";
+    String get_name() const override { return name; }
+
+    static FunctionPtr create() { return 
std::make_shared<FunctionQuantileStateFromBase64>(); }
+
+    DataTypePtr get_return_type_impl(const DataTypes& arguments) const 
override {
+        return 
std::make_shared<DataTypeNullable>(std::make_shared<DataTypeQuantileState>());
+    }
+
+    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,
+                        uint32_t result, size_t input_rows_count) const 
override {
+        auto res_null_map = ColumnUInt8::create(input_rows_count, 0);
+        auto res_data_column = ColumnQuantileState::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;
+        int64_t last_decode_buff_len = 0;
+        int64_t 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];
+
+            if (src_size == 0 || 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;
+            }
+            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::QuantileState quantile_state;
+                if (!quantile_state.deserialize(decoded_slice)) {
+                    return Status::RuntimeError(fmt::format(
+                            "quantile_state_from_base64 decode failed: base64: 
{}", src_str));
+                } else {
+                    res.emplace_back(std::move(quantile_state));
+                }
+            }
+        }
+
+        block.get_by_position(result).column =
+                ColumnNullable::create(std::move(res_data_column), 
std::move(res_null_map));
+        return Status::OK();
+    }
+};
+
+struct NameQuantileStateToBase64 {
+    static constexpr auto name = "quantile_state_to_base64";
+};
+
+struct QuantileStateToBase64 {
+    using ReturnType = DataTypeString;
+    static constexpr auto PrimitiveTypeImpl = 
PrimitiveType::TYPE_QUANTILE_STATE;
+    using Type = DataTypeQuantileState::FieldType;
+    using ReturnColumnType = ColumnString;
+    using Chars = ColumnString::Chars;
+    using Offsets = ColumnString::Offsets;
+
+    static Status vector(const std::vector<QuantileState>& 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& quantile_state_val = const_cast<QuantileState&>(data[i]);
+            auto ser_size = quantile_state_val.get_serialized_size();
+            output_char_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& quantile_state_val = const_cast<QuantileState&>(data[i]);
+
+            cur_ser_size = quantile_state_val.get_serialized_size();
+            if (cur_ser_size > last_ser_size) {
+                last_ser_size = cur_ser_size;
+                ser_buff.resize(cur_ser_size);
+            }
+            size_t real_size =
+                    
quantile_state_val.serialize(reinterpret_cast<uint8_t*>(ser_buff.data()));
+            auto outlen = base64_encode((const unsigned char*)ser_buff.data(), 
real_size,
+                                        chars_data + encoded_offset);
+            DCHECK(outlen > 0);
+
+            encoded_offset += outlen;
+            offsets[i] = cast_set<uint32_t>(encoded_offset);
+        }
+        return Status::OK();
+    }
+};
+
+using FunctionQuantileStateToBase64 =
+        FunctionUnaryToType<QuantileStateToBase64, NameQuantileStateToBase64>;
+
 void register_function_quantile_state(SimpleFunctionFactory& factory) {
     factory.register_function<FunctionConst<QuantileStateEmpty, false>>();
     factory.register_function<FunctionQuantileStatePercent>();
     factory.register_function<FunctionToQuantileState>();
+    factory.register_function<FunctionQuantileStateFromBase64>();
+    factory.register_function<FunctionQuantileStateToBase64>();
 }
 
 } // namespace doris::vectorized
diff --git a/be/test/vec/function/function_quantile_state_test.cpp 
b/be/test/vec/function/function_quantile_state_test.cpp
new file mode 100644
index 00000000000..e395ef79188
--- /dev/null
+++ b/be/test/vec/function/function_quantile_state_test.cpp
@@ -0,0 +1,216 @@
+// 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 <gtest/gtest.h>
+
+#include <string>
+
+#include "function_test_util.h"
+#include "runtime/define_primitive_type.h"
+#include "util/quantile_state.h"
+#include "util/url_coding.h"
+#include "vec/core/types.h"
+#include "vec/data_types/data_type_quantilestate.h"
+#include "vec/data_types/data_type_string.h"
+
+namespace doris::vectorized {
+
+TEST(function_quantile_state_test, function_quantile_state_to_base64) {
+    std::string func_name = "quantile_state_to_base64";
+    InputTypeSet input_types = {PrimitiveType::TYPE_QUANTILE_STATE};
+
+    QuantileState empty_quantile_state;
+
+    QuantileState single_quantile_state;
+    single_quantile_state.add_value(1.0);
+
+    QuantileState multi_quantile_state;
+    multi_quantile_state.add_value(1.0);
+    multi_quantile_state.add_value(2.0);
+    multi_quantile_state.add_value(3.0);
+    multi_quantile_state.add_value(4.0);
+    multi_quantile_state.add_value(5.0);
+
+    QuantileState explicit_quantile_state;
+    for (int i = 0; i < 100; i++) {
+        explicit_quantile_state.add_value(static_cast<double>(i));
+    }
+
+    QuantileState tdigest_quantile_state;
+    for (int i = 0; i < 3000; i++) {
+        tdigest_quantile_state.add_value(static_cast<double>(i));
+    }
+
+    uint8_t buf[65536];
+    unsigned char encoded_buf[131072];
+
+    std::string empty_base64;
+    {
+        size_t len = empty_quantile_state.serialize(buf);
+        size_t encoded_len = base64_encode(buf, len, encoded_buf);
+        empty_base64 = std::string(reinterpret_cast<char*>(encoded_buf), 
encoded_len);
+    }
+
+    std::string single_base64;
+    {
+        size_t len = single_quantile_state.serialize(buf);
+        size_t encoded_len = base64_encode(buf, len, encoded_buf);
+        single_base64 = std::string(reinterpret_cast<char*>(encoded_buf), 
encoded_len);
+    }
+
+    std::string multi_base64;
+    {
+        size_t len = multi_quantile_state.serialize(buf);
+        size_t encoded_len = base64_encode(buf, len, encoded_buf);
+        multi_base64 = std::string(reinterpret_cast<char*>(encoded_buf), 
encoded_len);
+    }
+
+    std::string explicit_base64;
+    {
+        size_t len = explicit_quantile_state.serialize(buf);
+        size_t encoded_len = base64_encode(buf, len, encoded_buf);
+        explicit_base64 = std::string(reinterpret_cast<char*>(encoded_buf), 
encoded_len);
+    }
+
+    std::string tdigest_base64;
+    {
+        size_t len = tdigest_quantile_state.serialize(buf);
+        size_t encoded_len = base64_encode(buf, len, encoded_buf);
+        tdigest_base64 = std::string(reinterpret_cast<char*>(encoded_buf), 
encoded_len);
+    }
+
+    {
+        DataSet data_set = {{{&empty_quantile_state}, empty_base64},
+                            {{&single_quantile_state}, single_base64},
+                            {{&multi_quantile_state}, multi_base64},
+                            {{&explicit_quantile_state}, explicit_base64},
+                            {{&tdigest_quantile_state}, tdigest_base64}};
+
+        static_cast<void>(check_function<DataTypeString, true>(func_name, 
input_types, data_set));
+    }
+}
+
+TEST(function_quantile_state_test, function_quantile_state_from_base64) {
+    std::string func_name = "quantile_state_from_base64";
+    InputTypeSet input_types = {PrimitiveType::TYPE_STRING};
+
+    // Create quantile states for comparison
+    QuantileState empty_quantile_state;
+
+    QuantileState single_quantile_state;
+    single_quantile_state.add_value(1.0);
+
+    QuantileState multi_quantile_state;
+    multi_quantile_state.add_value(1.0);
+    multi_quantile_state.add_value(2.0);
+    multi_quantile_state.add_value(3.0);
+    multi_quantile_state.add_value(4.0);
+    multi_quantile_state.add_value(5.0);
+
+    uint8_t buf[65536];
+    unsigned char encoded_buf[131072];
+    std::string empty_base64;
+    std::string single_base64;
+    std::string multi_base64;
+
+    {
+        size_t len = empty_quantile_state.serialize(buf);
+        size_t encoded_len = base64_encode(buf, len, encoded_buf);
+        empty_base64 = std::string(reinterpret_cast<char*>(encoded_buf), 
encoded_len);
+    }
+
+    {
+        size_t len = single_quantile_state.serialize(buf);
+        size_t encoded_len = base64_encode(buf, len, encoded_buf);
+        single_base64 = std::string(reinterpret_cast<char*>(encoded_buf), 
encoded_len);
+    }
+
+    {
+        size_t len = multi_quantile_state.serialize(buf);
+        size_t encoded_len = base64_encode(buf, len, encoded_buf);
+        multi_base64 = std::string(reinterpret_cast<char*>(encoded_buf), 
encoded_len);
+    }
+
+    {
+        char decoded_buf[65536];
+        int decoded_len = base64_decode(empty_base64.c_str(), 
empty_base64.length(), decoded_buf);
+        EXPECT_GT(decoded_len, 0);
+
+        QuantileState decoded_empty;
+        doris::Slice slice(decoded_buf, decoded_len);
+        EXPECT_TRUE(decoded_empty.deserialize(slice));
+
+        
EXPECT_TRUE(std::isnan(empty_quantile_state.get_value_by_percentile(0.5)));
+        EXPECT_TRUE(std::isnan(decoded_empty.get_value_by_percentile(0.5)));
+    }
+
+    {
+        char decoded_buf[65536];
+        int decoded_len = base64_decode(single_base64.c_str(), 
single_base64.length(), decoded_buf);
+        EXPECT_GT(decoded_len, 0);
+
+        QuantileState decoded_single;
+        doris::Slice slice(decoded_buf, decoded_len);
+        EXPECT_TRUE(decoded_single.deserialize(slice));
+
+        EXPECT_NEAR(single_quantile_state.get_value_by_percentile(0.5),
+                    decoded_single.get_value_by_percentile(0.5), 0.01);
+    }
+
+    {
+        char decoded_buf[65536];
+        int decoded_len = base64_decode(multi_base64.c_str(), 
multi_base64.length(), decoded_buf);
+        EXPECT_GT(decoded_len, 0);
+
+        QuantileState decoded_multi;
+        doris::Slice slice(decoded_buf, decoded_len);
+        EXPECT_TRUE(decoded_multi.deserialize(slice));
+
+        EXPECT_NEAR(multi_quantile_state.get_value_by_percentile(0.5),
+                    decoded_multi.get_value_by_percentile(0.5), 0.01);
+        EXPECT_NEAR(multi_quantile_state.get_value_by_percentile(0.9),
+                    decoded_multi.get_value_by_percentile(0.9), 0.01);
+    }
+}
+
+TEST(function_quantile_state_test, function_quantile_state_roundtrip) {
+    QuantileState original;
+    for (int i = 0; i < 50; i++) {
+        original.add_value(static_cast<double>(i * 2));
+    }
+
+    uint8_t ser_buf[65536];
+    size_t ser_len = original.serialize(ser_buf);
+
+    unsigned char encoded_buf[131072];
+    size_t encoded_len = base64_encode(ser_buf, ser_len, encoded_buf);
+    std::string base64_str(reinterpret_cast<char*>(encoded_buf), encoded_len);
+
+    char decoded_buf[65536];
+    int decoded_len = base64_decode(base64_str.c_str(), base64_str.length(), 
decoded_buf);
+    EXPECT_GT(decoded_len, 0);
+
+    QuantileState recovered;
+    doris::Slice slice(decoded_buf, decoded_len);
+    EXPECT_TRUE(recovered.deserialize(slice));
+
+    EXPECT_NEAR(original.get_value_by_percentile(0.5), 
recovered.get_value_by_percentile(0.5),
+                0.01);
+    EXPECT_NEAR(original.get_value_by_percentile(0.9), 
recovered.get_value_by_percentile(0.9),
+                0.01);
+}
+
+} // namespace doris::vectorized
diff --git a/be/test/vec/function/function_test_util.cpp 
b/be/test/vec/function/function_test_util.cpp
index 53e8e2d8571..2e0c3180167 100644
--- a/be/test/vec/function/function_test_util.cpp
+++ b/be/test/vec/function/function_test_util.cpp
@@ -26,6 +26,7 @@
 #include "runtime/jsonb_value.h"
 #include "runtime/runtime_state.h"
 #include "util/bitmap_value.h"
+#include "util/quantile_state.h"
 #include "vec/columns/column.h"
 #include "vec/columns/column_nullable.h"
 #include "vec/columns/column_struct.h"
@@ -44,6 +45,7 @@
 #include "vec/data_types/data_type_ipv6.h"
 #include "vec/data_types/data_type_jsonb.h"
 #include "vec/data_types/data_type_map.h"
+#include "vec/data_types/data_type_quantilestate.h"
 #include "vec/data_types/data_type_string.h"
 #include "vec/data_types/data_type_struct.h"
 #include "vec/data_types/data_type_time.h"
@@ -106,6 +108,10 @@ static size_t type_index_to_data_type(const 
std::vector<AnyType>& input_types, s
         type = std::make_shared<DataTypeHLL>();
         desc = type;
         return 1;
+    case PrimitiveType::TYPE_QUANTILE_STATE:
+        type = std::make_shared<DataTypeQuantileState>();
+        desc = type;
+        return 1;
     case PrimitiveType::TYPE_IPV4:
         type = std::make_shared<DataTypeIPv4>();
         desc = type;
@@ -407,6 +413,11 @@ bool insert_cell(MutableColumnPtr& column, DataTypePtr 
type_ptr, const AnyType&
             column->insert_data((char*)hll, sizeof(HyperLogLog));
             break;
         }
+        case PrimitiveType::TYPE_QUANTILE_STATE: {
+            auto* quantile_state = any_cast<QuantileState*>(cell);
+            column->insert_data((char*)quantile_state, sizeof(QuantileState));
+            break;
+        }
         case PrimitiveType::TYPE_IPV4: {
             auto value = any_cast<ut_type::IPV4>(cell);
             column->insert_data(reinterpret_cast<char*>(&value), 0);
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 48bb1375e3c..9722294f375 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
@@ -386,6 +386,8 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.Power;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Protocol;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.QuantilePercent;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.QuantileStateEmpty;
+import 
org.apache.doris.nereids.trees.expressions.functions.scalar.QuantileStateFromBase64;
+import 
org.apache.doris.nereids.trees.expressions.functions.scalar.QuantileStateToBase64;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Quarter;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.QuarterCeil;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.QuarterFloor;
@@ -932,6 +934,8 @@ public class BuiltinScalarFunctions implements 
FunctionHelper {
             scalar(Protocol.class, "protocol"),
             scalar(QuantilePercent.class, "quantile_percent"),
             scalar(QuantileStateEmpty.class, "quantile_state_empty"),
+            scalar(QuantileStateToBase64.class, "quantile_state_to_base64"),
+            scalar(QuantileStateFromBase64.class, 
"quantile_state_from_base64"),
             scalar(Quarter.class, "quarter"),
             scalar(QuartersAdd.class, "quarters_add"),
             scalar(QuarterCeil.class, "quarter_ceil"),
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/QuantileStateFromBase64.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/QuantileStateFromBase64.java
new file mode 100644
index 00000000000..dd8a3bfc0fb
--- /dev/null
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/QuantileStateFromBase64.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.QuantileStateType;
+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;
+
+/**
+ * Function 'quantile_state_from_base64'.
+ */
+public class QuantileStateFromBase64 extends ScalarFunction
+        implements UnaryExpression, ExplicitlyCastableSignature, 
AlwaysNullable {
+
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            
FunctionSignature.ret(QuantileStateType.INSTANCE).args(VarcharType.SYSTEM_DEFAULT),
+            
FunctionSignature.ret(QuantileStateType.INSTANCE).args(StringType.INSTANCE)
+    );
+
+    /**
+     * constructor with 1 argument.
+     */
+    public QuantileStateFromBase64(Expression arg) {
+        super("quantile_state_from_base64", arg);
+    }
+
+    /**
+     * withChildren.
+     */
+    @Override
+    public QuantileStateFromBase64 withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 1);
+        return new QuantileStateFromBase64(children.get(0));
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitQuantileStateFromBase64(this, context);
+    }
+}
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/QuantileStateToBase64.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/QuantileStateToBase64.java
new file mode 100644
index 00000000000..42c1500884d
--- /dev/null
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/QuantileStateToBase64.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.QuantileStateType;
+import org.apache.doris.nereids.types.StringType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * Function 'quantile_state_to_base64'.
+ */
+public class QuantileStateToBase64 extends ScalarFunction
+        implements UnaryExpression, ExplicitlyCastableSignature, 
PropagateNullable {
+
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            
FunctionSignature.ret(StringType.INSTANCE).args(QuantileStateType.INSTANCE)
+    );
+
+    /**
+     * constructor with 1 argument.
+     */
+    public QuantileStateToBase64(Expression arg) {
+        super("quantile_state_to_base64", arg);
+    }
+
+    /**
+     * withChildren.
+     */
+    @Override
+    public QuantileStateToBase64 withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 1);
+        return new QuantileStateToBase64(children.get(0));
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitQuantileStateToBase64(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 122450fe1b8..1bf08ceceaf 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
@@ -391,6 +391,8 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.Power;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Protocol;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.QuantilePercent;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.QuantileStateEmpty;
+import 
org.apache.doris.nereids.trees.expressions.functions.scalar.QuantileStateFromBase64;
+import 
org.apache.doris.nereids.trees.expressions.functions.scalar.QuantileStateToBase64;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Quarter;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.QuarterCeil;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.QuarterFloor;
@@ -1940,6 +1942,14 @@ public interface ScalarFunctionVisitor<R, C> {
         return visitScalarFunction(quantileEmpty, context);
     }
 
+    default R visitQuantileStateFromBase64(QuantileStateFromBase64 
quantileStateFromBase64, C context) {
+        return visitScalarFunction(quantileStateFromBase64, context);
+    }
+
+    default R visitQuantileStateToBase64(QuantileStateToBase64 
quantileStateToBase64, C context) {
+        return visitScalarFunction(quantileStateToBase64, context);
+    }
+
     default R visitQuarter(Quarter quarter, C context) {
         return visitScalarFunction(quarter, context);
     }
diff --git 
a/regression-test/data/query_p0/sql_functions/quantile_state_functions/test_quantile_state_function.out
 
b/regression-test/data/query_p0/sql_functions/quantile_state_functions/test_quantile_state_function.out
new file mode 100644
index 00000000000..1e9b9630e05
--- /dev/null
+++ 
b/regression-test/data/query_p0/sql_functions/quantile_state_functions/test_quantile_state_function.out
@@ -0,0 +1,37 @@
+-- This file is automatically generated. You should know what you did if you 
want to edit this
+-- !sql_quantile_state_base64_1 --
+AAAARQA=
+
+-- !sql_quantile_state_base64_2 --
+\N
+
+-- !sql_quantile_state_base64_3 --
+true
+
+-- !sql_quantile_state_base64_4 --
+true
+
+-- !sql_quantile_state_base64_5 --
+AAAARQEAAAAAAADwPw==
+
+-- !sql_quantile_state_base64_6 --
+\N
+
+-- !sql_quantile_state_base64_7 --
+\N
+
+-- !sql_quantile_state_base64_8 --
+\N
+
+-- !sql_quantile_state_base64_9 --
+true
+
+-- !sql_quantile_state_base64_10 --
+\N
+
+-- !sql_quantile_state_base64_11 --
+AAAARQEAAAAAAAAkQA==
+
+-- !sql_quantile_state_base64_12 --
+true
+
diff --git 
a/regression-test/suites/query_p0/sql_functions/quantile_state_functions/test_quantile_state_function.groovy
 
b/regression-test/suites/query_p0/sql_functions/quantile_state_functions/test_quantile_state_function.groovy
new file mode 100644
index 00000000000..b1480f0971b
--- /dev/null
+++ 
b/regression-test/suites/query_p0/sql_functions/quantile_state_functions/test_quantile_state_function.groovy
@@ -0,0 +1,82 @@
+// 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_quantile_state_function") {
+    qt_sql_quantile_state_base64_1 """
+        select quantile_state_to_base64(quantile_state_empty())
+    """
+
+    qt_sql_quantile_state_base64_2 """
+        select quantile_state_from_base64(null)
+    """
+
+    qt_sql_quantile_state_base64_3 """
+        select quantile_state_to_base64(
+            quantile_state_from_base64(
+                quantile_state_to_base64(quantile_state_empty())
+            )
+        ) = quantile_state_to_base64(quantile_state_empty())
+    """
+
+    qt_sql_quantile_state_base64_4 """
+        select quantile_state_to_base64(
+            quantile_state_from_base64(
+                quantile_state_to_base64(to_quantile_state(1.0, 2048))
+            )
+        ) = quantile_state_to_base64(to_quantile_state(1.0, 2048))
+    """
+
+    qt_sql_quantile_state_base64_5 """
+        select quantile_state_to_base64(to_quantile_state(1.0, 2048))
+    """
+
+    qt_sql_quantile_state_base64_6 """
+        select quantile_state_from_base64('invalid')
+    """
+
+    qt_sql_quantile_state_base64_7 """
+        select quantile_state_from_base64('not_base64!')
+    """
+
+    qt_sql_quantile_state_base64_8 """
+        select quantile_state_from_base64('')
+    """
+
+    qt_sql_quantile_state_base64_9 """
+        select length(quantile_state_to_base64(to_quantile_state(1.0, 2048))) 
> 0
+    """
+
+    qt_sql_quantile_state_base64_10 """
+        select quantile_state_from_base64(quantile_state_to_base64(null))
+    """
+
+    qt_sql_quantile_state_base64_11 """
+        select quantile_state_to_base64(to_quantile_state(10.0, 2048))
+    """
+
+    qt_sql_quantile_state_base64_12 """
+         select quantile_state_to_base64(
+            quantile_state_from_base64(
+                quantile_state_to_base64(
+                    quantile_state_from_base64(
+                        quantile_state_to_base64(to_quantile_state(10.0, 2048))
+                    )
+                )
+            )
+        ) = quantile_state_to_base64(to_quantile_state(10.0, 2048))
+    """
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to