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:
Robert Schulze 2022-11-04 09:09:48 +00:00
parent 3ed11c7a7b
commit a8098db25d
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
3 changed files with 184 additions and 199 deletions

View File

@ -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;
}

View File

@ -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,9 +155,10 @@ 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;
}
else
{
const auto & regexp = Regexps::Regexp(Regexps::createRegexp<is_like, /*no_capture*/ true, case_insensitive>(needle));
String required_substring;
@ -172,10 +170,7 @@ struct MatchImpl
if (required_substring.empty())
{
if (!regexp.getRE2()) /// An empty regexp. Always matches.
{
if (haystack_size)
memset(res.data(), !negate, haystack_size * sizeof(res[0]));
}
else
{
size_t prev_offset = 0;
@ -221,7 +216,6 @@ struct MatchImpl
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
@ -258,7 +252,6 @@ struct MatchImpl
memset(&res[i], negate, (res.size() - i) * sizeof(res[0]));
}
}
}
/// Very carefully crafted copy-paste.
static void vectorFixedConstant(
@ -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,9 +309,10 @@ 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;
}
else
{
const auto & regexp = Regexps::Regexp(Regexps::createRegexp<is_like, /*no_capture*/ true, case_insensitive>(needle));
String required_substring;
@ -330,10 +324,7 @@ struct MatchImpl
if (required_substring.empty())
{
if (!regexp.getRE2()) /// An empty regexp. Always matches.
{
if (haystack_size)
memset(res.data(), !negate, haystack_size * sizeof(res[0]));
}
else
{
size_t offset = 0;
@ -383,7 +374,6 @@ struct MatchImpl
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
@ -420,7 +410,6 @@ struct MatchImpl
memset(&res[i], negate, (res.size() - i) * sizeof(res[0]));
}
}
}
static void vectorVector(
const ColumnString::Chars & haystack_data,
@ -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;

View File

@ -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>;
}