mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Minor cleanups
Semantics are unchanged. Some special case handling was changed to early-out, because of that the indentation changed but the logic is the same as before.
This commit is contained in:
parent
3ed11c7a7b
commit
a8098db25d
@ -91,23 +91,30 @@ public:
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (arguments.size() < 2 || 3 < arguments.size())
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 2 or 3.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
throw Exception(
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Number of arguments for function {} doesn't match: passed {}, should be 2 or 3.",
|
||||
getName(), arguments.size());
|
||||
|
||||
if (!isStringOrFixedString(arguments[0]))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of argument of function {}",
|
||||
arguments[0]->getName(), getName());
|
||||
|
||||
if (!isString(arguments[1]))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of argument of function {}",
|
||||
arguments[1]->getName(), getName());
|
||||
|
||||
if (arguments.size() >= 3)
|
||||
{
|
||||
if (!isUnsignedInteger(arguments[2]))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[2]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of argument of function {}",
|
||||
arguments[2]->getName(), getName());
|
||||
}
|
||||
|
||||
return std::make_shared<DataTypeNumber<typename Impl::ResultType>>();
|
||||
@ -196,9 +203,11 @@ public:
|
||||
vec_res);
|
||||
else
|
||||
throw Exception(
|
||||
"Illegal columns " + arguments[0].column->getName() + " and "
|
||||
+ arguments[1].column->getName() + " of arguments of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal columns {} and {} of arguments of function {}",
|
||||
arguments[0].column->getName(),
|
||||
arguments[1].column->getName(),
|
||||
getName());
|
||||
|
||||
return col_res;
|
||||
}
|
||||
|
@ -25,7 +25,7 @@ namespace impl
|
||||
/// Is the [I]LIKE expression reduced to finding a substring in a string?
|
||||
inline bool likePatternIsSubstring(std::string_view pattern, String & res)
|
||||
{
|
||||
if (pattern.size() < 2 || pattern.front() != '%' || pattern.back() != '%')
|
||||
if (pattern.size() < 2 || !pattern.starts_with('%') || !pattern.ends_with('%'))
|
||||
return false;
|
||||
|
||||
res.clear();
|
||||
@ -101,9 +101,7 @@ struct MatchImpl
|
||||
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>;
|
||||
using Searcher = std::conditional_t<case_insensitive, VolnitskyCaseInsensitiveUTF8, VolnitskyUTF8>;
|
||||
|
||||
static void vectorConstant(
|
||||
const ColumnString::Chars & haystack_data,
|
||||
@ -115,13 +113,12 @@ struct MatchImpl
|
||||
const size_t haystack_size = haystack_offsets.size();
|
||||
|
||||
assert(haystack_size == res.size());
|
||||
|
||||
assert(start_pos_ == nullptr);
|
||||
|
||||
if (haystack_offsets.empty())
|
||||
return;
|
||||
|
||||
/// A simple case where the [I]LIKE expression reduces to finding a substring in a string
|
||||
/// Special case that the [I]LIKE expression reduces to finding a substring in a string
|
||||
String strstr_pattern;
|
||||
if (is_like && impl::likePatternIsSubstring(needle, strstr_pattern))
|
||||
{
|
||||
@ -158,105 +155,101 @@ struct MatchImpl
|
||||
/// Tail, in which there can be no substring.
|
||||
if (i < res.size())
|
||||
memset(&res[i], negate, (res.size() - i) * sizeof(res[0]));
|
||||
|
||||
return;
|
||||
}
|
||||
|
||||
const auto & regexp = Regexps::Regexp(Regexps::createRegexp<is_like, /*no_capture*/ true, case_insensitive>(needle));
|
||||
|
||||
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.
|
||||
memset(res.data(), !negate, haystack_size * sizeof(res[0]));
|
||||
else
|
||||
{
|
||||
size_t prev_offset = 0;
|
||||
for (size_t i = 0; i < haystack_size; ++i)
|
||||
{
|
||||
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];
|
||||
}
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto & regexp = Regexps::Regexp(Regexps::createRegexp<is_like, /*no_capture*/ true, case_insensitive>(needle));
|
||||
/// NOTE This almost matches with the case of impl::likePatternIsSubstring.
|
||||
|
||||
String required_substring;
|
||||
bool is_trivial;
|
||||
bool required_substring_is_prefix; /// for `anchored` execution of the regexp.
|
||||
const UInt8 * const begin = haystack_data.data();
|
||||
const UInt8 * const end = haystack_data.begin() + haystack_data.size();
|
||||
const UInt8 * pos = begin;
|
||||
|
||||
regexp.getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix);
|
||||
/// The current index in the array of strings.
|
||||
size_t i = 0;
|
||||
|
||||
if (required_substring.empty())
|
||||
Searcher 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)))
|
||||
{
|
||||
if (!regexp.getRE2()) /// An empty regexp. Always matches.
|
||||
/// Determine which index it refers to.
|
||||
while (begin + haystack_offsets[i] <= pos)
|
||||
{
|
||||
if (haystack_size)
|
||||
memset(res.data(), !negate, haystack_size * sizeof(res[0]));
|
||||
res[i] = negate;
|
||||
++i;
|
||||
}
|
||||
else
|
||||
|
||||
/// We check that the entry does not pass through the boundaries of strings.
|
||||
if (pos + required_substring.size() < begin + haystack_offsets[i])
|
||||
{
|
||||
size_t prev_offset = 0;
|
||||
for (size_t i = 0; i < haystack_size; ++i)
|
||||
/// And if it does not, if necessary, we check the regexp.
|
||||
if (is_trivial)
|
||||
res[i] = !negate;
|
||||
else
|
||||
{
|
||||
const char * str_data = reinterpret_cast<const char *>(&haystack_data[haystack_offsets[i - 1]]);
|
||||
size_t str_size = haystack_offsets[i] - haystack_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.
|
||||
*/
|
||||
const size_t start_pos = (required_substring_is_prefix) ? (reinterpret_cast<const char *>(pos) - str_data) : 0;
|
||||
const size_t end_pos = str_size;
|
||||
|
||||
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,
|
||||
{str_data, str_size},
|
||||
start_pos,
|
||||
end_pos,
|
||||
re2_st::RE2::UNANCHORED,
|
||||
nullptr,
|
||||
0);
|
||||
res[i] = negate ^ match;
|
||||
|
||||
prev_offset = haystack_offsets[i];
|
||||
}
|
||||
}
|
||||
else
|
||||
res[i] = negate;
|
||||
|
||||
pos = begin + haystack_offsets[i];
|
||||
++i;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// NOTE This almost matches with the case of impl::likePatternIsSubstring.
|
||||
|
||||
const UInt8 * const begin = haystack_data.data();
|
||||
const UInt8 * const end = haystack_data.begin() + haystack_data.size();
|
||||
const UInt8 * pos = begin;
|
||||
|
||||
/// The current index in the array of strings.
|
||||
size_t i = 0;
|
||||
|
||||
Searcher 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 + haystack_offsets[i] <= pos)
|
||||
{
|
||||
res[i] = negate;
|
||||
++i;
|
||||
}
|
||||
|
||||
/// We check that the entry does not pass through the boundaries of strings.
|
||||
if (pos + required_substring.size() < begin + haystack_offsets[i])
|
||||
{
|
||||
/// And if it does not, if necessary, we check the regexp.
|
||||
|
||||
if (is_trivial)
|
||||
res[i] = !negate;
|
||||
else
|
||||
{
|
||||
const char * str_data = reinterpret_cast<const char *>(&haystack_data[haystack_offsets[i - 1]]);
|
||||
size_t str_size = haystack_offsets[i] - haystack_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.
|
||||
*/
|
||||
const size_t start_pos = (required_substring_is_prefix) ? (reinterpret_cast<const char *>(pos) - str_data) : 0;
|
||||
const size_t end_pos = str_size;
|
||||
|
||||
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
|
||||
res[i] = negate;
|
||||
|
||||
pos = begin + haystack_offsets[i];
|
||||
++i;
|
||||
}
|
||||
|
||||
/// Tail, in which there can be no substring.
|
||||
if (i < res.size())
|
||||
memset(&res[i], negate, (res.size() - i) * sizeof(res[0]));
|
||||
}
|
||||
/// Tail, in which there can be no substring.
|
||||
if (i < res.size())
|
||||
memset(&res[i], negate, (res.size() - i) * sizeof(res[0]));
|
||||
}
|
||||
}
|
||||
|
||||
@ -274,7 +267,7 @@ struct MatchImpl
|
||||
if (haystack.empty())
|
||||
return;
|
||||
|
||||
/// A simple case where the LIKE expression reduces to finding a substring in a string
|
||||
/// Special case that the [I]LIKE expression reduces to finding a substring in a string
|
||||
String strstr_pattern;
|
||||
if (is_like && impl::likePatternIsSubstring(needle, strstr_pattern))
|
||||
{
|
||||
@ -316,109 +309,105 @@ struct MatchImpl
|
||||
/// Tail, in which there can be no substring.
|
||||
if (i < res.size())
|
||||
memset(&res[i], negate, (res.size() - i) * sizeof(res[0]));
|
||||
|
||||
return;
|
||||
}
|
||||
|
||||
const auto & regexp = Regexps::Regexp(Regexps::createRegexp<is_like, /*no_capture*/ true, case_insensitive>(needle));
|
||||
|
||||
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.
|
||||
memset(res.data(), !negate, haystack_size * sizeof(res[0]));
|
||||
else
|
||||
{
|
||||
size_t offset = 0;
|
||||
for (size_t i = 0; i < haystack_size; ++i)
|
||||
{
|
||||
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;
|
||||
}
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto & regexp = Regexps::Regexp(Regexps::createRegexp<is_like, /*no_capture*/ true, case_insensitive>(needle));
|
||||
/// NOTE This almost matches with the case of likePatternIsSubstring.
|
||||
|
||||
String required_substring;
|
||||
bool is_trivial;
|
||||
bool required_substring_is_prefix; /// for `anchored` execution of the regexp.
|
||||
const UInt8 * const begin = haystack.data();
|
||||
const UInt8 * const end = haystack.data() + haystack.size();
|
||||
const UInt8 * pos = begin;
|
||||
|
||||
regexp.getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix);
|
||||
size_t i = 0;
|
||||
const UInt8 * next_pos = begin;
|
||||
|
||||
if (required_substring.empty())
|
||||
/// If required substring is larger than string size - it cannot be found.
|
||||
if (required_substring.size() <= N)
|
||||
{
|
||||
if (!regexp.getRE2()) /// An empty regexp. Always matches.
|
||||
Searcher 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)))
|
||||
{
|
||||
if (haystack_size)
|
||||
memset(res.data(), !negate, haystack_size * sizeof(res[0]));
|
||||
}
|
||||
else
|
||||
{
|
||||
size_t offset = 0;
|
||||
for (size_t i = 0; i < haystack_size; ++i)
|
||||
/// Let's determine which index it refers to.
|
||||
while (next_pos + N <= pos)
|
||||
{
|
||||
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;
|
||||
}
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
/// NOTE This almost matches with the case of likePatternIsSubstring.
|
||||
|
||||
const UInt8 * const begin = haystack.data();
|
||||
const UInt8 * const end = haystack.data() + haystack.size();
|
||||
const UInt8 * pos = begin;
|
||||
|
||||
size_t i = 0;
|
||||
const UInt8 * next_pos = begin;
|
||||
|
||||
/// If required substring is larger than string size - it cannot be found.
|
||||
if (required_substring.size() <= N)
|
||||
{
|
||||
Searcher 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] = negate;
|
||||
next_pos += N;
|
||||
++i;
|
||||
}
|
||||
res[i] = negate;
|
||||
next_pos += N;
|
||||
|
||||
if (pos + required_substring.size() <= next_pos)
|
||||
{
|
||||
/// And if it does not, if necessary, we check the regexp.
|
||||
|
||||
if (is_trivial)
|
||||
res[i] = !negate;
|
||||
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.
|
||||
*/
|
||||
const size_t start_pos = (required_substring_is_prefix) ? (reinterpret_cast<const char *>(pos) - str_data) : 0;
|
||||
const size_t end_pos = N;
|
||||
|
||||
const bool match = regexp.getRE2()->Match(
|
||||
{str_data, N},
|
||||
start_pos,
|
||||
end_pos,
|
||||
re2_st::RE2::UNANCHORED,
|
||||
nullptr,
|
||||
0);
|
||||
res[i] = negate ^ match;
|
||||
}
|
||||
}
|
||||
else
|
||||
res[i] = negate;
|
||||
|
||||
pos = next_pos;
|
||||
++i;
|
||||
}
|
||||
}
|
||||
next_pos += N;
|
||||
|
||||
/// Tail, in which there can be no substring.
|
||||
if (i < res.size())
|
||||
memset(&res[i], negate, (res.size() - i) * sizeof(res[0]));
|
||||
if (pos + required_substring.size() <= next_pos)
|
||||
{
|
||||
/// And if it does not, if necessary, we check the regexp.
|
||||
if (is_trivial)
|
||||
res[i] = !negate;
|
||||
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.
|
||||
*/
|
||||
const size_t start_pos = (required_substring_is_prefix) ? (reinterpret_cast<const char *>(pos) - str_data) : 0;
|
||||
const size_t end_pos = N;
|
||||
|
||||
const bool match = regexp.getRE2()->Match(
|
||||
{str_data, N},
|
||||
start_pos,
|
||||
end_pos,
|
||||
re2_st::RE2::UNANCHORED,
|
||||
nullptr,
|
||||
0);
|
||||
res[i] = negate ^ match;
|
||||
}
|
||||
}
|
||||
else
|
||||
res[i] = negate;
|
||||
|
||||
pos = next_pos;
|
||||
++i;
|
||||
}
|
||||
}
|
||||
|
||||
/// Tail, in which there can be no substring.
|
||||
if (i < res.size())
|
||||
memset(&res[i], negate, (res.size() - i) * sizeof(res[0]));
|
||||
}
|
||||
}
|
||||
|
||||
@ -434,7 +423,6 @@ struct MatchImpl
|
||||
|
||||
assert(haystack_size == needle_offset.size());
|
||||
assert(haystack_size == res.size());
|
||||
|
||||
assert(start_pos_ == nullptr);
|
||||
|
||||
if (haystack_offsets.empty())
|
||||
@ -481,9 +469,7 @@ struct MatchImpl
|
||||
if (required_substr.empty())
|
||||
{
|
||||
if (!regexp->getRE2()) /// An empty regexp. Always matches.
|
||||
{
|
||||
res[i] = !negate;
|
||||
}
|
||||
else
|
||||
{
|
||||
const bool match = regexp->getRE2()->Match(
|
||||
@ -502,15 +488,11 @@ struct MatchImpl
|
||||
const auto * match = searcher.search(cur_haystack_data, cur_haystack_length);
|
||||
|
||||
if (match == cur_haystack_data + cur_haystack_length)
|
||||
{
|
||||
res[i] = negate; // no match
|
||||
}
|
||||
else
|
||||
{
|
||||
if (is_trivial)
|
||||
{
|
||||
res[i] = !negate; // no wildcards in pattern
|
||||
}
|
||||
else
|
||||
{
|
||||
const size_t start_pos = (required_substring_is_prefix) ? (match - cur_haystack_data) : 0;
|
||||
@ -546,7 +528,6 @@ struct MatchImpl
|
||||
|
||||
assert(haystack_size == needle_offset.size());
|
||||
assert(haystack_size == res.size());
|
||||
|
||||
assert(start_pos_ == nullptr);
|
||||
|
||||
if (haystack.empty())
|
||||
@ -593,9 +574,7 @@ struct MatchImpl
|
||||
if (required_substr.empty())
|
||||
{
|
||||
if (!regexp->getRE2()) /// An empty regexp. Always matches.
|
||||
{
|
||||
res[i] = !negate;
|
||||
}
|
||||
else
|
||||
{
|
||||
const bool match = regexp->getRE2()->Match(
|
||||
@ -614,15 +593,11 @@ struct MatchImpl
|
||||
const auto * match = searcher.search(cur_haystack_data, cur_haystack_length);
|
||||
|
||||
if (match == cur_haystack_data + cur_haystack_length)
|
||||
{
|
||||
res[i] = negate; // no match
|
||||
}
|
||||
else
|
||||
{
|
||||
if (is_trivial)
|
||||
{
|
||||
res[i] = !negate; // no wildcards in pattern
|
||||
}
|
||||
else
|
||||
{
|
||||
const size_t start_pos = (required_substring_is_prefix) ? (match - cur_haystack_data) : 0;
|
||||
|
@ -12,7 +12,8 @@ struct NameNotLike
|
||||
static constexpr auto name = "notLike";
|
||||
};
|
||||
|
||||
using FunctionNotLike = FunctionsStringSearch<MatchImpl<NameNotLike, MatchTraits::Syntax::Like, MatchTraits::Case::Sensitive, MatchTraits::Result::Negate>>;
|
||||
using NotLikeImpl = MatchImpl<NameNotLike, MatchTraits::Syntax::Like, MatchTraits::Case::Sensitive, MatchTraits::Result::Negate>;
|
||||
using FunctionNotLike = FunctionsStringSearch<NotLikeImpl>;
|
||||
|
||||
}
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user