replace{One/All}(): allow non-const pattern and replacement arguments

This commit is contained in:
Robert Schulze 2023-02-20 09:10:25 +00:00
parent 659ba65137
commit a7c153f88c
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
12 changed files with 697 additions and 116 deletions

View File

@ -13,17 +13,18 @@ Functions for [searching](../../sql-reference/functions/string-search-functions.
## replaceOne(haystack, pattern, replacement) ## replaceOne(haystack, pattern, replacement)
Replaces the first occurrence of the substring pattern (if it exists) in haystack by the replacement string. Replaces the first occurrence of the substring pattern (if it exists) in haystack by the replacement string.
pattern and replacement must be constants.
## replaceAll(haystack, pattern, replacement), replace(haystack, pattern, replacement) ## replaceAll(haystack, pattern, replacement), replace(haystack, pattern, replacement)
Replaces all occurrences of the substring pattern in haystack by the replacement string. Replaces all occurrences of the substring pattern in haystack by the replacement string.
Alias: `replace`.
## replaceRegexpOne(haystack, pattern, replacement) ## replaceRegexpOne(haystack, pattern, replacement)
Replaces the first occurrence of the substring matching the regular expression pattern in haystack by the replacement string. Replaces the first occurrence of the substring matching the regular expression pattern in haystack by the replacement string.
pattern must be a constant [re2 regular expression](https://github.com/google/re2/wiki/Syntax). pattern must be a [re2 regular expression](https://github.com/google/re2/wiki/Syntax).
replacement must be a plain constant string or a constant string containing substitutions `\0-\9`. replacement must be a plain string or a string containing substitutions `\0-\9`.
Substitutions `\1-\9` correspond to the 1st to 9th capturing group (submatch), substitution `\0` corresponds to the entire match. Substitutions `\1-\9` correspond to the 1st to 9th capturing group (submatch), substitution `\0` corresponds to the entire match.
To use a verbatim `\` character in the pattern or replacement string, escape it using `\`. To use a verbatim `\` character in the pattern or replacement string, escape it using `\`.
Also keep in mind that string literals require an extra escaping. Also keep in mind that string literals require an extra escaping.
@ -88,6 +89,8 @@ SELECT replaceRegexpAll('Hello, World!', '^', 'here: ') AS res
└─────────────────────┘ └─────────────────────┘
``` ```
Alias: `REGEXP_REPLACE`.
## regexpQuoteMeta(s) ## regexpQuoteMeta(s)
The function adds a backslash before some predefined characters in the string. The function adds a backslash before some predefined characters in the string.

View File

@ -5,6 +5,7 @@
#include <Columns/ColumnConst.h> #include <Columns/ColumnConst.h>
#include <DataTypes/DataTypeString.h> #include <DataTypes/DataTypeString.h>
#include <Functions/FunctionFactory.h> #include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
namespace DB namespace DB
@ -13,16 +14,14 @@ namespace DB
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_COLUMN;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
} }
template <typename Impl, typename Name> template <typename Impl, typename Name>
class FunctionStringReplace : public IFunction class FunctionStringReplace : public IFunction
{ {
public: public:
static constexpr auto name = Name::name; static constexpr auto name = Name::name;
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionStringReplace>(); } static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionStringReplace>(); }
String getName() const override { return name; } String getName() const override { return name; }
@ -32,65 +31,80 @@ public:
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
bool useDefaultImplementationForConstants() const override { return true; } bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{ {
if (!isStringOrFixedString(arguments[0])) FunctionArgumentDescriptors args{
throw Exception( {"haystack", &isStringOrFixedString<IDataType>, nullptr, "String or FixedString"},
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, {"pattern", &isString<IDataType>, nullptr, "String"},
"Illegal type {} of first argument of function {}", {"replacement", &isString<IDataType>, nullptr, "String"}
arguments[0]->getName(), getName()); };
if (!isStringOrFixedString(arguments[1])) validateFunctionArgumentTypes(*this, arguments, args);
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of second argument of function {}",
arguments[1]->getName(), getName());
if (!isStringOrFixedString(arguments[2]))
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of third argument of function {}",
arguments[2]->getName(), getName());
return std::make_shared<DataTypeString>(); return std::make_shared<DataTypeString>();
} }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
{ {
const ColumnPtr column_src = arguments[0].column; const ColumnPtr column_haystack = arguments[0].column;
const ColumnPtr column_needle = arguments[1].column; const ColumnPtr column_needle = arguments[1].column;
const ColumnPtr column_replacement = arguments[2].column; const ColumnPtr column_replacement = arguments[2].column;
if (!isColumnConst(*column_needle) || !isColumnConst(*column_replacement)) const ColumnString * col_haystack = checkAndGetColumn<ColumnString>(column_haystack.get());
throw Exception( const ColumnFixedString * col_haystack_fixed = checkAndGetColumn<ColumnFixedString>(column_haystack.get());
ErrorCodes::ILLEGAL_COLUMN,
"2nd and 3rd arguments of function {} must be constants.",
getName());
const IColumn * c1 = arguments[1].column.get(); const ColumnString * col_needle_vector = checkAndGetColumn<ColumnString>(column_needle.get());
const IColumn * c2 = arguments[2].column.get(); const ColumnConst * col_needle_const = checkAndGetColumn<ColumnConst>(column_needle.get());
const ColumnConst * c1_const = typeid_cast<const ColumnConst *>(c1);
const ColumnConst * c2_const = typeid_cast<const ColumnConst *>(c2);
String needle = c1_const->getValue<String>();
String replacement = c2_const->getValue<String>();
if (needle.empty()) const ColumnString * col_replacement_vector = checkAndGetColumn<ColumnString>(column_replacement.get());
throw Exception( const ColumnConst * col_replacement_const = checkAndGetColumn<ColumnConst>(column_replacement.get());
ErrorCodes::ARGUMENT_OUT_OF_BOUND,
"Length of the second argument of function replace must be greater than 0.");
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column_src.get())) auto col_res = ColumnString::create();
if (col_haystack && col_needle_const && col_replacement_const)
{ {
auto col_res = ColumnString::create(); Impl::vectorConstantConstant(
Impl::vector(col->getChars(), col->getOffsets(), needle, replacement, col_res->getChars(), col_res->getOffsets()); col_haystack->getChars(), col_haystack->getOffsets(),
col_needle_const->getValue<String>(),
col_replacement_const->getValue<String>(),
col_res->getChars(), col_res->getOffsets());
return col_res; return col_res;
} }
else if (const ColumnFixedString * col_fixed = checkAndGetColumn<ColumnFixedString>(column_src.get())) else if (col_haystack && col_needle_vector && col_replacement_const)
{ {
auto col_res = ColumnString::create(); Impl::vectorVectorConstant(
Impl::vectorFixed(col_fixed->getChars(), col_fixed->getN(), needle, replacement, col_res->getChars(), col_res->getOffsets()); col_haystack->getChars(), col_haystack->getOffsets(),
col_needle_vector->getChars(), col_needle_vector->getOffsets(),
col_replacement_const->getValue<String>(),
col_res->getChars(), col_res->getOffsets());
return col_res;
}
else if (col_haystack && col_needle_const && col_replacement_vector)
{
Impl::vectorConstantVector(
col_haystack->getChars(), col_haystack->getOffsets(),
col_needle_const->getValue<String>(),
col_replacement_vector->getChars(), col_replacement_vector->getOffsets(),
col_res->getChars(), col_res->getOffsets());
return col_res;
}
else if (col_haystack && col_needle_vector && col_replacement_vector)
{
Impl::vectorVectorVector(
col_haystack->getChars(), col_haystack->getOffsets(),
col_needle_vector->getChars(), col_needle_vector->getOffsets(),
col_replacement_vector->getChars(), col_replacement_vector->getOffsets(),
col_res->getChars(), col_res->getOffsets());
return col_res;
}
else if (col_haystack_fixed && col_needle_const && col_replacement_const)
{
Impl::vectorFixedConstantConstant(
col_haystack_fixed->getChars(), col_haystack_fixed->getN(),
col_needle_const->getValue<String>(),
col_replacement_const->getValue<String>(),
col_res->getChars(), col_res->getOffsets());
return col_res; return col_res;
} }
else else

View File

@ -13,6 +13,7 @@ namespace DB
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int BAD_ARGUMENTS; extern const int BAD_ARGUMENTS;
} }
@ -28,9 +29,11 @@ struct ReplaceRegexpTraits
/** Replace all matches of regexp 'needle' to string 'replacement'. 'needle' and 'replacement' are constants. /** Replace all matches of regexp 'needle' to string 'replacement'. 'needle' and 'replacement' are constants.
* 'replacement' can contain substitutions, for example: '\2-\3-\1' * 'replacement' can contain substitutions, for example: '\2-\3-\1'
*/ */
template <ReplaceRegexpTraits::Replace replace> template <typename Name, ReplaceRegexpTraits::Replace replace>
struct ReplaceRegexpImpl struct ReplaceRegexpImpl
{ {
static constexpr auto name = Name::name;
struct Instruction struct Instruction
{ {
/// If not negative, perform substitution of n-th subpattern from the regexp match. /// If not negative, perform substitution of n-th subpattern from the regexp match.
@ -162,18 +165,21 @@ struct ReplaceRegexpImpl
++res_offset; ++res_offset;
} }
static void vector( static void vectorConstantConstant(
const ColumnString::Chars & data, const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & offsets, const ColumnString::Offsets & haystack_offsets,
const String & needle, const String & needle,
const String & replacement, const String & replacement,
ColumnString::Chars & res_data, ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets) ColumnString::Offsets & res_offsets)
{ {
if (needle.empty())
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Length of the pattern argument in function {} must be greater than 0.", name);
ColumnString::Offset res_offset = 0; ColumnString::Offset res_offset = 0;
res_data.reserve(data.size()); res_data.reserve(haystack_data.size());
size_t size = offsets.size(); size_t haystack_size = haystack_offsets.size();
res_offsets.resize(size); res_offsets.resize(haystack_size);
re2_st::RE2::Options regexp_options; re2_st::RE2::Options regexp_options;
/// Don't write error messages to stderr. /// Don't write error messages to stderr.
@ -182,39 +188,89 @@ struct ReplaceRegexpImpl
re2_st::RE2 searcher(needle, regexp_options); re2_st::RE2 searcher(needle, regexp_options);
if (!searcher.ok()) if (!searcher.ok())
throw Exception( throw Exception(ErrorCodes::BAD_ARGUMENTS, "The pattern argument is not a valid re2 pattern: {}", searcher.error());
ErrorCodes::BAD_ARGUMENTS,
"The pattern argument is not a valid re2 pattern: {}",
searcher.error());
int num_captures = std::min(searcher.NumberOfCapturingGroups() + 1, max_captures); int num_captures = std::min(searcher.NumberOfCapturingGroups() + 1, max_captures);
Instructions instructions = createInstructions(replacement, num_captures); Instructions instructions = createInstructions(replacement, num_captures);
/// Cannot perform search for whole columns. Will process each string separately. /// Cannot perform search for whole columns. Will process each string separately.
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < haystack_size; ++i)
{ {
size_t from = i > 0 ? offsets[i - 1] : 0; size_t from = i > 0 ? haystack_offsets[i - 1] : 0;
const char * haystack_data = reinterpret_cast<const char *>(data.data() + from);
const size_t haystack_length = static_cast<unsigned>(offsets[i] - from - 1);
processString(haystack_data, haystack_length, res_data, res_offset, searcher, num_captures, instructions); const char * hs_data = reinterpret_cast<const char *>(haystack_data.data() + from);
const size_t hs_length = static_cast<unsigned>(haystack_offsets[i] - from - 1);
processString(hs_data, hs_length, res_data, res_offset, searcher, num_captures, instructions);
res_offsets[i] = res_offset; res_offsets[i] = res_offset;
} }
} }
static void vectorFixed( static void vectorVectorConstant(
const ColumnString::Chars & data, const ColumnString::Chars & haystack_data,
size_t n, const ColumnString::Offsets & haystack_offsets,
const String & needle, const ColumnString::Chars & needle_data,
const ColumnString::Offsets & needle_offsets,
const String & replacement, const String & replacement,
ColumnString::Chars & res_data, ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets) ColumnString::Offsets & res_offsets)
{ {
assert(haystack_offsets.size() == needle_offsets.size());
ColumnString::Offset res_offset = 0; ColumnString::Offset res_offset = 0;
size_t size = data.size() / n; res_data.reserve(haystack_data.size());
res_data.reserve(data.size()); size_t haystack_size = haystack_offsets.size();
res_offsets.resize(size); res_offsets.resize(haystack_size);
re2_st::RE2::Options regexp_options;
/// Don't write error messages to stderr.
regexp_options.set_log_errors(false);
/// Cannot perform search for whole columns. Will process each string separately.
for (size_t i = 0; i < haystack_size; ++i)
{
size_t hs_from = i > 0 ? haystack_offsets[i - 1] : 0;
const char * hs_data = reinterpret_cast<const char *>(haystack_data.data() + hs_from);
const size_t hs_length = static_cast<unsigned>(haystack_offsets[i] - hs_from - 1);
size_t ndl_from = i > 0 ? needle_offsets[i - 1] : 0;
const char * ndl_data = reinterpret_cast<const char *>(needle_data.data() + ndl_from);
const size_t ndl_length = static_cast<unsigned>(needle_offsets[i] - ndl_from - 1);
std::string_view needle(ndl_data, ndl_length);
if (needle.empty())
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Length of the pattern argument in function {} must be greater than 0.", name);
re2_st::RE2 searcher(needle, regexp_options);
if (!searcher.ok())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The pattern argument is not a valid re2 pattern: {}", searcher.error());
int num_captures = std::min(searcher.NumberOfCapturingGroups() + 1, max_captures);
Instructions instructions = createInstructions(replacement, num_captures);
processString(hs_data, hs_length, res_data, res_offset, searcher, num_captures, instructions);
res_offsets[i] = res_offset;
}
}
static void vectorConstantVector(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const String & needle,
const ColumnString::Chars & replacement_data,
const ColumnString::Offsets & replacement_offsets,
ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets)
{
assert(haystack_offsets.size() == replacement_offsets.size());
if (needle.empty())
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Length of the pattern argument in function {} must be greater than 0.", name);
ColumnString::Offset res_offset = 0;
res_data.reserve(haystack_data.size());
size_t haystack_size = haystack_offsets.size();
res_offsets.resize(haystack_size);
re2_st::RE2::Options regexp_options; re2_st::RE2::Options regexp_options;
/// Don't write error messages to stderr. /// Don't write error messages to stderr.
@ -223,22 +279,116 @@ struct ReplaceRegexpImpl
re2_st::RE2 searcher(needle, regexp_options); re2_st::RE2 searcher(needle, regexp_options);
if (!searcher.ok()) if (!searcher.ok())
throw Exception( throw Exception(ErrorCodes::BAD_ARGUMENTS, "The pattern argument is not a valid re2 pattern: {}", searcher.error());
ErrorCodes::BAD_ARGUMENTS,
"The pattern argument is not a valid re2 pattern: {}", int num_captures = std::min(searcher.NumberOfCapturingGroups() + 1, max_captures);
searcher.error());
/// Cannot perform search for whole columns. Will process each string separately.
for (size_t i = 0; i < haystack_size; ++i)
{
size_t hs_from = i > 0 ? haystack_offsets[i - 1] : 0;
const char * hs_data = reinterpret_cast<const char *>(haystack_data.data() + hs_from);
const size_t hs_length = static_cast<unsigned>(haystack_offsets[i] - hs_from - 1);
size_t repl_from = i > 0 ? replacement_offsets[i - 1] : 0;
const char * repl_data = reinterpret_cast<const char *>(replacement_data.data() + repl_from);
const size_t repl_length = static_cast<unsigned>(replacement_offsets[i] - repl_from - 1);
Instructions instructions = createInstructions(std::string_view(repl_data, repl_length), num_captures);
processString(hs_data, hs_length, res_data, res_offset, searcher, num_captures, instructions);
res_offsets[i] = res_offset;
}
}
static void vectorVectorVector(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const ColumnString::Chars & needle_data,
const ColumnString::Offsets & needle_offsets,
const ColumnString::Chars & replacement_data,
const ColumnString::Offsets & replacement_offsets,
ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets)
{
assert(haystack_offsets.size() == needle_offsets.size());
assert(needle_offsets.size() == replacement_offsets.size());
ColumnString::Offset res_offset = 0;
res_data.reserve(haystack_data.size());
size_t haystack_size = haystack_offsets.size();
res_offsets.resize(haystack_size);
re2_st::RE2::Options regexp_options;
/// Don't write error messages to stderr.
regexp_options.set_log_errors(false);
/// Cannot perform search for whole columns. Will process each string separately.
for (size_t i = 0; i < haystack_size; ++i)
{
size_t hs_from = i > 0 ? haystack_offsets[i - 1] : 0;
const char * hs_data = reinterpret_cast<const char *>(haystack_data.data() + hs_from);
const size_t hs_length = static_cast<unsigned>(haystack_offsets[i] - hs_from - 1);
size_t ndl_from = i > 0 ? needle_offsets[i - 1] : 0;
const char * ndl_data = reinterpret_cast<const char *>(needle_data.data() + ndl_from);
const size_t ndl_length = static_cast<unsigned>(needle_offsets[i] - ndl_from - 1);
std::string_view needle(ndl_data, ndl_length);
if (needle.empty())
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Length of the pattern argument in function {} must be greater than 0.", name);
size_t repl_from = i > 0 ? replacement_offsets[i - 1] : 0;
const char * repl_data = reinterpret_cast<const char *>(replacement_data.data() + repl_from);
const size_t repl_length = static_cast<unsigned>(replacement_offsets[i] - repl_from - 1);
re2_st::RE2 searcher(needle, regexp_options);
if (!searcher.ok())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The pattern argument is not a valid re2 pattern: {}", searcher.error());
int num_captures = std::min(searcher.NumberOfCapturingGroups() + 1, max_captures);
Instructions instructions = createInstructions(std::string_view(repl_data, repl_length), num_captures);
processString(hs_data, hs_length, res_data, res_offset, searcher, num_captures, instructions);
res_offsets[i] = res_offset;
}
}
static void vectorFixedConstantConstant(
const ColumnString::Chars & haystack_data,
size_t n,
const String & needle,
const String & replacement,
ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets)
{
if (needle.empty())
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Length of the pattern argument in function {} must be greater than 0.", name);
ColumnString::Offset res_offset = 0;
size_t haystack_size = haystack_data.size() / n;
res_data.reserve(haystack_data.size());
res_offsets.resize(haystack_size);
re2_st::RE2::Options regexp_options;
/// Don't write error messages to stderr.
regexp_options.set_log_errors(false);
re2_st::RE2 searcher(needle, regexp_options);
if (!searcher.ok())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The pattern argument is not a valid re2 pattern: {}", searcher.error());
int num_captures = std::min(searcher.NumberOfCapturingGroups() + 1, max_captures); int num_captures = std::min(searcher.NumberOfCapturingGroups() + 1, max_captures);
Instructions instructions = createInstructions(replacement, num_captures); Instructions instructions = createInstructions(replacement, num_captures);
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < haystack_size; ++i)
{ {
size_t from = i * n; size_t from = i * n;
const char * haystack_data = reinterpret_cast<const char *>(data.data() + from); const char * hs_data = reinterpret_cast<const char *>(haystack_data.data() + from);
const size_t haystack_length = n; const size_t hs_length = n;
processString(haystack_data, haystack_length, res_data, res_offset, searcher, num_captures, instructions); processString(hs_data, hs_length, res_data, res_offset, searcher, num_captures, instructions);
res_offsets[i] = res_offset; res_offsets[i] = res_offset;
} }
} }

View File

@ -8,6 +8,11 @@
namespace DB namespace DB
{ {
namespace ErrorCodes
{
extern const int ARGUMENT_OUT_OF_BOUND;
}
struct ReplaceStringTraits struct ReplaceStringTraits
{ {
enum class Replace enum class Replace
@ -16,27 +21,33 @@ struct ReplaceStringTraits
All All
}; };
}; };
/** Replace one or all occurencies of substring 'needle' to 'replacement'. 'needle' and 'replacement' are constants.
/** Replace one or all occurencies of substring 'needle' to 'replacement'.
*/ */
template <ReplaceStringTraits::Replace replace> template <typename Name, ReplaceStringTraits::Replace replace>
struct ReplaceStringImpl struct ReplaceStringImpl
{ {
static void vector( static constexpr auto name = Name::name;
const ColumnString::Chars & data,
const ColumnString::Offsets & offsets, static void vectorConstantConstant(
const std::string & needle, const ColumnString::Chars & haystack_data,
const std::string & replacement, const ColumnString::Offsets & haystack_offsets,
const String & needle,
const String & replacement,
ColumnString::Chars & res_data, ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets) ColumnString::Offsets & res_offsets)
{ {
const UInt8 * begin = data.data(); if (needle.empty())
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Length of the pattern argument in function {} must be greater than 0.", name);
const UInt8 * const begin = haystack_data.data();
const UInt8 * const end = haystack_data.data() + haystack_data.size();
const UInt8 * pos = begin; const UInt8 * pos = begin;
const UInt8 * end = pos + data.size();
ColumnString::Offset res_offset = 0; ColumnString::Offset res_offset = 0;
res_data.reserve(data.size()); res_data.reserve(haystack_data.size());
size_t size = offsets.size(); const size_t haystack_size = haystack_offsets.size();
res_offsets.resize(size); res_offsets.resize(haystack_size);
/// The current index in the array of strings. /// The current index in the array of strings.
size_t i = 0; size_t i = 0;
@ -53,22 +64,22 @@ struct ReplaceStringImpl
memcpy(&res_data[res_offset], pos, match - pos); memcpy(&res_data[res_offset], pos, match - pos);
/// Determine which index it belongs to. /// Determine which index it belongs to.
while (i < offsets.size() && begin + offsets[i] <= match) while (i < haystack_offsets.size() && begin + haystack_offsets[i] <= match)
{ {
res_offsets[i] = res_offset + ((begin + offsets[i]) - pos); res_offsets[i] = res_offset + ((begin + haystack_offsets[i]) - pos);
++i; ++i;
} }
res_offset += (match - pos); res_offset += (match - pos);
/// If you have reached the end, it's time to stop /// If you have reached the end, it's time to stop
if (i == offsets.size()) if (i == haystack_offsets.size())
break; break;
/// Is it true that this string no longer needs to perform transformations. /// Is it true that this string no longer needs to perform transformations.
bool can_finish_current_string = false; bool can_finish_current_string = false;
/// We check that the entry does not go through the boundaries of strings. /// We check that the entry does not go through the boundaries of strings.
if (match + needle.size() < begin + offsets[i]) if (match + needle.size() < begin + haystack_offsets[i])
{ {
res_data.resize(res_data.size() + replacement.size()); res_data.resize(res_data.size() + replacement.size());
memcpy(&res_data[res_offset], replacement.data(), replacement.size()); memcpy(&res_data[res_offset], replacement.data(), replacement.size());
@ -85,34 +96,268 @@ struct ReplaceStringImpl
if (can_finish_current_string) if (can_finish_current_string)
{ {
res_data.resize(res_data.size() + (begin + offsets[i] - pos)); res_data.resize(res_data.size() + (begin + haystack_offsets[i] - pos));
memcpy(&res_data[res_offset], pos, (begin + offsets[i] - pos)); memcpy(&res_data[res_offset], pos, (begin + haystack_offsets[i] - pos));
res_offset += (begin + offsets[i] - pos); res_offset += (begin + haystack_offsets[i] - pos);
res_offsets[i] = res_offset; res_offsets[i] = res_offset;
pos = begin + offsets[i]; pos = begin + haystack_offsets[i];
++i; ++i;
} }
} }
} }
/// Note: this function converts fixed-length strings to variable-length strings template <typename CharT>
/// and each variable-length string should ends with zero byte. requires (sizeof(CharT) == 1)
static void vectorFixed( static void copyToOutput(
const ColumnString::Chars & data, const CharT * what_start, size_t what_size,
size_t n, ColumnString::Chars & output, size_t & output_offset)
const std::string & needle, {
const std::string & replacement, output.resize(output.size() + what_size);
memcpy(&output[output_offset], what_start, what_size);
output_offset += what_size;
}
static void vectorVectorConstant(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const ColumnString::Chars & needle_data,
const ColumnString::Offsets & needle_offsets,
const String & replacement,
ColumnString::Chars & res_data, ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets) ColumnString::Offsets & res_offsets)
{ {
const UInt8 * begin = data.data(); chassert(haystack_offsets.size() == needle_offsets.size());
const UInt8 * pos = begin;
const UInt8 * end = pos + data.size(); const size_t haystack_size = haystack_offsets.size();
res_data.reserve(haystack_data.size());
res_offsets.resize(haystack_size);
ColumnString::Offset res_offset = 0; ColumnString::Offset res_offset = 0;
size_t count = data.size() / n;
res_data.reserve(data.size()); size_t prev_haystack_offset = 0;
res_offsets.resize(count); size_t prev_needle_offset = 0;
for (size_t i = 0; i < haystack_size; ++i)
{
const auto * const cur_haystack_data = &haystack_data[prev_haystack_offset];
const size_t cur_haystack_length = haystack_offsets[i] - prev_haystack_offset - 1;
const auto * const cur_needle_data = &needle_data[prev_needle_offset];
const size_t cur_needle_length = needle_offsets[i] - prev_needle_offset - 1;
if (cur_needle_length == 0)
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Length of the pattern argument in function {} must be greater than 0.", name);
/// Using "slow" "stdlib searcher instead of Volnitsky because there is a different pattern in each row
StdLibASCIIStringSearcher</*CaseInsensitive*/ false> searcher(cur_needle_data, cur_needle_length);
const auto * last_match = static_cast<UInt8 *>(nullptr);
const auto * start_pos = cur_haystack_data;
const auto * const cur_haystack_end = cur_haystack_data + cur_haystack_length;
while (start_pos < cur_haystack_end)
{
if (const auto * const match = searcher.search(start_pos, cur_haystack_end); match != cur_haystack_end)
{
/// Copy prefix before match
copyToOutput(start_pos, match - start_pos, res_data, res_offset);
/// Insert replacement for match
copyToOutput(replacement.data(), replacement.size(), res_data, res_offset);
last_match = match;
start_pos = match + cur_needle_length;
if constexpr (replace == ReplaceStringTraits::Replace::First)
break;
}
else
break;
}
/// Copy suffix after last match
size_t bytes = (last_match == nullptr) ? (cur_haystack_end - cur_haystack_data + 1)
: (cur_haystack_end - last_match - cur_needle_length + 1);
copyToOutput(start_pos, bytes, res_data, res_offset);
res_offsets[i] = res_offset;
prev_haystack_offset = haystack_offsets[i];
prev_needle_offset = needle_offsets[i];
}
}
static void vectorConstantVector(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const String & needle,
const ColumnString::Chars & replacement_data,
const ColumnString::Offsets & replacement_offsets,
ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets)
{
chassert(haystack_offsets.size() == replacement_offsets.size());
if (needle.empty())
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Length of the pattern argument in function {} must be greater than 0.", name);
const size_t haystack_size = haystack_offsets.size();
res_data.reserve(haystack_data.size());
res_offsets.resize(haystack_size);
ColumnString::Offset res_offset = 0;
size_t prev_haystack_offset = 0;
size_t prev_replacement_offset = 0;
for (size_t i = 0; i < haystack_size; ++i)
{
const auto * const cur_haystack_data = &haystack_data[prev_haystack_offset];
const size_t cur_haystack_length = haystack_offsets[i] - prev_haystack_offset - 1;
const auto * const cur_replacement_data = &replacement_data[prev_replacement_offset];
const size_t cur_replacement_length = replacement_offsets[i] - prev_replacement_offset - 1;
/// Using "slow" "stdlib searcher instead of Volnitsky just to keep things simple
StdLibASCIIStringSearcher</*CaseInsensitive*/ false> searcher(needle.data(), needle.size());
const auto * last_match = static_cast<UInt8 *>(nullptr);
const auto * start_pos = cur_haystack_data;
const auto * const cur_haystack_end = cur_haystack_data + cur_haystack_length;
while (start_pos < cur_haystack_end)
{
if (const auto * const match = searcher.search(start_pos, cur_haystack_end); match != cur_haystack_end)
{
/// Copy prefix before match
copyToOutput(start_pos, match - start_pos, res_data, res_offset);
/// Insert replacement for match
copyToOutput(cur_replacement_data, cur_replacement_length, res_data, res_offset);
last_match = match;
start_pos = match + needle.size();
if constexpr (replace == ReplaceStringTraits::Replace::First)
break;
}
else
break;
}
/// Copy suffix after last match
size_t bytes = (last_match == nullptr) ? (cur_haystack_end - cur_haystack_data + 1)
: (cur_haystack_end - last_match - needle.size() + 1);
copyToOutput(start_pos, bytes, res_data, res_offset);
res_offsets[i] = res_offset;
prev_haystack_offset = haystack_offsets[i];
prev_replacement_offset = replacement_offsets[i];
}
}
static void vectorVectorVector(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const ColumnString::Chars & needle_data,
const ColumnString::Offsets & needle_offsets,
const ColumnString::Chars & replacement_data,
const ColumnString::Offsets & replacement_offsets,
ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets)
{
chassert(haystack_offsets.size() == needle_offsets.size());
chassert(needle_offsets.size() == replacement_offsets.size());
const size_t haystack_size = haystack_offsets.size();
res_data.reserve(haystack_data.size());
res_offsets.resize(haystack_size);
ColumnString::Offset res_offset = 0;
size_t prev_haystack_offset = 0;
size_t prev_needle_offset = 0;
size_t prev_replacement_offset = 0;
for (size_t i = 0; i < haystack_size; ++i)
{
const auto * const cur_haystack_data = &haystack_data[prev_haystack_offset];
const size_t cur_haystack_length = haystack_offsets[i] - prev_haystack_offset - 1;
const auto * const cur_needle_data = &needle_data[prev_needle_offset];
const size_t cur_needle_length = needle_offsets[i] - prev_needle_offset - 1;
const auto * const cur_replacement_data = &replacement_data[prev_replacement_offset];
const size_t cur_replacement_length = replacement_offsets[i] - prev_replacement_offset - 1;
if (cur_needle_length == 0)
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Length of the pattern argument in function {} must be greater than 0.", name);
/// Using "slow" "stdlib searcher instead of Volnitsky because there is a different pattern in each row
StdLibASCIIStringSearcher</*CaseInsensitive*/ false> searcher(cur_needle_data, cur_needle_length);
const auto * last_match = static_cast<UInt8 *>(nullptr);
const auto * start_pos = cur_haystack_data;
const auto * const cur_haystack_end = cur_haystack_data + cur_haystack_length;
while (start_pos < cur_haystack_end)
{
if (const auto * const match = searcher.search(start_pos, cur_haystack_end); match != cur_haystack_end)
{
/// Copy prefix before match
copyToOutput(start_pos, match - start_pos, res_data, res_offset);
/// Insert replacement for match
copyToOutput(cur_replacement_data, cur_replacement_length, res_data, res_offset);
last_match = match;
start_pos = match + cur_needle_length;
if constexpr (replace == ReplaceStringTraits::Replace::First)
break;
}
else
break;
}
/// Copy suffix after last match
size_t bytes = (last_match == nullptr) ? (cur_haystack_end - cur_haystack_data + 1)
: (cur_haystack_end - last_match - cur_needle_length + 1);
copyToOutput(start_pos, bytes, res_data, res_offset);
res_offsets[i] = res_offset;
prev_haystack_offset = haystack_offsets[i];
prev_needle_offset = needle_offsets[i];
prev_replacement_offset = replacement_offsets[i];
}
}
/// Note: this function converts fixed-length strings to variable-length strings
/// and each variable-length string should ends with zero byte.
static void vectorFixedConstantConstant(
const ColumnString::Chars & haystack_data,
size_t n,
const String & needle,
const String & replacement,
ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets)
{
if (needle.empty())
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Length of the pattern argument in function {} must be greater than 0.", name);
const UInt8 * const begin = haystack_data.data();
const UInt8 * const end = haystack_data.data() + haystack_data.size();
const UInt8 * pos = begin;
ColumnString::Offset res_offset = 0;
size_t haystack_size = haystack_data.size() / n;
res_data.reserve(haystack_data.size());
res_offsets.resize(haystack_size);
/// The current index in the string array. /// The current index in the string array.
size_t i = 0; size_t i = 0;
@ -139,13 +384,13 @@ struct ReplaceStringImpl
/// Copy skipped strings without any changes but /// Copy skipped strings without any changes but
/// add zero byte to the end of each string. /// add zero byte to the end of each string.
while (i < count && begin + n * (i + 1) <= match) while (i < haystack_size && begin + n * (i + 1) <= match)
{ {
COPY_REST_OF_CURRENT_STRING(); COPY_REST_OF_CURRENT_STRING();
} }
/// If you have reached the end, it's time to stop /// If you have reached the end, it's time to stop
if (i == count) if (i == haystack_size)
break; break;
/// Copy unchanged part of current string. /// Copy unchanged part of current string.

View File

@ -13,7 +13,7 @@ struct NameReplaceAll
static constexpr auto name = "replaceAll"; static constexpr auto name = "replaceAll";
}; };
using FunctionReplaceAll = FunctionStringReplace<ReplaceStringImpl<ReplaceStringTraits::Replace::All>, NameReplaceAll>; using FunctionReplaceAll = FunctionStringReplace<ReplaceStringImpl<NameReplaceAll, ReplaceStringTraits::Replace::All>, NameReplaceAll>;
} }

View File

@ -13,7 +13,7 @@ struct NameReplaceOne
static constexpr auto name = "replaceOne"; static constexpr auto name = "replaceOne";
}; };
using FunctionReplaceOne = FunctionStringReplace<ReplaceStringImpl<ReplaceStringTraits::Replace::First>, NameReplaceOne>; using FunctionReplaceOne = FunctionStringReplace<ReplaceStringImpl<NameReplaceOne, ReplaceStringTraits::Replace::First>, NameReplaceOne>;
} }

View File

@ -13,7 +13,7 @@ struct NameReplaceRegexpAll
static constexpr auto name = "replaceRegexpAll"; static constexpr auto name = "replaceRegexpAll";
}; };
using FunctionReplaceRegexpAll = FunctionStringReplace<ReplaceRegexpImpl<ReplaceRegexpTraits::Replace::All>, NameReplaceRegexpAll>; using FunctionReplaceRegexpAll = FunctionStringReplace<ReplaceRegexpImpl<NameReplaceRegexpAll, ReplaceRegexpTraits::Replace::All>, NameReplaceRegexpAll>;
} }

View File

@ -13,7 +13,7 @@ struct NameReplaceRegexpOne
static constexpr auto name = "replaceRegexpOne"; static constexpr auto name = "replaceRegexpOne";
}; };
using FunctionReplaceRegexpOne = FunctionStringReplace<ReplaceRegexpImpl<ReplaceRegexpTraits::Replace::First>, NameReplaceRegexpOne>; using FunctionReplaceRegexpOne = FunctionStringReplace<ReplaceRegexpImpl<NameReplaceRegexpOne, ReplaceRegexpTraits::Replace::First>, NameReplaceRegexpOne>;
} }

View File

@ -3,6 +3,7 @@ FOO
foo foo
FOO FOO
baz baz
zzz
2 2
fo fo
oo oo

View File

@ -5,6 +5,7 @@ select ucase('foo');
select LOWER('Foo'); select LOWER('Foo');
select UPPER('Foo'); select UPPER('Foo');
select REPLACE('bar', 'r', 'z'); select REPLACE('bar', 'r', 'z');
select REGEXP_REPLACE('bar', '.', 'z');
select Locate('foo', 'o'); select Locate('foo', 'o');
select SUBSTRING('foo', 1, 2); select SUBSTRING('foo', 1, 2);
select Substr('foo', 2); select Substr('foo', 2);

View File

@ -0,0 +1,77 @@
** replaceAll() **
- non-const needle, const replacement
1 Hello World l x Hexxo Worxd
2 Hello World ll x Hexo World
3 Hello World not_found x Hello World
4 Hello World [eo] x Hello World
5 Hello World . x Hello World
- const needle, non-const replacement
1 Hello World l xx Hexxxxo Worxxd
2 Hello World l x Hexxo Worxd
3 Hello World l x Hexxo Worxd
4 Hello World l x Hexxo Worxd
5 Hello World l x Hexxo Worxd
- non-const needle, non-const replacement
1 Hello World l xx Hexxxxo Worxxd
2 Hello World ll x Hexo World
3 Hello World not_found x Hello World
4 Hello World [eo] x Hello World
5 Hello World . x Hello World
** replaceOne() **
- non-const needle, const replacement
1 Hello World l x Hexlo World
2 Hello World ll x Hexo World
3 Hello World not_found x Hello World
4 Hello World [eo] x Hello World
5 Hello World . x Hello World
- const needle, non-const replacement
1 Hello World l xx Hexxlo World
2 Hello World l x Hexlo World
3 Hello World l x Hexlo World
4 Hello World l x Hexlo World
5 Hello World l x Hexlo World
- non-const needle, non-const replacement
1 Hello World l xx Hexxlo World
2 Hello World ll x Hexo World
3 Hello World not_found x Hello World
4 Hello World [eo] x Hello World
5 Hello World . x Hello World
** replaceRegexpAll() **
- non-const needle, const replacement
1 Hello World l x Hexxo Worxd
2 Hello World ll x Hexo World
3 Hello World not_found x Hello World
4 Hello World [eo] x Hxllx Wxrld
5 Hello World . x xxxxxxxxxxx
- const needle, non-const replacement
1 Hello World l xx Hexxxxo Worxxd
2 Hello World l x Hexxo Worxd
3 Hello World l x Hexxo Worxd
4 Hello World l x Hexxo Worxd
5 Hello World l x Hexxo Worxd
- non-const needle, non-const replacement
1 Hello World l xx Hexxxxo Worxxd
2 Hello World ll x Hexo World
3 Hello World not_found x Hello World
4 Hello World [eo] x Hxllx Wxrld
5 Hello World . x xxxxxxxxxxx
** replaceRegexpOne() **
- non-const needle, const replacement
1 Hello World l x Hexlo World
2 Hello World ll x Hexo World
3 Hello World not_found x Hello World
4 Hello World [eo] x Hxllo World
5 Hello World . x xello World
- const needle, non-const replacement
1 Hello World l xx Hexxlo World
2 Hello World l x Hexlo World
3 Hello World l x Hexlo World
4 Hello World l x Hexlo World
5 Hello World l x Hexlo World
- non-const needle, non-const replacement
1 Hello World l xx Hexxlo World
2 Hello World ll x Hexo World
3 Hello World not_found x Hello World
4 Hello World [eo] x Hxllo World
5 Hello World . x xello World
Check that an exception is thrown if the needle is empty

View File

@ -0,0 +1,90 @@
-- Tests that functions replaceOne(), replaceAll(), replaceRegexpOne(), replaceRegexpAll() work with with non-const pattern and replacement arguments
DROP TABLE IF EXISTS test_tab;
CREATE TABLE test_tab
(id UInt32, haystack String, needle String, replacement String)
engine = MergeTree()
ORDER BY id;
INSERT INTO test_tab VALUES (1, 'Hello World', 'l', 'xx') (2, 'Hello World', 'll', 'x') (3, 'Hello World', 'not_found', 'x') (4, 'Hello World', '[eo]', 'x') (5, 'Hello World', '.', 'x')
SELECT '** replaceAll() **';
SELECT '- non-const needle, const replacement';
SELECT id, haystack, needle, 'x', replaceAll(haystack, needle, 'x') FROM test_tab ORDER BY id;
SELECT '- const needle, non-const replacement';
SELECT id, haystack, 'l', replacement, replaceAll(haystack, 'l', replacement) FROM test_tab ORDER BY id;
SELECT '- non-const needle, non-const replacement';
SELECT id, haystack, needle, replacement, replaceAll(haystack, needle, replacement) FROM test_tab ORDER BY id;
SELECT '** replaceOne() **';
SELECT '- non-const needle, const replacement';
SELECT id, haystack, needle, 'x', replaceOne(haystack, needle, 'x') FROM test_tab ORDER BY id;
SELECT '- const needle, non-const replacement';
SELECT id, haystack, 'l', replacement, replaceOne(haystack, 'l', replacement) FROM test_tab ORDER BY id;
SELECT '- non-const needle, non-const replacement';
SELECT id, haystack, needle, replacement, replaceOne(haystack, needle, replacement) FROM test_tab ORDER BY id;
SELECT '** replaceRegexpAll() **';
SELECT '- non-const needle, const replacement';
SELECT id, haystack, needle, 'x', replaceRegexpAll(haystack, needle, 'x') FROM test_tab ORDER BY id;
SELECT '- const needle, non-const replacement';
SELECT id, haystack, 'l', replacement, replaceRegexpAll(haystack, 'l', replacement) FROM test_tab ORDER BY id;
SELECT '- non-const needle, non-const replacement';
SELECT id, haystack, needle, replacement, replaceRegexpAll(haystack, needle, replacement) FROM test_tab ORDER BY id;
SELECT '** replaceRegexpOne() **';
SELECT '- non-const needle, const replacement';
SELECT id, haystack, needle, 'x', replaceRegexpOne(haystack, needle, 'x') FROM test_tab ORDER BY id;
SELECT '- const needle, non-const replacement';
SELECT id, haystack, 'l', replacement, replaceRegexpOne(haystack, 'l', replacement) FROM test_tab ORDER BY id;
SELECT '- non-const needle, non-const replacement';
SELECT id, haystack, needle, replacement, replaceRegexpOne(haystack, needle, replacement) FROM test_tab ORDER BY id;
DROP TABLE IF EXISTS test_tab;
SELECT 'Check that an exception is thrown if the needle is empty';
CREATE TABLE test_tab
(id UInt32, haystack String, needle String, replacement String)
engine = MergeTree()
ORDER BY id;
INSERT INTO test_tab VALUES (1, 'Hello World', 'l', 'x') (2, 'Hello World', '', 'y')
-- needle: non-const, replacement: const
SELECT replaceAll(haystack, needle, 'x') FROM test_tab; -- { serverError ARGUMENT_OUT_OF_BOUND }
SELECT replaceOne(haystack, needle, 'x') FROM test_tab; -- { serverError ARGUMENT_OUT_OF_BOUND }
SELECT replaceRegexpAll(haystack, needle, 'x') FROM test_tab; -- { serverError ARGUMENT_OUT_OF_BOUND }
SELECT replaceRegexpOne(haystack, needle, 'x') FROM test_tab; -- { serverError ARGUMENT_OUT_OF_BOUND }
-- needle: const, replacement: non-const
SELECT replaceAll(haystack, '', replacement) FROM test_tab; -- { serverError ARGUMENT_OUT_OF_BOUND }
SELECT replaceOne(haystack, '', replacement) FROM test_tab; -- { serverError ARGUMENT_OUT_OF_BOUND }
SELECT replaceRegexpAll(haystack, '', replacement) FROM test_tab; -- { serverError ARGUMENT_OUT_OF_BOUND }
SELECT replaceRegexpOne(haystack, '', replacement) FROM test_tab; -- { serverError ARGUMENT_OUT_OF_BOUND }
-- needle: non-const, replacement: non-const
SELECT replaceAll(haystack, needle, replacement) FROM test_tab; -- { serverError ARGUMENT_OUT_OF_BOUND }
SELECT replaceOne(haystack, needle, replacement) FROM test_tab; -- { serverError ARGUMENT_OUT_OF_BOUND }
SELECT replaceRegexpAll(haystack, needle, replacement) FROM test_tab; -- { serverError ARGUMENT_OUT_OF_BOUND }
SELECT replaceRegexpOne(haystack, needle, replacement) FROM test_tab; -- { serverError ARGUMENT_OUT_OF_BOUND }
DROP TABLE IF EXISTS test_tab;