Rename trigramDistance to ngramDistance, add more functions with CaseInsensitive and UTF, update docs, more job done in perf, added some perf tests for string search that I would like to see

This commit is contained in:
Danila Kutenin 2019-03-06 01:42:28 +03:00
parent 5ca8933dda
commit 952b5ea24a
9 changed files with 878 additions and 250 deletions

View File

@ -8,164 +8,271 @@
#include <Core/Defines.h>
#include <common/unaligned.h>
#include <algorithm>
#include <climits>
#include <cstring>
#include <limits>
#include <memory>
#include <utility>
#ifdef __SSE4_2__
#include <nmmintrin.h>
# include <nmmintrin.h>
#endif
namespace DB
{
/** Distance function implementation.
* We calculate all the trigrams from left string and count by the index of
* We calculate all the n-grams from left string and count by the index of
* 16 bits hash of them in the map.
* Then calculate all the trigrams from the right string and calculate
* the trigram distance on the flight by adding and subtracting from the hashmap.
* Then calculate all the n-grams from the right string and calculate
* the n-gram distance on the flight by adding and subtracting from the hashmap.
* Then return the map into the condition of which it was after the left string
* calculation. If the right string size is big (more than 2**15 bytes),
* the strings are not similar at all and we return 1.
*/
struct TrigramDistanceImpl
template <size_t N, class CodePoint, bool UTF8, bool CaseInsensitive>
struct NgramDistanceImpl
{
using ResultType = Float32;
using CodePoint = UInt32;
/// map_size for trigram difference
/// map_size for ngram difference.
static constexpr size_t map_size = 1u << 16;
/// If the haystack size is bigger than this, behaviour is unspecified for this function
/// If the haystack size is bigger than this, behaviour is unspecified for this function.
static constexpr size_t max_string_size = 1u << 15;
/// Default padding to read safely.
static constexpr size_t default_padding = 16;
/// Max codepoints to store at once. 16 is for batching usage and PODArray has this padding.
static constexpr size_t simultaneously_codepoints_num = default_padding + N - 1;
/** This fits mostly in L2 cache all the time.
* Actually use UInt16 as addings and subtractions do not UB overflow. But think of it as a signed
* integer array.
*/
using TrigramStats = UInt16[map_size];
using NgramStats = UInt16[map_size];
static ALWAYS_INLINE UInt16 trigramHash(CodePoint one, CodePoint two, CodePoint three)
static ALWAYS_INLINE UInt16 ASCIIHash(const CodePoint * code_points)
{
UInt64 combined = (static_cast<UInt64>(one) << 32) | two;
return intHashCRC32(unalignedLoad<UInt32>(code_points)) & 0xFFFFu;
}
static ALWAYS_INLINE UInt16 UTF8Hash(const CodePoint * code_points)
{
UInt64 combined = (static_cast<UInt64>(code_points[0]) << 32) | code_points[1];
#ifdef __SSE4_2__
return _mm_crc32_u64(three, combined) & 0xFFFFu;
return _mm_crc32_u64(code_points[2], combined) & 0xFFFFu;
#else
return (intHashCRC32(combined) ^ intHashCRC32(three)) & 0xFFFFu;
return (intHashCRC32(combined) ^ intHashCRC32(code_points[2])) & 0xFFFFu;
#endif
}
static ALWAYS_INLINE CodePoint readCodePoint(const char *& pos, const char * end) noexcept
template <size_t Offset, class Container, size_t... I>
static ALWAYS_INLINE inline void unrollLowering(Container & cont, const std::index_sequence<I...> &)
{
size_t length = UTF8::seqLength(*pos);
if (pos + length > end)
length = end - pos;
CodePoint res;
/// This is faster than just memcpy because of compiler optimizations with moving bytes.
switch (length)
{
case 1:
res = 0;
memcpy(&res, pos, 1);
break;
case 2:
res = 0;
memcpy(&res, pos, 2);
break;
case 3:
res = 0;
memcpy(&res, pos, 3);
break;
default:
memcpy(&res, pos, 4);
}
pos += length;
return res;
((cont[Offset + I] = std::tolower(cont[Offset + I])), ...);
}
static inline size_t calculateNeedleStats(const char * data, const size_t size, TrigramStats & trigram_stats) noexcept
static ALWAYS_INLINE size_t readASCIICodePoints(CodePoint * code_points, const char *& pos, const char * end)
{
size_t len = 0;
const char * start = data;
const char * end = data + size;
CodePoint cp1 = 0;
CodePoint cp2 = 0;
CodePoint cp3 = 0;
/// Offset before which we copy some data.
constexpr size_t padding_offset = default_padding - N + 1;
/// We have an array like this for ASCII (N == 4, other cases are similar)
/// |a0|a1|a2|a3|a4|a5|a6|a7|a8|a9|a10|a11|a12|a13|a14|a15|a16|a17|a18|
/// And we copy ^^^^^^^^^^^^^^^ these bytes to the start
/// Actually it is enough to copy 3 bytes, but memcpy for 4 bytes translates into 1 instruction
memcpy(code_points, code_points + padding_offset, roundUpToPowerOfTwoOrZero(N - 1) * sizeof(CodePoint));
/// Now we have an array
/// |a13|a14|a15|a16|a4|a5|a6|a7|a8|a9|a10|a11|a12|a13|a14|a15|a16|a17|a18|
/// ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
/// Doing unaligned read of 16 bytes and copy them like above
/// 16 is also chosen to do two `movups`.
/// Such copying allow us to have 3 codepoints from the previous read to produce the n-gram with them.
memcpy(code_points + (N - 1), pos, default_padding * sizeof(CodePoint));
while (start != end)
if constexpr (CaseInsensitive)
{
cp1 = cp2;
cp2 = cp3;
cp3 = readCodePoint(start, end);
++len;
if (len < 3)
continue;
++trigram_stats[trigramHash(cp1, cp2, cp3)];
/// We really need template lambdas with C++20 to do it inline
unrollLowering<N - 1>(code_points, std::make_index_sequence<padding_offset>());
}
return std::max(static_cast<Int64>(0), static_cast<Int64>(len) - 2);
pos += padding_offset;
if (pos > end)
return default_padding - (pos - end);
return default_padding;
}
static inline UInt64 calculateHaystackStatsAndMetric(const char * data, const size_t size, TrigramStats & trigram_stats, size_t & distance)
static ALWAYS_INLINE size_t readUTF8CodePoints(CodePoint * code_points, const char *& pos, const char * end)
{
size_t len = 0;
size_t trigram_cnt = 0;
/// The same copying as described in the function above.
memcpy(code_points, code_points + default_padding - N + 1, roundUpToPowerOfTwoOrZero(N - 1) * sizeof(CodePoint));
size_t num = N - 1;
while (num < default_padding && pos < end)
{
size_t length = UTF8::seqLength(*pos);
if (pos + length > end)
length = end - pos;
CodePoint res;
/// This is faster than just memcpy because of compiler optimizations with moving bytes.
switch (length)
{
case 1:
res = 0;
memcpy(&res, pos, 1);
break;
case 2:
res = 0;
memcpy(&res, pos, 2);
break;
case 3:
res = 0;
memcpy(&res, pos, 3);
break;
default:
memcpy(&res, pos, 4);
}
/// This is not a really true case insensitive utf8. We zero the 5-th bit of every byte.
/// For ASCII it works https://catonmat.net/ascii-case-conversion-trick. For most cyrrilic letters also does.
/// For others, we don't care now. Lowering UTF is not a cheap operation.
if constexpr (CaseInsensitive)
{
switch (length)
{
case 4:
res &= ~(1u << (5 + 3 * CHAR_BIT));
[[fallthrough]];
case 3:
res &= ~(1u << (5 + 2 * CHAR_BIT));
[[fallthrough]];
case 2:
res &= ~(1u << (5 + CHAR_BIT));
[[fallthrough]];
default:
res &= ~(1u << 5);
}
}
pos += length;
code_points[num++] = res;
}
return num;
}
static ALWAYS_INLINE inline size_t calculateNeedleStats(
const char * data,
const size_t size,
NgramStats & ngram_stats,
size_t (*read_code_points)(CodePoint *, const char *&, const char *),
UInt16 (*hash_functor)(const CodePoint *))
{
// To prevent size_t overflow below.
if (size < N)
return 0;
const char * start = data;
const char * end = data + size;
CodePoint cp1 = 0;
CodePoint cp2 = 0;
CodePoint cp3 = 0;
CodePoint cp[simultaneously_codepoints_num] = {};
/// read_code_points returns the position of cp where it stopped reading codepoints.
size_t found = read_code_points(cp, start, end);
/// We need to start for the first time here, because first N - 1 codepoints mean nothing.
size_t i = N - 1;
/// Initialize with this value because for the first time `found` does not initialize first N - 1 codepoints.
size_t len = -N + 1;
do
{
len += found - N + 1;
for (; i + N <= found; ++i)
++ngram_stats[hash_functor(cp + i)];
i = 0;
} while (start < end && (found = read_code_points(cp, start, end)));
return len;
}
static ALWAYS_INLINE inline UInt64 calculateHaystackStatsAndMetric(
const char * data,
const size_t size,
NgramStats & ngram_stats,
size_t & distance,
size_t (*read_code_points)(CodePoint *, const char *&, const char *),
UInt16 (*hash_functor)(const CodePoint *))
{
size_t ngram_cnt = 0;
const char * start = data;
const char * end = data + size;
CodePoint cp[simultaneously_codepoints_num] = {};
/// allocation tricks, most strings are relatively small
static constexpr size_t small_buffer_size = 256;
std::unique_ptr<UInt16[]> big_buffer;
UInt16 small_buffer[small_buffer_size];
UInt16 * trigram_storage = small_buffer;
UInt16 * ngram_storage = small_buffer;
if (size > small_buffer_size)
{
trigram_storage = new UInt16[size];
big_buffer.reset(trigram_storage);
ngram_storage = new UInt16[size];
big_buffer.reset(ngram_storage);
}
while (start != end)
/// read_code_points returns the position of cp where it stopped reading codepoints.
size_t found = read_code_points(cp, start, end);
/// We need to start for the first time here, because first N - 1 codepoints mean nothing.
size_t iter = N - 1;
do
{
cp1 = cp2;
cp2 = cp3;
cp3 = readCodePoint(start, end);
++len;
if (len < 3)
continue;
for (; iter + N <= found; ++iter)
{
UInt16 hash = hash_functor(cp + iter);
if (static_cast<Int16>(ngram_stats[hash]) > 0)
--distance;
else
++distance;
UInt16 hash = trigramHash(cp1, cp2, cp3);
if (static_cast<Int16>(trigram_stats[hash]) > 0)
--distance;
else
++distance;
trigram_storage[trigram_cnt++] = hash;
--trigram_stats[hash];
}
ngram_storage[ngram_cnt++] = hash;
--ngram_stats[hash];
}
iter = 0;
} while (start < end && (found = read_code_points(cp, start, end)));
/// Return the state of hash map to its initial.
for (size_t i = 0; i < trigram_cnt; ++i)
++trigram_stats[trigram_storage[i]];
return trigram_cnt;
for (size_t i = 0; i < ngram_cnt; ++i)
++ngram_stats[ngram_storage[i]];
return ngram_cnt;
}
static void constant_constant(const std::string & data, const std::string & needle, Float32 & res)
template <class Callback, class... Args>
static inline size_t dispatchSearcher(Callback callback, Args &&... args)
{
TrigramStats common_stats;
if constexpr (!UTF8)
return callback(std::forward<Args>(args)..., readASCIICodePoints, ASCIIHash);
else
return callback(std::forward<Args>(args)..., readUTF8CodePoints, UTF8Hash);
}
static void constant_constant(std::string data, std::string needle, Float32 & res)
{
NgramStats common_stats;
memset(common_stats, 0, sizeof(common_stats));
size_t second_size = calculateNeedleStats(needle.data(), needle.size(), common_stats);
/// We use unsafe versions of getting ngrams, so I decided to use padded strings.
const size_t needle_size = needle.size();
const size_t data_size = data.size();
needle.resize(needle_size + default_padding);
data.resize(data_size + default_padding);
size_t second_size = dispatchSearcher(calculateNeedleStats, needle.data(), needle_size, common_stats);
size_t distance = second_size;
if (data.size() <= max_string_size)
if (data_size <= max_string_size)
{
size_t first_size = calculateHaystackStatsAndMetric(data.data(), data.size(), common_stats, distance);
size_t first_size = dispatchSearcher(calculateHaystackStatsAndMetric, data.data(), data_size, common_stats, distance);
res = distance * 1.f / std::max(first_size + second_size, size_t(1));
}
else
@ -175,11 +282,18 @@ struct TrigramDistanceImpl
}
static void vector_constant(
const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & needle, PaddedPODArray<Float32> & res)
const ColumnString::Chars & data, const ColumnString::Offsets & offsets, std::string needle, PaddedPODArray<Float32> & res)
{
TrigramStats common_stats;
/// zeroing our map
NgramStats common_stats;
memset(common_stats, 0, sizeof(common_stats));
const size_t needle_stats_size = calculateNeedleStats(needle.data(), needle.size(), common_stats);
/// We use unsafe versions of getting ngrams, so I decided to use padded_data even in needle case.
const size_t needle_size = needle.size();
needle.resize(needle_size + default_padding);
const size_t needle_stats_size = dispatchSearcher(calculateNeedleStats, needle.data(), needle_size, common_stats);
size_t distance = needle_stats_size;
size_t prev_offset = 0;
for (size_t i = 0; i < offsets.size(); ++i)
@ -188,12 +302,13 @@ struct TrigramDistanceImpl
const size_t haystack_size = offsets[i] - prev_offset - 1;
if (haystack_size <= max_string_size)
{
size_t haystack_stats_size
= calculateHaystackStatsAndMetric(reinterpret_cast<const char *>(haystack), haystack_size, common_stats, distance);
size_t haystack_stats_size = dispatchSearcher(
calculateHaystackStatsAndMetric, reinterpret_cast<const char *>(haystack), haystack_size, common_stats, distance);
res[i] = distance * 1.f / std::max(haystack_stats_size + needle_stats_size, size_t(1));
}
else
{
/// if the strings are too big, we say they are completely not the same
res[i] = 1.f;
}
distance = needle_stats_size;
@ -203,16 +318,39 @@ struct TrigramDistanceImpl
};
struct TrigramDistanceName
struct NgramDistanceName
{
static constexpr auto name = "trigramDistance";
static constexpr auto name = "ngramDistance";
};
using FunctionTrigramsDistance = FunctionsStringSimilarity<TrigramDistanceImpl, TrigramDistanceName>;
struct NgramDistanceCaseInsensitiveName
{
static constexpr auto name = "ngramDistanceCaseInsensitive";
};
struct NgramDistanceUTF8Name
{
static constexpr auto name = "ngramDistanceUTF8";
};
struct NgramDistanceUTF8CaseInsensitiveName
{
static constexpr auto name = "ngramDistanceCaseInsensitiveUTF8";
};
using FunctionNgramDistance = FunctionsStringSimilarity<NgramDistanceImpl<4, UInt8, false, false>, NgramDistanceName>;
using FunctionNgramDistanceCaseInsensitive
= FunctionsStringSimilarity<NgramDistanceImpl<4, UInt8, false, true>, NgramDistanceCaseInsensitiveName>;
using FunctionNgramDistanceUTF8 = FunctionsStringSimilarity<NgramDistanceImpl<3, UInt32, true, false>, NgramDistanceUTF8Name>;
using FunctionNgramDistanceCaseInsensitiveUTF8
= FunctionsStringSimilarity<NgramDistanceImpl<3, UInt32, true, true>, NgramDistanceUTF8CaseInsensitiveName>;
void registerFunctionsStringSimilarity(FunctionFactory & factory)
{
factory.registerFunction<FunctionTrigramsDistance>();
factory.registerFunction<FunctionNgramDistance>();
factory.registerFunction<FunctionNgramDistanceCaseInsensitive>();
factory.registerFunction<FunctionNgramDistanceUTF8>();
factory.registerFunction<FunctionNgramDistanceCaseInsensitiveUTF8>();
}
}

View File

@ -12,8 +12,9 @@ namespace DB
/** Calculate similarity metrics:
*
* trigramDistance(haystack, needle) --- calculate so called 3-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.
*/
namespace ErrorCodes

View File

@ -78,5 +78,11 @@
<query>SELECT URLHash, EventDate, count() AS PageViews FROM {table} WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate &lt;= '2013-07-31' AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = halfMD5('http://example.ru/') GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100</query>
<query>SELECT WindowClientWidth, WindowClientHeight, count() AS PageViews FROM {table} WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate &lt;= '2013-07-31' AND NOT Refresh AND NOT DontCountHits AND URLHash = halfMD5('http://example.ru/') GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000</query>
<query>SELECT toStartOfMinute(EventTime) AS Minute, count() AS PageViews FROM {table} WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate &lt;= '2013-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute</query>
<query>SELECT count(multiSearch(URL, ['yandex', 'google', 'rambler'])) from {table};</query>
<query>SELECT count(match(URL, 'google|yandex|rambler')) from hits_100m_single;</query>
<query>SELECT count(match(URL, 'google')), count(match(URL, 'yandex')), count(match(URL, 'rambler')) from {table}</query>
<query>SELECT DISTINCT Title, ngramDistance(Title, 'what is love') AS distance FROM {table} ORDER BY distance ASC LIMIT 50</query>
<query>SELECT DISTINCT Title, ngramDistanceUTF8(Title, 'метрика') AS distance FROM {table} ORDER BY distance ASC LIMIT 50</query>
</test>

View File

@ -0,0 +1,509 @@
0
0
0
0
0
1000
1000
1000
1000
1000
1000
1000
1000
1000
1000
0
0
0
0
0
77
77
77
77
77
636
636
636
636
636
1000
1000
1000
1000
1000
0
1000
1000
0
77
636
1000
привет как дела?... Херсон 297
пап привет как дела - Яндекс.Видео 422
привет как дела клип - Яндекс.Видео 435
привет братан как дела - Яндекс.Видео 500
привет 529
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
1000
привет как дела?... Херсон 459
пап привет как дела - Яндекс.Видео 511
привет 529
привет как дела клип - Яндекс.Видео 565
привет братан как дела - Яндекс.Видео 583
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
1000
http://metrika.ru/ 524
http://metric.ru/ 700
http://metris.ru/ 700
http://autometric.ru/ 750
http://metrica.yandex.com/ 793
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
http://metric.ru/ 600
http://metrica.yandex.com/ 655
http://autometric.ru/ 667
http://metris.ru/ 700
http://metrika.ru/ 714
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
http://metrika.ru/ 619
http://metric.ru/ 700
http://metris.ru/ 700
http://autometric.ru/ 750
http://metrica.yandex.com/ 793
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
http://metric.ru/ 600
http://autometric.ru/ 667
http://metris.ru/ 700
http://metrika.ru/ 714
http://metrica.yandex.com/ 724
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
http://metrica.yandex.com/ 714
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
1000
0
0
0
0
0
1000
1000
1000
1000
1000
1000
1000
1000
1000
1000
0
0
0
0
0
77
77
77
77
77
636
636
636
636
636
1000
1000
1000
1000
1000
0
1000
1000
429
77
636
1000
привет как дела?... Херсон 297
пап привет как дела - Яндекс.Видео 422
привет как дела клип - Яндекс.Видео 435
привет братан как дела - Яндекс.Видео 500
привет 529
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
1000
привет как дела?... Херсон 676
пап привет как дела - Яндекс.Видео 733
привет как дела клип - Яндекс.Видео 739
привет братан как дела - Яндекс.Видео 750
привет 882
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
1000
http://metrika.ru/ 524
http://metric.ru/ 700
http://metris.ru/ 700
http://autometric.ru/ 750
http://metrica.yandex.com/ 793
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
http://metrika.ru/ 524
http://metric.ru/ 700
http://metris.ru/ 700
http://autometric.ru/ 750
http://metrica.yandex.com/ 793
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
http://metric.ru/ 600
http://metrica.yandex.com/ 655
http://autometric.ru/ 667
http://metris.ru/ 700
http://metrika.ru/ 714
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
http://metrika.ru/ 619
http://metric.ru/ 700
http://metris.ru/ 700
http://autometric.ru/ 750
http://metrica.yandex.com/ 793
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
http://metric.ru/ 600
http://autometric.ru/ 667
http://metris.ru/ 700
http://metrika.ru/ 714
http://metrica.yandex.com/ 724
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
http://metrica.yandex.com/ 714
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
1000
привет как дела клип - Яндекс.Видео 182
пап привет как дела - Яндекс.Видео 354
привет братан как дела - Яндекс.Видео 382
привет как дела?... Херсон 649
привет 838
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
1000
0
0
0
0
0
0
0
0
0
0
0
0
0
0
0
0
0
0
0
0
111
111
111
111
111
429
429
429
429
429
1000
1000
1000
1000
1000
0
0
0
0
111
429
1000
привет как дела?... Херсон 254
пап привет как дела - Яндекс.Видео 398
привет как дела клип - Яндекс.Видео 412
привет братан как дела - Яндекс.Видео 461
привет 471
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
1000
привет как дела?... Херсон 343
пап привет как дела - Яндекс.Видео 446
привет 471
привет как дела клип - Яндекс.Видео 482
привет братан как дела - Яндекс.Видео 506
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
1000
http://metrika.ru/ 579
http://metric.ru/ 778
http://metris.ru/ 778
http://autometric.ru/ 818
http://metrica.yandex.com/ 852
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
http://metric.ru/ 667
http://metrica.yandex.com/ 704
http://autometric.ru/ 727
http://metris.ru/ 778
http://metrika.ru/ 789
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
http://metrika.ru/ 684
http://metric.ru/ 778
http://metris.ru/ 778
http://autometric.ru/ 818
http://metrica.yandex.com/ 852
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
http://metric.ru/ 667
http://autometric.ru/ 727
http://metrica.yandex.com/ 778
http://metris.ru/ 778
http://metrika.ru/ 789
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
http://metrica.yandex.com/ 769
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
1000
0
0
0
0
0
0
0
0
0
0
0
0
0
0
0
0
0
0
0
0
111
111
111
111
111
600
600
600
600
600
1000
1000
1000
1000
1000
0
0
0
0
111
600
1000
привет как дела?... Херсон 910
пап привет как дела - Яндекс.Видео 928
привет как дела клип - Яндекс.Видео 929
привет братан как дела - Яндекс.Видео 955
привет 1000
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
1000
привет как дела?... Херсон 672
пап привет как дела - Яндекс.Видео 735
привет как дела клип - Яндекс.Видео 741
привет братан как дела - Яндекс.Видео 753
привет 1000
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
1000
http://metrika.ru/ 579
http://metric.ru/ 778
http://metris.ru/ 778
http://autometric.ru/ 818
http://metrica.yandex.com/ 852
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
http://metrika.ru/ 579
http://metric.ru/ 778
http://metris.ru/ 778
http://autometric.ru/ 818
http://metrica.yandex.com/ 852
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
http://metric.ru/ 667
http://metrica.yandex.com/ 704
http://autometric.ru/ 727
http://metris.ru/ 778
http://metrika.ru/ 789
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
http://metrika.ru/ 684
http://metric.ru/ 778
http://metris.ru/ 778
http://autometric.ru/ 818
http://metrica.yandex.com/ 852
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
http://metric.ru/ 667
http://autometric.ru/ 727
http://metrica.yandex.com/ 778
http://metris.ru/ 778
http://metrika.ru/ 789
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
http://metrica.yandex.com/ 769
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
1000

View File

@ -0,0 +1,106 @@
select round(1000 * ngramDistanceUTF8(materialize(''), '')) from system.numbers limit 5;
select round(1000 * ngramDistanceUTF8(materialize('абв'), '')) from system.numbers limit 5;
select round(1000 * ngramDistanceUTF8(materialize(''), 'абв')) from system.numbers limit 5;
select round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), 'абвгдеёжз')) from system.numbers limit 5;
select round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), 'абвгдеёж')) from system.numbers limit 5;
select round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), 'гдеёзд')) from system.numbers limit 5;
select round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), 'ёёёёёёёё')) from system.numbers limit 5;
select round(1000 * ngramDistanceUTF8('', ''));
select round(1000 * ngramDistanceUTF8('абв', ''));
select round(1000 * ngramDistanceUTF8('', 'абв'));
select round(1000 * ngramDistanceUTF8('абвгдеёжз', 'абвгдеёжз'));
select round(1000 * ngramDistanceUTF8('абвгдеёжз', 'абвгдеёж'));
select round(1000 * ngramDistanceUTF8('абвгдеёжз', 'гдеёзд'));
select round(1000 * ngramDistanceUTF8('абвгдеёжз', 'ёёёёёёёё'));
drop table if exists test.test_distance;
create table test.test_distance (Title String) engine = Memory;
insert into test.test_distance values ('привет как дела?... Херсон'), ('привет как дела клип - Яндекс.Видео'), ('привет'), ('пап привет как дела - Яндекс.Видео'), ('привет братан как дела - Яндекс.Видео'), ('http://metric.ru/'), ('http://autometric.ru/'), ('http://metrica.yandex.com/'), ('http://metris.ru/'), ('http://metrika.ru/'), ('');
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistanceUTF8(Title, 'привет как дела') as distance;
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistanceUTF8(Title, 'как привет дела') as distance;
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistanceUTF8(Title, 'metrika') as distance;
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistanceUTF8(Title, 'metrica') as distance;
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistanceUTF8(Title, 'metriks') as distance;
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistanceUTF8(Title, 'metrics') as distance;
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistanceUTF8(Title, 'yandex') as distance;
select round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize(''), '')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('абв'), '')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize(''), 'абв')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('абвГДЕёжз'), 'АбвгдЕёжз')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('аБВГдеёЖз'), 'АбвГдеёж')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('абвгдеёжз'), 'гдеёЗД')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('абвгдеёжз'), 'ЁЁЁЁЁЁЁЁ')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitiveUTF8('', ''));
select round(1000 * ngramDistanceCaseInsensitiveUTF8('абв', ''));
select round(1000 * ngramDistanceCaseInsensitiveUTF8('', 'абв'));
select round(1000 * ngramDistanceCaseInsensitiveUTF8('абвГДЕёжз', 'АбвгдЕЁжз'));
select round(1000 * ngramDistanceCaseInsensitiveUTF8('аБВГдеёЖз', 'АбвГдеёж'));
select round(1000 * ngramDistanceCaseInsensitiveUTF8('абвгдеёжз', 'гдеёЗД'));
select round(1000 * ngramDistanceCaseInsensitiveUTF8('АБВГДеёжз', 'ЁЁЁЁЁЁЁЁ'));
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'ПрИвЕт кАК ДЕЛа') as distance;
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'как ПРИВЕТ дела') as distance;
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'metrika') as distance;
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'Metrika') as distance;
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'mEtrica') as distance;
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'metriKS') as distance;
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'metrics') as distance;
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'YanDEX') as distance;
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'приВЕТ КАк ДеЛа КлИп - яндеКс.видео') as distance;
select round(1000 * ngramDistance(materialize(''), '')) from system.numbers limit 5;
select round(1000 * ngramDistance(materialize('abc'), '')) from system.numbers limit 5;
select round(1000 * ngramDistance(materialize(''), 'abc')) from system.numbers limit 5;
select round(1000 * ngramDistance(materialize('abcdefgh'), 'abcdefgh')) from system.numbers limit 5;
select round(1000 * ngramDistance(materialize('abcdefgh'), 'abcdefg')) from system.numbers limit 5;
select round(1000 * ngramDistance(materialize('abcdefgh'), 'defgh')) from system.numbers limit 5;
select round(1000 * ngramDistance(materialize('abcdefgh'), 'aaaaaaaa')) from system.numbers limit 5;
select round(1000 * ngramDistance('', ''));
select round(1000 * ngramDistance('abc', ''));
select round(1000 * ngramDistance('', 'abc'));
select round(1000 * ngramDistance('abcdefgh', 'abcdefgh'));
select round(1000 * ngramDistance('abcdefgh', 'abcdefg'));
select round(1000 * ngramDistance('abcdefgh', 'defgh'));
select round(1000 * ngramDistance('abcdefgh', 'aaaaaaaa'));
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistance(Title, 'привет как дела') as distance;
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistance(Title, 'как привет дела') as distance;
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistance(Title, 'metrika') as distance;
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistance(Title, 'metrica') as distance;
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistance(Title, 'metriks') as distance;
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistance(Title, 'metrics') as distance;
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistance(Title, 'yandex') as distance;
select round(1000 * ngramDistanceCaseInsensitive(materialize(''), '')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitive(materialize('abc'), '')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitive(materialize(''), 'abc')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitive(materialize('abCdefgH'), 'Abcdefgh')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitive(materialize('abcdefgh'), 'abcdeFG')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitive(materialize('AAAAbcdefgh'), 'defgh')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitive(materialize('ABCdefgH'), 'aaaaaaaa')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitive('', ''));
select round(1000 * ngramDistanceCaseInsensitive('abc', ''));
select round(1000 * ngramDistanceCaseInsensitive('', 'abc'));
select round(1000 * ngramDistanceCaseInsensitive('abCdefgH', 'Abcdefgh'));
select round(1000 * ngramDistanceCaseInsensitive('abcdefgh', 'abcdeFG'));
select round(1000 * ngramDistanceCaseInsensitive('AAAAbcdefgh', 'defgh'));
select round(1000 * ngramDistanceCaseInsensitive('ABCdefgH', 'aaaaaaaa'));
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistanceCaseInsensitive(Title, 'ПрИвЕт кАК ДЕЛа') as distance;
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistanceCaseInsensitive(Title, 'как ПРИВЕТ дела') as distance;
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistanceCaseInsensitive(Title, 'metrika') as distance;
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistanceCaseInsensitive(Title, 'Metrika') as distance;
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistanceCaseInsensitive(Title, 'mEtrica') as distance;
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistanceCaseInsensitive(Title, 'metriKS') as distance;
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistanceCaseInsensitive(Title, 'metrics') as distance;
SELECT Title, round(1000 * distance) FROM test.test_distance ORDER BY ngramDistanceCaseInsensitive(Title, 'YanDEX') as distance;
drop table if exists test.test_distance;

View File

@ -1,119 +0,0 @@
0
0
0
0
0
1000
1000
1000
1000
1000
1000
1000
1000
1000
1000
0
0
0
0
0
77
77
77
77
77
636
636
636
636
636
1000
1000
1000
1000
1000
0
1000
1000
0
77
636
1000
привет как дела?... Херсон
пап привет как дела - Яндекс.Видео
привет как дела клип - Яндекс.Видео
привет братан как дела - Яндекс.Видео
привет
http://metric.ru/
http://autometric.ru/
http://metrica.yandex.com/
http://metris.ru/
http://metrika.ru/
привет как дела?... Херсон
пап привет как дела - Яндекс.Видео
привет
привет как дела клип - Яндекс.Видео
привет братан как дела - Яндекс.Видео
http://metric.ru/
http://autometric.ru/
http://metrica.yandex.com/
http://metris.ru/
http://metrika.ru/
http://metrika.ru/
http://metric.ru/
http://metris.ru/
http://autometric.ru/
http://metrica.yandex.com/
привет как дела?... Херсон
привет как дела клип - Яндекс.Видео
привет
пап привет как дела - Яндекс.Видео
привет братан как дела - Яндекс.Видео
http://metric.ru/
http://metrica.yandex.com/
http://autometric.ru/
http://metris.ru/
http://metrika.ru/
привет как дела?... Херсон
привет как дела клип - Яндекс.Видео
привет
пап привет как дела - Яндекс.Видео
привет братан как дела - Яндекс.Видео
http://metrika.ru/
http://metric.ru/
http://metris.ru/
http://autometric.ru/
http://metrica.yandex.com/
привет как дела?... Херсон
привет как дела клип - Яндекс.Видео
привет
пап привет как дела - Яндекс.Видео
привет братан как дела - Яндекс.Видео
http://metric.ru/
http://autometric.ru/
http://metris.ru/
http://metrika.ru/
http://metrica.yandex.com/
привет как дела?... Херсон
привет как дела клип - Яндекс.Видео
привет
пап привет как дела - Яндекс.Видео
привет братан как дела - Яндекс.Видео
http://metrica.yandex.com/
привет как дела?... Херсон
привет как дела клип - Яндекс.Видео
привет
пап привет как дела - Яндекс.Видео
привет братан как дела - Яндекс.Видео
http://metric.ru/
http://autometric.ru/
http://metris.ru/
http://metrika.ru/

View File

@ -1,29 +0,0 @@
select round(1000 * trigramDistance(materialize(''), '')) from system.numbers limit 5;
select round(1000 * trigramDistance(materialize('абв'), '')) from system.numbers limit 5;
select round(1000 * trigramDistance(materialize(''), 'абв')) from system.numbers limit 5;
select round(1000 * trigramDistance(materialize('абвгдеёжз'), 'абвгдеёжз')) from system.numbers limit 5;
select round(1000 * trigramDistance(materialize('абвгдеёжз'), 'абвгдеёж')) from system.numbers limit 5;
select round(1000 * trigramDistance(materialize('абвгдеёжз'), 'гдеёзд')) from system.numbers limit 5;
select round(1000 * trigramDistance(materialize('абвгдеёжз'), 'ёёёёёёёё')) from system.numbers limit 5;
select round(1000 * trigramDistance('', ''));
select round(1000 * trigramDistance('абв', ''));
select round(1000 * trigramDistance('', 'абв'));
select round(1000 * trigramDistance('абвгдеёжз', 'абвгдеёжз'));
select round(1000 * trigramDistance('абвгдеёжз', 'абвгдеёж'));
select round(1000 * trigramDistance('абвгдеёжз', 'гдеёзд'));
select round(1000 * trigramDistance('абвгдеёжз', 'ёёёёёёёё'));
drop table if exists test.test_distance;
create table test.test_distance (Title String) engine = Memory;
insert into test.test_distance values ('привет как дела?... Херсон'), ('привет как дела клип - Яндекс.Видео'), ('привет'), ('пап привет как дела - Яндекс.Видео'), ('привет братан как дела - Яндекс.Видео'), ('http://metric.ru/'), ('http://autometric.ru/'), ('http://metrica.yandex.com/'), ('http://metris.ru/'), ('http://metrika.ru/'), ('');
SELECT Title FROM test.test_distance ORDER BY trigramDistance(Title, 'привет как дела');
SELECT Title FROM test.test_distance ORDER BY trigramDistance(Title, 'как привет дела');
SELECT Title FROM test.test_distance ORDER BY trigramDistance(Title, 'metrika');
SELECT Title FROM test.test_distance ORDER BY trigramDistance(Title, 'metrica');
SELECT Title FROM test.test_distance ORDER BY trigramDistance(Title, 'metriks');
SELECT Title FROM test.test_distance ORDER BY trigramDistance(Title, 'metrics');
SELECT Title FROM test.test_distance ORDER BY trigramDistance(Title, 'yandex');
drop table if exists test.test_distance;

View File

@ -70,5 +70,13 @@ For other regular expressions, the code is the same as for the 'match' function.
The same thing as 'like', but negative.
## ngramDistance(haystack, needle)
Calculate the 4-gram distance between `haystack` and `needle`: count the symmetric difference between two sets of 4-grams and normalize it by the sum of their cardinality. Returns float number from 0 to 1 - the closer to zero, the more strings are similar to each other. If the `needle` is more than 32Kb, throw an exception. If some of the `haystack` strings are more than 32Kb, the distance is always one.
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 bytes 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 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-->

View File

@ -59,4 +59,12 @@
## notLike(haystack, pattern), оператор haystack NOT LIKE pattern
То же, что like, но с отрицанием.
## ngramDistance(haystack, needle)
Вычисление 4-граммного расстояния между `haystack` и `needle`: считается симметрическая разность между двумя множествами 4-грам и нормализается на сумму их мощностей. Возвращает число float от 0 до 1 - чем ближе к нулю, тем больше строки похожи друг на друга. Если `needle` больше чем 32КБ, кидается исключение. Если некоторые строка из `haystack` больше 32КБ, расстояние всегда равно единице.
Для поиска без учета регистра и/или в формате UTF-8 используйте функции `ngramDistanceCaseInsensitive, ngramDistanceUTF8, ngramDistanceCaseInsensitiveUTF8`.
Примечание: для случая UTF-8 мы используем триграммное расстояние. Вычисление n-граммовного расстояния не совсем честное. Мы используем 2-х байтные хэши для хэширования n-грамм, а затем вычисляем симметричную разность между хэш-таблицами - могут возникнуть коллизии. В формате UTF-8 без учета регистра мы не используем честную функцию `tolower` - мы обнуляем 5-й бит каждого байта кодовой точки - это работает для латиницы и почти для всех кириллических букв.
[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/string_search_functions/) <!--hide-->