This is an automated email from the ASF dual-hosted git repository.
morningman pushed a commit to branch branch-3.0
in repository https://gitbox.apache.org/repos/asf/doris.git
The following commit(s) were added to refs/heads/branch-3.0 by this push:
new c3740bade94 [cherry-pick](branch3.0) impl translate and url encode
(#41657)
c3740bade94 is described below
commit c3740bade945f3910867a08b64f3d9b354764c77
Author: Socrates <[email protected]>
AuthorDate: Sun Oct 13 19:59:41 2024 +0800
[cherry-pick](branch3.0) impl translate and url encode (#41657)
## Proposed changes
pick from master:
https://github.com/apache/doris/pull/40567
---
be/src/util/url_coding.cpp | 42 +-
be/src/util/url_coding.h | 12 +-
be/src/vec/functions/function_string.cpp | 2 +
be/src/vec/functions/function_string.h | 245 ++++++++--
.../doris/catalog/BuiltinScalarFunctions.java | 4 +
.../functions/executable/ExecutableFunctions.java | 2 +-
.../functions/executable/StringArithmetic.java | 13 +
.../expressions/functions/scalar/Translate.java | 72 +++
.../expressions/functions/scalar/UrlEncode.java | 70 +++
.../expressions/visitor/ScalarFunctionVisitor.java | 10 +
.../string_functions/test_translate.out | 493 +++++++++++++++++++++
.../string_functions/test_url_decode.out | 43 ++
.../string_functions/test_url_encode.out | 43 ++
.../fold_constant_string_arithmatic.groovy | 10 +-
.../string_functions/test_translate.groovy | 125 ++++++
.../string_functions/test_url_decode.groovy | 47 ++
.../string_functions/test_url_encode.groovy | 47 ++
17 files changed, 1205 insertions(+), 75 deletions(-)
diff --git a/be/src/util/url_coding.cpp b/be/src/util/url_coding.cpp
index d0bbf5aae63..5871b4b9d32 100644
--- a/be/src/util/url_coding.cpp
+++ b/be/src/util/url_coding.cpp
@@ -17,41 +17,33 @@
#include "util/url_coding.h"
+#include <curl/curl.h>
#include <libbase64.h>
-#include <math.h>
-#include <memory>
#include <sstream>
namespace doris {
-static inline void url_encode(const char* in, int in_len, std::string* out) {
- (*out).reserve(in_len);
- std::stringstream ss;
-
- for (int i = 0; i < in_len; ++i) {
- const char ch = in[i];
-
- // Escape the character iff a) we are in Hive-compat mode and the
- // character is in the Hive whitelist or b) we are not in
- // Hive-compat mode, and the character is not alphanumeric or one
- // of the four commonly excluded characters.
- ss << ch;
- }
-
- (*out) = ss.str();
+inline unsigned char to_hex(unsigned char x) {
+ return x + (x > 9 ? ('A' - 10) : '0');
}
-void url_encode(const std::vector<uint8_t>& in, std::string* out) {
- if (in.empty()) {
- *out = "";
- } else {
- url_encode(reinterpret_cast<const char*>(&in[0]), in.size(), out);
+// Adapted from http://dlib.net/dlib/server/server_http.cpp.html
+void url_encode(const std::string_view& in, std::string* out) {
+ std::ostringstream os;
+ for (auto c : in) {
+ // impl as
https://docs.oracle.com/javase/8/docs/api/java/net/URLEncoder.html
+ if ((c >= 'a' && c <= 'z') || (c >= 'A' && c <= 'Z') || (c >= '0' && c
<= '9') ||
+ c == '.' || c == '-' || c == '*' || c == '_') { // allowed
+ os << c;
+ } else if (c == ' ') {
+ os << '+';
+ } else {
+ os << '%' << to_hex(c >> 4) << to_hex(c % 16);
+ }
}
-}
-void url_encode(const std::string& in, std::string* out) {
- url_encode(in.c_str(), in.size(), out);
+ *out = os.str();
}
// Adapted from
diff --git a/be/src/util/url_coding.h b/be/src/util/url_coding.h
index 1a9fb4943b1..b7e5136ecad 100644
--- a/be/src/util/url_coding.h
+++ b/be/src/util/url_coding.h
@@ -17,27 +17,19 @@
#pragma once
-#include <stddef.h>
-
+#include <cstddef>
#include <cstdint>
#include <iosfwd>
#include <string>
-#include <vector>
namespace doris {
// Utility method to URL-encode a string (that is, replace special
// characters with %<hex value in ascii>).
-// The optional parameter hive_compat controls whether we mimic Hive's
-// behaviour when encoding a string, which is only to encode certain
-// characters (excluding, e.g., ' ')
-void url_encode(const std::string& in, std::string* out);
+void url_encode(const std::string_view& in, std::string* out);
// Utility method to decode a string that was URL-encoded. Returns
// true unless the string could not be correctly decoded.
-// The optional parameter hive_compat controls whether or not we treat
-// the strings as encoded by Hive, which means selectively ignoring
-// certain characters like ' '.
bool url_decode(const std::string& in, std::string* out);
void base64_encode(const std::string& in, std::string* out);
diff --git a/be/src/vec/functions/function_string.cpp
b/be/src/vec/functions/function_string.cpp
index d0e12bb4984..1a62c9daaf6 100644
--- a/be/src/vec/functions/function_string.cpp
+++ b/be/src/vec/functions/function_string.cpp
@@ -1046,6 +1046,7 @@ void register_function_string(SimpleFunctionFactory&
factory) {
factory.register_function<FunctionExtractURLParameter>();
factory.register_function<FunctionStringParseUrl>();
factory.register_function<FunctionUrlDecode>();
+ factory.register_function<FunctionUrlEncode>();
factory.register_function<FunctionRandomBytes>();
factory.register_function<FunctionMoneyFormat<MoneyFormatDoubleImpl>>();
factory.register_function<FunctionMoneyFormat<MoneyFormatInt64Impl>>();
@@ -1057,6 +1058,7 @@ void register_function_string(SimpleFunctionFactory&
factory) {
factory.register_function<FunctionStringDigestSHA2>();
factory.register_function<FunctionReplace<ReplaceImpl, true>>();
factory.register_function<FunctionReplace<ReplaceEmptyImpl, false>>();
+ factory.register_function<FunctionTranslate>();
factory.register_function<FunctionMask>();
factory.register_function<FunctionMaskPartial<true>>();
factory.register_function<FunctionMaskPartial<false>>();
diff --git a/be/src/vec/functions/function_string.h
b/be/src/vec/functions/function_string.h
index e4258484739..2157db9ed0c 100644
--- a/be/src/vec/functions/function_string.h
+++ b/be/src/vec/functions/function_string.h
@@ -17,28 +17,25 @@
#pragma once
-#include <limits.h>
-#include <stdlib.h>
-#include <string.h>
#include <sys/types.h>
#include <algorithm>
#include <array>
#include <boost/iterator/iterator_facade.hpp>
+#include <climits>
#include <cmath>
#include <codecvt>
#include <cstddef>
#include <cstdlib>
+#include <cstring>
#include <iomanip>
-#include <limits>
#include <memory>
#include <ostream>
#include <random>
-#include <regex>
#include <sstream>
-#include <stdexcept>
#include <tuple>
#include <type_traits>
+#include <unordered_map>
#include <utility>
#include <variant>
#include <vector>
@@ -49,7 +46,6 @@
#include "gutil/strings/numbers.h"
#include "gutil/strings/substitute.h"
#include "runtime/decimalv2_value.h"
-#include "runtime/runtime_state.h"
#include "runtime/string_search.hpp"
#include "util/sha.h"
#include "util/string_util.h"
@@ -64,17 +60,11 @@
#include "vec/common/memcpy_small.h"
#include "vec/common/pod_array.h"
#include "vec/common/pod_array_fwd.h"
-#include "vec/common/string_utils/string_utils.h"
-#include "vec/common/typeid_cast.h"
#include "vec/core/block.h"
#include "vec/core/column_numbers.h"
#include "vec/core/column_with_type_and_name.h"
-#include "vec/core/field.h"
#include "vec/core/types.h"
#include "vec/data_types/data_type.h"
-#include "vec/functions/function_binary_arithmetic.h"
-#include "vec/functions/round.h"
-#include "vec/io/io_helper.h"
#include "vec/utils/template_helpers.hpp"
#ifndef USE_LIBCPP
@@ -2729,43 +2719,60 @@ public:
static FunctionPtr create() { return
std::make_shared<FunctionUrlDecode>(); }
String get_name() const override { return name; }
size_t get_number_of_arguments() const override { return 1; }
- bool is_variadic() const override { return false; }
-
DataTypePtr get_return_type_impl(const DataTypes& arguments) const
override {
return std::make_shared<DataTypeString>();
}
- Status execute_impl(FunctionContext* context, Block& block,
-
- const ColumnNumbers& arguments, size_t result,
- size_t input_rows_count) const override {
+ Status execute_impl(FunctionContext* context, Block& block, const
ColumnNumbers& arguments,
+ size_t result, size_t input_rows_count) const override
{
auto res = ColumnString::create();
- auto& res_offsets = res->get_offsets();
- auto& res_chars = res->get_chars();
- res_offsets.resize(input_rows_count);
+ res->get_offsets().reserve(input_rows_count);
- ColumnPtr argument_column =
-
block.get_by_position(arguments[0]).column->convert_to_full_column_if_const();
- const auto* url_col =
check_and_get_column<ColumnString>(argument_column.get());
+ const auto* url_col =
+ assert_cast<const
ColumnString*>(block.get_by_position(arguments[0]).column.get());
- if (!url_col) {
- return Status::InternalError("Not supported input argument type");
+ std::string decoded_url;
+ for (size_t i = 0; i < input_rows_count; ++i) {
+ auto url = url_col->get_data_at(i);
+ if (!url_decode(url.to_string(), &decoded_url)) {
+ return Status::InternalError("Decode url failed");
+ }
+ res->insert_data(decoded_url.data(), decoded_url.size());
+ decoded_url.clear();
}
- std::string decoded_url;
+ block.get_by_position(result).column = std::move(res);
+ return Status::OK();
+ }
+};
- for (size_t i = 0; i < input_rows_count; ++i) {
- auto source = url_col->get_data_at(i);
- StringRef url_val(const_cast<char*>(source.data), source.size);
+class FunctionUrlEncode : public IFunction {
+public:
+ static constexpr auto name = "url_encode";
+ static FunctionPtr create() { return
std::make_shared<FunctionUrlEncode>(); }
+ String get_name() const override { return name; }
+ size_t get_number_of_arguments() const override { return 1; }
+ DataTypePtr get_return_type_impl(const DataTypes& arguments) const
override {
+ return std::make_shared<DataTypeString>();
+ }
+
+ Status execute_impl(FunctionContext* context, Block& block, const
ColumnNumbers& arguments,
+ size_t result, size_t input_rows_count) const override
{
+ auto res = ColumnString::create();
+ res->get_offsets().reserve(input_rows_count);
- url_decode(url_val.to_string(), &decoded_url);
+ const auto* url_col =
+ assert_cast<const
ColumnString*>(block.get_by_position(arguments[0]).column.get());
- StringOP::push_value_string(decoded_url, i, res_chars,
res_offsets);
- decoded_url.clear();
+ std::string encoded_url;
+ for (size_t i = 0; i < input_rows_count; ++i) {
+ auto url = url_col->get_data_at(i);
+ url_encode(url.to_string_view(), &encoded_url);
+ res->insert_data(encoded_url.data(), encoded_url.size());
+ encoded_url.clear();
}
block.get_by_position(result).column = std::move(res);
-
return Status::OK();
}
};
@@ -4259,4 +4266,172 @@ private:
}
};
+class FunctionTranslate : public IFunction {
+public:
+ static constexpr auto name = "translate";
+ static FunctionPtr create() { return
std::make_shared<FunctionTranslate>(); }
+ String get_name() const override { return name; }
+ size_t get_number_of_arguments() const override { return 3; }
+
+ DataTypePtr get_return_type_impl(const DataTypes& arguments) const
override {
+ return std::make_shared<DataTypeString>();
+ };
+
+ DataTypes get_variadic_argument_types_impl() const override {
+ return {std::make_shared<DataTypeString>(),
std::make_shared<DataTypeString>(),
+ std::make_shared<DataTypeString>()};
+ }
+
+ Status execute_impl(FunctionContext* context, Block& block, const
ColumnNumbers& arguments,
+ size_t result, size_t input_rows_count) const override
{
+ CHECK_EQ(arguments.size(), 3);
+ auto col_res = ColumnString::create();
+ bool col_const[3];
+ ColumnPtr argument_columns[3];
+ for (int i = 0; i < 3; ++i) {
+ col_const[i] =
is_column_const(*block.get_by_position(arguments[i]).column);
+ }
+ argument_columns[0] = col_const[0] ? static_cast<const ColumnConst&>(
+
*block.get_by_position(arguments[0]).column)
+ .convert_to_full_column()
+ :
block.get_by_position(arguments[0]).column;
+ default_preprocess_parameter_columns(argument_columns, col_const, {1,
2}, block, arguments);
+
+ const auto* col_source = assert_cast<const
ColumnString*>(argument_columns[0].get());
+ const auto* col_from = assert_cast<const
ColumnString*>(argument_columns[1].get());
+ const auto* col_to = assert_cast<const
ColumnString*>(argument_columns[2].get());
+
+ bool is_ascii = simd::VStringFunctions::is_ascii(
+ {col_source->get_chars().data(),
col_source->get_chars().size()}) &&
+ simd::VStringFunctions::is_ascii(
+ {col_from->get_chars().data(),
col_from->get_chars().size()}) &&
+ simd::VStringFunctions::is_ascii(
+ {col_to->get_chars().data(),
col_to->get_chars().size()});
+ auto impl_vectors = impl_vectors_utf8<false>;
+ if (col_const[1] && col_const[2] && is_ascii) {
+ impl_vectors = impl_vectors_ascii<true>;
+ } else if (col_const[1] && col_const[2]) {
+ impl_vectors = impl_vectors_utf8<true>;
+ } else if (is_ascii) {
+ impl_vectors = impl_vectors_ascii<false>;
+ }
+ impl_vectors(col_source, col_from, col_to, col_res);
+ block.get_by_position(result).column = std::move(col_res);
+ return Status::OK();
+ }
+
+private:
+ template <bool IsConst>
+ static void impl_vectors_ascii(const ColumnString* col_source, const
ColumnString* col_from,
+ const ColumnString* col_to, ColumnString*
col_res) {
+ col_res->get_chars().reserve(col_source->get_chars().size());
+ col_res->get_offsets().reserve(col_source->get_offsets().size());
+ std::unordered_map<char, char> translate_map;
+ if (IsConst) {
+ const auto& from_str = col_from->get_data_at(0);
+ const auto& to_str = col_to->get_data_at(0);
+ translate_map =
+ build_translate_map_ascii(from_str.to_string_view(),
to_str.to_string_view());
+ }
+ for (size_t i = 0; i < col_source->size(); ++i) {
+ const auto& source_str = col_source->get_data_at(i);
+ if (!IsConst) {
+ const auto& from_str = col_from->get_data_at(i);
+ const auto& to_str = col_to->get_data_at(i);
+ translate_map =
build_translate_map_ascii(from_str.to_string_view(),
+
to_str.to_string_view());
+ }
+ auto translated_str = translate_ascii(source_str.to_string_view(),
translate_map);
+ col_res->insert_data(translated_str.data(), translated_str.size());
+ }
+ }
+
+ static std::unordered_map<char, char> build_translate_map_ascii(
+ const std::string_view& from_str, const std::string_view& to_str) {
+ std::unordered_map<char, char> translate_map;
+ for (size_t i = 0; i < from_str.size(); ++i) {
+ if (translate_map.find(from_str[i]) == translate_map.end()) {
+ translate_map[from_str[i]] = i < to_str.size() ? to_str[i] : 0;
+ }
+ }
+ return translate_map;
+ }
+
+ static std::string translate_ascii(const std::string_view& source_str,
+ std::unordered_map<char, char>&
translate_map) {
+ std::string result;
+ result.reserve(source_str.size());
+ for (auto const& c : source_str) {
+ if (translate_map.find(c) != translate_map.end()) {
+ if (translate_map[c]) {
+ result.push_back(translate_map[c]);
+ }
+ } else {
+ result.push_back(c);
+ }
+ }
+ return result;
+ }
+
+ template <bool IsConst>
+ static void impl_vectors_utf8(const ColumnString* col_source, const
ColumnString* col_from,
+ const ColumnString* col_to, ColumnString*
col_res) {
+ col_res->get_chars().reserve(col_source->get_chars().size());
+ col_res->get_offsets().reserve(col_source->get_offsets().size());
+ std::unordered_map<std::string_view, std::string_view> translate_map;
+ if (IsConst) {
+ const auto& from_str = col_from->get_data_at(0);
+ const auto& to_str = col_to->get_data_at(0);
+ translate_map =
+ build_translate_map_utf8(from_str.to_string_view(),
to_str.to_string_view());
+ }
+ for (size_t i = 0; i < col_source->size(); ++i) {
+ const auto& source_str = col_source->get_data_at(i);
+ if (!IsConst) {
+ const auto& from_str = col_from->get_data_at(i);
+ const auto& to_str = col_to->get_data_at(i);
+ translate_map =
build_translate_map_utf8(from_str.to_string_view(),
+
to_str.to_string_view());
+ }
+ auto translated_str = translate_utf8(source_str.to_string_view(),
translate_map);
+ col_res->insert_data(translated_str.data(), translated_str.size());
+ }
+ }
+
+ static std::unordered_map<std::string_view, std::string_view>
build_translate_map_utf8(
+ const std::string_view& from_str, const std::string_view& to_str) {
+ std::unordered_map<std::string_view, std::string_view> translate_map;
+ for (size_t i = 0, from_char_size = 0, j = 0, to_char_size = 0; i <
from_str.size();
+ i += from_char_size, j += to_char_size) {
+ from_char_size = get_utf8_byte_length(from_str[i]);
+ to_char_size = j < to_str.size() ? get_utf8_byte_length(to_str[j])
: 0;
+ auto from_char = from_str.substr(i, from_char_size);
+ if (translate_map.find(from_char) == translate_map.end()) {
+ translate_map[from_char] =
+ j < to_str.size() ? to_str.substr(j, to_char_size) :
std::string_view();
+ }
+ }
+ return translate_map;
+ }
+
+ static std::string translate_utf8(
+ const std::string_view& source_str,
+ std::unordered_map<std::string_view, std::string_view>&
translate_map) {
+ std::string result;
+ result.reserve(source_str.size());
+ for (size_t i = 0, char_size = 0; i < source_str.size(); i +=
char_size) {
+ char_size = get_utf8_byte_length(source_str[i]);
+ auto c = source_str.substr(i, char_size);
+ if (translate_map.find(c) != translate_map.end()) {
+ if (!translate_map[c].empty()) {
+ result.append(translate_map[c]);
+ }
+ } else {
+ result.append(c);
+ }
+ }
+ return result;
+ }
+};
+
} // namespace doris::vectorized
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 58c367fe868..1b49917b85f 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
@@ -430,12 +430,14 @@ import
org.apache.doris.nereids.trees.expressions.functions.scalar.ToIso8601;
import org.apache.doris.nereids.trees.expressions.functions.scalar.ToMonday;
import
org.apache.doris.nereids.trees.expressions.functions.scalar.ToQuantileState;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Tokenize;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.Translate;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Trim;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Truncate;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Unhex;
import
org.apache.doris.nereids.trees.expressions.functions.scalar.UnixTimestamp;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Upper;
import org.apache.doris.nereids.trees.expressions.functions.scalar.UrlDecode;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.UrlEncode;
import org.apache.doris.nereids.trees.expressions.functions.scalar.User;
import
org.apache.doris.nereids.trees.expressions.functions.scalar.UtcTimestamp;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Uuid;
@@ -904,6 +906,7 @@ public class BuiltinScalarFunctions implements
FunctionHelper {
scalar(Tokenize.class, "tokenize"),
scalar(ToMonday.class, "to_monday"),
scalar(ToQuantileState.class, "to_quantile_state"),
+ scalar(Translate.class, "translate"),
scalar(Trim.class, "trim"),
scalar(Truncate.class, "truncate"),
scalar(Unhex.class, "unhex"),
@@ -911,6 +914,7 @@ public class BuiltinScalarFunctions implements
FunctionHelper {
scalar(Upper.class, "ucase", "upper"),
scalar(Quote.class, "quote"),
scalar(UrlDecode.class, "url_decode"),
+ scalar(UrlEncode.class, "url_encode"),
scalar(User.class, "user"),
scalar(UtcTimestamp.class, "utc_timestamp"),
scalar(Uuid.class, "uuid"),
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/ExecutableFunctions.java
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/ExecutableFunctions.java
index d54ea4681c5..42636cfa1b5 100644
---
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/ExecutableFunctions.java
+++
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/ExecutableFunctions.java
@@ -125,7 +125,7 @@ public class ExecutableFunctions {
return new DoubleLiteral(Math.E);
}
- @ExecFunction(name = "p1", argTypes = {}, returnType = "DOUBLE")
+ @ExecFunction(name = "pi", argTypes = {}, returnType = "DOUBLE")
public static Expression pi() {
return new DoubleLiteral(Math.PI);
}
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/StringArithmetic.java
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/StringArithmetic.java
index d2f42e44a26..c405ac40b09 100644
---
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/StringArithmetic.java
+++
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/StringArithmetic.java
@@ -44,6 +44,7 @@ import java.net.MalformedURLException;
import java.net.URI;
import java.net.URISyntaxException;
import java.net.URLDecoder;
+import java.net.URLEncoder;
import java.nio.charset.StandardCharsets;
import java.security.MessageDigest;
import java.security.NoSuchAlgorithmException;
@@ -789,6 +790,18 @@ public class StringArithmetic {
}
}
+ /**
+ * Executable arithmetic functions urlencode
+ */
+ @ExecFunction(name = "url_encode")
+ public static Expression urlEncode(StringLikeLiteral first) {
+ try {
+ return castStringLikeLiteral(first,
URLEncoder.encode(first.getValue(), StandardCharsets.UTF_8.name()));
+ } catch (UnsupportedEncodingException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
/**
* Executable arithmetic functions append_trailing_char_if_absent
*/
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Translate.java
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Translate.java
new file mode 100644
index 00000000000..2edb474a71f
--- /dev/null
+++
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Translate.java
@@ -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.
+
+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.TernaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+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 'translate'. This class is generated by GenerateFunction.
+ */
+public class Translate extends ScalarFunction
+ implements TernaryExpression, ExplicitlyCastableSignature,
PropagateNullable {
+
+ public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+ FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT)
+ .args(VarcharType.SYSTEM_DEFAULT,
VarcharType.SYSTEM_DEFAULT, VarcharType.SYSTEM_DEFAULT),
+ FunctionSignature.ret(StringType.INSTANCE)
+ .args(StringType.INSTANCE, StringType.INSTANCE,
StringType.INSTANCE)
+ );
+
+ /**
+ * constructor with 3 arguments.
+ */
+ public Translate(Expression arg0, Expression arg1, Expression arg2) {
+ super("translate", arg0, arg1, arg2);
+ }
+
+ /**
+ * withChildren.
+ */
+ @Override
+ public Translate withChildren(List<Expression> children) {
+ Preconditions.checkArgument(children.size() == 3);
+ return new Translate(children.get(0), children.get(1),
children.get(2));
+ }
+
+ @Override
+ public List<FunctionSignature> getSignatures() {
+ return SIGNATURES;
+ }
+
+ @Override
+ public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+ return visitor.visitTranslate(this, context);
+ }
+}
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/UrlEncode.java
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/UrlEncode.java
new file mode 100644
index 00000000000..fd32e953cf9
--- /dev/null
+++
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/UrlEncode.java
@@ -0,0 +1,70 @@
+// 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.visitor.ExpressionVisitor;
+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 'url_encode'. This class is generated by GenerateFunction.
+ */
+public class UrlEncode extends ScalarFunction
+ implements ExplicitlyCastableSignature, PropagateNullable {
+
+ public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+
FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(VarcharType.SYSTEM_DEFAULT),
+
FunctionSignature.ret(StringType.INSTANCE).args(StringType.INSTANCE)
+ );
+
+ /**
+ * constructor with 1 argument.
+ */
+ public UrlEncode(Expression arg0) {
+ super("url_encode", arg0);
+ }
+
+
+ /**
+ * withChildren.
+ */
+ @Override
+ public UrlEncode withChildren(List<Expression> children) {
+ Preconditions.checkArgument(children.size() == 1);
+ return new UrlEncode(children.get(0));
+ }
+
+ @Override
+ public List<FunctionSignature> getSignatures() {
+ return SIGNATURES;
+ }
+
+ @Override
+ public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+ return visitor.visitUrlEncode(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 5ae78a8e351..a905ae28d8f 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
@@ -427,12 +427,14 @@ import
org.apache.doris.nereids.trees.expressions.functions.scalar.ToIso8601;
import org.apache.doris.nereids.trees.expressions.functions.scalar.ToMonday;
import
org.apache.doris.nereids.trees.expressions.functions.scalar.ToQuantileState;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Tokenize;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.Translate;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Trim;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Truncate;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Unhex;
import
org.apache.doris.nereids.trees.expressions.functions.scalar.UnixTimestamp;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Upper;
import org.apache.doris.nereids.trees.expressions.functions.scalar.UrlDecode;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.UrlEncode;
import org.apache.doris.nereids.trees.expressions.functions.scalar.User;
import
org.apache.doris.nereids.trees.expressions.functions.scalar.UtcTimestamp;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Uuid;
@@ -1658,6 +1660,10 @@ public interface ScalarFunctionVisitor<R, C> {
return visitScalarFunction(urlDecode, context);
}
+ default R visitUrlEncode(UrlEncode urlEncode, C context) {
+ return visitScalarFunction(urlEncode, context);
+ }
+
default R visitRandomBytes(RandomBytes randomBytes, C context) {
return visitScalarFunction(randomBytes, context);
}
@@ -2058,6 +2064,10 @@ public interface ScalarFunctionVisitor<R, C> {
return visitScalarFunction(toQuantileState, context);
}
+ default R visitTranslate(Translate translate, C context) {
+ return visitScalarFunction(translate, context);
+ }
+
default R visitTrim(Trim trim, C context) {
return visitScalarFunction(trim, context);
}
diff --git
a/regression-test/data/query_p0/sql_functions/string_functions/test_translate.out
b/regression-test/data/query_p0/sql_functions/string_functions/test_translate.out
new file mode 100644
index 00000000000..951d1012c3b
--- /dev/null
+++
b/regression-test/data/query_p0/sql_functions/string_functions/test_translate.out
@@ -0,0 +1,493 @@
+-- This file is automatically generated. You should know what you did if you
want to edit this
+-- !empty_nullable --
+
+-- !empty_not_nullable --
+
+-- !empty_partial_nullable --
+
+-- !nullable --
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+!!!
+!@
+!@!@!$!^
+!@#!@#
+!@#@#$#^$%%$^
+!@#@#$#^$%%$^
+!@#@#$#^$%%$^
+!@#@#$#^$%%$^
+!@#@#$#^$%%$^
+!@#@#$#^$%%$^
+!@#@#$#^$%%$^
+!@#@#$#^$%%$^
+!@#@#$#^$%%$^
+!@#@#$#^$%%$^
+!@#@#$#^$%%$^
+!@#@#$#^$%%$^
+!@#@#$#^$%%$^
+!@#@#$#^$%%$^
+!@#@#$#^$%%$^
+!@#@#$#^$%%$^
+!@#@#$#^$%%$^
+!@#@#$#^$%%$^
+!@#@#$#^$%%$^
+!@#@#$#^$%%$^
+!@#@#$#^$%%$^
+!@#@#$#^$%%$^
+!@#@#$#^$%%$^
+!@#@#$#^$%%$^
+!@#@#$#^$%%$^
+!@#@#$#^$%%$^
+!@#@#$#^$%%$^
+!@#@#$#^$%%$^
+!@#@#$#^$%%$^
+!@#@#$#^$%%$^
+!@#@#$#^$%%$^
+111
+12
+1211131
+123123
+123123
+123123
+123123
+123123
+123123
+123123
+123123
+123123
+123123
+123123
+123123
+123123
+123123
+123123
+123123
+123123
+123123
+123123
+123123
+123123
+123123
+123123
+123123
+123123
+123123
+123123
+123123
+123123
+123123
+123123
+123233333
+\\\\\\
+\\a
+\\a\\\\a\\
+\\a\\a\\c\\dccd
+\\a\\b\\c\\d
+\\a\\b\\c\\d
+\\a\\b\\c\\d
+\\a\\b\\c\\d
+\\a\\b\\c\\d
+\\a\\b\\c\\d
+\\a\\b\\c\\d
+\\a\\b\\c\\d
+\\a\\b\\c\\d
+\\a\\b\\c\\d
+\\a\\b\\c\\d
+\\a\\b\\c\\d
+\\a\\b\\c\\d
+\\a\\b\\c\\d
+\\a\\b\\c\\d
+\\a\\b\\c\\d
+\\a\\b\\c\\d
+\\a\\b\\c\\d
+\\a\\b\\c\\d
+\\a\\b\\c\\d
+\\a\\b\\c\\d
+\\a\\b\\c\\d
+\\a\\b\\c\\d
+\\a\\b\\c\\d
+\\a\\b\\c\\d
+\\a\\b\\c\\d
+\\a\\b\\c\\d
+\\a\\b\\c\\d
+\\a\\b\\c\\d
+\\a\\b\\c\\d
+\\a\\b\\c\\d
+中中中
+中文
+中文
+中文
+中文
+中文
+中文
+中文
+中文
+中文
+中文
+中文
+中文
+中文
+中文
+中文
+中文
+中文
+中文
+中文
+中文
+中文
+中文
+中文
+中文
+中文
+中文
+中文
+中文
+中文
+中文
+中文
+中文中中中
+中文中文
+中文文
+
+-- !not_nullable --
+
+
+
+!@#@#$#^$%%$^
+123123
+\\a\\b\\c\\d
+中文
+
+-- !partial_nullable --
+\N
+
+
+!@#@#$#^$%%$^
+123123
+\\a\\b\\c\\d
+中文
+
+-- !nullable_no_null --
+
+
+
+!@#@#$#^$%%$^
+123123
+\\a\\b\\c\\d
+中文
+
+-- !const_nullable --
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+
+-- !partial_const_nullable --
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+
+-- !const_not_nullable --
+a
+a
+a
+a
+a
+a
+a
+
+-- !const_other_nullable --
+\N
+x
+x
+x
+x
+x
+x
+
+-- !const_other_not_nullable --
+
+
+
+!
+1
+\\
+中
+
+-- !const_nullable_no_null --
+abc
+
+-- !const_partial_nullable_no_null --
+xyz
+
+-- !const1 --
+\N
+xyz
+xyz
+xyz
+xyz
+xyz
+xyz
+
+-- !const12 --
+\N
+xyz
+xyz
+xyz
+xyz
+xyz
+xyz
+
+-- !const23 --
+
+
+
+!@#@#$#^$%%$^
+123123
+\\a\\b\\c\\d
+中文
+
+-- !const3 --
+\N
+
+aaa
+ab
+abaaa
+abcabc
+abcbcc
+
+-- !1 --
+abcd
+
+-- !2 --
+zbcd
+
+-- !3 --
+zbcdz
+
+-- !4 --
+zbd
+
+-- !5 --
+zbxd
+
+-- !6 --
+中bxd
+
+-- !7 --
+文文
+
+-- !8 --
+a文
+
+-- !9 --
+tttttt
+
diff --git
a/regression-test/data/query_p0/sql_functions/string_functions/test_url_decode.out
b/regression-test/data/query_p0/sql_functions/string_functions/test_url_decode.out
new file mode 100644
index 00000000000..7199df82b4b
--- /dev/null
+++
b/regression-test/data/query_p0/sql_functions/string_functions/test_url_decode.out
@@ -0,0 +1,43 @@
+-- This file is automatically generated. You should know what you did if you
want to edit this
+-- !empty_nullable --
+
+-- !empty_not_nullable --
+
+-- !nullable --
+\N
+
+/home/doris/directory/
+1234567890
+ABCDEFGHIJKLMNOPQRSTUWXYZ
+~!@#%^&*()<>?,./:{}|[]\\_+-=
+
+-- !not_nullable --
+
+
+/home/doris/directory/
+1234567890
+ABCDEFGHIJKLMNOPQRSTUWXYZ
+~!@#%^&*()<>?,./:{}|[]\\_+-=
+
+-- !nullable_no_null --
+
+
+/home/doris/directory/
+1234567890
+ABCDEFGHIJKLMNOPQRSTUWXYZ
+~!@#%^&*()<>?,./:{}|[]\\_+-=
+
+-- !const_nullable --
+
+
+
+
+
+
+
+-- !const_not_nullable --
+/home/doris/directory/
+
+-- !const_nullable_no_null --
+/home/doris/directory/
+
diff --git
a/regression-test/data/query_p0/sql_functions/string_functions/test_url_encode.out
b/regression-test/data/query_p0/sql_functions/string_functions/test_url_encode.out
new file mode 100644
index 00000000000..23b82546e3a
--- /dev/null
+++
b/regression-test/data/query_p0/sql_functions/string_functions/test_url_encode.out
@@ -0,0 +1,43 @@
+-- This file is automatically generated. You should know what you did if you
want to edit this
+-- !empty_nullable --
+
+-- !empty_not_nullable --
+
+-- !nullable --
+\N
+
+%2Fhome%2Fdoris%2Fdirectory%2F
+%7E%21%40%23%25%5E%26*%28%29%3C%3E%3F%2C.%2F%3A%7B%7D%7C%5B%5D%5C_%2B-%3D
+1234567890
+ABCDEFGHIJKLMNOPQRSTUWXYZ
+
+-- !not_nullable --
+
+
+%2Fhome%2Fdoris%2Fdirectory%2F
+%7E%21%40%23%25%5E%26*%28%29%3C%3E%3F%2C.%2F%3A%7B%7D%7C%5B%5D%5C_%2B-%3D
+1234567890
+ABCDEFGHIJKLMNOPQRSTUWXYZ
+
+-- !nullable_no_null --
+
+
+%2Fhome%2Fdoris%2Fdirectory%2F
+%7E%21%40%23%25%5E%26*%28%29%3C%3E%3F%2C.%2F%3A%7B%7D%7C%5B%5D%5C_%2B-%3D
+1234567890
+ABCDEFGHIJKLMNOPQRSTUWXYZ
+
+-- !const_nullable --
+
+
+
+
+
+
+
+-- !const_not_nullable --
+%2Fhome%2Fdoris%2Fdirectory%2F
+
+-- !const_nullable_no_null --
+%2Fhome%2Fdoris%2Fdirectory%2F
+
diff --git
a/regression-test/suites/nereids_p0/expression/fold_constant/fold_constant_string_arithmatic.groovy
b/regression-test/suites/nereids_p0/expression/fold_constant/fold_constant_string_arithmatic.groovy
index 2bcdfc2fd24..2f707c56fe5 100644
---
a/regression-test/suites/nereids_p0/expression/fold_constant/fold_constant_string_arithmatic.groovy
+++
b/regression-test/suites/nereids_p0/expression/fold_constant/fold_constant_string_arithmatic.groovy
@@ -79,7 +79,8 @@ suite("fold_constant_string_arithmatic") {
testFoldConst("SELECT StrRight('Hello World', 5)")
testFoldConst("SELECT Overlay('abcdef', '123', 3, 2)")
testFoldConst("SELECT Parse_Url('http://www.example.com/path?query=abc',
'HOST')")
- testFoldConst("SELECT Url_Decode('%20Hello%20World%20')")
+ testFoldConst("SELECT Url_Decode('+Hello+World+')")
+ testFoldConst("SELECT Url_Encode(' Hello World ')")
// Substring with negative start index
// Expected behavior: Depending on the SQL engine, might return an empty
string or error.
@@ -187,7 +188,7 @@ suite("fold_constant_string_arithmatic") {
// UrlDecode with an invalid percent-encoded string
// Expected behavior: Return NULL or error due to invalid encoding.
- testFoldConst("SELECT Url_Decode('%ZZHello%20World')")
+ // testFoldConst("SELECT Url_Decode('%ZZHello%20World')")
testFoldConst("select elt(0, \"hello\", \"doris\")")
testFoldConst("select elt(1, \"hello\", \"doris\")")
@@ -437,7 +438,8 @@ suite("fold_constant_string_arithmatic") {
testFoldConst("SELECT StrRight(cast('Hello World' as string), 5)")
testFoldConst("SELECT Overlay(cast('abcdef' as string), cast('123' as
string), 3, 2)")
testFoldConst("SELECT
Parse_Url(cast('http://www.example.com/path?query=abc' as string), cast('HOST'
as string))")
- testFoldConst("SELECT Url_Decode(cast('%20Hello%20World%20' as string))")
+ testFoldConst("SELECT Url_Decode(cast('+Hello+World+' as string))")
+ testFoldConst("SELECT Url_Encode(cast(' Hello World ' as string))")
// Substring with negative start index
// Expected behavior: Depending on the SQL engine, might return an empty
string or error.
@@ -525,7 +527,7 @@ suite("fold_constant_string_arithmatic") {
testFoldConst("SELECT Unhex(cast('GHIJ' as string))")
// UrlDecode with an invalid percent-encoded string
- testFoldConst("SELECT Url_Decode(cast('%ZZHello%20World' as string))")
+ // testFoldConst("SELECT Url_Decode(cast('%ZZHello%20World' as string))")
// Additional function tests
testFoldConst("SELECT Elt(0, cast('hello' as string), cast('doris' as
string))")
diff --git
a/regression-test/suites/query_p0/sql_functions/string_functions/test_translate.groovy
b/regression-test/suites/query_p0/sql_functions/string_functions/test_translate.groovy
new file mode 100644
index 00000000000..e63f42ae5b4
--- /dev/null
+++
b/regression-test/suites/query_p0/sql_functions/string_functions/test_translate.groovy
@@ -0,0 +1,125 @@
+// 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_translate") {
+ // this table has nothing todo. just make it eaiser to generate query
+ sql " drop table if exists hits_three_args "
+ sql """ create table hits_three_args(
+ nothing boolean
+ )
+ properties("replication_num" = "1");
+ """
+ sql "insert into hits_three_args values(true);"
+
+ sql " drop table if exists test_translate"
+ sql """
+ create table test_translate (
+ k0 int,
+ a varchar not null,
+ b varchar null,
+ )
+ DISTRIBUTED BY HASH(k0)
+ PROPERTIES
+ (
+ "replication_num" = "1"
+ );
+ """
+
+ order_qt_empty_nullable "select translate(a, a, a) from test_translate"
+ order_qt_empty_not_nullable "select translate(b, b, b) from test_translate"
+ order_qt_empty_partial_nullable "select translate(a, b, b) from
test_translate"
+
+ sql """ insert into test_translate values (1, "", ""), (2, "中文", "中文"),
(3, "123123", "123123"),
+ (4, "\\\\a\\\\b\\\\c\\\\d", "\\\\a\\\\b\\\\c\\\\d"),
+ (5, "!@#@#\$#^\$%%\$^", "!@#@#\$#^\$%%\$^"), (6, " ", " "),
+ (7, "", NULL);
+ """
+
+ order_qt_nullable """
+ SELECT translate(t.test_translate, t.ARG2, t.ARG3) as result
+ FROM (
+ SELECT hits_three_args.nothing, TABLE1.test_translate,
TABLE1.order1, TABLE2.ARG2, TABLE2.order2, TABLE3.ARG3, TABLE3.order3
+ FROM hits_three_args
+ CROSS JOIN (
+ SELECT b as test_translate, k0 as order1
+ FROM test_translate
+ ) as TABLE1
+ CROSS JOIN (
+ SELECT b as ARG2, k0 as order2
+ FROM test_translate
+ ) as TABLE2
+ CROSS JOIN (
+ SELECT b as ARG3, k0 as order3
+ FROM test_translate
+ ) as TABLE3
+ )t;
+ """
+
+ /// nullables
+ order_qt_not_nullable "select translate(a, a, a) from test_translate"
+ order_qt_partial_nullable "select translate(a, b, b) from test_translate"
+ order_qt_nullable_no_null "select translate(a, nullable(a), nullable(a))
from test_translate"
+
+ /// consts. most by BE-UT
+ order_qt_const_nullable "select translate(NULL, NULL, NULL) from
test_translate"
+ order_qt_partial_const_nullable "select translate(NULL, b, b) from
test_translate"
+ order_qt_const_not_nullable "select translate('a', 'b', 'c') from
test_translate"
+ order_qt_const_other_nullable "select translate('x', b, b) from
test_translate"
+ order_qt_const_other_not_nullable "select translate('x', 'x', a) from
test_translate"
+ order_qt_const_nullable_no_null "select translate(nullable('abc'),
nullable('中文'), nullable('xxx'))"
+ order_qt_const_partial_nullable_no_null "select translate('xyz',
nullable('a'), nullable('a'))"
+ order_qt_const1 "select translate('xyz', a, b) from test_translate"
+ order_qt_const12 "select translate('xyz', 'abc', b) from test_translate"
+ order_qt_const23 "select translate(a, 'xyz', 'abc') from test_translate"
+ order_qt_const3 "select translate(b, a, 'abc') from test_translate"
+
+ /// folding
+ def re_fe
+ def re_be
+ def re_no_fold
+ def check_three_ways = { test_sql ->
+ sql "set enable_fold_constant_by_be=false;"
+ re_fe = order_sql "select ${test_sql}"
+ sql "set enable_fold_constant_by_be=true;"
+ re_be = order_sql "select ${test_sql}"
+ sql "set debug_skip_fold_constant=true;"
+ re_no_fold = order_sql "select ${test_sql}"
+ logger.info("check on sql \${test_sql}")
+ assertEquals(re_fe, re_be)
+ assertEquals(re_fe, re_no_fold)
+ }
+
+ check_three_ways "translate('abcd', '', '');"
+ check_three_ways "translate('abcda', 'a', 'z');"
+ check_three_ways "translate('abcd', 'ac', 'z');"
+ check_three_ways "translate('abcd', 'aac', 'zq');"
+ check_three_ways "translate('abcd', 'aac', 'zqx');"
+ check_three_ways "translate('abcd', 'aac', '中文x');"
+ check_three_ways "translate('中文', '中', '文');"
+ check_three_ways "translate('中文', '中', 'a');"
+ check_three_ways "translate('\tt\tt\tt', '\t', 't');"
+
+ order_qt_1 "select translate('abcd', '', '');"
+ order_qt_2 "select translate('abcd', 'a', 'z')"
+ order_qt_3 "select translate('abcda', 'a', 'z');"
+ order_qt_4 "select translate('abcd', 'aac', 'zq');"
+ order_qt_5 "select translate('abcd', 'aac', 'zqx');"
+ order_qt_6 "select translate('abcd', 'aac', '中文x');"
+ order_qt_7 "select translate('中文', '中', '文');"
+ order_qt_8 "select translate('中文', '中', 'ab');"
+ order_qt_9 "select translate('\tt\tt\tt', '\t', 't');"
+}
diff --git
a/regression-test/suites/query_p0/sql_functions/string_functions/test_url_decode.groovy
b/regression-test/suites/query_p0/sql_functions/string_functions/test_url_decode.groovy
new file mode 100644
index 00000000000..dd5cb9d3521
--- /dev/null
+++
b/regression-test/suites/query_p0/sql_functions/string_functions/test_url_decode.groovy
@@ -0,0 +1,47 @@
+// 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_url_decode") {
+ sql " drop table if exists test_url_decode"
+ sql """
+ create table test_url_decode (
+ k0 int,
+ a string not null,
+ b string null
+ )
+ DISTRIBUTED BY HASH(k0)
+ PROPERTIES
+ (
+ "replication_num" = "1"
+ );
+ """
+
+ order_qt_empty_nullable "select url_decode(b) from test_url_decode"
+ order_qt_empty_not_nullable "select url_decode(a) from test_url_decode"
+
+ sql """ insert into test_url_decode values (1,
'ABCDEFGHIJKLMNOPQRSTUWXYZ', 'ABCDEFGHIJKLMNOPQRSTUWXYZ'), (2, '1234567890',
'1234567890'),
+ (3,
'~%21%40%23%25%5E%26%2A%28%29%3C%3E%3F%2C.%2F%3A%7B%7D%7C%5B%5D%5C_%2B-%3D',
'~%21%40%23%25%5E%26%2A%28%29%3C%3E%3F%2C.%2F%3A%7B%7D%7C%5B%5D%5C_%2B-%3D'),
+ (4, '', ''), (5, '%2Fhome%2Fdoris%2Fdirectory%2F',
'%2Fhome%2Fdoris%2Fdirectory%2F'), (6, '', null);
+ """
+
+ order_qt_nullable "select url_decode(b) from test_url_decode"
+ order_qt_not_nullable "select url_decode(a) from test_url_decode"
+ order_qt_nullable_no_null "select url_decode(nullable(a)) from
test_url_decode"
+ order_qt_const_nullable "select url_decode('') from test_url_decode" //
choose one case to test const multi-rows
+ order_qt_const_not_nullable "select
url_decode('%2Fhome%2Fdoris%2Fdirectory%2F')"
+ order_qt_const_nullable_no_null "select
url_decode('%2Fhome%2Fdoris%2Fdirectory%2F')"
+}
diff --git
a/regression-test/suites/query_p0/sql_functions/string_functions/test_url_encode.groovy
b/regression-test/suites/query_p0/sql_functions/string_functions/test_url_encode.groovy
new file mode 100644
index 00000000000..18b8a615d5a
--- /dev/null
+++
b/regression-test/suites/query_p0/sql_functions/string_functions/test_url_encode.groovy
@@ -0,0 +1,47 @@
+// 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_url_encode") {
+ sql " drop table if exists test_url_encode"
+ sql """
+ create table test_url_encode (
+ k0 int,
+ a string not null,
+ b string null
+ )
+ DISTRIBUTED BY HASH(k0)
+ PROPERTIES
+ (
+ "replication_num" = "1"
+ );
+ """
+
+ order_qt_empty_nullable "select url_encode(b) from test_url_encode"
+ order_qt_empty_not_nullable "select url_encode(a) from test_url_encode"
+
+ sql """ insert into test_url_encode values (1,
'ABCDEFGHIJKLMNOPQRSTUWXYZ', 'ABCDEFGHIJKLMNOPQRSTUWXYZ'),
+ (2, '1234567890', '1234567890'), (3, '~!@#%^&*()<>?,./:{}|[]\\_+-=',
'~!@#%^&*()<>?,./:{}|[]\\_+-='),
+ (4, '', ''), (5, '/home/doris/directory/', '/home/doris/directory/'),
(6, '', null);
+ """
+
+ order_qt_nullable "select url_encode(b) from test_url_encode"
+ order_qt_not_nullable "select url_encode(a) from test_url_encode"
+ order_qt_nullable_no_null "select url_encode(nullable(a)) from
test_url_encode"
+ order_qt_const_nullable "select url_encode('') from test_url_encode" //
choose one case to test const multi-rows
+ order_qt_const_not_nullable "select url_encode('/home/doris/directory/')"
+ order_qt_const_nullable_no_null "select
url_encode('/home/doris/directory/')"
+}
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]