未验证 提交 19ecc3e6 编写于 作者: D Danila Kutenin 提交者: GitHub

Merge pull request #7299 from ClickHouse/multi_match_all_indices

Add multi{Fuzzy}MatchAllIndices functions
......@@ -268,14 +268,22 @@ 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 ReturnType()
{
return std::make_shared<DataTypeNumber<ResultType>>();
}
static void vector_constant(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const std::vector<StringRef> & needles,
PaddedPODArray<Type> & res)
PaddedPODArray<Type> & res,
PaddedPODArray<UInt64> & offsets)
{
vector_constant(haystack_data, haystack_offsets, needles, res, std::nullopt);
vector_constant(haystack_data, haystack_offsets, needles, res, offsets, std::nullopt);
}
static void vector_constant(
......@@ -283,10 +291,12 @@ struct MultiMatchAnyImpl
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;
......@@ -307,15 +317,18 @@ struct MultiMatchAnyImpl
*reinterpret_cast<Type *>(context) = id;
else if constexpr (FindAny)
*reinterpret_cast<Type *>(context) = 1;
return 0;
/// 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(),
......@@ -325,7 +338,7 @@ struct MultiMatchAnyImpl
smart_scratch.get(),
on_match,
&res[i]);
if (err != HS_SUCCESS)
if (err != HS_SUCCESS && err != HS_SCAN_TERMINATED)
throw Exception("Failed to scan with hyperscan", ErrorCodes::HYPERSCAN_CANNOT_SCAN_TEXT);
offset = haystack_offsets[i];
}
......@@ -333,7 +346,7 @@ struct MultiMatchAnyImpl
/// Fallback if do not use hyperscan
if constexpr (MultiSearchDistance)
throw Exception(
"Edit distance multi-search is not implemented when hyperscan is off (is it Intel processor?)",
"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()));
......@@ -353,6 +366,92 @@ struct MultiMatchAnyImpl
}
};
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 ReturnType()
{
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
}
static void vector_constant(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const std::vector<StringRef> & needles,
PaddedPODArray<Type> & res,
PaddedPODArray<UInt64> & offsets)
{
vector_constant(haystack_data, haystack_offsets, needles, res, offsets, std::nullopt);
}
static void vector_constant(
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 */,
unsigned long long /* to */,
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
}
};
struct ExtractImpl
{
......@@ -866,6 +965,10 @@ struct NameMultiMatchAnyIndex
{
static constexpr auto name = "multiMatchAnyIndex";
};
struct NameMultiMatchAllIndices
{
static constexpr auto name = "multiMatchAllIndices";
};
struct NameMultiFuzzyMatchAny
{
static constexpr auto name = "multiFuzzyMatchAny";
......@@ -874,6 +977,10 @@ struct NameMultiFuzzyMatchAnyIndex
{
static constexpr auto name = "multiFuzzyMatchAnyIndex";
};
struct NameMultiFuzzyMatchAllIndices
{
static constexpr auto name = "multiFuzzyMatchAllIndices";
};
struct NameExtract
{
static constexpr auto name = "extract";
......@@ -908,6 +1015,11 @@ using FunctionMultiMatchAnyIndex = FunctionsMultiStringSearch<
NameMultiMatchAnyIndex,
std::numeric_limits<UInt32>::max()>;
using FunctionMultiMatchAllIndices = FunctionsMultiStringSearch<
MultiMatchAllIndicesImpl<UInt64, false>,
NameMultiMatchAllIndices,
std::numeric_limits<UInt32>::max()>;
using FunctionMultiFuzzyMatchAny = FunctionsMultiStringFuzzySearch<
MultiMatchAnyImpl<UInt8, true, false, true>,
NameMultiFuzzyMatchAny,
......@@ -918,6 +1030,11 @@ using FunctionMultiFuzzyMatchAnyIndex = FunctionsMultiStringFuzzySearch<
NameMultiFuzzyMatchAnyIndex,
std::numeric_limits<UInt32>::max()>;
using FunctionMultiFuzzyMatchAllIndices = FunctionsMultiStringFuzzySearch<
MultiMatchAllIndicesImpl<UInt64, true>,
NameMultiFuzzyMatchAllIndices,
std::numeric_limits<UInt32>::max()>;
using FunctionLike = FunctionsStringSearch<MatchImpl<true>, NameLike>;
using FunctionNotLike = FunctionsStringSearch<MatchImpl<true, true>, NameNotLike>;
using FunctionExtract = FunctionsStringSearchToString<ExtractImpl, NameExtract>;
......@@ -940,8 +1057,10 @@ void registerFunctionsStringRegex(FunctionFactory & factory)
factory.registerFunction<FunctionMultiMatchAny>();
factory.registerFunction<FunctionMultiMatchAnyIndex>();
factory.registerFunction<FunctionMultiMatchAllIndices>();
factory.registerFunction<FunctionMultiFuzzyMatchAny>();
factory.registerFunction<FunctionMultiFuzzyMatchAnyIndex>();
factory.registerFunction<FunctionMultiFuzzyMatchAllIndices>();
factory.registerAlias("replace", NameReplaceAll::name, FunctionFactory::CaseInsensitive);
}
}
......@@ -63,9 +63,7 @@ public:
if (!array_type || !checkAndGetDataType<DataTypeString>(array_type->getNestedType().get()))
throw Exception(
"Illegal type " + arguments[2]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeNumber<typename Impl::ResultType>>();
return Impl::ReturnType();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
......@@ -115,20 +113,23 @@ public:
for (const auto & el : src_arr)
refs.emplace_back(el.get<String>());
const size_t column_haystack_size = column_haystack->size();
auto col_res = ColumnVector<ResultType>::create();
auto col_offsets = ColumnArray::ColumnOffsets::create();
auto & vec_res = col_res->getData();
auto & offsets_res = col_offsets->getData();
vec_res.resize(column_haystack_size);
/// The blame for resizing output is for the callee.
if (col_haystack_vector)
Impl::vector_constant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res, edit_distance);
Impl::vector_constant(
col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res, offsets_res, edit_distance);
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN);
block.getByPosition(result).column = std::move(col_res);
if constexpr (Impl::is_column_array)
block.getByPosition(result).column = ColumnArray::create(std::move(col_res), std::move(col_offsets));
else
block.getByPosition(result).column = std::move(col_res);
}
};
......
......@@ -47,7 +47,7 @@ struct PositionCaseSensitiveASCII
/// Convert string to lowercase. Only for case-insensitive search.
/// Implementation is permitted to be inefficient because it is called for single string.
static void toLowerIfNeed(std::string &) {}
static void toLowerIfNeed(std::string &) { }
};
struct PositionCaseInsensitiveASCII
......@@ -107,7 +107,7 @@ struct PositionCaseSensitiveUTF8
return res;
}
static void toLowerIfNeed(std::string &) {}
static void toLowerIfNeed(std::string &) { }
};
struct PositionCaseInsensitiveUTF8
......@@ -335,15 +335,21 @@ struct MultiSearchImpl
{
using ResultType = UInt8;
static constexpr bool is_using_hyperscan = false;
/// 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 ReturnType() { return std::make_shared<DataTypeNumber<ResultType>>(); }
static void vector_constant(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const std::vector<StringRef> & needles,
PaddedPODArray<UInt8> & res)
PaddedPODArray<UInt8> & res,
[[maybe_unused]] PaddedPODArray<UInt64> & offsets)
{
auto searcher = Impl::createMultiSearcherInBigHaystack(needles);
const size_t haystack_string_size = haystack_offsets.size();
res.resize(haystack_string_size);
size_t iteration = 0;
while (searcher.hasMoreToSearch())
{
......@@ -366,12 +372,17 @@ struct MultiSearchFirstPositionImpl
{
using ResultType = UInt64;
static constexpr bool is_using_hyperscan = false;
/// 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 ReturnType() { return std::make_shared<DataTypeNumber<ResultType>>(); }
static void vector_constant(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const std::vector<StringRef> & needles,
PaddedPODArray<UInt64> & res)
PaddedPODArray<UInt64> & res,
[[maybe_unused]] PaddedPODArray<UInt64> & offsets)
{
auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64
{
......@@ -379,6 +390,7 @@ struct MultiSearchFirstPositionImpl
};
auto searcher = Impl::createMultiSearcherInBigHaystack(needles);
const size_t haystack_string_size = haystack_offsets.size();
res.resize(haystack_string_size);
size_t iteration = 0;
while (searcher.hasMoreToSearch())
{
......@@ -407,15 +419,21 @@ struct MultiSearchFirstIndexImpl
{
using ResultType = UInt64;
static constexpr bool is_using_hyperscan = false;
/// 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 ReturnType() { return std::make_shared<DataTypeNumber<ResultType>>(); }
static void vector_constant(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const std::vector<StringRef> & needles,
PaddedPODArray<UInt64> & res)
PaddedPODArray<UInt64> & res,
[[maybe_unused]] PaddedPODArray<UInt64> & offsets)
{
auto searcher = Impl::createMultiSearcherInBigHaystack(needles);
const size_t haystack_string_size = haystack_offsets.size();
res.resize(haystack_string_size);
size_t iteration = 0;
while (searcher.hasMoreToSearch())
{
......@@ -598,30 +616,48 @@ struct NameHasTokenCaseInsensitive
using FunctionPosition = FunctionsStringSearch<PositionImpl<PositionCaseSensitiveASCII>, NamePosition>;
using FunctionPositionUTF8 = FunctionsStringSearch<PositionImpl<PositionCaseSensitiveUTF8>, NamePositionUTF8>;
using FunctionPositionCaseInsensitive = FunctionsStringSearch<PositionImpl<PositionCaseInsensitiveASCII>, NamePositionCaseInsensitive>;
using FunctionPositionCaseInsensitiveUTF8 = FunctionsStringSearch<PositionImpl<PositionCaseInsensitiveUTF8>, NamePositionCaseInsensitiveUTF8>;
using FunctionMultiSearchAllPositions = FunctionsMultiStringPosition<MultiSearchAllPositionsImpl<PositionCaseSensitiveASCII>, NameMultiSearchAllPositions>;
using FunctionMultiSearchAllPositionsUTF8 = FunctionsMultiStringPosition<MultiSearchAllPositionsImpl<PositionCaseSensitiveUTF8>, NameMultiSearchAllPositionsUTF8>;
using FunctionMultiSearchAllPositionsCaseInsensitive = FunctionsMultiStringPosition<MultiSearchAllPositionsImpl<PositionCaseInsensitiveASCII>, NameMultiSearchAllPositionsCaseInsensitive>;
using FunctionMultiSearchAllPositionsCaseInsensitiveUTF8 = FunctionsMultiStringPosition<MultiSearchAllPositionsImpl<PositionCaseInsensitiveUTF8>, NameMultiSearchAllPositionsCaseInsensitiveUTF8>;
using FunctionPositionCaseInsensitiveUTF8
= FunctionsStringSearch<PositionImpl<PositionCaseInsensitiveUTF8>, NamePositionCaseInsensitiveUTF8>;
using FunctionMultiSearchAllPositions
= FunctionsMultiStringPosition<MultiSearchAllPositionsImpl<PositionCaseSensitiveASCII>, NameMultiSearchAllPositions>;
using FunctionMultiSearchAllPositionsUTF8
= FunctionsMultiStringPosition<MultiSearchAllPositionsImpl<PositionCaseSensitiveUTF8>, NameMultiSearchAllPositionsUTF8>;
using FunctionMultiSearchAllPositionsCaseInsensitive
= FunctionsMultiStringPosition<MultiSearchAllPositionsImpl<PositionCaseInsensitiveASCII>, NameMultiSearchAllPositionsCaseInsensitive>;
using FunctionMultiSearchAllPositionsCaseInsensitiveUTF8 = FunctionsMultiStringPosition<
MultiSearchAllPositionsImpl<PositionCaseInsensitiveUTF8>,
NameMultiSearchAllPositionsCaseInsensitiveUTF8>;
using FunctionMultiSearch = FunctionsMultiStringSearch<MultiSearchImpl<PositionCaseSensitiveASCII>, NameMultiSearchAny>;
using FunctionMultiSearchUTF8 = FunctionsMultiStringSearch<MultiSearchImpl<PositionCaseSensitiveUTF8>, NameMultiSearchAnyUTF8>;
using FunctionMultiSearchCaseInsensitive = FunctionsMultiStringSearch<MultiSearchImpl<PositionCaseInsensitiveASCII>, NameMultiSearchAnyCaseInsensitive>;
using FunctionMultiSearchCaseInsensitiveUTF8 = FunctionsMultiStringSearch<MultiSearchImpl<PositionCaseInsensitiveUTF8>, NameMultiSearchAnyCaseInsensitiveUTF8>;
using FunctionMultiSearchFirstIndex = FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseSensitiveASCII>, NameMultiSearchFirstIndex>;
using FunctionMultiSearchFirstIndexUTF8 = FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseSensitiveUTF8>, NameMultiSearchFirstIndexUTF8>;
using FunctionMultiSearchFirstIndexCaseInsensitive = FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseInsensitiveASCII>, NameMultiSearchFirstIndexCaseInsensitive>;
using FunctionMultiSearchFirstIndexCaseInsensitiveUTF8 = FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseInsensitiveUTF8>, NameMultiSearchFirstIndexCaseInsensitiveUTF8>;
using FunctionMultiSearchFirstPosition = FunctionsMultiStringSearch<MultiSearchFirstPositionImpl<PositionCaseSensitiveASCII>, NameMultiSearchFirstPosition>;
using FunctionMultiSearchFirstPositionUTF8 = FunctionsMultiStringSearch<MultiSearchFirstPositionImpl<PositionCaseSensitiveUTF8>, NameMultiSearchFirstPositionUTF8>;
using FunctionMultiSearchFirstPositionCaseInsensitive = FunctionsMultiStringSearch<MultiSearchFirstPositionImpl<PositionCaseInsensitiveASCII>, NameMultiSearchFirstPositionCaseInsensitive>;
using FunctionMultiSearchFirstPositionCaseInsensitiveUTF8 = FunctionsMultiStringSearch<MultiSearchFirstPositionImpl<PositionCaseInsensitiveUTF8>, NameMultiSearchFirstPositionCaseInsensitiveUTF8>;
using FunctionMultiSearchCaseInsensitive
= FunctionsMultiStringSearch<MultiSearchImpl<PositionCaseInsensitiveASCII>, NameMultiSearchAnyCaseInsensitive>;
using FunctionMultiSearchCaseInsensitiveUTF8
= FunctionsMultiStringSearch<MultiSearchImpl<PositionCaseInsensitiveUTF8>, NameMultiSearchAnyCaseInsensitiveUTF8>;
using FunctionMultiSearchFirstIndex
= FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseSensitiveASCII>, NameMultiSearchFirstIndex>;
using FunctionMultiSearchFirstIndexUTF8
= FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseSensitiveUTF8>, NameMultiSearchFirstIndexUTF8>;
using FunctionMultiSearchFirstIndexCaseInsensitive
= FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseInsensitiveASCII>, NameMultiSearchFirstIndexCaseInsensitive>;
using FunctionMultiSearchFirstIndexCaseInsensitiveUTF8
= FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseInsensitiveUTF8>, NameMultiSearchFirstIndexCaseInsensitiveUTF8>;
using FunctionMultiSearchFirstPosition
= FunctionsMultiStringSearch<MultiSearchFirstPositionImpl<PositionCaseSensitiveASCII>, NameMultiSearchFirstPosition>;
using FunctionMultiSearchFirstPositionUTF8
= FunctionsMultiStringSearch<MultiSearchFirstPositionImpl<PositionCaseSensitiveUTF8>, NameMultiSearchFirstPositionUTF8>;
using FunctionMultiSearchFirstPositionCaseInsensitive
= FunctionsMultiStringSearch<MultiSearchFirstPositionImpl<PositionCaseInsensitiveASCII>, NameMultiSearchFirstPositionCaseInsensitive>;
using FunctionMultiSearchFirstPositionCaseInsensitiveUTF8 = FunctionsMultiStringSearch<
MultiSearchFirstPositionImpl<PositionCaseInsensitiveUTF8>,
NameMultiSearchFirstPositionCaseInsensitiveUTF8>;
using FunctionHasToken = FunctionsStringSearch<HasTokenImpl<VolnitskyCaseSensitiveToken, false>, NameHasToken>;
using FunctionHasTokenCaseInsensitive = FunctionsStringSearch<HasTokenImpl<VolnitskyCaseInsensitiveToken, false>, NameHasTokenCaseInsensitive>;
using FunctionHasTokenCaseInsensitive
= FunctionsStringSearch<HasTokenImpl<VolnitskyCaseInsensitiveToken, false>, NameHasTokenCaseInsensitive>;
void registerFunctionsStringSearch(FunctionFactory & factory)
{
......
......@@ -28,6 +28,7 @@ namespace DB
* match(haystack, pattern) - search by regular expression re2; Returns 0 or 1.
* multiMatchAny(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- search by re2 regular expressions pattern_i; Returns 0 or 1 if any pattern_i matches.
* multiMatchAnyIndex(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- search by re2 regular expressions pattern_i; Returns index of any match or zero if none;
* multiMatchAllIndices(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- search by re2 regular expressions pattern_i; Returns an array of matched indices in any order;
*
* Applies regexp re2 and pulls:
* - the first subpattern, if the regexp has a subpattern;
......@@ -312,9 +313,7 @@ public:
if (!array_type || !checkAndGetDataType<DataTypeString>(array_type->getNestedType().get()))
throw Exception(
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeNumber<typename Impl::ResultType>>();
return Impl::ReturnType();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
......@@ -347,20 +346,22 @@ public:
for (const auto & el : src_arr)
refs.emplace_back(el.get<String>());
const size_t column_haystack_size = column_haystack->size();
auto col_res = ColumnVector<ResultType>::create();
auto col_offsets = ColumnArray::ColumnOffsets::create();
auto & vec_res = col_res->getData();
auto & offsets_res = col_offsets->getData();
vec_res.resize(column_haystack_size);
/// The blame for resizing output is for the callee.
if (col_haystack_vector)
Impl::vector_constant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res);
Impl::vector_constant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res, offsets_res);
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN);
block.getByPosition(result).column = std::move(col_res);
if constexpr (Impl::is_column_array)
block.getByPosition(result).column = ColumnArray::create(std::move(col_res), std::move(col_offsets));
else
block.getByPosition(result).column = std::move(col_res);
}
};
......
......@@ -8,10 +8,10 @@
#include <utility>
#include <vector>
#include <Functions/likePatternToRegexp.h>
#include <Common/Exception.h>
#include <Common/ObjectPool.h>
#include <Common/OptimizedRegularExpression.h>
#include <Common/ProfileEvents.h>
#include <Common/Exception.h>
#include <common/StringRef.h>
......@@ -87,18 +87,20 @@ namespace MultiRegexps
}
};
/// Helper unique pointers to correctly delete the allocated space when hyperscan cannot compile something and we throw an exception.
using CompilerError = std::unique_ptr<hs_compile_error_t, HyperscanDeleter<decltype(&hs_free_compile_error), &hs_free_compile_error>>;
using ScratchPtr = std::unique_ptr<hs_scratch_t, HyperscanDeleter<decltype(&hs_free_scratch), &hs_free_scratch>>;
using DataBasePtr = std::unique_ptr<hs_database_t, HyperscanDeleter<decltype(&hs_free_database), &hs_free_database>>;
/// Database is thread safe across multiple threads and Scratch is not but we can copy it whenever we use it in the searcher
/// Database is thread safe across multiple threads and Scratch is not but we can copy it whenever we use it in the searcher.
class Regexps
{
public:
Regexps(hs_database_t * db_, hs_scratch_t * scratch_) : db{db_}, scratch{scratch_} {}
Regexps(hs_database_t * db_, hs_scratch_t * scratch_) : db{db_}, scratch{scratch_} { }
hs_database_t * getDB() const { return db.get(); }
hs_scratch_t * getScratch() const { return scratch.get(); }
private:
DataBasePtr db;
ScratchPtr scratch;
......@@ -106,25 +108,25 @@ namespace MultiRegexps
struct Pool
{
/// Mutex for finding in map
/// Mutex for finding in map.
std::mutex mutex;
/// Patterns + possible edit_distance to database and scratch
/// Patterns + possible edit_distance to database and scratch.
std::map<std::pair<std::vector<String>, std::optional<UInt32>>, Regexps> storage;
};
template <bool FindAnyIndex, bool CompileForEditDistance>
template <bool SaveIndices, bool CompileForEditDistance>
inline Regexps constructRegexps(const std::vector<String> & str_patterns, std::optional<UInt32> edit_distance)
{
(void)edit_distance;
/// Common pointers
std::vector<const char *> ptrns;
std::vector<const char *> patterns;
std::vector<unsigned int> flags;
/// Pointer for external edit distance compilation
std::vector<hs_expr_ext> ext_exprs;
std::vector<const hs_expr_ext *> ext_exprs_ptrs;
ptrns.reserve(str_patterns.size());
patterns.reserve(str_patterns.size());
flags.reserve(str_patterns.size());
if constexpr (CompileForEditDistance)
......@@ -135,12 +137,22 @@ namespace MultiRegexps
for (const StringRef ref : str_patterns)
{
ptrns.push_back(ref.data);
flags.push_back(HS_FLAG_DOTALL | HS_FLAG_ALLOWEMPTY | HS_FLAG_SINGLEMATCH | HS_FLAG_UTF8);
patterns.push_back(ref.data);
/* Flags below are the pattern matching flags.
* HS_FLAG_DOTALL is a compile flag where matching a . will not exclude newlines. This is a good
* performance practice accrording to Hyperscan API. https://intel.github.io/hyperscan/dev-reference/performance.html#dot-all-mode
* HS_FLAG_ALLOWEMPTY is a compile flag where empty strings are allowed to match.
* HS_FLAG_UTF8 is a flag where UTF8 literals are matched.
* HS_FLAG_SINGLEMATCH is a compile flag where each pattern match will be returned only once. it is a good performance practice
* as it is said in the Hyperscan documentation. https://intel.github.io/hyperscan/dev-reference/performance.html#single-match-flag
*/
flags.push_back(HS_FLAG_DOTALL | HS_FLAG_SINGLEMATCH | HS_FLAG_ALLOWEMPTY | HS_FLAG_UTF8);
if constexpr (CompileForEditDistance)
{
/// Hyperscan currently does not support UTF8 matching with edit distance.
flags.back() &= ~HS_FLAG_UTF8;
ext_exprs.emplace_back();
/// HS_EXT_FLAG_EDIT_DISTANCE is a compile flag responsible for Levenstein distance.
ext_exprs.back().flags = HS_EXT_FLAG_EDIT_DISTANCE;
ext_exprs.back().edit_distance = edit_distance.value();
ext_exprs_ptrs.push_back(&ext_exprs.back());
......@@ -152,31 +164,32 @@ namespace MultiRegexps
std::unique_ptr<unsigned int[]> ids;
if constexpr (FindAnyIndex)
/// We mark the patterns to provide the callback results.
if constexpr (SaveIndices)
{
ids.reset(new unsigned int[ptrns.size()]);
for (size_t i = 0; i < ptrns.size(); ++i)
ids.reset(new unsigned int[patterns.size()]);
for (size_t i = 0; i < patterns.size(); ++i)
ids[i] = i + 1;
}
hs_error_t err;
if constexpr (!CompileForEditDistance)
err = hs_compile_multi(
ptrns.data(),
patterns.data(),
flags.data(),
ids.get(),
ptrns.size(),
patterns.size(),
HS_MODE_BLOCK,
nullptr,
&db,
&compile_error);
else
err = hs_compile_ext_multi(
ptrns.data(),
patterns.data(),
flags.data(),
ids.get(),
ext_exprs_ptrs.data(),
ptrns.size(),
patterns.size(),
HS_MODE_BLOCK,
nullptr,
&db,
......@@ -184,6 +197,7 @@ namespace MultiRegexps
if (err != HS_SUCCESS)
{
/// CompilerError is a unique_ptr, so correct memory free after the exception is thrown.
CompilerError error(compile_error);
if (error->expression < 0)
......@@ -196,9 +210,12 @@ namespace MultiRegexps
ProfileEvents::increment(ProfileEvents::RegexpCreated);
/// We allocate the scratch space only once, then copy it across multiple threads with hs_clone_scratch
/// function which is faster than allocating scratch space each time in each thread.
hs_scratch_t * scratch = nullptr;
err = hs_alloc_scratch(db, &scratch);
/// If not HS_SUCCESS, it is guaranteed that the memory would not be allocated for scratch.
if (err != HS_SUCCESS)
throw Exception("Could not allocate scratch space for hyperscan", ErrorCodes::CANNOT_ALLOCATE_MEMORY);
......@@ -206,7 +223,10 @@ namespace MultiRegexps
}
/// If CompileForEditDistance is False, edit_distance must be nullopt
template <bool FindAnyIndex, bool CompileForEditDistance>
/// Also, we use templates here because each instantiation of function
/// template has its own copy of local static variables which must not be the same
/// for different hyperscan compilations.
template <bool SaveIndices, bool CompileForEditDistance>
inline Regexps * get(const std::vector<StringRef> & patterns, std::optional<UInt32> edit_distance)
{
/// C++11 has thread-safe function-local statics on most modern compilers.
......@@ -217,15 +237,19 @@ namespace MultiRegexps
for (const StringRef & ref : patterns)
str_patterns.push_back(ref.toString());
/// Get the lock for finding database.
std::unique_lock lock(known_regexps.mutex);
auto it = known_regexps.storage.find({str_patterns, edit_distance});
/// If not found, compile and let other threads wait.
if (known_regexps.storage.end() == it)
it = known_regexps.storage.emplace(
std::pair{str_patterns, edit_distance},
constructRegexps<FindAnyIndex, CompileForEditDistance>(str_patterns, edit_distance)).first;
it = known_regexps.storage
.emplace(
std::pair{str_patterns, edit_distance},
constructRegexps<SaveIndices, CompileForEditDistance>(str_patterns, edit_distance))
.first;
/// If found, unlock and return the database.
lock.unlock();
return &it->second;
......
......@@ -600,3 +600,26 @@
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
All tests above must return 1, all tests below return something.
[]
[1,3]
[]
[1,2,3]
......@@ -73,10 +73,20 @@ select 1 = multiMatchAny(materialize('abcdef'), ['a......', 'a.....']) from syst
select 0 = multiMatchAny(materialize('aaaa'), ['.*aa.*aaa.*', 'aaaaaa{2}', '\(aa\){3}']) from system.numbers limit 10;
select 1 = multiMatchAny(materialize('abc'), ['a\0d']) from system.numbers limit 10;
select 1 = multiMatchAnyIndex(materialize('gogleuedeyandexgoogle'), ['google', 'yandex1']) from system.numbers limit 10;;
select 2 = multiMatchAnyIndex(materialize('gogleuedeyandexgoogle'), ['google1', 'yandex']) from system.numbers limit 10;;
select 0 != multiMatchAnyIndex(materialize('gogleuedeyandexgoogle'), ['.*goo.*', '.*yan.*']) from system.numbers limit 10;;
select 5 = multiMatchAnyIndex(materialize('vladizlvav dabe don\'t heart me no more'), ['what', 'is', 'love', 'baby', 'no mo??', 'dont', 'h.rt me']) from system.numbers limit 10;;
select 1 = multiMatchAnyIndex(materialize('gogleuedeyandexgoogle'), ['google', 'yandex1']) from system.numbers limit 10;
select 2 = multiMatchAnyIndex(materialize('gogleuedeyandexgoogle'), ['google1', 'yandex']) from system.numbers limit 10;
select 0 != multiMatchAnyIndex(materialize('gogleuedeyandexgoogle'), ['.*goo.*', '.*yan.*']) from system.numbers limit 10;
select 5 = multiMatchAnyIndex(materialize('vladizlvav dabe don\'t heart me no more'), ['what', 'is', 'love', 'baby', 'no mo??', 'dont', 'h.rt me']) from system.numbers limit 10;
SELECT multiMatchAny(materialize('/odezhda-dlya-bega/'), ['/odezhda-dlya-bega/', 'kurtki-i-vetrovki-dlya-bega', 'futbolki-i-mayki-dlya-bega']);
SELECT 1 = multiMatchAny('фабрикант', ['f[ae]b[ei]rl', 'ф[иаэе]б[еэи][рпл]', 'афиукд', 'a[ft],th', '^ф[аиеэ]?б?[еэи]?$', 'берлик', 'fab', 'фа[беьв]+е?[рлко]']);
-- All indices tests
SELECT [1, 2] = arraySort(multiMatchAllIndices(materialize('gogleuedeyandexgoogle'), ['.*goo.*', '.*yan.*'])) from system.numbers limit 5;
SELECT [1, 3] = arraySort(multiMatchAllIndices(materialize('gogleuedeyandexgoogle'), ['.*goo.*', 'neverexisted', '.*yan.*'])) from system.numbers limit 5;
SELECT [] = multiMatchAllIndices(materialize('gogleuedeyandexgoogle'), ['neverexisted', 'anotherone', 'andanotherone']) from system.numbers limit 5;
SELECT [1, 2, 3, 11] = arraySort(multiMatchAllIndices('фабрикант', ['', 'рикан', 'а', 'f[ae]b[ei]rl', 'ф[иаэе]б[еэи][рпл]', 'афиукд', 'a[ft],th', '^ф[аиеэ]?б?[еэи]?$', 'берлик', 'fab', 'фа[беьв]+е?[рлко]']));
SELECT [1] = multiMatchAllIndices(materialize('/odezhda-dlya-bega/'), ['/odezhda-dlya-bega/', 'kurtki-i-vetrovki-dlya-bega', 'futbolki-i-mayki-dlya-bega']);
SELECT [] = multiMatchAllIndices(materialize('aaaa'), ['.*aa.*aaa.*', 'aaaaaa{2}', '\(aa\){3}']);
SELECT 'All tests above must return 1, all tests below return something.';
SELECT arraySort(multiMatchAllIndices(arrayJoin(['aaaa', 'aaaaaa', 'bbbb', 'aaaaaaaaaaaaaa']), ['.*aa.*aaa.*', 'aaaaaa{2}', '\(aa\){3}']));
......@@ -2,5 +2,6 @@ SET allow_hyperscan = 1;
SELECT multiMatchAny(arrayJoin(['hello', 'world', 'hellllllllo', 'wororld', 'abc']), ['hel+o', 'w(or)*ld']);
SET allow_hyperscan = 0;
SELECT multiMatchAny(arrayJoin(['hello', 'world', 'hellllllllo', 'wororld', 'abc']), ['hel+o', 'w(or)*ld']); -- { serverError 446 }
SELECT multiMatchAllIndices(arrayJoin(['hello', 'world', 'hellllllllo', 'wororld', 'abc']), ['hel+o', 'w(or)*ld']); -- { serverError 446 }
SELECT multiSearchAny(arrayJoin(['hello', 'world', 'hello, world', 'abc']), ['hello', 'world']);
......@@ -24,3 +24,6 @@ select 1 = multiFuzzyMatchAny('string', 1, ['zorro$', '^tring', 'ip$', 'how.*',
select 2 = multiFuzzyMatchAnyIndex('string', 1, ['zorro$', '^tring', 'ip$', 'how.*', 'it{2}', 'works']);
select 2 = multiFuzzyMatchAnyIndex('halo some wrld', 2, ['^hello.*world$', '^halo.*world$']);
select 1 = multiFuzzyMatchAnyIndex('halo some wrld', 2, ['^halo.*world$', '^hello.*world$']);
select [2, 3, 4] = arraySort(multiFuzzyMatchAllIndices('halo some wrld', 2, ['some random string', '^halo.*world$', '^halo.*world$', '^halo.*world$', '^hallllo.*world$']));
select [] = multiFuzzyMatchAllIndices('halo some wrld', 2, ['^halllllo.*world$', 'some random string']);
......@@ -64,6 +64,10 @@ The same as `match`, but returns 0 if none of the regular expressions are matche
The same as `multiMatchAny`, but returns any index that matches the haystack.
## multiMatchAllIndices(haystack, [pattern<sub>1</sub>, pattern<sub>2</sub>, ..., pattern<sub>n</sub>])
The same as `multiMatchAny`, but returns the array of all indicies that match the haystack in any order.
## multiFuzzyMatchAny(haystack, distance, [pattern<sub>1</sub>, pattern<sub>2</sub>, ..., pattern<sub>n</sub>])
The same as `multiMatchAny`, but returns 1 if any pattern matches the haystack within a constant [edit distance](https://en.wikipedia.org/wiki/Edit_distance). This function is also in an experimental mode and can be extremely slow. For more information see [hyperscan documentation](https://intel.github.io/hyperscan/dev-reference/compilation.html#approximate-matching).
......@@ -72,6 +76,10 @@ The same as `multiMatchAny`, but returns 1 if any pattern matches the haystack w
The same as `multiFuzzyMatchAny`, but returns any index that matches the haystack within a constant edit distance.
## multiFuzzyMatchAllIndices(haystack, distance, [pattern<sub>1</sub>, pattern<sub>2</sub>, ..., pattern<sub>n</sub>])
The same as `multiFuzzyMatchAny`, but returns the array of all indices in any order that match the haystack within a constant edit distance.
!!! note "Note"
`multiFuzzyMatch*` functions do not support UTF-8 regular expressions, and such expressions are treated as bytes because of hyperscan restriction.
......
......@@ -57,6 +57,10 @@
То же, что и `multiMatchAny`, только возвращает любой индекс подходящего регулярного выражения.
## multiMatchAllIndices(haystack, [pattern<sub>1</sub>, pattern<sub>2</sub>, ..., pattern<sub>n</sub>])
То же, что и `multiMatchAny`, только возвращает массив всех индексов всех подходящих регулярных выражений в любом порядке.
## multiFuzzyMatchAny(haystack, distance, [pattern<sub>1</sub>, pattern<sub>2</sub>, ..., pattern<sub>n</sub>])
То же, что и `multiMatchAny`, но возвращает 1 если любой pattern соответствует haystack в пределах константного [редакционного расстояния](https://en.wikipedia.org/wiki/Edit_distance). Эта функция также находится в экспериментальном режиме и может быть очень медленной. За подробностями обращайтесь к [документации hyperscan](https://intel.github.io/hyperscan/dev-reference/compilation.html#approximate-matching).
......@@ -65,6 +69,10 @@
То же, что и `multiFuzzyMatchAny`, только возвращает любой индекс подходящего регулярного выражения в пределах константного редакционного расстояния.
## multiFuzzyMatchAllIndices(haystack, distance, [pattern<sub>1</sub>, pattern<sub>2</sub>, ..., pattern<sub>n</sub>])
То же, что и `multiFuzzyMatchAny`, только возвращает массив всех индексов всех подходящих регулярных выражений в любом порядке в пределах константного редакционного расстояния.
!!! note "Примечание"
`multiFuzzyMatch*` функции не поддерживают UTF-8 закодированные регулярные выражения, и такие выражения рассматриваются как байтовые из-за ограничения hyperscan.
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册