mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Measure and rework internal re2 caching
This commit is based on local benchmarks of ClickHouse's re2 caching. Question 1: ----------------------------------------------------------- Is pattern caching useful for queries with const LIKE/REGEX patterns? E.g. SELECT LIKE(col_haystack, '%HelloWorld') FROM T; The short answer is: no. Runtime is (unsurprisingly) dominated by pattern evaluation + other stuff going on in queries, but definitely not pattern compilation. For space reasons, I omit details of the local experiments. (Side note: the current caching scheme is unbounded in size which poses a DoS risk (think of multi-tenancy). This risk is more pronounced when unbounded caching is used with non-const patterns ..., see next question) Question 2: ----------------------------------------------------------- Is pattern caching useful for queries with non-const LIKE/REGEX patterns? E.g. SELECT LIKE(col_haystack, col_needle) FROM T; I benchmarked five caching strategies: 1. no caching as a baseline (= recompile for each row) 2. unbounded cache (= threadsafe global hash-map) 3. LRU cache (= threadsafe global hash-map + LRU queue) 4. lightweight local cache 1 (= not threadsafe local hashmap with collision list which grows to a certain size (here: 10 elements) and afterwards never changes) 5. lightweight local cache 2 (not threadsafe local hashmap without collision list in which a collision replaces the stored element, idea by Alexey) ... using a haystack of 2 mio strings and A). 2 mio distinct simple patterns B). 10 simple patterns C) 2 mio distinct complex patterns D) 10 complex patterns Fo A) and C), caching does not help but these queries still allow to judge the static overhead of caching on query runtimes. B) and D) are extreme but common cases in practice. They include queries like "SELECT ... WHERE LIKE (col_haystack, flag ? '%pattern1%' : '%pattern2%'). Caching should help significantly. Because LIKE patterns are internally translated to re2 expressions, I show only measurements for MATCH queries. Results in sec, averaged over on multiple measurements; 1.A): 2.12 B): 1.68 C): 9.75 D): 9.45 2.A): 2.17 B): 1.73 C): 9.78 D): 9.47 3.A): 9.8 B): 0.63 C): 31.8 D): 0.98 4.A): 2.14 B): 0.29 C): 9.82 D): 0.41 5.A) 2.12 / 2.15 / 2.26 B) 1.51 / 0.43 / 0.30 C) 9.97 / 9.88 / 10.13 D) 5.70 / 0.42 / 0.43 (10/100/1000 buckets, resp. 10/1/0.1% collision rate) Evaluation: 1. This is the baseline. It was surprised that complex patterns (C, D) slow down the queries so badly compared to simple patterns (A, B). The runtime includes evaluation costs, but as caching only helps with compilation, and looking at 4.D and 5.D, compilation makes up over 90% of the runtime! 2. No speedup compared to 1, probably due to locking overhead. The cache is unbounded, and in experiments with data sets > 2 mio rows, 2. is the only scheme to throw OOM exceptions which is not acceptable. 3. Unique patterns (A and C) lead to thrashing of the LRU cache and very bad runtimes due to LRU queue maintenance and locking. Works pretty well however with few distinct patterns (B and D). 4. This scheme is tailored to queries B and D where it performs pretty good. More importantly, the caching is lightweight enough to not deteriorate performance on datasets A and C. 5. After some tuning of the hash map size, 100 buckets seem optimal to be in the same ballpark with 10 distinct patterns as 4. Performance also does not deteriorate on A and C compared to the baseline. Unlike 4., this scheme behaves LRU-like and can adjust to changing pattern distributions. As a conclusion, this commit implementes two things: 1. Based on Q1, pattern search with const needle no longer uses caching. This applies to LIKE and MATCH + a few (exotic) other SQL functions. The code for the unbounded caching was removed. 2. Based on Q2, pattern search with non-const needles now use method 5.
This commit is contained in:
parent
ff228d63e8
commit
ad12adc31c
@ -477,7 +477,7 @@ public:
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
if (!col->getValue<String>().empty())
|
||||
re = Regexps::get<false, false, false>(col->getValue<String>());
|
||||
re = std::make_shared<Regexps::Regexp>(Regexps::createRegexp<false, false, false>(col->getValue<String>()));
|
||||
|
||||
}
|
||||
|
||||
@ -560,7 +560,7 @@ public:
|
||||
+ " of first argument of function " + getName() + ". Must be constant string.",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
re = Regexps::get<false, false, false>(col->getValue<String>());
|
||||
re = std::make_shared<Regexps::Regexp>(Regexps::createRegexp<false, false, false>(col->getValue<String>()));
|
||||
capture = re->getNumberOfSubpatterns() > 0 ? 1 : 0;
|
||||
|
||||
matches.resize(capture + 1);
|
||||
|
@ -166,17 +166,17 @@ struct MatchImpl
|
||||
}
|
||||
else
|
||||
{
|
||||
auto regexp = Regexps::get<is_like, /*no_capture*/ true, case_insensitive>(needle);
|
||||
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);
|
||||
regexp.getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix);
|
||||
|
||||
if (required_substring.empty())
|
||||
{
|
||||
if (!regexp->getRE2()) /// An empty regexp. Always matches.
|
||||
if (!regexp.getRE2()) /// An empty regexp. Always matches.
|
||||
{
|
||||
if (haystack_size)
|
||||
memset(res.data(), !negate, haystack_size * sizeof(res[0]));
|
||||
@ -186,7 +186,7 @@ struct MatchImpl
|
||||
size_t prev_offset = 0;
|
||||
for (size_t i = 0; i < haystack_size; ++i)
|
||||
{
|
||||
const bool match = 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,
|
||||
@ -241,7 +241,7 @@ 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;
|
||||
|
||||
const bool match = regexp->getRE2()->Match(
|
||||
const bool match = regexp.getRE2()->Match(
|
||||
{str_data, str_size},
|
||||
start_pos,
|
||||
end_pos,
|
||||
@ -325,17 +325,17 @@ struct MatchImpl
|
||||
}
|
||||
else
|
||||
{
|
||||
auto regexp = Regexps::get<is_like, /*no_capture*/ true, case_insensitive>(needle);
|
||||
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);
|
||||
regexp.getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix);
|
||||
|
||||
if (required_substring.empty())
|
||||
{
|
||||
if (!regexp->getRE2()) /// An empty regexp. Always matches.
|
||||
if (!regexp.getRE2()) /// An empty regexp. Always matches.
|
||||
{
|
||||
if (haystack_size)
|
||||
memset(res.data(), !negate, haystack_size * sizeof(res[0]));
|
||||
@ -345,7 +345,7 @@ struct MatchImpl
|
||||
size_t offset = 0;
|
||||
for (size_t i = 0; i < haystack_size; ++i)
|
||||
{
|
||||
const bool match = regexp->getRE2()->Match(
|
||||
const bool match = regexp.getRE2()->Match(
|
||||
{reinterpret_cast<const char *>(&haystack[offset]), N},
|
||||
0,
|
||||
N,
|
||||
@ -403,7 +403,7 @@ 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;
|
||||
|
||||
const bool match = regexp->getRE2()->Match(
|
||||
const bool match = regexp.getRE2()->Match(
|
||||
{str_data, N},
|
||||
start_pos,
|
||||
end_pos,
|
||||
@ -454,6 +454,9 @@ struct MatchImpl
|
||||
size_t prev_haystack_offset = 0;
|
||||
size_t prev_needle_offset = 0;
|
||||
|
||||
Regexps::LocalCacheTable cache;
|
||||
Regexps::RegexpPtr regexp;
|
||||
|
||||
for (size_t i = 0; i < haystack_size; ++i)
|
||||
{
|
||||
const auto * const cur_haystack_data = &haystack_data[prev_haystack_offset];
|
||||
@ -479,7 +482,7 @@ struct MatchImpl
|
||||
}
|
||||
else
|
||||
{
|
||||
auto regexp = Regexps::get<is_like, /*no_capture*/ true, case_insensitive>(needle);
|
||||
cache.getOrSet<is_like, /*no_capture*/ true, case_insensitive>(needle, regexp);
|
||||
|
||||
regexp->getAnalyzeResult(required_substr, is_trivial, required_substring_is_prefix);
|
||||
|
||||
@ -565,6 +568,9 @@ struct MatchImpl
|
||||
size_t prev_haystack_offset = 0;
|
||||
size_t prev_needle_offset = 0;
|
||||
|
||||
Regexps::LocalCacheTable cache;
|
||||
Regexps::RegexpPtr regexp;
|
||||
|
||||
for (size_t i = 0; i < haystack_size; ++i)
|
||||
{
|
||||
const auto * const cur_haystack_data = &haystack[prev_haystack_offset];
|
||||
@ -590,7 +596,7 @@ struct MatchImpl
|
||||
}
|
||||
else
|
||||
{
|
||||
auto regexp = Regexps::get<is_like, /*no_capture*/ true, case_insensitive>(needle);
|
||||
cache.getOrSet<is_like, /*no_capture*/ true, case_insensitive>(needle, regexp);
|
||||
|
||||
regexp->getAnalyzeResult(required_substr, is_trivial, required_substring_is_prefix);
|
||||
|
||||
|
@ -9,7 +9,6 @@
|
||||
#include <vector>
|
||||
#include <Functions/likePatternToRegexp.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/LRUCache.h>
|
||||
#include <Common/OptimizedRegularExpression.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/config.h>
|
||||
@ -39,39 +38,73 @@ namespace ErrorCodes
|
||||
namespace Regexps
|
||||
{
|
||||
using Regexp = OptimizedRegularExpressionSingleThreaded;
|
||||
using Cache = LRUCache<String, Regexp>;
|
||||
using RegexpPtr = Cache::MappedPtr;
|
||||
using RegexpPtr = std::shared_ptr<Regexp>;
|
||||
|
||||
template<bool no_capture, bool case_insensitive>
|
||||
inline int buildRe2Flags()
|
||||
template <bool like, bool no_capture, bool case_insensitive>
|
||||
inline Regexp createRegexp(const std::string & pattern)
|
||||
{
|
||||
int flags = OptimizedRegularExpression::RE_DOT_NL;
|
||||
if constexpr (no_capture)
|
||||
flags |= OptimizedRegularExpression::RE_NO_CAPTURE;
|
||||
if constexpr (case_insensitive)
|
||||
flags |= OptimizedRegularExpression::RE_CASELESS;
|
||||
return flags;
|
||||
}
|
||||
|
||||
/// Probes the cache of known compiled regexps for the given string pattern and returns a compiled regexp if
|
||||
/// found. Otherwise, a new cache entry is created.
|
||||
template <bool like, bool no_capture, bool case_insensitive>
|
||||
inline RegexpPtr get(const String & pattern)
|
||||
{
|
||||
static Cache known_regexps(42'000);
|
||||
|
||||
auto [regexp_ptr, _] = known_regexps.getOrSet(pattern, [&pattern]()
|
||||
{
|
||||
const int flags = buildRe2Flags<no_capture, case_insensitive>();
|
||||
ProfileEvents::increment(ProfileEvents::RegexpCreated);
|
||||
if constexpr (like)
|
||||
return std::make_shared<Regexp>(likePatternToRegexp(pattern), flags);
|
||||
return {likePatternToRegexp(pattern), flags};
|
||||
else
|
||||
return std::make_shared<Regexp>(pattern, flags);
|
||||
});
|
||||
return regexp_ptr;
|
||||
return {pattern, flags};
|
||||
}
|
||||
|
||||
/// Caches compiled re2 objects for given string patterns. Intended to support the common situation of a small set of patterns which are
|
||||
/// evaluated over and over within the same query. In these situations, usage of the cache will save unnecessary pattern re-compilation.
|
||||
/// However, we must be careful that caching does not add too much static overhead to overall pattern evaluation. Therefore, the cache is
|
||||
/// intentionally very lightweight: a) no thread-safety/mutexes, b) small & fixed capacity, c) no collision list, d) but also no open
|
||||
/// addressing, instead collisions simply replace the existing element.
|
||||
class LocalCacheTable
|
||||
{
|
||||
public:
|
||||
using RegexpPtr = std::shared_ptr<Regexp>;
|
||||
|
||||
LocalCacheTable()
|
||||
: known_regexps(max_regexp_cache_size, {"", nullptr})
|
||||
{
|
||||
}
|
||||
|
||||
template <bool like, bool no_capture, bool case_insensitive>
|
||||
void getOrSet(const String & pattern, RegexpPtr & regexp)
|
||||
{
|
||||
StringAndRegexp & bucket = known_regexps[hasher(pattern) % max_regexp_cache_size];
|
||||
|
||||
if (likely(bucket.regexp != nullptr))
|
||||
{
|
||||
if (pattern == bucket.pattern)
|
||||
regexp = bucket.regexp;
|
||||
else
|
||||
{
|
||||
regexp = std::make_shared<Regexp>(createRegexp<like, no_capture, case_insensitive>(pattern));
|
||||
bucket = {pattern, regexp};
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
regexp = std::make_shared<Regexp>(createRegexp<like, no_capture, case_insensitive>(pattern));
|
||||
bucket = {pattern, regexp};
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
std::hash<std::string> hasher;
|
||||
struct StringAndRegexp
|
||||
{
|
||||
std::string pattern;
|
||||
RegexpPtr regexp;
|
||||
};
|
||||
using CacheTable = std::vector<StringAndRegexp>;
|
||||
CacheTable known_regexps;
|
||||
|
||||
constexpr static size_t max_regexp_cache_size = 100; // collision probability
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
#if USE_HYPERSCAN
|
||||
|
@ -55,7 +55,7 @@ public:
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
||||
{
|
||||
const ColumnConst * column_pattern = checkAndGetColumnConstStringOrFixedString(arguments[1].column.get());
|
||||
Regexps::RegexpPtr re = Regexps::get<false /* like */, true /* is_no_capture */, CountMatchesBase::case_insensitive>(column_pattern->getValue<String>());
|
||||
const Regexps::Regexp re = Regexps::createRegexp</*is_like*/ false, /*no_capture*/ true, CountMatchesBase::case_insensitive>(column_pattern->getValue<String>());
|
||||
OptimizedRegularExpression::MatchVec matches;
|
||||
|
||||
const IColumn * column_haystack = arguments[0].column.get();
|
||||
@ -95,7 +95,7 @@ public:
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Error in FunctionCountMatches::getReturnTypeImpl()");
|
||||
}
|
||||
|
||||
static uint64_t countMatches(StringRef src, Regexps::RegexpPtr & re, OptimizedRegularExpression::MatchVec & matches)
|
||||
static uint64_t countMatches(StringRef src, const Regexps::Regexp & re, OptimizedRegularExpression::MatchVec & matches)
|
||||
{
|
||||
/// Only one match is required, no need to copy more.
|
||||
static const unsigned matches_limit = 1;
|
||||
@ -108,7 +108,7 @@ public:
|
||||
{
|
||||
if (pos >= end)
|
||||
break;
|
||||
if (!re->match(pos, end - pos, matches, matches_limit))
|
||||
if (!re.match(pos, end - pos, matches, matches_limit))
|
||||
break;
|
||||
/// Progress should be made, but with empty match the progress will not be done.
|
||||
/// Also note that simply check is pattern empty is not enough,
|
||||
|
@ -21,9 +21,9 @@ struct ExtractImpl
|
||||
res_data.reserve(data.size() / 5);
|
||||
res_offsets.resize(offsets.size());
|
||||
|
||||
const auto & regexp = Regexps::get<false, false, false>(pattern);
|
||||
const Regexps::Regexp regexp = Regexps::createRegexp<false, false, false>(pattern);
|
||||
|
||||
unsigned capture = regexp->getNumberOfSubpatterns() > 0 ? 1 : 0;
|
||||
unsigned capture = regexp.getNumberOfSubpatterns() > 0 ? 1 : 0;
|
||||
OptimizedRegularExpression::MatchVec matches;
|
||||
matches.reserve(capture + 1);
|
||||
size_t prev_offset = 0;
|
||||
@ -34,7 +34,7 @@ struct ExtractImpl
|
||||
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);
|
||||
= 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];
|
||||
|
@ -95,8 +95,8 @@ public:
|
||||
throw Exception("Length of 'needle' argument must be greater than 0.", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
using StringPiece = typename Regexps::Regexp::StringPieceType;
|
||||
auto holder = Regexps::get<false, false, false>(needle);
|
||||
const auto & regexp = holder->getRE2();
|
||||
const Regexps::Regexp holder = Regexps::createRegexp<false, false, false>(needle);
|
||||
const auto & regexp = holder.getRE2();
|
||||
|
||||
if (!regexp)
|
||||
throw Exception("There are no groups in regexp: " + needle, ErrorCodes::BAD_ARGUMENTS);
|
||||
|
@ -63,8 +63,8 @@ public:
|
||||
if (needle.empty())
|
||||
throw Exception(getName() + " length of 'needle' argument must be greater than 0.", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
auto regexp = Regexps::get<false, false, false>(needle);
|
||||
const auto & re2 = regexp->getRE2();
|
||||
const Regexps::Regexp regexp = Regexps::createRegexp<false, false, false>(needle);
|
||||
const auto & re2 = regexp.getRE2();
|
||||
|
||||
if (!re2)
|
||||
throw Exception("There are no groups in regexp: " + needle, ErrorCodes::BAD_ARGUMENTS);
|
||||
|
62
tests/performance/like_and_match_pattern_caching.xml
Normal file
62
tests/performance/like_and_match_pattern_caching.xml
Normal file
@ -0,0 +1,62 @@
|
||||
<!-- Simple benchmark to verify the caching of compiled re2 patterns (e.g. LIKE/MATCH) -->
|
||||
|
||||
<test>
|
||||
<substitutions>
|
||||
<substitution>
|
||||
<name>numbers</name>
|
||||
<values>
|
||||
<value>numbers_mt(2000000)</value>
|
||||
</values>
|
||||
</substitution>
|
||||
<substitution>
|
||||
<name>needle_like</name>
|
||||
<values>
|
||||
simple patterns, all unique
|
||||
<value>'%' || toString(number) || '_'</value>
|
||||
simple patterns, low distinctness (10 patterns)
|
||||
<value>'%' || toString(number % 10) || '_'</value>
|
||||
</values>
|
||||
</substitution>
|
||||
<substitution>
|
||||
<name>needle_match</name>
|
||||
<values>
|
||||
<!-- simple patterns, all unique -->
|
||||
<value>'.*' || toString(number) || '.'</value>
|
||||
<!-- simple patterns, low distinctness (10 patterns) -->
|
||||
<value>'.*' || toString(number % 10) || '.'</value>
|
||||
<!-- complex patterns, all unique -->
|
||||
<value>'([a-zA-Z][a-zA-Z0-9]*)://([^ /]+)(/[^ ]*)?([^ @]+)@([^ @]+)([0-9][0-9]?)/([0-9][0-9]?)/([0-9][0-9]([0-9][0-9])?)(?:(?:25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9])\.){3}(?:25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9])' || toString(number)</value>
|
||||
<!-- complex patterns, low distinctness -->
|
||||
<value>'([a-zA-Z][a-zA-Z0-9]*)://([^ /]+)(/[^ ]*)?([^ @]+)@([^ @]+)([0-9][0-9]?)/([0-9][0-9]?)/([0-9][0-9]([0-9][0-9])?)(?:(?:25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9])\.){3}(?:25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9])' || toString(number % 10)</value>
|
||||
<!-- Note: for this benchmark, we are only interested in compilation time, not correctness, evaluation time or the result.
|
||||
Therefore, this is a maximally expensive to compile "pattern from hell": a concatenation of || email || date ||
|
||||
ip4 || <number (% 10), see http://lh3lh3.users.sourceforge.net/reb.shtml and https://github.com/mariomka/regex-benchmark -->
|
||||
</values>
|
||||
</substitution>
|
||||
</substitutions>
|
||||
|
||||
<!-- const needle (just for reference) -->
|
||||
|
||||
<query>
|
||||
select toString(number) as haystack, like(haystack, '%x_')
|
||||
from(select * from {numbers})
|
||||
</query>
|
||||
|
||||
<query>
|
||||
select toString(number) as haystack, match(haystack, '.*x.')
|
||||
from(select * from {numbers})
|
||||
</query>
|
||||
|
||||
<!-- non-const needle -->
|
||||
|
||||
<query>
|
||||
select toString(number) as haystack, {needle_like} as needle, like(haystack, needle)
|
||||
from (select * from {numbers});
|
||||
</query>
|
||||
|
||||
<query>
|
||||
select toString(number) as haystack, {needle_match} as needle, match(haystack, needle)
|
||||
from (select * from {numbers});
|
||||
</query>
|
||||
|
||||
</test>
|
Loading…
Reference in New Issue
Block a user