未验证 提交 3e583ef4 编写于 作者: A alexey-milovidov 提交者: GitHub

Merge pull request #10712 from ClickHouse/merging-extract-groups

Merging #10534
......@@ -3,7 +3,7 @@
#include <Common/OptimizedRegularExpression.h>
#define MIN_LENGTH_FOR_STRSTR 3
#define MAX_SUBPATTERNS 5
#define MAX_SUBPATTERNS 1024
namespace DB
......@@ -454,7 +454,7 @@ unsigned OptimizedRegularExpressionImpl<thread_safe>::match(const char * subject
return 0;
}
DB::PODArrayWithStackMemory<StringPieceType, sizeof(StringPieceType) * (MAX_SUBPATTERNS + 1)> pieces(limit);
DB::PODArrayWithStackMemory<StringPieceType, 128> pieces(limit);
if (!re2->Match(StringPieceType(subject, subject_size), 0, subject_size, RegexType::UNANCHORED, pieces.data(), pieces.size()))
return 0;
......
......@@ -11,19 +11,20 @@ PEERDIR(
NO_COMPILER_WARNINGS()
# Autogenerated from: find . -name '*.cpp' | grep -v -F tests | grep -v -F Trie | sed 's/^\.\// /' | sort
SRCS(
CacheDictionary.cpp
CacheDictionary_generate1.cpp
CacheDictionary_generate2.cpp
CacheDictionary_generate3.cpp
CacheDictionary.cpp
ClickHouseDictionarySource.cpp
ComplexKeyCacheDictionary.cpp
ComplexKeyCacheDictionary_createAttributeWithType.cpp
ComplexKeyCacheDictionary_generate1.cpp
ComplexKeyCacheDictionary_generate2.cpp
ComplexKeyCacheDictionary_generate3.cpp
ComplexKeyCacheDictionary_setAttributeValue.cpp
ComplexKeyCacheDictionary_setDefaultAttributeValue.cpp
ComplexKeyCacheDictionary.cpp
ComplexKeyHashedDictionary.cpp
DictionaryBlockInputStreamBase.cpp
DictionaryFactory.cpp
......
#pragma once
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnConst.h>
#include <DataTypes/DataTypeString.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
template <typename Impl, typename Name>
class FunctionStringReplace : public IFunction
{
public:
static constexpr auto name = Name::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionStringReplace>(); }
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 3; }
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!isStringOrFixedString(arguments[0]))
throw Exception(
"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (!isStringOrFixedString(arguments[1]))
throw Exception(
"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (!isStringOrFixedString(arguments[2]))
throw Exception(
"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeString>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const ColumnPtr column_src = block.getByPosition(arguments[0]).column;
const ColumnPtr column_needle = block.getByPosition(arguments[1]).column;
const ColumnPtr column_replacement = block.getByPosition(arguments[2]).column;
if (!isColumnConst(*column_needle) || !isColumnConst(*column_replacement))
throw Exception("2nd and 3rd arguments of function " + getName() + " must be constants.", ErrorCodes::ILLEGAL_COLUMN);
const IColumn * c1 = block.getByPosition(arguments[1]).column.get();
const IColumn * c2 = block.getByPosition(arguments[2]).column.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())
throw Exception("Length of the second argument of function replace must be greater than 0.", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column_src.get()))
{
auto col_res = ColumnString::create();
Impl::vector(col->getChars(), col->getOffsets(), needle, replacement, col_res->getChars(), col_res->getOffsets());
block.getByPosition(result).column = std::move(col_res);
}
else if (const ColumnFixedString * col_fixed = checkAndGetColumn<ColumnFixedString>(column_src.get()))
{
auto col_res = ColumnString::create();
Impl::vectorFixed(col_fixed->getChars(), col_fixed->getN(), needle, replacement, col_res->getChars(), col_res->getOffsets());
block.getByPosition(result).column = std::move(col_res);
}
else
throw Exception(
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
};
}
此差异已折叠。
#pragma once
#include <common/types.h>
#include <Common/Volnitsky.h>
#include <Columns/ColumnString.h>
#include "Regexps.h"
#if !defined(ARCADIA_BUILD)
# include "config_functions.h"
# include <Common/config.h>
#endif
#if USE_RE2_ST
# include <re2_st/re2.h>
#else
# include <re2/re2.h>
# define re2_st re2
#endif
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
/// Is the LIKE expression reduced to finding a substring in a string?
static inline bool likePatternIsStrstr(const String & pattern, String & res)
{
res = "";
if (pattern.size() < 2 || pattern.front() != '%' || pattern.back() != '%')
return false;
res.reserve(pattern.size() * 2);
const char * pos = pattern.data();
const char * end = pos + pattern.size();
++pos;
--end;
while (pos < end)
{
switch (*pos)
{
case '%':
case '_':
return false;
case '\\':
++pos;
if (pos == end)
return false;
else
res += *pos;
break;
default:
res += *pos;
break;
}
++pos;
}
return true;
}
/** 'like' - if true, treat pattern as SQL LIKE; if false - treat pattern as re2 regexp.
* NOTE: We want to run regexp search for whole block by one call (as implemented in function 'position')
* but for that, regexp engine must support \0 bytes and their interpretation as string boundaries.
*/
template <bool like, bool revert = false>
struct MatchImpl
{
static constexpr bool use_default_implementation_for_constants = true;
using ResultType = UInt8;
static void vectorConstant(
const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & pattern, PaddedPODArray<UInt8> & res)
{
if (offsets.empty())
return;
String strstr_pattern;
/// A simple case where the LIKE expression reduces to finding a substring in a string
if (like && likePatternIsStrstr(pattern, strstr_pattern))
{
const UInt8 * begin = data.data();
const UInt8 * pos = begin;
const UInt8 * end = pos + data.size();
/// The current index in the array of strings.
size_t i = 0;
/// TODO You need to make that `searcher` is common to all the calls of the function.
Volnitsky searcher(strstr_pattern.data(), strstr_pattern.size(), end - pos);
/// We will search for the next occurrence in all rows at once.
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
{
/// Let's determine which index it refers to.
while (begin + offsets[i] <= pos)
{
res[i] = revert;
++i;
}
/// We check that the entry does not pass through the boundaries of strings.
if (pos + strstr_pattern.size() < begin + offsets[i])
res[i] = !revert;
else
res[i] = revert;
pos = begin + offsets[i];
++i;
}
/// Tail, in which there can be no substring.
if (i < res.size())
memset(&res[i], revert, (res.size() - i) * sizeof(res[0]));
}
else
{
size_t size = offsets.size();
const auto & regexp = Regexps::get<like, true>(pattern);
std::string required_substring;
bool is_trivial;
bool required_substring_is_prefix; /// for `anchored` execution of the regexp.
regexp->getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix);
if (required_substring.empty())
{
if (!regexp->getRE2()) /// An empty regexp. Always matches.
{
if (size)
memset(res.data(), 1, size * sizeof(res[0]));
}
else
{
size_t prev_offset = 0;
for (size_t i = 0; i < size; ++i)
{
res[i] = revert
^ regexp->getRE2()->Match(
re2_st::StringPiece(reinterpret_cast<const char *>(&data[prev_offset]), offsets[i] - prev_offset - 1),
0,
offsets[i] - prev_offset - 1,
re2_st::RE2::UNANCHORED,
nullptr,
0);
prev_offset = offsets[i];
}
}
}
else
{
/// NOTE This almost matches with the case of LikePatternIsStrstr.
const UInt8 * begin = data.data();
const UInt8 * pos = begin;
const UInt8 * end = pos + data.size();
/// The current index in the array of strings.
size_t i = 0;
Volnitsky searcher(required_substring.data(), required_substring.size(), end - pos);
/// We will search for the next occurrence in all rows at once.
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
{
/// Determine which index it refers to.
while (begin + offsets[i] <= pos)
{
res[i] = revert;
++i;
}
/// We check that the entry does not pass through the boundaries of strings.
if (pos + strstr_pattern.size() < begin + offsets[i])
{
/// And if it does not, if necessary, we check the regexp.
if (is_trivial)
res[i] = !revert;
else
{
const char * str_data = reinterpret_cast<const char *>(&data[offsets[i - 1]]);
size_t str_size = offsets[i] - offsets[i - 1] - 1;
/** Even in the case of `required_substring_is_prefix` use UNANCHORED check for regexp,
* so that it can match when `required_substring` occurs into the string several times,
* and at the first occurrence, the regexp is not a match.
*/
if (required_substring_is_prefix)
res[i] = revert
^ regexp->getRE2()->Match(
re2_st::StringPiece(str_data, str_size),
reinterpret_cast<const char *>(pos) - str_data,
str_size,
re2_st::RE2::UNANCHORED,
nullptr,
0);
else
res[i] = revert
^ regexp->getRE2()->Match(
re2_st::StringPiece(str_data, str_size), 0, str_size, re2_st::RE2::UNANCHORED, nullptr, 0);
}
}
else
res[i] = revert;
pos = begin + offsets[i];
++i;
}
if (i < res.size())
memset(&res[i], revert, (res.size() - i) * sizeof(res[0]));
}
}
}
/// Very carefully crafted copy-paste.
static void vectorFixedConstant(
const ColumnString::Chars & data, size_t n, const std::string & pattern, PaddedPODArray<UInt8> & res)
{
if (data.empty())
return;
String strstr_pattern;
/// A simple case where the LIKE expression reduces to finding a substring in a string
if (like && likePatternIsStrstr(pattern, strstr_pattern))
{
const UInt8 * begin = data.data();
const UInt8 * pos = begin;
const UInt8 * end = pos + data.size();
size_t i = 0;
const UInt8 * next_pos = begin;
/// If pattern is larger than string size - it cannot be found.
if (strstr_pattern.size() <= n)
{
Volnitsky searcher(strstr_pattern.data(), strstr_pattern.size(), end - pos);
/// We will search for the next occurrence in all rows at once.
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
{
/// Let's determine which index it refers to.
while (next_pos + n <= pos)
{
res[i] = revert;
next_pos += n;
++i;
}
next_pos += n;
/// We check that the entry does not pass through the boundaries of strings.
if (pos + strstr_pattern.size() <= next_pos)
res[i] = !revert;
else
res[i] = revert;
pos = next_pos;
++i;
}
}
/// Tail, in which there can be no substring.
if (i < res.size())
memset(&res[i], revert, (res.size() - i) * sizeof(res[0]));
}
else
{
size_t size = data.size() / n;
const auto & regexp = Regexps::get<like, true>(pattern);
std::string required_substring;
bool is_trivial;
bool required_substring_is_prefix; /// for `anchored` execution of the regexp.
regexp->getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix);
if (required_substring.empty())
{
if (!regexp->getRE2()) /// An empty regexp. Always matches.
{
if (size)
memset(res.data(), 1, size * sizeof(res[0]));
}
else
{
size_t offset = 0;
for (size_t i = 0; i < size; ++i)
{
res[i] = revert
^ regexp->getRE2()->Match(
re2_st::StringPiece(reinterpret_cast<const char *>(&data[offset]), n),
0,
n,
re2_st::RE2::UNANCHORED,
nullptr,
0);
offset += n;
}
}
}
else
{
/// NOTE This almost matches with the case of LikePatternIsStrstr.
const UInt8 * begin = data.data();
const UInt8 * pos = begin;
const UInt8 * end = pos + data.size();
size_t i = 0;
const UInt8 * next_pos = begin;
/// If required substring is larger than string size - it cannot be found.
if (strstr_pattern.size() <= n)
{
Volnitsky searcher(required_substring.data(), required_substring.size(), end - pos);
/// We will search for the next occurrence in all rows at once.
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
{
/// Let's determine which index it refers to.
while (next_pos + n <= pos)
{
res[i] = revert;
next_pos += n;
++i;
}
next_pos += n;
if (pos + strstr_pattern.size() <= next_pos)
{
/// And if it does not, if necessary, we check the regexp.
if (is_trivial)
res[i] = !revert;
else
{
const char * str_data = reinterpret_cast<const char *>(next_pos - n);
/** Even in the case of `required_substring_is_prefix` use UNANCHORED check for regexp,
* so that it can match when `required_substring` occurs into the string several times,
* and at the first occurrence, the regexp is not a match.
*/
if (required_substring_is_prefix)
res[i] = revert
^ regexp->getRE2()->Match(
re2_st::StringPiece(str_data, n),
reinterpret_cast<const char *>(pos) - str_data,
n,
re2_st::RE2::UNANCHORED,
nullptr,
0);
else
res[i] = revert
^ regexp->getRE2()->Match(
re2_st::StringPiece(str_data, n), 0, n, re2_st::RE2::UNANCHORED, nullptr, 0);
}
}
else
res[i] = revert;
pos = next_pos;
++i;
}
}
/// Tail, in which there can be no substring.
if (i < res.size())
memset(&res[i], revert, (res.size() - i) * sizeof(res[0]));
}
}
}
template <typename... Args>
static void vectorVector(Args &&...)
{
throw Exception("Functions 'like' and 'match' don't support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN);
}
/// Search different needles in single haystack.
template <typename... Args>
static void constantVector(Args &&...)
{
throw Exception("Functions 'like' and 'match' don't support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN);
}
};
}
#pragma once
#include <common/types.h>
#include <Columns/ColumnString.h>
#include <DataTypes/DataTypesNumber.h>
#include "Regexps.h"
#if !defined(ARCADIA_BUILD)
# include "config_functions.h"
# include <Common/config.h>
#endif
#if USE_HYPERSCAN
# include <hs.h>
#else
# include "MatchImpl.h"
#endif
namespace DB
{
namespace ErrorCodes
{
extern const int HYPERSCAN_CANNOT_SCAN_TEXT;
extern const int CANNOT_ALLOCATE_MEMORY;
extern const int NOT_IMPLEMENTED;
extern const int TOO_MANY_BYTES;
}
template <typename Type, bool MultiSearchDistance>
struct MultiMatchAllIndicesImpl
{
using ResultType = Type;
static constexpr bool is_using_hyperscan = true;
/// Variable for understanding, if we used offsets for the output, most
/// likely to determine whether the function returns ColumnVector of ColumnArray.
static constexpr bool is_column_array = true;
static auto getReturnType()
{
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
}
static void vectorConstant(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const std::vector<StringRef> & needles,
PaddedPODArray<Type> & res,
PaddedPODArray<UInt64> & offsets)
{
vectorConstant(haystack_data, haystack_offsets, needles, res, offsets, std::nullopt);
}
static void vectorConstant(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const std::vector<StringRef> & needles,
PaddedPODArray<Type> & res,
PaddedPODArray<UInt64> & offsets,
[[maybe_unused]] std::optional<UInt32> edit_distance)
{
offsets.resize(haystack_offsets.size());
#if USE_HYPERSCAN
const auto & hyperscan_regex = MultiRegexps::get</*SaveIndices=*/true, MultiSearchDistance>(needles, edit_distance);
hs_scratch_t * scratch = nullptr;
hs_error_t err = hs_clone_scratch(hyperscan_regex->getScratch(), &scratch);
if (err != HS_SUCCESS)
throw Exception("Could not clone scratch space for hyperscan", ErrorCodes::CANNOT_ALLOCATE_MEMORY);
MultiRegexps::ScratchPtr smart_scratch(scratch);
auto on_match = [](unsigned int id,
unsigned long long /* from */, // NOLINT
unsigned long long /* to */, // NOLINT
unsigned int /* flags */,
void * context) -> int
{
static_cast<PaddedPODArray<Type>*>(context)->push_back(id);
return 0;
};
const size_t haystack_offsets_size = haystack_offsets.size();
UInt64 offset = 0;
for (size_t i = 0; i < haystack_offsets_size; ++i)
{
UInt64 length = haystack_offsets[i] - offset - 1;
/// Hyperscan restriction.
if (length > std::numeric_limits<UInt32>::max())
throw Exception("Too long string to search", ErrorCodes::TOO_MANY_BYTES);
/// Scan, check, update the offsets array and the offset of haystack.
err = hs_scan(
hyperscan_regex->getDB(),
reinterpret_cast<const char *>(haystack_data.data()) + offset,
length,
0,
smart_scratch.get(),
on_match,
&res);
if (err != HS_SUCCESS)
throw Exception("Failed to scan with hyperscan", ErrorCodes::HYPERSCAN_CANNOT_SCAN_TEXT);
offsets[i] = res.size();
offset = haystack_offsets[i];
}
#else
(void)haystack_data;
(void)haystack_offsets;
(void)needles;
(void)res;
(void)offsets;
throw Exception(
"multi-search all indices is not implemented when hyperscan is off (is it x86 processor?)",
ErrorCodes::NOT_IMPLEMENTED);
#endif // USE_HYPERSCAN
}
};
}
#pragma once
#include <common/types.h>
#include <Columns/ColumnString.h>
#include <DataTypes/DataTypesNumber.h>
#include "Regexps.h"
#if !defined(ARCADIA_BUILD)
# include "config_functions.h"
# include <Common/config.h>
#endif
#if USE_HYPERSCAN
# include <hs.h>
#else
# include "MatchImpl.h"
#endif
namespace DB
{
namespace ErrorCodes
{
extern const int HYPERSCAN_CANNOT_SCAN_TEXT;
extern const int CANNOT_ALLOCATE_MEMORY;
extern const int NOT_IMPLEMENTED;
extern const int TOO_MANY_BYTES;
}
template <typename Type, bool FindAny, bool FindAnyIndex, bool MultiSearchDistance>
struct MultiMatchAnyImpl
{
static_assert(static_cast<int>(FindAny) + static_cast<int>(FindAnyIndex) == 1);
using ResultType = Type;
static constexpr bool is_using_hyperscan = true;
/// Variable for understanding, if we used offsets for the output, most
/// likely to determine whether the function returns ColumnVector of ColumnArray.
static constexpr bool is_column_array = false;
static auto getReturnType()
{
return std::make_shared<DataTypeNumber<ResultType>>();
}
static void vectorConstant(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const std::vector<StringRef> & needles,
PaddedPODArray<Type> & res,
PaddedPODArray<UInt64> & offsets)
{
vectorConstant(haystack_data, haystack_offsets, needles, res, offsets, std::nullopt);
}
static void vectorConstant(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const std::vector<StringRef> & needles,
PaddedPODArray<Type> & res,
[[maybe_unused]] PaddedPODArray<UInt64> & offsets,
[[maybe_unused]] std::optional<UInt32> edit_distance)
{
(void)FindAny;
(void)FindAnyIndex;
res.resize(haystack_offsets.size());
#if USE_HYPERSCAN
const auto & hyperscan_regex = MultiRegexps::get<FindAnyIndex, MultiSearchDistance>(needles, edit_distance);
hs_scratch_t * scratch = nullptr;
hs_error_t err = hs_clone_scratch(hyperscan_regex->getScratch(), &scratch);
if (err != HS_SUCCESS)
throw Exception("Could not clone scratch space for hyperscan", ErrorCodes::CANNOT_ALLOCATE_MEMORY);
MultiRegexps::ScratchPtr smart_scratch(scratch);
auto on_match = []([[maybe_unused]] unsigned int id,
unsigned long long /* from */, // NOLINT
unsigned long long /* to */, // NOLINT
unsigned int /* flags */,
void * context) -> int
{
if constexpr (FindAnyIndex)
*reinterpret_cast<Type *>(context) = id;
else if constexpr (FindAny)
*reinterpret_cast<Type *>(context) = 1;
/// Once we hit the callback, there is no need to search for others.
return 1;
};
const size_t haystack_offsets_size = haystack_offsets.size();
UInt64 offset = 0;
for (size_t i = 0; i < haystack_offsets_size; ++i)
{
UInt64 length = haystack_offsets[i] - offset - 1;
/// Hyperscan restriction.
if (length > std::numeric_limits<UInt32>::max())
throw Exception("Too long string to search", ErrorCodes::TOO_MANY_BYTES);
/// Zero the result, scan, check, update the offset.
res[i] = 0;
err = hs_scan(
hyperscan_regex->getDB(),
reinterpret_cast<const char *>(haystack_data.data()) + offset,
length,
0,
smart_scratch.get(),
on_match,
&res[i]);
if (err != HS_SUCCESS && err != HS_SCAN_TERMINATED)
throw Exception("Failed to scan with hyperscan", ErrorCodes::HYPERSCAN_CANNOT_SCAN_TEXT);
offset = haystack_offsets[i];
}
#else
/// Fallback if do not use hyperscan
if constexpr (MultiSearchDistance)
throw Exception(
"Edit distance multi-search is not implemented when hyperscan is off (is it x86 processor?)",
ErrorCodes::NOT_IMPLEMENTED);
PaddedPODArray<UInt8> accum(res.size());
memset(res.data(), 0, res.size() * sizeof(res.front()));
memset(accum.data(), 0, accum.size());
for (size_t j = 0; j < needles.size(); ++j)
{
MatchImpl<false, false>::vectorConstant(haystack_data, haystack_offsets, needles[j].toString(), accum);
for (size_t i = 0; i < res.size(); ++i)
{
if constexpr (FindAny)
res[i] |= accum[i];
else if (FindAnyIndex && accum[i])
res[i] = j + 1;
}
}
#endif // USE_HYPERSCAN
}
};
}
#pragma once
#include <common/types.h>
#include <Common/Volnitsky.h>
#include <Columns/ColumnString.h>
#include <IO/WriteHelpers.h>
#if !defined(ARCADIA_BUILD)
# include "config_functions.h"
# include <Common/config.h>
#endif
#if USE_RE2_ST
# include <re2_st/re2.h>
#else
# include <re2/re2.h>
# define re2_st re2
#endif
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
/** Replace all matches of regexp 'needle' to string 'replacement'. 'needle' and 'replacement' are constants.
* 'replacement' could contain substitutions, for example: '\2-\3-\1'
*/
template <bool replace_one = false>
struct ReplaceRegexpImpl
{
/// Sequence of instructions, describing how to get resulting string.
/// Each element is either:
/// - substitution (in that case first element of pair is their number and second element is empty)
/// - string that need to be inserted (in that case, first element of pair is that string and second element is -1)
using Instructions = std::vector<std::pair<int, std::string>>;
static const size_t max_captures = 10;
static Instructions createInstructions(const std::string & s, int num_captures)
{
Instructions instructions;
String now;
for (size_t i = 0; i < s.size(); ++i)
{
if (s[i] == '\\' && i + 1 < s.size())
{
if (isNumericASCII(s[i + 1])) /// Substitution
{
if (!now.empty())
{
instructions.emplace_back(-1, now);
now = "";
}
instructions.emplace_back(s[i + 1] - '0', String());
}
else
now += s[i + 1]; /// Escaping
++i;
}
else
now += s[i]; /// Plain character
}
if (!now.empty())
{
instructions.emplace_back(-1, now);
now = "";
}
for (const auto & it : instructions)
if (it.first >= num_captures)
throw Exception(
"Invalid replace instruction in replacement string. Id: " + toString(it.first) + ", but regexp has only "
+ toString(num_captures - 1) + " subpatterns",
ErrorCodes::BAD_ARGUMENTS);
return instructions;
}
static void processString(
const re2_st::StringPiece & input,
ColumnString::Chars & res_data,
ColumnString::Offset & res_offset,
re2_st::RE2 & searcher,
int num_captures,
const Instructions & instructions)
{
re2_st::StringPiece matches[max_captures];
size_t start_pos = 0;
while (start_pos < static_cast<size_t>(input.length()))
{
/// If no more replacements possible for current string
bool can_finish_current_string = false;
if (searcher.Match(input, start_pos, input.length(), re2_st::RE2::Anchor::UNANCHORED, matches, num_captures))
{
const auto & match = matches[0];
size_t bytes_to_copy = (match.data() - input.data()) - start_pos;
/// Copy prefix before matched regexp without modification
res_data.resize(res_data.size() + bytes_to_copy);
memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data() + start_pos, bytes_to_copy);
res_offset += bytes_to_copy;
start_pos += bytes_to_copy + match.length();
/// Do substitution instructions
for (const auto & it : instructions)
{
if (it.first >= 0)
{
res_data.resize(res_data.size() + matches[it.first].length());
memcpy(&res_data[res_offset], matches[it.first].data(), matches[it.first].length());
res_offset += matches[it.first].length();
}
else
{
res_data.resize(res_data.size() + it.second.size());
memcpy(&res_data[res_offset], it.second.data(), it.second.size());
res_offset += it.second.size();
}
}
if (replace_one || match.length() == 0) /// Stop after match of zero length, to avoid infinite loop.
can_finish_current_string = true;
}
else
can_finish_current_string = true;
/// If ready, append suffix after match to end of string.
if (can_finish_current_string)
{
res_data.resize(res_data.size() + input.length() - start_pos);
memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data() + start_pos, input.length() - start_pos);
res_offset += input.length() - start_pos;
start_pos = input.length();
}
}
res_data.resize(res_data.size() + 1);
res_data[res_offset] = 0;
++res_offset;
}
static void vector(
const ColumnString::Chars & data,
const ColumnString::Offsets & offsets,
const std::string & needle,
const std::string & replacement,
ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets)
{
ColumnString::Offset res_offset = 0;
res_data.reserve(data.size());
size_t size = offsets.size();
res_offsets.resize(size);
re2_st::RE2 searcher(needle);
int num_captures = std::min(searcher.NumberOfCapturingGroups() + 1, static_cast<int>(max_captures));
Instructions instructions = createInstructions(replacement, num_captures);
/// Cannot perform search for whole block. Will process each string separately.
for (size_t i = 0; i < size; ++i)
{
int from = i > 0 ? offsets[i - 1] : 0;
re2_st::StringPiece input(reinterpret_cast<const char *>(data.data() + from), offsets[i] - from - 1);
processString(input, res_data, res_offset, searcher, num_captures, instructions);
res_offsets[i] = res_offset;
}
}
static void vectorFixed(
const ColumnString::Chars & data,
size_t n,
const std::string & needle,
const std::string & replacement,
ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets)
{
ColumnString::Offset res_offset = 0;
size_t size = data.size() / n;
res_data.reserve(data.size());
res_offsets.resize(size);
re2_st::RE2 searcher(needle);
int num_captures = std::min(searcher.NumberOfCapturingGroups() + 1, static_cast<int>(max_captures));
Instructions instructions = createInstructions(replacement, num_captures);
for (size_t i = 0; i < size; ++i)
{
int from = i * n;
re2_st::StringPiece input(reinterpret_cast<const char *>(data.data() + from), n);
processString(input, res_data, res_offset, searcher, num_captures, instructions);
res_offsets[i] = res_offset;
}
}
};
}
#pragma once
#include <common/types.h>
#include <Common/Volnitsky.h>
#include <Columns/ColumnString.h>
namespace DB
{
/** Replace one or all occurencies of substring 'needle' to 'replacement'. 'needle' and 'replacement' are constants.
*/
template <bool replace_one = false>
struct ReplaceStringImpl
{
static void vector(
const ColumnString::Chars & data,
const ColumnString::Offsets & offsets,
const std::string & needle,
const std::string & replacement,
ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets)
{
const UInt8 * begin = data.data();
const UInt8 * pos = begin;
const UInt8 * end = pos + data.size();
ColumnString::Offset res_offset = 0;
res_data.reserve(data.size());
size_t size = offsets.size();
res_offsets.resize(size);
/// The current index in the array of strings.
size_t i = 0;
Volnitsky searcher(needle.data(), needle.size(), end - pos);
/// We will search for the next occurrence in all rows at once.
while (pos < end)
{
const UInt8 * match = searcher.search(pos, end - pos);
/// Copy the data without changing
res_data.resize(res_data.size() + (match - pos));
memcpy(&res_data[res_offset], pos, match - pos);
/// Determine which index it belongs to.
while (i < offsets.size() && begin + offsets[i] <= match)
{
res_offsets[i] = res_offset + ((begin + offsets[i]) - pos);
++i;
}
res_offset += (match - pos);
/// If you have reached the end, it's time to stop
if (i == offsets.size())
break;
/// Is it true that this string no longer needs to perform transformations.
bool can_finish_current_string = false;
/// We check that the entry does not go through the boundaries of strings.
if (match + needle.size() < begin + offsets[i])
{
res_data.resize(res_data.size() + replacement.size());
memcpy(&res_data[res_offset], replacement.data(), replacement.size());
res_offset += replacement.size();
pos = match + needle.size();
if (replace_one)
can_finish_current_string = true;
}
else
{
pos = match;
can_finish_current_string = true;
}
if (can_finish_current_string)
{
res_data.resize(res_data.size() + (begin + offsets[i] - pos));
memcpy(&res_data[res_offset], pos, (begin + offsets[i] - pos));
res_offset += (begin + offsets[i] - pos);
res_offsets[i] = res_offset;
pos = begin + offsets[i];
++i;
}
}
}
/// Note: this function converts fixed-length strings to variable-length strings
/// and each variable-length string should ends with zero byte.
static void vectorFixed(
const ColumnString::Chars & data,
size_t n,
const std::string & needle,
const std::string & replacement,
ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets)
{
const UInt8 * begin = data.data();
const UInt8 * pos = begin;
const UInt8 * end = pos + data.size();
ColumnString::Offset res_offset = 0;
size_t count = data.size() / n;
res_data.reserve(data.size());
res_offsets.resize(count);
/// The current index in the string array.
size_t i = 0;
Volnitsky searcher(needle.data(), needle.size(), end - pos);
/// We will search for the next occurrence in all rows at once.
while (pos < end)
{
const UInt8 * match = searcher.search(pos, end - pos);
#define COPY_REST_OF_CURRENT_STRING() \
do \
{ \
const size_t len = begin + n * (i + 1) - pos; \
res_data.resize(res_data.size() + len + 1); \
memcpy(&res_data[res_offset], pos, len); \
res_offset += len; \
res_data[res_offset++] = 0; \
res_offsets[i] = res_offset; \
pos = begin + n * (i + 1); \
++i; \
} while (false)
/// Copy skipped strings without any changes but
/// add zero byte to the end of each string.
while (i < count && begin + n * (i + 1) <= match)
{
COPY_REST_OF_CURRENT_STRING();
}
/// If you have reached the end, it's time to stop
if (i == count)
break;
/// Copy unchanged part of current string.
res_data.resize(res_data.size() + (match - pos));
memcpy(&res_data[res_offset], pos, match - pos);
res_offset += (match - pos);
/// Is it true that this string no longer needs to perform conversions.
bool can_finish_current_string = false;
/// We check that the entry does not pass through the boundaries of strings.
if (match + needle.size() <= begin + n * (i + 1))
{
res_data.resize(res_data.size() + replacement.size());
memcpy(&res_data[res_offset], replacement.data(), replacement.size());
res_offset += replacement.size();
pos = match + needle.size();
if (replace_one || pos == begin + n * (i + 1))
can_finish_current_string = true;
}
else
{
pos = match;
can_finish_current_string = true;
}
if (can_finish_current_string)
{
COPY_REST_OF_CURRENT_STRING();
}
#undef COPY_REST_OF_CURRENT_STRING
}
}
};
}
#include "FunctionsStringSearchToString.h"
#include "FunctionFactory.h"
#include "Regexps.h"
#include <Common/OptimizedRegularExpression.h>
namespace DB
{
struct ExtractImpl
{
static void vector(
const ColumnString::Chars & data,
const ColumnString::Offsets & offsets,
const std::string & pattern,
ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets)
{
res_data.reserve(data.size() / 5);
res_offsets.resize(offsets.size());
const auto & regexp = Regexps::get<false, false>(pattern);
unsigned capture = regexp->getNumberOfSubpatterns() > 0 ? 1 : 0;
OptimizedRegularExpression::MatchVec matches;
matches.reserve(capture + 1);
size_t prev_offset = 0;
size_t res_offset = 0;
for (size_t i = 0; i < offsets.size(); ++i)
{
size_t cur_offset = offsets[i];
unsigned count
= regexp->match(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1, matches, capture + 1);
if (count > capture && matches[capture].offset != std::string::npos)
{
const auto & match = matches[capture];
res_data.resize(res_offset + match.length + 1);
memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &data[prev_offset + match.offset], match.length);
res_offset += match.length;
}
else
{
res_data.resize(res_offset + 1);
}
res_data[res_offset] = 0;
++res_offset;
res_offsets[i] = res_offset;
prev_offset = cur_offset;
}
}
};
struct NameExtract
{
static constexpr auto name = "extract";
};
using FunctionExtract = FunctionsStringSearchToString<ExtractImpl, NameExtract>;
void registerFunctionExtract(FunctionFactory & factory)
{
factory.registerFunction<FunctionExtract>();
}
}
#include <Columns/ColumnString.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnConst.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeString.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/Regexps.h>
#include <memory>
#include <string>
#include <vector>
namespace DB
{
namespace ErrorCodes
{
extern const int ARGUMENT_OUT_OF_BOUND;
}
/** Match all groups of given input string with given re, return array of arrays of matches.
*
* SELECT extractAllGroups('abc=111, def=222, ghi=333', '("[^"]+"|\\w+)=("[^"]+"|\\w+)')
* should produce:
* [['abc', '111'], ['def', '222'], ['ghi', '333']]
*/
class FunctionExtractAllGroups : public IFunction
{
public:
static constexpr auto name = "extractAllGroups";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionExtractAllGroups>(); }
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 2; }
bool useDefaultImplementationForConstants() const override { return false; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
FunctionArgumentDescriptors args{
{"haystack", isStringOrFixedString, nullptr, "const String or const FixedString"},
{"needle", isStringOrFixedString, isColumnConst, "const String or const FixedString"},
};
validateFunctionArgumentTypes(*this, arguments, args);
/// Two-dimensional array of strings, each `row` of top array represents matching groups.
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()));
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const ColumnPtr column_haystack = block.getByPosition(arguments[0]).column;
const ColumnPtr column_needle = block.getByPosition(arguments[1]).column;
const auto needle = typeid_cast<const ColumnConst &>(*column_needle).getValue<String>();
if (needle.empty())
throw Exception(getName() + " length of 'needle' argument must be greater than 0.", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
const auto regexp = Regexps::get<false, false>(needle);
const auto & re2 = regexp->getRE2();
const size_t groups_count = re2->NumberOfCapturingGroups();
// Including 0-group, which is the whole regexp.
PODArrayWithStackMemory<re2_st::StringPiece, 128> matched_groups(groups_count + 1);
ColumnArray::ColumnOffsets::MutablePtr root_offsets_col = ColumnArray::ColumnOffsets::create();
ColumnArray::ColumnOffsets::MutablePtr nested_offsets_col = ColumnArray::ColumnOffsets::create();
ColumnString::MutablePtr data_col = ColumnString::create();
auto & root_offsets_data = root_offsets_col->getData();
auto & nested_offsets_data = nested_offsets_col->getData();
root_offsets_data.resize(input_rows_count);
ColumnArray::Offset current_root_offset = 0;
ColumnArray::Offset current_nested_offset = 0;
for (size_t i = 0; i < input_rows_count; ++i)
{
StringRef current_row = column_haystack->getDataAt(i);
// Extract all non-intersecting matches from haystack except group #0.
const auto * pos = current_row.data;
const auto * end = pos + current_row.size;
while (pos < end
&& re2->Match(re2_st::StringPiece(pos, end - pos),
0, end - pos, re2_st::RE2::UNANCHORED, matched_groups.data(), matched_groups.size()))
{
// 1 is to exclude group #0 which is whole re match.
for (size_t group = 1; group <= groups_count; ++group)
data_col->insertData(matched_groups[group].data(), matched_groups[group].size());
pos = matched_groups[0].data() + matched_groups[0].size();
current_nested_offset += groups_count;
nested_offsets_data.push_back(current_nested_offset);
++current_root_offset;
}
root_offsets_data[i] = current_root_offset;
}
ColumnArray::MutablePtr nested_array_col = ColumnArray::create(std::move(data_col), std::move(nested_offsets_col));
ColumnArray::MutablePtr root_array_col = ColumnArray::create(std::move(nested_array_col), std::move(root_offsets_col));
block.getByPosition(result).column = std::move(root_array_col);
}
};
void registerFunctionExtractAllGroups(FunctionFactory & factory)
{
factory.registerFunction<FunctionExtractAllGroups>();
}
}
#include <Columns/ColumnString.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnConst.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeString.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/Regexps.h>
#include <memory>
#include <string>
#include <vector>
namespace DB
{
namespace ErrorCodes
{
extern const int ARGUMENT_OUT_OF_BOUND;
}
/** Match all groups of given input string with given re, return array of arrays of matches.
*
* SELECT extractGroups('hello abc=111 world', '("[^"]+"|\\w+)=("[^"]+"|\\w+)')
* should produce:
* ['abc', '111']
*/
class FunctionExtractGroups : public IFunction
{
public:
static constexpr auto name = "extractGroups";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionExtractGroups>(); }
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 2; }
bool useDefaultImplementationForConstants() const override { return false; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
FunctionArgumentDescriptors args{
{"haystack", isStringOrFixedString, nullptr, "const String or const FixedString"},
{"needle", isStringOrFixedString, isColumnConst, "const String or const FixedString"},
};
validateFunctionArgumentTypes(*this, arguments, args);
/// Two-dimensional array of strings, each `row` of top array represents matching groups.
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>());
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const ColumnPtr column_haystack = block.getByPosition(arguments[0]).column;
const ColumnPtr column_needle = block.getByPosition(arguments[1]).column;
const auto needle = typeid_cast<const ColumnConst &>(*column_needle).getValue<String>();
if (needle.empty())
throw Exception(getName() + " length of 'needle' argument must be greater than 0.", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
const auto regexp = Regexps::get<false, false>(needle);
const auto & re2 = regexp->getRE2();
const size_t groups_count = re2->NumberOfCapturingGroups();
// Including 0-group, which is the whole regexp.
PODArrayWithStackMemory<re2_st::StringPiece, 128> matched_groups(groups_count + 1);
ColumnArray::ColumnOffsets::MutablePtr offsets_col = ColumnArray::ColumnOffsets::create();
ColumnString::MutablePtr data_col = ColumnString::create();
auto & offsets_data = offsets_col->getData();
offsets_data.resize(input_rows_count);
ColumnArray::Offset current_offset = 0;
for (size_t i = 0; i < input_rows_count; ++i)
{
StringRef current_row = column_haystack->getDataAt(i);
if (re2->Match(re2_st::StringPiece(current_row.data, current_row.size),
0, current_row.size, re2_st::RE2::UNANCHORED, matched_groups.data(), matched_groups.size()))
{
// 1 is to exclude group #0 which is whole re match.
for (size_t group = 1; group <= groups_count; ++group)
data_col->insertData(matched_groups[group].data(), matched_groups[group].size());
current_offset += groups_count;
}
offsets_data[i] = current_offset;
}
block.getByPosition(result).column = ColumnArray::create(std::move(data_col), std::move(offsets_col));
}
};
void registerFunctionExtractGroups(FunctionFactory & factory)
{
factory.registerFunction<FunctionExtractGroups>();
}
}
#include "FunctionsStringSearch.h"
#include "FunctionFactory.h"
#include "MatchImpl.h"
namespace DB
{
struct NameLike
{
static constexpr auto name = "like";
};
using FunctionLike = FunctionsStringSearch<MatchImpl<true>, NameLike>;
void registerFunctionLike(FunctionFactory & factory)
{
factory.registerFunction<FunctionLike>();
}
}
#include "FunctionsStringSearch.h"
#include "FunctionFactory.h"
#include "MatchImpl.h"
namespace DB
{
struct NameMatch
{
static constexpr auto name = "match";
};
using FunctionMatch = FunctionsStringSearch<MatchImpl<false>, NameMatch>;
void registerFunctionMatch(FunctionFactory & factory)
{
factory.registerFunction<FunctionMatch>();
}
}
#include "FunctionsMultiStringFuzzySearch.h"
#include "FunctionFactory.h"
#include "MultiMatchAllIndicesImpl.h"
namespace DB
{
struct NameMultiFuzzyMatchAllIndices
{
static constexpr auto name = "multiFuzzyMatchAllIndices";
};
using FunctionMultiFuzzyMatchAllIndices = FunctionsMultiStringFuzzySearch<
MultiMatchAllIndicesImpl<UInt64, true>,
NameMultiFuzzyMatchAllIndices,
std::numeric_limits<UInt32>::max()>;
void registerFunctionMultiFuzzyMatchAllIndices(FunctionFactory & factory)
{
factory.registerFunction<FunctionMultiFuzzyMatchAllIndices>();
}
}
#include "FunctionsMultiStringFuzzySearch.h"
#include "FunctionFactory.h"
#include "MultiMatchAnyImpl.h"
namespace DB
{
struct NameMultiFuzzyMatchAny
{
static constexpr auto name = "multiFuzzyMatchAny";
};
using FunctionMultiFuzzyMatchAny = FunctionsMultiStringFuzzySearch<
MultiMatchAnyImpl<UInt8, true, false, true>,
NameMultiFuzzyMatchAny,
std::numeric_limits<UInt32>::max()>;
void registerFunctionMultiFuzzyMatchAny(FunctionFactory & factory)
{
factory.registerFunction<FunctionMultiFuzzyMatchAny>();
}
}
#include "FunctionsMultiStringFuzzySearch.h"
#include "FunctionFactory.h"
#include "MultiMatchAnyImpl.h"
namespace DB
{
struct NameMultiFuzzyMatchAnyIndex
{
static constexpr auto name = "multiFuzzyMatchAnyIndex";
};
using FunctionMultiFuzzyMatchAnyIndex = FunctionsMultiStringFuzzySearch<
MultiMatchAnyImpl<UInt64, false, true, true>,
NameMultiFuzzyMatchAnyIndex,
std::numeric_limits<UInt32>::max()>;
void registerFunctionMultiFuzzyMatchAnyIndex(FunctionFactory & factory)
{
factory.registerFunction<FunctionMultiFuzzyMatchAnyIndex>();
}
}
#include "FunctionsMultiStringSearch.h"
#include "FunctionFactory.h"
#include "MultiMatchAllIndicesImpl.h"
namespace DB
{
struct NameMultiMatchAllIndices
{
static constexpr auto name = "multiMatchAllIndices";
};
using FunctionMultiMatchAllIndices = FunctionsMultiStringSearch<
MultiMatchAllIndicesImpl<UInt64, false>,
NameMultiMatchAllIndices,
std::numeric_limits<UInt32>::max()>;
void registerFunctionMultiMatchAllIndices(FunctionFactory & factory)
{
factory.registerFunction<FunctionMultiMatchAllIndices>();
}
}
#include "FunctionsMultiStringSearch.h"
#include "FunctionFactory.h"
#include "MultiMatchAnyImpl.h"
namespace DB
{
struct NameMultiMatchAny
{
static constexpr auto name = "multiMatchAny";
};
using FunctionMultiMatchAny = FunctionsMultiStringSearch<
MultiMatchAnyImpl<UInt8, true, false, false>,
NameMultiMatchAny,
std::numeric_limits<UInt32>::max()>;
void registerFunctionMultiMatchAny(FunctionFactory & factory)
{
factory.registerFunction<FunctionMultiMatchAny>();
}
}
#include "FunctionsMultiStringSearch.h"
#include "FunctionFactory.h"
#include "MultiMatchAnyImpl.h"
namespace DB
{
struct NameMultiMatchAnyIndex
{
static constexpr auto name = "multiMatchAnyIndex";
};
using FunctionMultiMatchAnyIndex = FunctionsMultiStringSearch<
MultiMatchAnyImpl<UInt64, false, true, false>,
NameMultiMatchAnyIndex,
std::numeric_limits<UInt32>::max()>;
void registerFunctionMultiMatchAnyIndex(FunctionFactory & factory)
{
factory.registerFunction<FunctionMultiMatchAnyIndex>();
}
}
#include "FunctionsStringSearch.h"
#include "FunctionFactory.h"
#include "MatchImpl.h"
namespace DB
{
struct NameNotLike
{
static constexpr auto name = "notLike";
};
using FunctionNotLike = FunctionsStringSearch<MatchImpl<true, true>, NameNotLike>;
void registerFunctionNotLike(FunctionFactory & factory)
{
factory.registerFunction<FunctionNotLike>();
}
}
......@@ -27,7 +27,7 @@ void registerFunctionsRound(FunctionFactory &);
void registerFunctionsString(FunctionFactory &);
void registerFunctionsStringArray(FunctionFactory &);
void registerFunctionsStringSearch(FunctionFactory &);
void registerFunctionsStringRegex(FunctionFactory &);
void registerFunctionsStringRegexp(FunctionFactory &);
void registerFunctionsStringSimilarity(FunctionFactory &);
void registerFunctionsURL(FunctionFactory &);
void registerFunctionsVisitParam(FunctionFactory &);
......@@ -68,7 +68,7 @@ void registerFunctions()
registerFunctionsString(factory);
registerFunctionsStringArray(factory);
registerFunctionsStringSearch(factory);
registerFunctionsStringRegex(factory);
registerFunctionsStringRegexp(factory);
registerFunctionsStringSimilarity(factory);
registerFunctionsURL(factory);
registerFunctionsVisitParam(factory);
......
namespace DB
{
class FunctionFactory;
void registerFunctionLike(FunctionFactory &);
void registerFunctionNotLike(FunctionFactory &);
void registerFunctionMatch(FunctionFactory &);
void registerFunctionExtract(FunctionFactory &);
void registerFunctionReplaceOne(FunctionFactory &);
void registerFunctionReplaceAll(FunctionFactory &);
void registerFunctionReplaceRegexpOne(FunctionFactory &);
void registerFunctionReplaceRegexpAll(FunctionFactory &);
void registerFunctionMultiMatchAny(FunctionFactory &);
void registerFunctionMultiMatchAnyIndex(FunctionFactory &);
void registerFunctionMultiMatchAllIndices(FunctionFactory &);
void registerFunctionMultiFuzzyMatchAny(FunctionFactory &);
void registerFunctionMultiFuzzyMatchAnyIndex(FunctionFactory &);
void registerFunctionMultiFuzzyMatchAllIndices(FunctionFactory &);
void registerFunctionExtractGroups(FunctionFactory &);
void registerFunctionExtractAllGroups(FunctionFactory &);
void registerFunctionsStringRegexp(FunctionFactory & factory)
{
registerFunctionLike(factory);
registerFunctionNotLike(factory);
registerFunctionMatch(factory);
registerFunctionExtract(factory);
registerFunctionReplaceOne(factory);
registerFunctionReplaceAll(factory);
registerFunctionReplaceRegexpOne(factory);
registerFunctionReplaceRegexpAll(factory);
registerFunctionMultiMatchAny(factory);
registerFunctionMultiMatchAnyIndex(factory);
registerFunctionMultiMatchAllIndices(factory);
registerFunctionMultiFuzzyMatchAny(factory);
registerFunctionMultiFuzzyMatchAnyIndex(factory);
registerFunctionMultiFuzzyMatchAllIndices(factory);
registerFunctionExtractGroups(factory);
registerFunctionExtractAllGroups(factory);
}
}
#include "FunctionStringReplace.h"
#include "FunctionFactory.h"
#include "ReplaceStringImpl.h"
namespace DB
{
struct NameReplaceAll
{
static constexpr auto name = "replaceAll";
};
using FunctionReplaceAll = FunctionStringReplace<ReplaceStringImpl<false>, NameReplaceAll>;
void registerFunctionReplaceAll(FunctionFactory & factory)
{
factory.registerFunction<FunctionReplaceAll>();
factory.registerAlias("replace", NameReplaceAll::name, FunctionFactory::CaseInsensitive);
}
}
#include "FunctionStringReplace.h"
#include "FunctionFactory.h"
#include "ReplaceStringImpl.h"
namespace DB
{
struct NameReplaceOne
{
static constexpr auto name = "replaceOne";
};
using FunctionReplaceOne = FunctionStringReplace<ReplaceStringImpl<true>, NameReplaceOne>;
void registerFunctionReplaceOne(FunctionFactory & factory)
{
factory.registerFunction<FunctionReplaceOne>();
}
}
#include "FunctionStringReplace.h"
#include "FunctionFactory.h"
#include "ReplaceRegexpImpl.h"
namespace DB
{
struct NameReplaceRegexpAll
{
static constexpr auto name = "replaceRegexpAll";
};
using FunctionReplaceRegexpAll = FunctionStringReplace<ReplaceRegexpImpl<false>, NameReplaceRegexpAll>;
void registerFunctionReplaceRegexpAll(FunctionFactory & factory)
{
factory.registerFunction<FunctionReplaceRegexpAll>();
}
}
#include "FunctionStringReplace.h"
#include "FunctionFactory.h"
#include "ReplaceRegexpImpl.h"
namespace DB
{
struct NameReplaceRegexpOne
{
static constexpr auto name = "replaceRegexpOne";
};
using FunctionReplaceRegexpOne = FunctionStringReplace<ReplaceRegexpImpl<true>, NameReplaceRegexpOne>;
void registerFunctionReplaceRegexpOne(FunctionFactory & factory)
{
factory.registerFunction<FunctionReplaceRegexpOne>();
}
}
......@@ -24,13 +24,27 @@ PEERDIR(
library/consistent_hashing
)
# Autogenerated from: find . -name '*.cpp' | grep -v -F tests | sed 's/^\.\// /' | sort
SRCS(
array/array.cpp
abs.cpp
acos.cpp
addDays.cpp
addHours.cpp
addMinutes.cpp
addMonths.cpp
addQuarters.cpp
addressToLine.cpp
addressToSymbol.cpp
addSeconds.cpp
addWeeks.cpp
addYears.cpp
appendTrailingCharIfAbsent.cpp
array/arrayAll.cpp
array/arrayAUC.cpp
array/arrayCompact.cpp
array/arrayConcat.cpp
array/arrayCount.cpp
array/array.cpp
array/arrayCumSum.cpp
array/arrayCumSumNonNegative.cpp
array/arrayDifference.cpp
......@@ -69,65 +83,13 @@ SRCS(
array/countEqual.cpp
array/emptyArray.cpp
array/emptyArrayToSingle.cpp
array/has.cpp
array/hasAll.cpp
array/hasAny.cpp
array/has.cpp
array/indexOf.cpp
array/length.cpp
array/range.cpp
array/registerFunctionsArray.cpp
GatherUtils/concat.cpp
GatherUtils/createArraySink.cpp
GatherUtils/createArraySource.cpp
GatherUtils/createValueSource.cpp
GatherUtils/has.cpp
GatherUtils/push.cpp
GatherUtils/resizeConstantSize.cpp
GatherUtils/resizeDynamicSize.cpp
GatherUtils/sliceDynamicOffsetBounded.cpp
GatherUtils/sliceDynamicOffsetUnbounded.cpp
GatherUtils/sliceFromLeftConstantOffsetBounded.cpp
GatherUtils/sliceFromLeftConstantOffsetUnbounded.cpp
GatherUtils/sliceFromRightConstantOffsetBounded.cpp
GatherUtils/sliceFromRightConstantOffsetUnbounded.cpp
URL/basename.cpp
URL/cutFragment.cpp
URL/cutQueryString.cpp
URL/cutQueryStringAndFragment.cpp
URL/cutToFirstSignificantSubdomain.cpp
URL/cutURLParameter.cpp
URL/cutWWW.cpp
URL/decodeURLComponent.cpp
URL/domain.cpp
URL/domainWithoutWWW.cpp
URL/extractURLParameter.cpp
URL/extractURLParameterNames.cpp
URL/extractURLParameters.cpp
URL/firstSignificantSubdomain.cpp
URL/fragment.cpp
URL/path.cpp
URL/pathFull.cpp
URL/protocol.cpp
URL/queryString.cpp
URL/queryStringAndFragment.cpp
URL/registerFunctionsURL.cpp
URL/tldLookup.generated.cpp
URL/topLevelDomain.cpp
URL/URLHierarchy.cpp
URL/URLPathHierarchy.cpp
abs.cpp
acos.cpp
addDays.cpp
addHours.cpp
addMinutes.cpp
addMonths.cpp
addQuarters.cpp
addressToLine.cpp
addressToSymbol.cpp
addSeconds.cpp
addWeeks.cpp
addYears.cpp
appendTrailingCharIfAbsent.cpp
asin.cpp
assumeNotNull.cpp
atan.cpp
......@@ -145,9 +107,9 @@ SRCS(
bitShiftLeft.cpp
bitShiftRight.cpp
bitSwapLastTwo.cpp
bitTest.cpp
bitTestAll.cpp
bitTestAny.cpp
bitTest.cpp
bitWrapperFunc.cpp
bitXor.cpp
blockNumber.cpp
......@@ -173,12 +135,15 @@ SRCS(
empty.cpp
endsWith.cpp
equals.cpp
erf.cpp
erfc.cpp
erf.cpp
evalMLMethod.cpp
exp.cpp
exp2.cpp
exp10.cpp
exp2.cpp
exp.cpp
extractAllGroups.cpp
extract.cpp
extractGroups.cpp
extractTimeZoneFromFunctionArguments.cpp
filesystem.cpp
finalizeAggregation.cpp
......@@ -188,6 +153,7 @@ SRCS(
FunctionFQDN.cpp
FunctionHelpers.cpp
FunctionJoinGet.cpp
FunctionsBitmap.cpp
FunctionsCoding.cpp
FunctionsConversion.cpp
FunctionsEmbeddedDictionaries.cpp
......@@ -200,14 +166,28 @@ SRCS(
FunctionsRandom.cpp
FunctionsRound.cpp
FunctionsStringArray.cpp
FunctionsStringRegex.cpp
FunctionsStringSimilarity.cpp
GatherUtils/concat.cpp
GatherUtils/createArraySink.cpp
GatherUtils/createArraySource.cpp
GatherUtils/createValueSource.cpp
GatherUtils/has.cpp
GatherUtils/push.cpp
GatherUtils/resizeConstantSize.cpp
GatherUtils/resizeDynamicSize.cpp
GatherUtils/sliceDynamicOffsetBounded.cpp
GatherUtils/sliceDynamicOffsetUnbounded.cpp
GatherUtils/sliceFromLeftConstantOffsetBounded.cpp
GatherUtils/sliceFromLeftConstantOffsetUnbounded.cpp
GatherUtils/sliceFromRightConstantOffsetBounded.cpp
GatherUtils/sliceFromRightConstantOffsetUnbounded.cpp
gcd.cpp
generateUUIDv4.cpp
GeoHash.cpp
geohashDecode.cpp
geohashEncode.cpp
geohashesInBox.cpp
geoToH3.cpp
getMacro.cpp
getScalar.cpp
getSizeOfEnumType.cpp
......@@ -215,9 +195,20 @@ SRCS(
greater.cpp
greaterOrEquals.cpp
greatest.cpp
h3EdgeAngle.cpp
h3EdgeLengthM.cpp
h3GetBaseCell.cpp
h3GetResolution.cpp
h3HexAreaM2.cpp
h3IndexesAreNeighbors.cpp
h3IsValid.cpp
h3kRing.cpp
h3ToChildren.cpp
h3ToParent.cpp
h3ToString.cpp
hasColumnInTable.cpp
hasToken.cpp
hasTokenCaseInsensitive.cpp
hasToken.cpp
hostName.cpp
identity.cpp
if.cpp
......@@ -229,8 +220,8 @@ SRCS(
in.cpp
intDiv.cpp
intDivOrZero.cpp
intExp2.cpp
intExp10.cpp
intExp2.cpp
isConstant.cpp
isFinite.cpp
isInfinite.cpp
......@@ -245,61 +236,70 @@ SRCS(
less.cpp
lessOrEquals.cpp
lgamma.cpp
log.cpp
log2.cpp
like.cpp
log10.cpp
log2.cpp
log.cpp
lowCardinalityIndices.cpp
lowCardinalityKeys.cpp
lower.cpp
lowerUTF8.cpp
match.cpp
materialize.cpp
minus.cpp
modulo.cpp
moduloOrZero.cpp
multiFuzzyMatchAllIndices.cpp
multiFuzzyMatchAny.cpp
multiFuzzyMatchAnyIndex.cpp
multiIf.cpp
multiMatchAllIndices.cpp
multiMatchAny.cpp
multiMatchAnyIndex.cpp
multiply.cpp
multiSearchAllPositions.cpp
multiSearchAllPositionsCaseInsensitive.cpp
multiSearchAllPositionsCaseInsensitiveUTF8.cpp
multiSearchAllPositions.cpp
multiSearchAllPositionsUTF8.cpp
multiSearchAny.cpp
multiSearchAnyCaseInsensitive.cpp
multiSearchAnyCaseInsensitiveUTF8.cpp
multiSearchAny.cpp
multiSearchAnyUTF8.cpp
multiSearchFirstIndex.cpp
multiSearchFirstIndexCaseInsensitive.cpp
multiSearchFirstIndexCaseInsensitiveUTF8.cpp
multiSearchFirstIndex.cpp
multiSearchFirstIndexUTF8.cpp
multiSearchFirstPosition.cpp
multiSearchFirstPositionCaseInsensitive.cpp
multiSearchFirstPositionCaseInsensitiveUTF8.cpp
multiSearchFirstPosition.cpp
multiSearchFirstPositionUTF8.cpp
negate.cpp
neighbor.cpp
notEmpty.cpp
notEquals.cpp
now.cpp
notLike.cpp
now64.cpp
now.cpp
nullIf.cpp
pi.cpp
plus.cpp
pointInEllipses.cpp
pointInPolygon.cpp
position.cpp
positionCaseInsensitive.cpp
positionCaseInsensitiveUTF8.cpp
position.cpp
positionUTF8.cpp
pow.cpp
rand.cpp
rand64.cpp
randConstant.cpp
rand.cpp
randomPrintableASCII.cpp
regexpQuoteMeta.cpp
registerFunctions.cpp
registerFunctionsArithmetic.cpp
registerFunctionsComparison.cpp
registerFunctionsConditional.cpp
registerFunctionsConsistentHashing.cpp
registerFunctions.cpp
registerFunctionsDateTime.cpp
registerFunctionsGeo.cpp
registerFunctionsHigherOrder.cpp
......@@ -310,6 +310,7 @@ SRCS(
registerFunctionsRandom.cpp
registerFunctionsReinterpret.cpp
registerFunctionsString.cpp
registerFunctionsStringRegexp.cpp
registerFunctionsStringSearch.cpp
registerFunctionsTuple.cpp
registerFunctionsVisitParam.cpp
......@@ -317,6 +318,10 @@ SRCS(
reinterpretAsString.cpp
reinterpretStringAs.cpp
repeat.cpp
replaceAll.cpp
replaceOne.cpp
replaceRegexpAll.cpp
replaceRegexpOne.cpp
replicate.cpp
reverse.cpp
reverseUTF8.cpp
......@@ -334,6 +339,7 @@ SRCS(
sleepEachRow.cpp
sqrt.cpp
startsWith.cpp
stringToH3.cpp
substring.cpp
subtractDays.cpp
subtractHours.cpp
......@@ -343,6 +349,7 @@ SRCS(
subtractSeconds.cpp
subtractWeeks.cpp
subtractYears.cpp
sumburConsistentHash.cpp
tan.cpp
tanh.cpp
tgamma.cpp
......@@ -402,6 +409,31 @@ SRCS(
upper.cpp
upperUTF8.cpp
uptime.cpp
URL/basename.cpp
URL/cutFragment.cpp
URL/cutQueryStringAndFragment.cpp
URL/cutQueryString.cpp
URL/cutToFirstSignificantSubdomain.cpp
URL/cutURLParameter.cpp
URL/cutWWW.cpp
URL/decodeURLComponent.cpp
URL/domain.cpp
URL/domainWithoutWWW.cpp
URL/extractURLParameter.cpp
URL/extractURLParameterNames.cpp
URL/extractURLParameters.cpp
URL/firstSignificantSubdomain.cpp
URL/fragment.cpp
URL/path.cpp
URL/pathFull.cpp
URL/protocol.cpp
URL/queryStringAndFragment.cpp
URL/queryString.cpp
URL/registerFunctionsURL.cpp
URL/tldLookup.generated.cpp
URL/topLevelDomain.cpp
URL/URLHierarchy.cpp
URL/URLPathHierarchy.cpp
version.cpp
visibleWidth.cpp
visitParamExtractBool.cpp
......
<test>
<preconditions>
<table_exists>test.hits</table_exists>
</preconditions>
<query>SELECT count() FROM test.hits WHERE NOT ignore(extract(URL, '(\\w+=\\w+)'))</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(extractAll(URL, '(\\w+=\\w+)'))</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(extractGroups(URL, '(\\w+)=(\\w+)'))</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(extractAllGroups(URL, '(\\w+)=(\\w+)'))</query>
</test>
0 groups, zero matches
[]
[[],[]]
1 group, multiple matches, String and FixedString
[['hello','world']]
[['hello','world']]
[['hello','world']]
[['hello','world']]
[['hello','world']]
[['hello','world']]
[['hello'],['world']]
[['hello'],['world']]
[['hello'],['world']]
[['hello'],['world']]
[['hello'],['world']]
[['hello'],['world']]
mutiple groups, multiple matches
[['abc','def','ghi','"jkl mno"'],['111','222','333','"444 foo bar"']]
[['abc','111'],['def','222'],['ghi','333'],['"jkl mno"','"444 foo bar"']]
big match
0 1 0 []
0 0 0 []
260 1 1 [156]
520 1 3 [156,156,156]
lots of matches
0 1 0 0
0 0 0 0
260 1 260 260
520 1 520 520
lots of groups
0 100 0 []
0 0 0 []
260 100 2 [1,1]
520 100 5 [1,1,1,1,1]
......@@ -22,30 +22,30 @@ SELECT extractAllGroups('abc=111, def=222, ghi=333 "jkl mno"="444 foo bar"', '("
SELECT 'big match';
SELECT
length(haystack), length(matches), length(matches[1]), arrayMap((x) -> length(x), matches[1])
length(haystack), length(matches[1]), length(matches), arrayMap((x) -> length(x), arrayMap(x -> x[1], matches))
FROM (
SELECT
repeat('abcdefghijklmnopqrstuvwxyz', number * 10) AS haystack,
extractAllGroups(haystack, '(abcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyz)') AS matches
FROM numbers(3)
SELECT
repeat('abcdefghijklmnopqrstuvwxyz', number * 10) AS haystack,
extractAllGroups(haystack, '(abcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyz)') AS matches
FROM numbers(3)
);
SELECT 'lots of matches';
SELECT
length(haystack), length(matches), length(matches[1]), arrayReduce('sum', arrayMap((x) -> length(x), matches[1]))
length(haystack), length(matches[1]), length(matches), arrayReduce('sum', arrayMap((x) -> length(x), arrayMap(x -> x[1], matches)))
FROM (
SELECT
repeat('abcdefghijklmnopqrstuvwxyz', number * 10) AS haystack,
extractAllGroups(haystack, '(\\w)') AS matches
FROM numbers(3)
SELECT
repeat('abcdefghijklmnopqrstuvwxyz', number * 10) AS haystack,
extractAllGroups(haystack, '(\\w)') AS matches
FROM numbers(3)
);
SELECT 'lots of groups';
SELECT
length(haystack), length(matches), length(matches[1]), arrayMap((x) -> length(x), matches[1])
length(haystack), length(matches[1]), length(matches), arrayMap((x) -> length(x), arrayMap(x -> x[1], matches))
FROM (
SELECT
repeat('abcdefghijklmnopqrstuvwxyz', number * 10) AS haystack,
extractAllGroups(haystack, repeat('(\\w)', 100)) AS matches
FROM numbers(3)
SELECT
repeat('abcdefghijklmnopqrstuvwxyz', number * 10) AS haystack,
extractAllGroups(haystack, repeat('(\\w)', 100)) AS matches
FROM numbers(3)
);
0 groups, zero matches
[]
1 group, multiple matches, String and FixedString
['hello','world']
['hello','world']
['hello','world']
['hello','world']
['hello','world']
['hello','world']
multiple matches
['abc','111']
big match
0 0 []
260 1 [156]
520 1 [156]
lots of matches
0 0 0
260 1 1
520 1 1
lots of groups
0 0 []
260 100 [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1]
520 100 [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1]
-- error cases
SELECT extractGroups(); --{serverError 42} not enough arguments
SELECT extractGroups('hello'); --{serverError 42} not enough arguments
SELECT extractGroups('hello', 123); --{serverError 43} invalid argument type
SELECT extractGroups(123, 'world'); --{serverError 43} invalid argument type
SELECT extractGroups('hello world', '((('); --{serverError 427} invalid re
SELECT extractGroups('hello world', materialize('\\w+')); --{serverError 44} non-cons needle
SELECT '0 groups, zero matches';
SELECT extractGroups('hello world', '\\w+');
SELECT '1 group, multiple matches, String and FixedString';
SELECT extractGroups('hello world', '(\\w+) (\\w+)');
SELECT extractGroups('hello world', CAST('(\\w+) (\\w+)' as FixedString(11)));
SELECT extractGroups(CAST('hello world' AS FixedString(12)), '(\\w+) (\\w+)');
SELECT extractGroups(CAST('hello world' AS FixedString(12)), CAST('(\\w+) (\\w+)' as FixedString(11)));
SELECT extractGroups(materialize(CAST('hello world' AS FixedString(12))), '(\\w+) (\\w+)');
SELECT extractGroups(materialize(CAST('hello world' AS FixedString(12))), CAST('(\\w+) (\\w+)' as FixedString(11)));
SELECT 'multiple matches';
SELECT extractGroups('abc=111, def=222, ghi=333 "jkl mno"="444 foo bar"', '("[^"]+"|\\w+)=("[^"]+"|\\w+)');
SELECT 'big match';
SELECT
length(haystack), length(matches), arrayMap((x) -> length(x), matches)
FROM (
SELECT
repeat('abcdefghijklmnopqrstuvwxyz', number * 10) AS haystack,
extractGroups(haystack, '(abcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyz)') AS matches
FROM numbers(3)
);
SELECT 'lots of matches';
SELECT
length(haystack), length(matches), arrayReduce('sum', arrayMap((x) -> length(x), matches))
FROM (
SELECT
repeat('abcdefghijklmnopqrstuvwxyz', number * 10) AS haystack,
extractGroups(haystack, '(\\w)') AS matches
FROM numbers(3)
);
SELECT 'lots of groups';
SELECT
length(haystack), length(matches), arrayMap((x) -> length(x), matches)
FROM (
SELECT
repeat('abcdefghijklmnopqrstuvwxyz', number * 10) AS haystack,
extractGroups(haystack, repeat('(\\w)', 100)) AS matches
FROM numbers(3)
);
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册