mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge pull request #37491 from ClickHouse/match_refactoring
Refactorings of LIKE/MATCH code
This commit is contained in:
commit
abf2558fba
@ -17,7 +17,7 @@ namespace DB
|
||||
|
||||
template <bool thread_safe>
|
||||
void OptimizedRegularExpressionImpl<thread_safe>::analyze(
|
||||
const std::string & regexp,
|
||||
std::string_view regexp,
|
||||
std::string & required_substring,
|
||||
bool & is_trivial,
|
||||
bool & required_substring_is_prefix)
|
||||
|
@ -86,8 +86,6 @@ public:
|
||||
/// Get the regexp re2 or nullptr if the pattern is trivial (for output to the log).
|
||||
const std::unique_ptr<RegexType> & getRE2() const { return re2; }
|
||||
|
||||
static void analyze(const std::string & regexp_, std::string & required_substring, bool & is_trivial, bool & required_substring_is_prefix);
|
||||
|
||||
void getAnalyzeResult(std::string & out_required_substring, bool & out_is_trivial, bool & out_required_substring_is_prefix) const
|
||||
{
|
||||
out_required_substring = required_substring;
|
||||
@ -104,6 +102,8 @@ private:
|
||||
std::optional<DB::StringSearcher<false, true>> case_insensitive_substring_searcher;
|
||||
std::unique_ptr<RegexType> re2;
|
||||
unsigned number_of_subpatterns;
|
||||
|
||||
static void analyze(std::string_view regexp_, std::string & required_substring, bool & is_trivial, bool & required_substring_is_prefix);
|
||||
};
|
||||
|
||||
using OptimizedRegularExpression = OptimizedRegularExpressionImpl<true>;
|
||||
|
@ -63,14 +63,33 @@ inline bool likePatternIsSubstring(std::string_view pattern, String & res)
|
||||
|
||||
}
|
||||
|
||||
/** 'like' - if true, treat pattern as SQL LIKE, otherwise as re2 regexp.
|
||||
* 'negate' - if true, negate result
|
||||
* 'case_insensitive' - if true, match case insensitively
|
||||
*
|
||||
// For more readable instantiations of MatchImpl<>
|
||||
struct MatchTraits
|
||||
{
|
||||
enum class Syntax
|
||||
{
|
||||
Like,
|
||||
Re2
|
||||
};
|
||||
|
||||
enum class Case
|
||||
{
|
||||
Sensitive,
|
||||
Insensitive
|
||||
};
|
||||
|
||||
enum class Result
|
||||
{
|
||||
DontNegate,
|
||||
Negate
|
||||
};
|
||||
};
|
||||
|
||||
/**
|
||||
* NOTE: We want to run regexp search for whole columns by one call (as implemented in function 'position')
|
||||
* but for that, regexp engine must support \0 bytes and their interpretation as string boundaries.
|
||||
*/
|
||||
template <typename Name, bool like, bool negate, bool case_insensitive>
|
||||
template <typename Name, MatchTraits::Syntax syntax_, MatchTraits::Case case_, MatchTraits::Result result_>
|
||||
struct MatchImpl
|
||||
{
|
||||
static constexpr bool use_default_implementation_for_constants = true;
|
||||
@ -81,6 +100,10 @@ struct MatchImpl
|
||||
|
||||
using ResultType = UInt8;
|
||||
|
||||
static constexpr bool is_like = (syntax_ == MatchTraits::Syntax::Like);
|
||||
static constexpr bool case_insensitive = (case_ == MatchTraits::Case::Insensitive);
|
||||
static constexpr bool negate = (result_ == MatchTraits::Result::Negate);
|
||||
|
||||
using Searcher = std::conditional_t<case_insensitive,
|
||||
VolnitskyCaseInsensitiveUTF8,
|
||||
VolnitskyUTF8>;
|
||||
@ -92,16 +115,20 @@ struct MatchImpl
|
||||
const ColumnPtr & start_pos_,
|
||||
PaddedPODArray<UInt8> & res)
|
||||
{
|
||||
const size_t haystack_size = haystack_offsets.size();
|
||||
|
||||
if (haystack_size != res.size())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Function '{}' unexpectedly received a different number of haystacks and results", name);
|
||||
|
||||
if (start_pos_ != nullptr)
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Function '{}' doesn't support start_pos argument", name);
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function '{}' doesn't support start_pos argument", name);
|
||||
|
||||
if (haystack_offsets.empty())
|
||||
return;
|
||||
|
||||
/// A simple case where the [I]LIKE expression reduces to finding a substring in a string
|
||||
String strstr_pattern;
|
||||
if (like && impl::likePatternIsSubstring(needle, strstr_pattern))
|
||||
if (is_like && impl::likePatternIsSubstring(needle, strstr_pattern))
|
||||
{
|
||||
const UInt8 * const begin = haystack_data.data();
|
||||
const UInt8 * const end = haystack_data.data() + haystack_data.size();
|
||||
@ -139,7 +166,7 @@ struct MatchImpl
|
||||
}
|
||||
else
|
||||
{
|
||||
auto regexp = Regexps::get<like, true, case_insensitive>(needle);
|
||||
auto regexp = Regexps::get<is_like, true, case_insensitive>(needle);
|
||||
|
||||
String required_substring;
|
||||
bool is_trivial;
|
||||
@ -147,28 +174,26 @@ struct MatchImpl
|
||||
|
||||
regexp->getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix);
|
||||
|
||||
size_t haystack_size = haystack_offsets.size();
|
||||
|
||||
if (required_substring.empty())
|
||||
{
|
||||
if (!regexp->getRE2()) /// An empty regexp. Always matches.
|
||||
{
|
||||
if (haystack_size)
|
||||
memset(res.data(), 1, haystack_size * sizeof(res[0]));
|
||||
memset(res.data(), !negate, haystack_size * sizeof(res[0]));
|
||||
}
|
||||
else
|
||||
{
|
||||
size_t prev_offset = 0;
|
||||
for (size_t i = 0; i < haystack_size; ++i)
|
||||
{
|
||||
res[i] = negate
|
||||
^ regexp->getRE2()->Match(
|
||||
const bool match = regexp->getRE2()->Match(
|
||||
{reinterpret_cast<const char *>(&haystack_data[prev_offset]), haystack_offsets[i] - prev_offset - 1},
|
||||
0,
|
||||
haystack_offsets[i] - prev_offset - 1,
|
||||
re2_st::RE2::UNANCHORED,
|
||||
nullptr,
|
||||
0);
|
||||
res[i] = negate ^ match;
|
||||
|
||||
prev_offset = haystack_offsets[i];
|
||||
}
|
||||
@ -216,14 +241,14 @@ struct MatchImpl
|
||||
const size_t start_pos = (required_substring_is_prefix) ? (reinterpret_cast<const char *>(pos) - str_data) : 0;
|
||||
const size_t end_pos = str_size;
|
||||
|
||||
res[i] = negate
|
||||
^ regexp->getRE2()->Match(
|
||||
const bool match = regexp->getRE2()->Match(
|
||||
{str_data, str_size},
|
||||
start_pos,
|
||||
end_pos,
|
||||
re2_st::RE2::UNANCHORED,
|
||||
nullptr,
|
||||
0);
|
||||
res[i] = negate ^ match;
|
||||
}
|
||||
}
|
||||
else
|
||||
@ -247,12 +272,17 @@ struct MatchImpl
|
||||
const String & needle,
|
||||
PaddedPODArray<UInt8> & res)
|
||||
{
|
||||
const size_t haystack_size = haystack.size() / N;
|
||||
|
||||
if (haystack_size != res.size())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Function '{}' unexpectedly received a different number of haystacks and results", name);
|
||||
|
||||
if (haystack.empty())
|
||||
return;
|
||||
|
||||
/// A simple case where the LIKE expression reduces to finding a substring in a string
|
||||
String strstr_pattern;
|
||||
if (like && impl::likePatternIsSubstring(needle, strstr_pattern))
|
||||
if (is_like && impl::likePatternIsSubstring(needle, strstr_pattern))
|
||||
{
|
||||
const UInt8 * const begin = haystack.data();
|
||||
const UInt8 * const end = haystack.data() + haystack.size();
|
||||
@ -295,7 +325,7 @@ struct MatchImpl
|
||||
}
|
||||
else
|
||||
{
|
||||
auto regexp = Regexps::get<like, true, case_insensitive>(needle);
|
||||
auto regexp = Regexps::get<is_like, true, case_insensitive>(needle);
|
||||
|
||||
String required_substring;
|
||||
bool is_trivial;
|
||||
@ -303,28 +333,26 @@ struct MatchImpl
|
||||
|
||||
regexp->getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix);
|
||||
|
||||
const size_t haystack_size = haystack.size() / N;
|
||||
|
||||
if (required_substring.empty())
|
||||
{
|
||||
if (!regexp->getRE2()) /// An empty regexp. Always matches.
|
||||
{
|
||||
if (haystack_size)
|
||||
memset(res.data(), 1, haystack_size * sizeof(res[0]));
|
||||
memset(res.data(), !negate, haystack_size * sizeof(res[0]));
|
||||
}
|
||||
else
|
||||
{
|
||||
size_t offset = 0;
|
||||
for (size_t i = 0; i < haystack_size; ++i)
|
||||
{
|
||||
res[i] = negate
|
||||
^ regexp->getRE2()->Match(
|
||||
const bool match = regexp->getRE2()->Match(
|
||||
{reinterpret_cast<const char *>(&haystack[offset]), N},
|
||||
0,
|
||||
N,
|
||||
re2_st::RE2::UNANCHORED,
|
||||
nullptr,
|
||||
0);
|
||||
res[i] = negate ^ match;
|
||||
|
||||
offset += N;
|
||||
}
|
||||
@ -375,14 +403,14 @@ struct MatchImpl
|
||||
const size_t start_pos = (required_substring_is_prefix) ? (reinterpret_cast<const char *>(pos) - str_data) : 0;
|
||||
const size_t end_pos = N;
|
||||
|
||||
res[i] = negate
|
||||
^ regexp->getRE2()->Match(
|
||||
const bool match = regexp->getRE2()->Match(
|
||||
{str_data, N},
|
||||
start_pos,
|
||||
end_pos,
|
||||
re2_st::RE2::UNANCHORED,
|
||||
nullptr,
|
||||
0);
|
||||
res[i] = negate ^ match;
|
||||
}
|
||||
}
|
||||
else
|
||||
@ -410,13 +438,11 @@ struct MatchImpl
|
||||
{
|
||||
const size_t haystack_size = haystack_offsets.size();
|
||||
|
||||
if (haystack_size != needle_offset.size())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Function '{}' unexpectedly received a different number of haystacks and needles", name);
|
||||
if (haystack_size != needle_offset.size() || haystack_size != res.size())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Function '{}' unexpectedly received a different number of haystacks, needles and results", name);
|
||||
|
||||
if (start_pos_ != nullptr)
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Function '{}' doesn't support start_pos argument", name);
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function '{}' doesn't support start_pos argument", name);
|
||||
|
||||
if (haystack_offsets.empty())
|
||||
return;
|
||||
@ -440,7 +466,7 @@ struct MatchImpl
|
||||
reinterpret_cast<const char *>(cur_needle_data),
|
||||
cur_needle_length);
|
||||
|
||||
if (like && impl::likePatternIsSubstring(needle, required_substr))
|
||||
if (is_like && impl::likePatternIsSubstring(needle, required_substr))
|
||||
{
|
||||
if (required_substr.size() > cur_haystack_length)
|
||||
res[i] = negate;
|
||||
@ -448,16 +474,15 @@ struct MatchImpl
|
||||
{
|
||||
Searcher searcher(required_substr.data(), required_substr.size(), cur_haystack_length);
|
||||
const auto * match = searcher.search(cur_haystack_data, cur_haystack_length);
|
||||
res[i] = negate
|
||||
^ (match != cur_haystack_data + cur_haystack_length);
|
||||
res[i] = negate ^ (match != cur_haystack_data + cur_haystack_length);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
// each row is expected to contain a different like/re2 pattern
|
||||
// --> bypass the regexp cache, instead construct the pattern on-the-fly
|
||||
const int flags = Regexps::buildRe2Flags<true, case_insensitive>();
|
||||
const auto & regexp = Regexps::Regexp(Regexps::createRegexp<like>(needle, flags));
|
||||
const int flags = Regexps::buildRe2Flags</*no_capture*/ true, case_insensitive>();
|
||||
const auto & regexp = Regexps::Regexp(Regexps::createRegexp<is_like>(needle, flags));
|
||||
|
||||
regexp.getAnalyzeResult(required_substr, is_trivial, required_substring_is_prefix);
|
||||
|
||||
@ -465,18 +490,18 @@ struct MatchImpl
|
||||
{
|
||||
if (!regexp.getRE2()) /// An empty regexp. Always matches.
|
||||
{
|
||||
res[i] = 1;
|
||||
res[i] = !negate;
|
||||
}
|
||||
else
|
||||
{
|
||||
res[i] = negate
|
||||
^ regexp.getRE2()->Match(
|
||||
const bool match = regexp.getRE2()->Match(
|
||||
{reinterpret_cast<const char *>(cur_haystack_data), cur_haystack_length},
|
||||
0,
|
||||
cur_haystack_length,
|
||||
re2_st::RE2::UNANCHORED,
|
||||
nullptr,
|
||||
0);
|
||||
res[i] = negate ^ match;
|
||||
}
|
||||
}
|
||||
else
|
||||
@ -499,14 +524,14 @@ struct MatchImpl
|
||||
const size_t start_pos = (required_substring_is_prefix) ? (match - cur_haystack_data) : 0;
|
||||
const size_t end_pos = cur_haystack_length;
|
||||
|
||||
res[i] = negate
|
||||
^ regexp.getRE2()->Match(
|
||||
const bool match2 = regexp.getRE2()->Match(
|
||||
{reinterpret_cast<const char *>(cur_haystack_data), cur_haystack_length},
|
||||
start_pos,
|
||||
end_pos,
|
||||
re2_st::RE2::UNANCHORED,
|
||||
nullptr,
|
||||
0);
|
||||
res[i] = negate ^ match2;
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -527,13 +552,11 @@ struct MatchImpl
|
||||
{
|
||||
const size_t haystack_size = haystack.size()/N;
|
||||
|
||||
if (haystack_size != needle_offset.size())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Function '{}' unexpectedly received a different number of haystacks and needles", name);
|
||||
if (haystack_size != needle_offset.size() || haystack_size != res.size())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Function '{}' unexpectedly received a different number of haystacks, needles and results", name);
|
||||
|
||||
if (start_pos_ != nullptr)
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Function '{}' doesn't support start_pos argument", name);
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function '{}' doesn't support start_pos argument", name);
|
||||
|
||||
if (haystack.empty())
|
||||
return;
|
||||
@ -557,7 +580,7 @@ struct MatchImpl
|
||||
reinterpret_cast<const char *>(cur_needle_data),
|
||||
cur_needle_length);
|
||||
|
||||
if (like && impl::likePatternIsSubstring(needle, required_substr))
|
||||
if (is_like && impl::likePatternIsSubstring(needle, required_substr))
|
||||
{
|
||||
if (required_substr.size() > cur_haystack_length)
|
||||
res[i] = negate;
|
||||
@ -565,16 +588,15 @@ struct MatchImpl
|
||||
{
|
||||
Searcher searcher(required_substr.data(), required_substr.size(), cur_haystack_length);
|
||||
const auto * match = searcher.search(cur_haystack_data, cur_haystack_length);
|
||||
res[i] = negate
|
||||
^ (match != cur_haystack_data + cur_haystack_length);
|
||||
res[i] = negate ^ (match != cur_haystack_data + cur_haystack_length);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
// each row is expected to contain a different like/re2 pattern
|
||||
// --> bypass the regexp cache, instead construct the pattern on-the-fly
|
||||
const int flags = Regexps::buildRe2Flags<true, case_insensitive>();
|
||||
const auto & regexp = Regexps::Regexp(Regexps::createRegexp<like>(needle, flags));
|
||||
const int flags = Regexps::buildRe2Flags</*no_capture*/ true, case_insensitive>();
|
||||
const auto & regexp = Regexps::Regexp(Regexps::createRegexp<is_like>(needle, flags));
|
||||
|
||||
regexp.getAnalyzeResult(required_substr, is_trivial, required_substring_is_prefix);
|
||||
|
||||
@ -582,18 +604,18 @@ struct MatchImpl
|
||||
{
|
||||
if (!regexp.getRE2()) /// An empty regexp. Always matches.
|
||||
{
|
||||
res[i] = 1;
|
||||
res[i] = !negate;
|
||||
}
|
||||
else
|
||||
{
|
||||
res[i] = negate
|
||||
^ regexp.getRE2()->Match(
|
||||
const bool match = regexp.getRE2()->Match(
|
||||
{reinterpret_cast<const char *>(cur_haystack_data), cur_haystack_length},
|
||||
0,
|
||||
cur_haystack_length,
|
||||
re2_st::RE2::UNANCHORED,
|
||||
nullptr,
|
||||
0);
|
||||
res[i] = negate ^ match;
|
||||
}
|
||||
}
|
||||
else
|
||||
@ -616,14 +638,14 @@ struct MatchImpl
|
||||
const size_t start_pos = (required_substring_is_prefix) ? (match - cur_haystack_data) : 0;
|
||||
const size_t end_pos = cur_haystack_length;
|
||||
|
||||
res[i] = negate
|
||||
^ regexp.getRE2()->Match(
|
||||
const bool match2 = regexp.getRE2()->Match(
|
||||
{reinterpret_cast<const char *>(cur_haystack_data), cur_haystack_length},
|
||||
start_pos,
|
||||
end_pos,
|
||||
re2_st::RE2::UNANCHORED,
|
||||
nullptr,
|
||||
0);
|
||||
res[i] = negate ^ match2;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -120,7 +120,7 @@ struct MultiMatchAnyImpl
|
||||
memset(accum.data(), 0, accum.size());
|
||||
for (size_t j = 0; j < needles.size(); ++j)
|
||||
{
|
||||
MatchImpl<Name, false, false, false>::vectorConstant(haystack_data, haystack_offsets, needles[j].toString(), nullptr, accum);
|
||||
MatchImpl<Name, MatchTraits::Syntax::Re2, MatchTraits::Case::Sensitive, MatchTraits::Result::DontNegate>::vectorConstant(haystack_data, haystack_offsets, needles[j].toString(), nullptr, accum);
|
||||
for (size_t i = 0; i < res.size(); ++i)
|
||||
{
|
||||
if constexpr (FindAny)
|
||||
|
@ -12,7 +12,7 @@ struct NameILike
|
||||
static constexpr auto name = "ilike";
|
||||
};
|
||||
|
||||
using ILikeImpl = MatchImpl<NameILike, true, false, true>;
|
||||
using ILikeImpl = MatchImpl<NameILike, MatchTraits::Syntax::Like, MatchTraits::Case::Insensitive, MatchTraits::Result::DontNegate>;
|
||||
using FunctionILike = FunctionsStringSearch<ILikeImpl>;
|
||||
|
||||
}
|
||||
|
@ -11,7 +11,7 @@ struct NameLike
|
||||
static constexpr auto name = "like";
|
||||
};
|
||||
|
||||
using LikeImpl = MatchImpl<NameLike, true, false, false>;
|
||||
using LikeImpl = MatchImpl<NameLike, MatchTraits::Syntax::Like, MatchTraits::Case::Sensitive, MatchTraits::Result::DontNegate>;
|
||||
using FunctionLike = FunctionsStringSearch<LikeImpl>;
|
||||
|
||||
}
|
||||
|
@ -6,7 +6,7 @@ namespace DB
|
||||
{
|
||||
|
||||
/// Transforms the [I]LIKE expression into regexp re2. For example, abc%def -> ^abc.*def$
|
||||
inline String likePatternToRegexp(const String & pattern)
|
||||
inline String likePatternToRegexp(std::string_view pattern)
|
||||
{
|
||||
String res;
|
||||
res.reserve(pattern.size() * 2);
|
||||
|
@ -13,7 +13,7 @@ struct NameMatch
|
||||
static constexpr auto name = "match";
|
||||
};
|
||||
|
||||
using FunctionMatch = FunctionsStringSearch<MatchImpl<NameMatch, false, false, false>>;
|
||||
using FunctionMatch = FunctionsStringSearch<MatchImpl<NameMatch, MatchTraits::Syntax::Re2, MatchTraits::Case::Sensitive, MatchTraits::Result::DontNegate>>;
|
||||
|
||||
}
|
||||
|
||||
|
@ -12,7 +12,7 @@ struct NameNotILike
|
||||
static constexpr auto name = "notILike";
|
||||
};
|
||||
|
||||
using NotILikeImpl = MatchImpl<NameNotILike, true, true, true>;
|
||||
using NotILikeImpl = MatchImpl<NameNotILike, MatchTraits::Syntax::Like, MatchTraits::Case::Insensitive, MatchTraits::Result::Negate>;
|
||||
using FunctionNotILike = FunctionsStringSearch<NotILikeImpl>;
|
||||
|
||||
}
|
||||
|
@ -12,7 +12,7 @@ struct NameNotLike
|
||||
static constexpr auto name = "notLike";
|
||||
};
|
||||
|
||||
using FunctionNotLike = FunctionsStringSearch<MatchImpl<NameNotLike, true, true, false>>;
|
||||
using FunctionNotLike = FunctionsStringSearch<MatchImpl<NameNotLike, MatchTraits::Syntax::Like, MatchTraits::Case::Sensitive, MatchTraits::Result::Negate>>;
|
||||
|
||||
}
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user