未验证 提交 7aefd666 编写于 作者: A alexey-milovidov 提交者: GitHub

Merge pull request #4841 from danlark1/master

Added hyperscan fuzzy search
此差异已折叠。
#pragma once
#include <Columns/ColumnArray.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnsNumber.h>
#include <Core/Field.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunction.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/Context.h>
#include <common/StringRef.h>
#include <optional>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int FUNCTION_NOT_ALLOWED;
}
template <typename Impl, typename Name, size_t LimitArgs>
class FunctionsMultiStringFuzzySearch : public IFunction
{
static_assert(LimitArgs > 0);
public:
static constexpr auto name = Name::name;
static FunctionPtr create(const Context & context)
{
if (Impl::is_using_hyperscan && !context.getSettingsRef().allow_hyperscan)
throw Exception(
"Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0", ErrorCodes::FUNCTION_NOT_ALLOWED);
return std::make_shared<FunctionsMultiStringFuzzySearch>();
}
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 3; }
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!isString(arguments[0]))
throw Exception(
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (!isUnsignedInteger(arguments[1]))
throw Exception(
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(arguments[2].get());
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>>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
using ResultType = typename Impl::ResultType;
const ColumnPtr & column_haystack = block.getByPosition(arguments[0]).column;
const ColumnString * col_haystack_vector = checkAndGetColumn<ColumnString>(&*column_haystack);
const ColumnPtr & num_ptr = block.getByPosition(arguments[1]).column;
const ColumnConst * col_const_num = nullptr;
UInt32 edit_distance = 0;
if ((col_const_num = checkAndGetColumnConst<ColumnUInt8>(num_ptr.get())))
edit_distance = col_const_num->getValue<UInt8>();
else if ((col_const_num = checkAndGetColumnConst<ColumnUInt16>(num_ptr.get())))
edit_distance = col_const_num->getValue<UInt16>();
else if ((col_const_num = checkAndGetColumnConst<ColumnUInt32>(num_ptr.get())))
edit_distance = col_const_num->getValue<UInt32>();
else
throw Exception(
"Illegal column " + block.getByPosition(arguments[1]).column->getName()
+ ". The number is not const or does not fit in UInt32",
ErrorCodes::ILLEGAL_COLUMN);
const ColumnPtr & arr_ptr = block.getByPosition(arguments[2]).column;
const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(arr_ptr.get());
if (!col_const_arr)
throw Exception(
"Illegal column " + block.getByPosition(arguments[2]).column->getName() + ". The array is not const",
ErrorCodes::ILLEGAL_COLUMN);
Array src_arr = col_const_arr->getValue<Array>();
if (src_arr.size() > LimitArgs)
throw Exception(
"Number of arguments for function " + getName() + " doesn't match: passed " + std::to_string(src_arr.size())
+ ", should be at most " + std::to_string(LimitArgs),
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
std::vector<StringRef> refs;
refs.reserve(src_arr.size());
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 & vec_res = col_res->getData();
vec_res.resize(column_haystack_size);
if (col_haystack_vector)
Impl::vector_constant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_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);
}
};
}
......@@ -46,7 +46,7 @@ namespace DB
* multiSearchAllPositionsUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
* multiSearchAllPositionsCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n])
* multiSearchAllPositionsCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
*
* multiSearchFirstPosition(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- returns the first position of the haystack matched by strings or zero if nothing was found
* multiSearchFirstPositionUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
* multiSearchFirstPositionCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n])
......@@ -291,7 +291,8 @@ public:
static FunctionPtr create(const Context & context)
{
if (Impl::is_using_hyperscan && !context.getSettingsRef().allow_hyperscan)
throw Exception("Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0", ErrorCodes::FUNCTION_NOT_ALLOWED);
throw Exception(
"Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0", ErrorCodes::FUNCTION_NOT_ALLOWED);
return std::make_shared<FunctionsMultiStringSearch>();
}
......@@ -355,7 +356,6 @@ public:
vec_res.resize(column_haystack_size);
/// TODO support constant_constant version
if (col_haystack_vector)
Impl::vector_constant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res);
else
......
......@@ -318,32 +318,32 @@ struct NgramDistanceImpl
};
struct NgramDistanceName
struct NameNgramDistance
{
static constexpr auto name = "ngramDistance";
};
struct NgramDistanceCaseInsensitiveName
struct NameNgramDistanceCaseInsensitive
{
static constexpr auto name = "ngramDistanceCaseInsensitive";
};
struct NgramDistanceUTF8Name
struct NameNgramDistanceUTF8
{
static constexpr auto name = "ngramDistanceUTF8";
};
struct NgramDistanceUTF8CaseInsensitiveName
struct NameNgramDistanceUTF8CaseInsensitive
{
static constexpr auto name = "ngramDistanceCaseInsensitiveUTF8";
};
using FunctionNgramDistance = FunctionsStringSimilarity<NgramDistanceImpl<4, UInt8, false, false>, NgramDistanceName>;
using FunctionNgramDistance = FunctionsStringSimilarity<NgramDistanceImpl<4, UInt8, false, false>, NameNgramDistance>;
using FunctionNgramDistanceCaseInsensitive
= FunctionsStringSimilarity<NgramDistanceImpl<4, UInt8, false, true>, NgramDistanceCaseInsensitiveName>;
using FunctionNgramDistanceUTF8 = FunctionsStringSimilarity<NgramDistanceImpl<3, UInt32, true, false>, NgramDistanceUTF8Name>;
= FunctionsStringSimilarity<NgramDistanceImpl<4, UInt8, false, true>, NameNgramDistanceCaseInsensitive>;
using FunctionNgramDistanceUTF8 = FunctionsStringSimilarity<NgramDistanceImpl<3, UInt32, true, false>, NameNgramDistanceUTF8>;
using FunctionNgramDistanceCaseInsensitiveUTF8
= FunctionsStringSimilarity<NgramDistanceImpl<3, UInt32, true, true>, NgramDistanceUTF8CaseInsensitiveName>;
= FunctionsStringSimilarity<NgramDistanceImpl<3, UInt32, true, true>, NameNgramDistanceUTF8CaseInsensitive>;
void registerFunctionsStringSimilarity(FunctionFactory & factory)
{
......
......@@ -9,12 +9,14 @@
namespace DB
{
/** Calculate similarity metrics:
*
* ngramDistance(haystack, needle) --- calculate n-gram distance between haystack and needle.
* ngramDistance(haystack, needle) - calculate n-gram distance between haystack and needle.
* Returns float number from 0 to 1 - the closer to zero, the more strings are similar to each other.
* Also support CaseInsensitive and UTF8 formats.
* ngramDistanceCaseInsensitive(haystack, needle)
* ngramDistanceUTF8(haystack, needle)
* ngramDistanceCaseInsensitiveUTF8(haystack, needle)
*/
namespace ErrorCodes
......@@ -70,11 +72,13 @@ public:
if (needle.size() > Impl::max_string_size)
{
throw Exception(
"String size of needle is too big for function " + getName() + ". Should be at most " + std::to_string(Impl::max_string_size),
ErrorCodes::TOO_LARGE_STRING_SIZE);
"String size of needle is too big for function " + getName() + ". Should be at most "
+ std::to_string(Impl::max_string_size),
ErrorCodes::TOO_LARGE_STRING_SIZE);
}
Impl::constant_constant(col_haystack_const->getValue<String>(), needle, res);
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(col_haystack_const->size(), toField(res));
block.getByPosition(result).column
= block.getByPosition(result).type->createColumnConst(col_haystack_const->size(), toField(res));
return;
}
......@@ -91,11 +95,11 @@ public:
if (needle.size() > Impl::max_string_size)
{
throw Exception(
"String size of needle is too big for function " + getName() + ". Should be at most " + std::to_string(Impl::max_string_size),
ErrorCodes::TOO_LARGE_STRING_SIZE);
"String size of needle is too big for function " + getName() + ". Should be at most "
+ std::to_string(Impl::max_string_size),
ErrorCodes::TOO_LARGE_STRING_SIZE);
}
Impl::vector_constant(
col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), needle, vec_res);
Impl::vector_constant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), needle, vec_res);
}
else
{
......
#pragma once
#include <memory>
#include <optional>
#include <string>
#include <utility>
#include <vector>
#include <Functions/likePatternToRegexp.h>
#include <Common/ObjectPool.h>
#include <Common/OptimizedRegularExpression.h>
#include <Common/ProfileEvents.h>
#include <common/StringRef.h>
#include <memory>
#include <string>
#include <vector>
#include <Common/config.h>
#if USE_HYPERSCAN
# if __has_include(<hs/hs.h>)
# include <hs/hs.h>
# else
# include <hs.h>
# endif
# if __has_include(<hs/hs.h>)
# include <hs/hs.h>
# else
# include <hs.h>
# endif
#endif
namespace ProfileEvents
......@@ -81,12 +83,15 @@ namespace MultiRegexps
}
};
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, DB::MultiRegexps::HyperscanDeleter<decltype(&hs_free_scratch), &hs_free_scratch>>;
using Regexps = std::unique_ptr<hs_database_t, HyperscanDeleter<decltype(&hs_free_database), &hs_free_database>>;
using Pool = ObjectPoolMap<Regexps, std::vector<String>>;
using Pool = ObjectPoolMap<Regexps, std::pair<std::vector<String>, std::optional<UInt32>>>;
template <bool FindAnyIndex>
inline Pool::Pointer get(const std::vector<StringRef> & patterns)
/// If CompileForEditDistance is False, edit_distance must be nullopt
template <bool FindAnyIndex, bool CompileForEditDistance>
inline Pool::Pointer get(const std::vector<StringRef> & patterns, std::optional<UInt32> edit_distance)
{
/// C++11 has thread-safe function-local statics on most modern compilers.
static Pool known_regexps; /// Different variables for different pattern parameters.
......@@ -96,16 +101,37 @@ namespace MultiRegexps
for (const StringRef & ref : patterns)
str_patterns.push_back(ref.toString());
return known_regexps.get(str_patterns, [&str_patterns]
return known_regexps.get({str_patterns, edit_distance}, [&str_patterns, edit_distance]
{
(void)edit_distance;
/// Common pointers
std::vector<const char *> ptrns;
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());
flags.reserve(str_patterns.size());
if constexpr (CompileForEditDistance)
{
ext_exprs.reserve(str_patterns.size());
ext_exprs_ptrs.reserve(str_patterns.size());
}
for (const StringRef ref : str_patterns)
{
ptrns.push_back(ref.data);
flags.push_back(HS_FLAG_DOTALL | HS_FLAG_ALLOWEMPTY | HS_FLAG_SINGLEMATCH);
if constexpr (CompileForEditDistance)
{
ext_exprs.emplace_back();
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());
}
}
hs_database_t * db = nullptr;
hs_compile_error_t * compile_error;
......@@ -120,13 +146,32 @@ namespace MultiRegexps
ids[i] = i + 1;
}
hs_error_t err
= hs_compile_multi(ptrns.data(), flags.data(), ids.get(), ptrns.size(), HS_MODE_BLOCK, nullptr, &db, &compile_error);
hs_error_t err;
if constexpr (!CompileForEditDistance)
err = hs_compile_multi(
ptrns.data(),
flags.data(),
ids.get(),
ptrns.size(),
HS_MODE_BLOCK,
nullptr,
&db,
&compile_error);
else
err = hs_compile_ext_multi(
ptrns.data(),
flags.data(),
ids.get(),
ext_exprs_ptrs.data(),
ptrns.size(),
HS_MODE_BLOCK,
nullptr,
&db,
&compile_error);
if (err != HS_SUCCESS)
{
std::unique_ptr<
hs_compile_error_t,
HyperscanDeleter<decltype(&hs_free_compile_error), &hs_free_compile_error>> error(compile_error);
CompilerError error(compile_error);
if (error->expression < 0)
throw Exception(String(error->message), ErrorCodes::LOGICAL_ERROR);
......
#include <Common/config.h>
#include <Functions/registerFunctions.h>
#include <Functions/FunctionFactory.h>
#include <Functions/registerFunctions.h>
#include <Common/config.h>
namespace DB
{
/** These functions are defined in a separate translation units.
* This is done in order to reduce the consumption of RAM during build, and to speed up the parallel build.
*/
......@@ -34,6 +32,7 @@ void registerFunctionsRound(FunctionFactory &);
void registerFunctionsString(FunctionFactory &);
void registerFunctionsStringArray(FunctionFactory &);
void registerFunctionsStringSearch(FunctionFactory &);
void registerFunctionsStringRegex(FunctionFactory &);
void registerFunctionsStringSimilarity(FunctionFactory &);
void registerFunctionsURL(FunctionFactory &);
void registerFunctionsVisitParam(FunctionFactory &);
......@@ -75,6 +74,7 @@ void registerFunctions()
registerFunctionsString(factory);
registerFunctionsStringArray(factory);
registerFunctionsStringSearch(factory);
registerFunctionsStringRegex(factory);
registerFunctionsStringSimilarity(factory);
registerFunctionsURL(factory);
registerFunctionsVisitParam(factory);
......
......@@ -53,6 +53,13 @@
<query><![CDATA[SELECT count() FROM hits_100m_single WHERE multiMatchAny(URL, ['chelyabinsk\\.74\\.ru', 'doctor\\.74\\.ru', 'transport\\.74\\.ru', 'm\\.74\\.ru', '//74\\.ru/', 'chel\\.74\\.ru', 'afisha\\.74\\.ru', 'diplom\\.74\\.ru', 'chelfin\\.ru', '//chel\\.ru', 'chelyabinsk\\.ru', 'cheldoctor\\.ru', '//mychel\\.ru', 'cheldiplom\\.ru', '74\\.ru/video', 'market', 'poll', 'mail', 'conference', 'consult', 'contest', 'tags', 'feedback', 'pages', 'text'])]]></query>
<query><![CDATA[SELECT count() FROM hits_100m_single WHERE multiSearchAny(URL, ['chelyabinsk.74.ru', 'doctor.74.ru', 'transport.74.ru', 'm.74.ru', '//74.ru/', 'chel.74.ru', 'afisha.74.ru', 'diplom.74.ru', 'chelfin.ru', '//chel.ru', 'chelyabinsk.ru', 'cheldoctor.ru', '//mychel.ru', 'cheldiplom.ru', '74.ru/video', 'market', 'poll', 'mail', 'conference', 'consult', 'contest', 'tags', 'feedback', 'pages', 'text'])]]></query>
<query><![CDATA[SELECT DISTINCT Title, multiFuzzyMatchAny(Title, 2, ['^metrika\\.ry$']) AS distance FROM hits_100m_single WHERE distance = 1]]></query>
<query><![CDATA[SELECT DISTINCT Title, multiFuzzyMatchAny(Title, 5, ['^metrika\\.ry$']) AS distance FROM hits_100m_single WHERE distance = 1]]></query>
<query><![CDATA[SELECT sum(multiFuzzyMatchAny(Title, 3, ['hello$', 'world$', '^hello'])) FROM hits_100m_single]]></query>
<query><![CDATA[SELECT count() FROM hits_100m_single WHERE multiFuzzyMatchAny(URL, 2, ['about/address', 'for_woman', '^https?://lm-company.ruy/$', 'ultimateguitar.com'])]]></query>
<main_metric>
<min_time/>
......
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
SET send_logs_level = 'none';
select 0 = multiFuzzyMatchAny('abc', 0, ['a1c']) from system.numbers limit 5;
select 1 = multiFuzzyMatchAny('abc', 1, ['a1c']) from system.numbers limit 5;
select 1 = multiFuzzyMatchAny('abc', 2, ['a1c']) from system.numbers limit 5;
select 1 = multiFuzzyMatchAny('abc', 3, ['a1c']) from system.numbers limit 5; -- { serverError 49 }
select 1 = multiFuzzyMatchAny('abc', 4, ['a1c']) from system.numbers limit 5; -- { serverError 49 }
select 1 = multiFuzzyMatchAny('leftabcright', 1, ['a1c']) from system.numbers limit 5;
select 1 = multiFuzzyMatchAny('hello some world', 0, ['^hello.*world$']);
select 1 = multiFuzzyMatchAny('hallo some world', 1, ['^hello.*world$']);
select 0 = multiFuzzyMatchAny('halo some wrld', 2, ['^hello.*world$']);
select 1 = multiFuzzyMatchAny('halo some wrld', 2, ['^hello.*world$', '^halo.*world$']);
select 1 = multiFuzzyMatchAny('halo some wrld', 2, ['^halo.*world$', '^hello.*world$']);
select 1 = multiFuzzyMatchAny('halo some wrld', 3, ['^hello.*world$']);
select 1 = multiFuzzyMatchAny('hello some world', 10, ['^hello.*world$']); -- { serverError 49 }
select 1 = multiFuzzyMatchAny('hello some world', -1, ['^hello.*world$']); -- { serverError 43 }
select 1 = multiFuzzyMatchAny('hello some world', 10000000000, ['^hello.*world$']); -- { serverError 44 }
select 1 = multiFuzzyMatchAny('http://hyperscan_is_nice.ru/st', 2, ['http://hyperscan_is_nice.ru/(st\\d\\d$|st\\d\\d\\.|st1[0-4]\\d|st150|st\\d$|gl|rz|ch)']);
select 0 = multiFuzzyMatchAny('string', 0, ['zorro$', '^tring', 'in$', 'how.*', 'it{2}', 'works']);
select 1 = multiFuzzyMatchAny('string', 1, ['zorro$', '^tring', 'ip$', 'how.*', 'it{2}', 'works']);
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$']);
......@@ -39,7 +39,7 @@ Returns 1, if at least one string `needle_i` matches the string `haystack` and 0
For a case-insensitive search or/and in UTF-8 format use functions `multiSearchAnyCaseInsensitive, multiSearchAnyUTF8, multiSearchAnyCaseInsensitiveUTF8`.
Note: in all `multiSearch*` functions the number of needles should be less than 2<sup>8</sup> because of implementation specification.
**Note: in all `multiSearch*` functions the number of needles should be less than 2<sup>8</sup> because of implementation specification.**
## match(haystack, pattern)
......@@ -56,12 +56,22 @@ For patterns to search for substrings in a string, it is better to use LIKE or '
The same as `match`, but returns 0 if none of the regular expressions are matched and 1 if any of the patterns matches. It uses [hyperscan](https://github.com/intel/hyperscan) library. For patterns to search substrings in a string, it is better to use `multiSearchAny` since it works much faster.
Note: this function is in experimental mode because of some [issues](https://github.com/intel/hyperscan/issues/141). The length of any of the `haystack` string must be less than 2<sup>32</sup> bytes otherwise the exception is thrown. This restriction takes place because of hyperscan API.
**Note: this function is in experimental mode because of some [issues](https://github.com/intel/hyperscan/issues/141). The length of any of the `haystack` string must be less than 2<sup>32</sup> bytes otherwise the exception is thrown. This restriction takes place because of hyperscan API.**
## multiMatchAnyIndex(haystack, [pattern_1, pattern_2, ..., pattern_n])
The same as `multiMatchAny`, but returns any index that matches the haystack.
## multiFuzzyMatchAny(haystack, distance, [pattern_1, pattern_2, ..., pattern_n])
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).
## multiFuzzyMatchAnyIndex(haystack, distance, [pattern_1, pattern_2, ..., pattern_n])
The same as `multiFuzzyMatchAny`, but returns any index that matches the haystack within a constant edit distance.
**Note: to turn off all functions that use hyperscan, use setting `SET allow_hyperscan = 0;`.**
## extract(haystack, pattern)
Extracts a fragment of a string using a regular expression. If 'haystack' doesn't match the 'pattern' regex, an empty string is returned. If the regex doesn't contain subpatterns, it takes the fragment that matches the entire regex. Otherwise, it takes the fragment that matches the first subpattern.
......@@ -94,7 +104,7 @@ Calculates the 4-gram distance between `haystack` and `needle`: counts the symme
For case-insensitive search or/and in UTF-8 format use functions `ngramDistanceCaseInsensitive, ngramDistanceUTF8, ngramDistanceCaseInsensitiveUTF8`.
Notes: For UTF-8 case we use 3-gram distance. All these are not perfectly fair n-gram distances. We use 2-byte hashes to hash n-grams and then calculate the symmetric difference between these hash tables -- collisions may occur. With UTF-8 case-insensitive format we do not use fair `tolower` function -- we zero the 5-th bit (starting from zero) of each codepoint byte -- this works for Latin and mostly for all Cyrillic letters.
**Note: For UTF-8 case we use 3-gram distance. All these are not perfectly fair n-gram distances. We use 2-byte hashes to hash n-grams and then calculate the symmetric difference between these hash tables -- collisions may occur. With UTF-8 case-insensitive format we do not use fair `tolower` function -- we zero the 5-th bit (starting from zero) of each codepoint byte -- this works for Latin and mostly for all Cyrillic letters.**
[Original article](https://clickhouse.yandex/docs/en/query_language/functions/string_search_functions/) <!--hide-->
......@@ -34,7 +34,7 @@
Для поиска без учета регистра и/или в кодировке UTF-8 используйте функции `multiSearchAnyCaseInsensitive, multiSearchAnyUTF8, multiSearchAnyCaseInsensitiveUTF8`.
Примечание: во всех функциях `multiSearch*` количество needles должно быть меньше 2<sup>8</sup> из-за внутренностей реализации.
**Примечание: во всех функциях `multiSearch*` количество needles должно быть меньше 2<sup>8</sup> из-за внутренностей реализации.**
## match(haystack, pattern)
Проверка строки на соответствие регулярному выражению pattern. Регулярное выражение **re2**. Синтаксис регулярных выражений **re2** является более ограниченным по сравнению с регулярными выражениями **Perl** ([подробнее](https://github.com/google/re2/wiki/Syntax)).
......@@ -49,11 +49,21 @@
То же, что и `match`, но возвращает ноль, если ни одно регулярное выражение не подошло и один, если хотя бы одно. Используется библиотека [hyperscan](https://github.com/intel/hyperscan) для соответствия регулярных выражений. Для шаблонов на поиск многих подстрок в строке, лучше используйте `multiSearchAny`, так как она работает существенно быстрее.
Примечание: эта функция находится пока в экспериментальном режиме из-за некоторых [проблем](https://github.com/intel/hyperscan/issues/141). Длина любой строки из `haystack` должна быть меньше 2<sup>32</sup> байт, иначе бросается исключение. Это ограничение связано с ограничением hyperscan API.
**Примечание: эта функция находится пока в экспериментальном режиме из-за некоторых [проблем](https://github.com/intel/hyperscan/issues/141). Длина любой строки из `haystack` должна быть меньше 2<sup>32</sup> байт, иначе бросается исключение. Это ограничение связано с ограничением hyperscan API.**
## multiMatchAnyIndex(haystack, [pattern_1, pattern_2, ..., pattern_n])
То же, что `multiMatchAny`, только возвращает любой индекс подходящего регулярного выражения.
То же, что и `multiMatchAny`, только возвращает любой индекс подходящего регулярного выражения.
## multiFuzzyMatchAny(haystack, distance, [pattern_1, pattern_2, ..., pattern_n])
То же, что и `multiMatchAny`, но возвращает 1 если любой pattern соответствует haystack в пределах константного [редакционного расстояния](https://en.wikipedia.org/wiki/Edit_distance). Эта функция также находится в экспериментальном режиме и может быть очень медленной. За подробностями обращайтесь к [документации hyperscan](https://intel.github.io/hyperscan/dev-reference/compilation.html#approximate-matching).
## multiFuzzyMatchAnyIndex(haystack, distance, [pattern_1, pattern_2, ..., pattern_n])
То же, что и `multiFuzzyMatchAny`, только возвращает любой индекс подходящего регулярного выражения в пределах константного редакционного расстояния.
**Примечание: чтобы выключить все функции, использующие hyperscan, используйте настройку `SET allow_hyperscan = 0;`.**
## extract(haystack, pattern)
Извлечение фрагмента строки по регулярному выражению. Если haystack не соответствует регулярному выражению pattern, то возвращается пустая строка. Если регулярное выражение не содержит subpattern-ов, то вынимается фрагмент, который подпадает под всё регулярное выражение. Иначе вынимается фрагмент, который подпадает под первый subpattern.
......@@ -83,6 +93,6 @@
Для поиска без учета регистра и/или в формате UTF-8 используйте функции `ngramDistanceCaseInsensitive, ngramDistanceUTF8, ngramDistanceCaseInsensitiveUTF8`.
Примечание: для случая UTF-8 мы используем триграммное расстояние. Вычисление n-граммного расстояния не совсем честное. Мы используем 2-х байтные хэши для хэширования n-грамм, а затем вычисляем симметричную разность между хэш таблицами -- могут возникнуть коллизии. В формате UTF-8 без учета регистра мы не используем честную функцию `tolower` -- мы обнуляем 5-й бит (нумерация с нуля) каждого байта кодовой точки -- это работает для латиницы и почти для всех кириллических букв.
**Примечание: для случая UTF-8 мы используем триграммное расстояние. Вычисление n-граммного расстояния не совсем честное. Мы используем 2-х байтные хэши для хэширования n-грамм, а затем вычисляем симметричную разность между хэш таблицами -- могут возникнуть коллизии. В формате UTF-8 без учета регистра мы не используем честную функцию `tolower` -- мы обнуляем 5-й бит (нумерация с нуля) каждого байта кодовой точки -- это работает для латиницы и почти для всех кириллических букв.**
[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/string_search_functions/) <!--hide-->
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册