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]