提交 f613c06c 编写于 作者: D Danila Kutenin

ngram vector_vector distance added

上级 0bb10c7f
......@@ -164,43 +164,46 @@ struct NgramDistanceImpl
return num;
}
template <bool SaveNgrams>
static ALWAYS_INLINE inline size_t calculateNeedleStats(
const char * data,
const size_t size,
NgramStats & ngram_stats,
[[maybe_unused]] UInt16 * ngram_storage,
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 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;
size_t len = 0;
do
{
len += found - N + 1;
for (; i + N <= found; ++i)
++ngram_stats[hash_functor(cp + i)];
{
++len;
UInt16 hash = hash_functor(cp + i);
if constexpr (SaveNgrams)
*ngram_storage++ = hash;
++ngram_stats[hash];
}
i = 0;
} while (start < end && (found = read_code_points(cp, start, end)));
return len;
}
template <bool ReuseStats>
static ALWAYS_INLINE inline UInt64 calculateHaystackStatsAndMetric(
const char * data,
const size_t size,
NgramStats & ngram_stats,
size_t & distance,
[[maybe_unused]] UInt16 * ngram_storage,
size_t (*read_code_points)(CodePoint *, const char *&, const char *),
UInt16 (*hash_functor)(const CodePoint *))
{
......@@ -209,18 +212,6 @@ struct NgramDistanceImpl
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 * ngram_storage = small_buffer;
if (size > small_buffer_size)
{
ngram_storage = new UInt16[size];
big_buffer.reset(ngram_storage);
}
/// 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.
......@@ -235,21 +226,25 @@ struct NgramDistanceImpl
--distance;
else
++distance;
ngram_storage[ngram_cnt++] = hash;
if constexpr (ReuseStats)
ngram_storage[ngram_cnt] = hash;
++ngram_cnt;
--ngram_stats[hash];
}
iter = 0;
} while (start < end && (found = read_code_points(cp, start, end)));
/// Return the state of hash map to its initial.
if constexpr (ReuseStats)
{
for (size_t i = 0; i < ngram_cnt; ++i)
++ngram_stats[ngram_storage[i]];
}
return ngram_cnt;
}
template <class Callback, class... Args>
static inline size_t dispatchSearcher(Callback callback, Args &&... args)
static inline auto dispatchSearcher(Callback callback, Args &&... args)
{
if constexpr (!UTF8)
return callback(std::forward<Args>(args)..., readASCIICodePoints, ASCIIHash);
......@@ -268,11 +263,11 @@ struct NgramDistanceImpl
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 second_size = dispatchSearcher(calculateNeedleStats<false>, needle.data(), needle_size, common_stats, nullptr);
size_t distance = second_size;
if (data_size <= max_string_size)
{
size_t first_size = dispatchSearcher(calculateHaystackStatsAndMetric, data.data(), data_size, common_stats, distance);
size_t first_size = dispatchSearcher(calculateHaystackStatsAndMetric<false>, data.data(), data_size, common_stats, distance, nullptr);
res = distance * 1.f / std::max(first_size + second_size, size_t(1));
}
else
......@@ -281,18 +276,94 @@ struct NgramDistanceImpl
}
}
static void vector_vector(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const ColumnString::Chars & needle_data,
const ColumnString::Offsets & needle_offsets,
PaddedPODArray<Float32> & res)
{
const size_t haystack_offsets_size = haystack_offsets.size();
size_t prev_haystack_offset = 0;
size_t prev_needle_offset = 0;
NgramStats common_stats;
memset(common_stats, 0, sizeof(common_stats));
/// The main motivation is to not allocate more on stack because we have already allocated a lot (128Kb).
/// And we can reuse these storages in one thread because we care only about what was written to first places.
std::unique_ptr<UInt16[]> needle_ngram_storage;
std::unique_ptr<UInt16[]> haystack_ngram_storage;
needle_ngram_storage.reset(new UInt16[max_string_size]);
haystack_ngram_storage.reset(new UInt16[max_string_size]);
for (size_t i = 0; i < haystack_offsets_size; ++i)
{
const char * haystack = reinterpret_cast<const char *>(&haystack_data[prev_haystack_offset]);
const size_t haystack_size = haystack_offsets[i] - prev_haystack_offset - 1;
const char * needle = reinterpret_cast<const char *>(&needle_data[prev_needle_offset]);
const size_t needle_size = needle_offsets[i] - prev_needle_offset - 1;
if (needle_size <= max_string_size && haystack_size <= max_string_size)
{
/// Get needle stats.
const size_t needle_stats_size = dispatchSearcher(
calculateNeedleStats<true>,
needle,
needle_size,
common_stats,
needle_ngram_storage.get());
size_t distance = needle_stats_size;
/// Combine with haystack stats, return to initial needle stats.
const size_t haystack_stats_size = dispatchSearcher(
calculateHaystackStatsAndMetric<true>,
haystack,
haystack_size,
common_stats,
distance,
haystack_ngram_storage.get());
/// Return to zero array stats.
for (size_t j = 0; j < needle_stats_size; ++j)
--common_stats[needle_ngram_storage[j]];
/// For now, common stats is a zero array.
res[i] = distance * 1.f / std::max(haystack_stats_size + needle_stats_size, size_t(1));
}
else
{
/// String are too big, we are assuming they are not the same. This is done because of limiting number
/// of bigrams added and not allocating too much memory.
res[i] = 1.f;
}
prev_needle_offset = needle_offsets[i];
prev_haystack_offset = haystack_offsets[i];
}
}
static void vector_constant(
const ColumnString::Chars & data, const ColumnString::Offsets & offsets, std::string needle, PaddedPODArray<Float32> & res)
const ColumnString::Chars & data,
const ColumnString::Offsets & offsets,
std::string needle,
PaddedPODArray<Float32> & res)
{
/// zeroing our map
NgramStats common_stats;
memset(common_stats, 0, sizeof(common_stats));
/// The main motivation is to not allocate more on stack because we have already allocated a lot (128Kb).
/// And we can reuse these storages in one thread because we care only about what was written to first places.
std::unique_ptr<UInt16[]> ngram_storage;
ngram_storage.reset(new UInt16[max_string_size]);
/// 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);
const size_t needle_stats_size = dispatchSearcher(calculateNeedleStats<false>, needle.data(), needle_size, common_stats, nullptr);
size_t distance = needle_stats_size;
size_t prev_offset = 0;
......@@ -303,7 +374,11 @@ struct NgramDistanceImpl
if (haystack_size <= max_string_size)
{
size_t haystack_stats_size = dispatchSearcher(
calculateHaystackStatsAndMetric, reinterpret_cast<const char *>(haystack), haystack_size, common_stats, distance);
calculateHaystackStatsAndMetric<true>,
reinterpret_cast<const char *>(haystack),
haystack_size, common_stats,
distance,
ngram_storage.get());
res[i] = distance * 1.f / std::max(haystack_stats_size + needle_stats_size, size_t(1));
}
else
......
......@@ -62,10 +62,7 @@ public:
const ColumnConst * col_haystack_const = typeid_cast<const ColumnConst *>(&*column_haystack);
const ColumnConst * col_needle_const = typeid_cast<const ColumnConst *>(&*column_needle);
if (!col_needle_const)
throw Exception("Second argument of function " + getName() + " must be constant string.", ErrorCodes::ILLEGAL_COLUMN);
if (col_haystack_const)
if (col_haystack_const && col_needle_const)
{
ResultType res{};
const String & needle = col_needle_const->getValue<String>();
......@@ -88,8 +85,9 @@ public:
vec_res.resize(column_haystack->size());
const ColumnString * col_haystack_vector = checkAndGetColumn<ColumnString>(&*column_haystack);
const ColumnString * col_needle_vector = checkAndGetColumn<ColumnString>(&*column_needle);
if (col_haystack_vector)
if (col_haystack_vector && col_needle_const)
{
const String & needle = col_needle_const->getValue<String>();
if (needle.size() > Impl::max_string_size)
......@@ -101,6 +99,27 @@ public:
}
Impl::vector_constant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), needle, vec_res);
}
else if (col_haystack_vector && col_needle_vector)
{
Impl::vector_vector(
col_haystack_vector->getChars(),
col_haystack_vector->getOffsets(),
col_needle_vector->getChars(),
col_needle_vector->getOffsets(),
vec_res);
}
else if (col_haystack_const && col_needle_vector)
{
const String & needle = col_haystack_const->getValue<String>();
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);
}
Impl::vector_constant(col_needle_vector->getChars(), col_needle_vector->getOffsets(), needle, vec_res);
}
else
{
throw Exception(
......
......@@ -21,13 +21,16 @@
<total_time_ms>60000</total_time_ms>
</any_of>
</stop_conditions>
<query>SELECT DISTINCT URL,Title, ngramDistance(Title, URL) AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50</query>
<query>SELECT DISTINCT SearchPhrase,Title, ngramDistance(Title, SearchPhrase) AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50</query>
<query>SELECT DISTINCT Title, ngramDistance(Title, 'what is love') AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50</query>
<query>SELECT DISTINCT Title, ngramDistance(Title, 'baby dont hurt me') AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50</query>
<query>SELECT DISTINCT Title, ngramDistance(Title, 'no more') AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50</query>
<query>SELECT DISTINCT Title, ngramDistanceCaseInsensitive(Title, 'wHAt Is lovE') AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50</query>
<query>SELECT DISTINCT Title, ngramDistanceCaseInsensitive(Title, 'BABY DonT hUrT me') AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50</query>
<query>SELECT DISTINCT Title, ngramDistanceCaseInsensitive(Title, 'nO MOrE') AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50</query>
<query>SELECT DISTINCT URL,Title, ngramDistanceUTF8(Title, URL) AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50</query>
<query>SELECT DISTINCT SearchPhrase,Title, ngramDistanceUTF8(Title, SearchPhrase) AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50</query>
<query>SELECT DISTINCT Title, ngramDistanceUTF8(Title, 'метрика') AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50</query>
<query>SELECT DISTINCT URL, ngramDistanceUTF8(URL, 'как дела') AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50</query>
<query>SELECT DISTINCT URL, ngramDistanceUTF8(URL, 'чем занимаешься') AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50</query>
......
......@@ -33,6 +33,76 @@
1000
1000
1000
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
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
1
1
1
1
1
1
0
1000
1000
......@@ -40,6 +110,39 @@
77
636
1000
привет как дела?... Херсон 0
привет как дела клип - Яндекс.Видео 0
привет 0
пап привет как дела - Яндекс.Видео 0
привет братан как дела - Яндекс.Видео 0
http://metric.ru/ 0
http://autometric.ru/ 0
http://metrica.yandex.com/ 0
http://metris.ru/ 0
http://metrika.ru/ 0
0
0
привет как дела?... Херсон 600
пап привет как дела - Яндекс.Видео 684
привет как дела клип - Яндекс.Видео 692
привет братан как дела - Яндекс.Видео 707
привет 1000
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
0
http://metric.ru/ 765
http://metris.ru/ 765
http://metrika.ru/ 778
http://autometric.ru/ 810
http://metrica.yandex.com/ 846
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела?... Херсон 297
пап привет как дела - Яндекс.Видео 422
привет как дела клип - Яндекс.Видео 435
......@@ -152,6 +255,76 @@ http://metrika.ru/ 1000
1000
1000
1000
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
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
1
1
1
1
1
1
0
1000
1000
......@@ -159,6 +332,39 @@ http://metrika.ru/ 1000
77
636
1000
привет как дела?... Херсон 0
привет как дела клип - Яндекс.Видео 0
привет 0
пап привет как дела - Яндекс.Видео 0
привет братан как дела - Яндекс.Видео 0
http://metric.ru/ 0
http://autometric.ru/ 0
http://metrica.yandex.com/ 0
http://metris.ru/ 0
http://metrika.ru/ 0
0
0
привет как дела?... Херсон 600
пап привет как дела - Яндекс.Видео 684
привет как дела клип - Яндекс.Видео 692
привет братан как дела - Яндекс.Видео 707
привет 1000
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
0
http://metric.ru/ 765
http://metris.ru/ 765
http://metrika.ru/ 778
http://autometric.ru/ 810
http://metrica.yandex.com/ 846
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела?... Херсон 297
пап привет как дела - Яндекс.Видео 422
привет как дела клип - Яндекс.Видео 435
......@@ -293,6 +499,76 @@ http://metrika.ru/ 1000
1000
1000
1000
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
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
1
1
1
1
1
1
0
0
0
......@@ -412,6 +688,76 @@ http://metrika.ru/ 1000
1000
1000
1000
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
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
1
1
1
1
1
1
0
0
0
......
......@@ -6,6 +6,22 @@ select round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), 'абв
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(''), materialize('')))=round(1000 * ngramDistanceUTF8(materialize(''), '')) from system.numbers limit 5;
select round(1000 * ngramDistanceUTF8(materialize('абв'), materialize('')))=round(1000 * ngramDistanceUTF8(materialize('абв'), '')) from system.numbers limit 5;
select round(1000 * ngramDistanceUTF8(materialize(''), materialize('абв')))=round(1000 * ngramDistanceUTF8(materialize(''), 'абв')) from system.numbers limit 5;
select round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), materialize('абвгдеёжз')))=round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), 'абвгдеёжз')) from system.numbers limit 5;
select round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), materialize('абвгдеёж')))=round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), 'абвгдеёж')) from system.numbers limit 5;
select round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), materialize('гдеёзд')))=round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), 'гдеёзд')) from system.numbers limit 5;
select round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), materialize('ёёёёёёёё')))=round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), 'ёёёёёёёё')) from system.numbers limit 5;
select round(1000 * ngramDistanceUTF8('', materialize('')))=round(1000 * ngramDistanceUTF8(materialize(''), '')) from system.numbers limit 5;
select round(1000 * ngramDistanceUTF8('абв', materialize('')))=round(1000 * ngramDistanceUTF8(materialize('абв'), '')) from system.numbers limit 5;
select round(1000 * ngramDistanceUTF8('', materialize('абв')))=round(1000 * ngramDistanceUTF8(materialize(''), 'абв')) from system.numbers limit 5;
select round(1000 * ngramDistanceUTF8('абвгдеёжз', materialize('абвгдеёжз')))=round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), 'абвгдеёжз')) from system.numbers limit 5;
select round(1000 * ngramDistanceUTF8('абвгдеёжз', materialize('абвгдеёж')))=round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), 'абвгдеёж')) from system.numbers limit 5;
select round(1000 * ngramDistanceUTF8('абвгдеёжз', materialize('гдеёзд')))=round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), 'гдеёзд')) from system.numbers limit 5;
select round(1000 * ngramDistanceUTF8('абвгдеёжз', materialize('ёёёёёёёё')))=round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), 'ёёёёёёёё')) from system.numbers limit 5;
select round(1000 * ngramDistanceUTF8('', ''));
select round(1000 * ngramDistanceUTF8('абв', ''));
select round(1000 * ngramDistanceUTF8('', 'абв'));
......@@ -18,6 +34,10 @@ drop table if exists test_distance;
create table test_distance (Title String) engine = Memory;
insert into 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_distance ORDER BY ngramDistanceUTF8(Title, Title) as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, extract(Title, 'как дела')) as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, extract(Title, 'metr')) as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, 'привет как дела') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, 'как привет дела') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, 'metrika') as distance;
......@@ -35,6 +55,23 @@ select round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('аБВГдеё
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(''),materialize(''))) = round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize(''), '')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('абв'),materialize(''))) = round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('абв'), '')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize(''), materialize('абв'))) = round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize(''), 'абв')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('абвГДЕёжз'), materialize('АбвгдЕёжз'))) = round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('абвГДЕёжз'), 'АбвгдЕёжз')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('аБВГдеёЖз'), materialize('АбвГдеёж'))) = round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('аБВГдеёЖз'), 'АбвГдеёж')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('абвгдеёжз'), materialize('гдеёЗД'))) = round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('абвгдеёжз'), 'гдеёЗД')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('абвгдеёжз'), materialize('ЁЁЁЁЁЁЁЁ'))) = round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('абвгдеёжз'), 'ЁЁЁЁЁЁЁЁ')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitiveUTF8('', materialize(''))) = round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize(''), '')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitiveUTF8('абв',materialize(''))) = round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('абв'), '')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitiveUTF8('', materialize('абв'))) = round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize(''), 'абв')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitiveUTF8('абвГДЕёжз', materialize('АбвгдЕёжз'))) = round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('абвГДЕёжз'), 'АбвгдЕёжз')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitiveUTF8('аБВГдеёЖз', materialize('АбвГдеёж'))) = round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('аБВГдеёЖз'), 'АбвГдеёж')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitiveUTF8('абвгдеёжз', materialize('гдеёЗД'))) = round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('абвгдеёжз'), 'гдеёЗД')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitiveUTF8('абвгдеёжз', materialize('ЁЁЁЁЁЁЁЁ'))) = round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('абвгдеёжз'), 'ЁЁЁЁЁЁЁЁ')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitiveUTF8('', ''));
select round(1000 * ngramDistanceCaseInsensitiveUTF8('абв', ''));
select round(1000 * ngramDistanceCaseInsensitiveUTF8('', 'абв'));
......@@ -43,6 +80,10 @@ select round(1000 * ngramDistanceCaseInsensitiveUTF8('аБВГдеёЖз', 'Аб
select round(1000 * ngramDistanceCaseInsensitiveUTF8('абвгдеёжз', 'гдеёЗД'));
select round(1000 * ngramDistanceCaseInsensitiveUTF8('АБВГДеёжз', 'ЁЁЁЁЁЁЁЁ'));
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, Title) as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, extract(Title, 'как дела')) as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, extract(Title, 'metr')) as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'ПрИвЕт кАК ДЕЛа') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'как ПРИВЕТ дела') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'metrika') as distance;
......@@ -62,6 +103,23 @@ select round(1000 * ngramDistance(materialize('abcdefgh'), 'abcdefg')) from syst
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(materialize(''),materialize('')))=round(1000 * ngramDistance(materialize(''), '')) from system.numbers limit 5;
select round(1000 * ngramDistance(materialize('abc'),materialize('')))=round(1000 * ngramDistance(materialize('abc'), '')) from system.numbers limit 5;
select round(1000 * ngramDistance(materialize(''), materialize('abc')))=round(1000 * ngramDistance(materialize(''), 'abc')) from system.numbers limit 5;
select round(1000 * ngramDistance(materialize('abcdefgh'), materialize('abcdefgh')))=round(1000 * ngramDistance(materialize('abcdefgh'), 'abcdefgh')) from system.numbers limit 5;
select round(1000 * ngramDistance(materialize('abcdefgh'), materialize('abcdefg')))=round(1000 * ngramDistance(materialize('abcdefgh'), 'abcdefg')) from system.numbers limit 5;
select round(1000 * ngramDistance(materialize('abcdefgh'), materialize('defgh')))=round(1000 * ngramDistance(materialize('abcdefgh'), 'defgh')) from system.numbers limit 5;
select round(1000 * ngramDistance(materialize('abcdefgh'), materialize('aaaaaaaa')))=round(1000 * ngramDistance(materialize('abcdefgh'), 'aaaaaaaa')) from system.numbers limit 5;
select round(1000 * ngramDistance('',materialize('')))=round(1000 * ngramDistance(materialize(''), '')) from system.numbers limit 5;
select round(1000 * ngramDistance('abc', materialize('')))=round(1000 * ngramDistance(materialize('abc'), '')) from system.numbers limit 5;
select round(1000 * ngramDistance('', materialize('abc')))=round(1000 * ngramDistance(materialize(''), 'abc')) from system.numbers limit 5;
select round(1000 * ngramDistance('abcdefgh', materialize('abcdefgh')))=round(1000 * ngramDistance(materialize('abcdefgh'), 'abcdefgh')) from system.numbers limit 5;
select round(1000 * ngramDistance('abcdefgh', materialize('abcdefg')))=round(1000 * ngramDistance(materialize('abcdefgh'), 'abcdefg')) from system.numbers limit 5;
select round(1000 * ngramDistance('abcdefgh', materialize('defgh')))=round(1000 * ngramDistance(materialize('abcdefgh'), 'defgh')) from system.numbers limit 5;
select round(1000 * ngramDistance('abcdefgh', materialize('aaaaaaaa')))=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'));
......@@ -86,6 +144,22 @@ select round(1000 * ngramDistanceCaseInsensitive(materialize('abcdefgh'), 'abcde
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(materialize(''), materialize('')))=round(1000 * ngramDistanceCaseInsensitive(materialize(''), '')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitive(materialize('abc'), materialize('')))=round(1000 * ngramDistanceCaseInsensitive(materialize('abc'), '')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitive(materialize(''), materialize('abc')))=round(1000 * ngramDistanceCaseInsensitive(materialize(''), 'abc')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitive(materialize('abCdefgH'), materialize('Abcdefgh')))=round(1000 * ngramDistanceCaseInsensitive(materialize('abCdefgH'), 'Abcdefgh')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitive(materialize('abcdefgh'), materialize('abcdeFG')))=round(1000 * ngramDistanceCaseInsensitive(materialize('abcdefgh'), 'abcdeFG')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitive(materialize('AAAAbcdefgh'), materialize('defgh')))=round(1000 * ngramDistanceCaseInsensitive(materialize('AAAAbcdefgh'), 'defgh')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitive(materialize('ABCdefgH'), materialize('aaaaaaaa')))=round(1000 * ngramDistanceCaseInsensitive(materialize('ABCdefgH'), 'aaaaaaaa')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitive('', materialize('')))=round(1000 * ngramDistanceCaseInsensitive(materialize(''), '')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitive('abc', materialize('')))=round(1000 * ngramDistanceCaseInsensitive(materialize('abc'), '')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitive('', materialize('abc')))=round(1000 * ngramDistanceCaseInsensitive(materialize(''), 'abc')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitive('abCdefgH', materialize('Abcdefgh')))=round(1000 * ngramDistanceCaseInsensitive(materialize('abCdefgH'), 'Abcdefgh')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitive('abcdefgh', materialize('abcdeFG')))=round(1000 * ngramDistanceCaseInsensitive(materialize('abcdefgh'), 'abcdeFG')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitive('AAAAbcdefgh', materialize('defgh')))=round(1000 * ngramDistanceCaseInsensitive(materialize('AAAAbcdefgh'), 'defgh')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitive('ABCdefgH', materialize('aaaaaaaa')))=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'));
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册