mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Optimize distance function, some allocation, hashmap tricks
This commit is contained in:
parent
3f636b3c34
commit
ada849263b
@ -6,15 +6,35 @@
|
||||
#include <Common/HashTable/Hash.h>
|
||||
#include <Common/UTF8Helpers.h>
|
||||
|
||||
#include <algorithm>
|
||||
#include <cstring>
|
||||
#include <limits>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct TrigramDistanceImpl
|
||||
/** Distance function implementation.
|
||||
* We calculate all the trigrams 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 return the map into the condition of which it was after 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 DistanceImpl
|
||||
{
|
||||
using ResultType = Float32;
|
||||
using CodePoint = UInt32;
|
||||
|
||||
using TrigramMap = ClearableHashMap<UInt64, UInt64, TrivialHash>;
|
||||
/// MapSize for trigram difference
|
||||
static constexpr size_t MapSize = 1u << 16;
|
||||
|
||||
/// If the haystack size is bigger than this, behaviour is unspecified for this function
|
||||
static constexpr size_t MaxStringSize = 1u << 15;
|
||||
|
||||
/// This fits mostly in L2 cache all the time
|
||||
using TrigramDiff = UInt16[MapSize];
|
||||
|
||||
static inline CodePoint readCodePoint(const char *& pos, const char * end) noexcept
|
||||
{
|
||||
@ -23,20 +43,31 @@ struct TrigramDistanceImpl
|
||||
if (pos + length > end)
|
||||
length = end - pos;
|
||||
|
||||
if (length > sizeof(CodePoint))
|
||||
length = sizeof(CodePoint);
|
||||
|
||||
CodePoint res = 0;
|
||||
memcpy(&res, pos, length);
|
||||
/// this is faster than just memcpy because of compiler optimizations with moving bytes
|
||||
switch (length) {
|
||||
case 1:
|
||||
memcpy(&res, pos, 1);
|
||||
break;
|
||||
case 2:
|
||||
memcpy(&res, pos, 2);
|
||||
break;
|
||||
case 3:
|
||||
memcpy(&res, pos, 3);
|
||||
break;
|
||||
default:
|
||||
memcpy(&res, pos, 4);
|
||||
break;
|
||||
}
|
||||
|
||||
pos += length;
|
||||
return res;
|
||||
}
|
||||
|
||||
static inline size_t calculateStats(const char * data, const size_t size, TrigramMap & ans)
|
||||
static inline size_t calculateNeedleStats(const char * data, const size_t size, TrigramDiff & ans) noexcept
|
||||
{
|
||||
ans.clear();
|
||||
size_t len = 0;
|
||||
size_t trigramCnt = 0;
|
||||
size_t trigram_cnt = 0;
|
||||
const char * start = data;
|
||||
const char * end = data + size;
|
||||
CodePoint cp1 = 0;
|
||||
@ -50,68 +81,107 @@ struct TrigramDistanceImpl
|
||||
++len;
|
||||
if (len < 3)
|
||||
continue;
|
||||
++trigramCnt;
|
||||
++ans[intHashCRC32(intHashCRC32(cp1) ^ cp2) ^ cp3];
|
||||
++trigram_cnt;
|
||||
++ans[(intHashCRC32(intHashCRC32(cp1) ^ cp2) ^ cp3) & 0xFFFFu];
|
||||
}
|
||||
return trigramCnt;
|
||||
return trigram_cnt;
|
||||
}
|
||||
|
||||
static inline UInt64 calculateMetric(const TrigramMap & lhs, const TrigramMap & rhs)
|
||||
static inline UInt64 calculateHaystackStatsAndMetric(const char * data, const size_t size, TrigramDiff & ans, size_t & distance)
|
||||
{
|
||||
UInt64 res = 0;
|
||||
size_t len = 0;
|
||||
size_t trigram_cnt = 0;
|
||||
const char * start = data;
|
||||
const char * end = data + size;
|
||||
CodePoint cp1 = 0;
|
||||
CodePoint cp2 = 0;
|
||||
CodePoint cp3 = 0;
|
||||
|
||||
for (const auto & [trigram, count] : lhs)
|
||||
/// 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;
|
||||
|
||||
if (size > small_buffer_size)
|
||||
{
|
||||
if (auto it = rhs.find(trigram); it != rhs.end())
|
||||
res += std::abs(static_cast<Int64>(count) - static_cast<Int64>(it->second));
|
||||
else
|
||||
res += count;
|
||||
trigram_storage = new UInt16[size];
|
||||
big_buffer.reset(trigram_storage);
|
||||
}
|
||||
|
||||
for (const auto & [trigram, count] : rhs)
|
||||
if (!lhs.has(trigram))
|
||||
res += count;
|
||||
while (start != end)
|
||||
{
|
||||
cp1 = cp2;
|
||||
cp2 = cp3;
|
||||
cp3 = readCodePoint(start, end);
|
||||
++len;
|
||||
if (len < 3)
|
||||
continue;
|
||||
UInt16 hash = (intHashCRC32(intHashCRC32(cp1) ^ cp2) ^ cp3) & 0xFFFFu;
|
||||
|
||||
return res;
|
||||
/// Unsigned integer tricks
|
||||
if (ans[hash] < std::numeric_limits<UInt16>::max() / 2) {
|
||||
--distance;
|
||||
} else {
|
||||
++distance;
|
||||
}
|
||||
trigram_storage[trigram_cnt++] = hash;
|
||||
--ans[hash];
|
||||
}
|
||||
for (size_t i = 0; i < trigram_cnt; ++i) {
|
||||
++ans[trigram_storage[i]];
|
||||
}
|
||||
return trigram_cnt;
|
||||
}
|
||||
|
||||
static void constant_constant(const std::string & data, const std::string & needle, Float32 & res)
|
||||
{
|
||||
TrigramMap haystack_stats;
|
||||
TrigramMap needle_stats;
|
||||
size_t first_size = calculateStats(data.data(), data.size(), haystack_stats);
|
||||
size_t second_size = calculateStats(needle.data(), needle.size(), needle_stats);
|
||||
res = calculateMetric(needle_stats, haystack_stats) * 1.0 / std::max(first_size + second_size, size_t(1));
|
||||
TrigramDiff common_stats;
|
||||
memset(common_stats, std::numeric_limits<UInt8>::max(), sizeof(common_stats));
|
||||
size_t second_size = calculateNeedleStats(needle.data(), needle.size(), common_stats);
|
||||
size_t distance = second_size;
|
||||
if (data.size() <= MaxStringSize) {
|
||||
size_t first_size = calculateHaystackStatsAndMetric(data.data(), data.size(), common_stats, distance);
|
||||
res = distance * 1.0 / std::max(first_size + second_size, size_t(1));
|
||||
} else {
|
||||
res = 1.f;
|
||||
}
|
||||
}
|
||||
|
||||
static void vector_constant(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & needle, PaddedPODArray<Float32> & res)
|
||||
{
|
||||
TrigramMap needle_stats;
|
||||
TrigramMap haystack_stats;
|
||||
const size_t needle_stats_size = calculateStats(needle.data(), needle.size(), needle_stats);
|
||||
TrigramDiff common_stats;
|
||||
memset(common_stats, std::numeric_limits<UInt8>::max(), sizeof(common_stats));
|
||||
const size_t needle_stats_size = 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)
|
||||
{
|
||||
const auto * haystack = &data[prev_offset];
|
||||
const size_t haystack_size = offsets[i] - prev_offset - 1;
|
||||
size_t haystack_stats_size = calculateStats(reinterpret_cast<const char *>(haystack), haystack_size, haystack_stats);
|
||||
res[i] = calculateMetric(haystack_stats, needle_stats) * 1.0 / std::max(haystack_stats_size + needle_stats_size, size_t(1));
|
||||
if (haystack_size <= MaxStringSize) {
|
||||
size_t haystack_stats_size = calculateHaystackStatsAndMetric(reinterpret_cast<const char *>(haystack), haystack_size, common_stats, distance);
|
||||
res[i] = distance * 1.0 / std::max(haystack_stats_size + needle_stats_size, size_t(1));
|
||||
} else {
|
||||
res[i] = 1.f;
|
||||
}
|
||||
distance = needle_stats_size;
|
||||
prev_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
struct TrigramDistanceName
|
||||
struct DistanceName
|
||||
{
|
||||
static constexpr auto name = "trigramDistance";
|
||||
static constexpr auto name = "distance";
|
||||
};
|
||||
|
||||
using FunctionTrigramDistance = FunctionsStringSimilarity<TrigramDistanceImpl, TrigramDistanceName>;
|
||||
using FunctionDistance = FunctionsStringSimilarity<DistanceImpl, DistanceName>;
|
||||
|
||||
void registerFunctionsStringSimilarity(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionTrigramDistance>();
|
||||
factory.registerFunction<FunctionDistance>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -12,7 +12,8 @@ namespace DB
|
||||
|
||||
/** Calculate similarity metrics:
|
||||
*
|
||||
* trigramSimilarity(haystack, needle) --- calculate trigram distance between haystack and needle
|
||||
* distance(haystack, needle) --- calculate so called "distance" between haystack and needle.
|
||||
* Returns float number from 0 to 1 - the closer to zero, the more strings are similar to each other.
|
||||
*/
|
||||
|
||||
namespace ErrorCodes
|
||||
@ -20,6 +21,7 @@ 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 TOO_LARGE_STRING_SIZE;
|
||||
}
|
||||
|
||||
template <typename Impl, typename Name>
|
||||
@ -63,7 +65,14 @@ public:
|
||||
if (col_haystack_const)
|
||||
{
|
||||
ResultType res{};
|
||||
Impl::constant_constant(col_haystack_const->getValue<String>(), col_needle_const->getValue<String>(), res);
|
||||
const String & needle = col_needle_const->getValue<String>();
|
||||
if (needle.size() > Impl::MaxStringSize)
|
||||
{
|
||||
throw Exception(
|
||||
"String size of needle is too big for function " + getName() + ". Should be at most " + std::to_string(Impl::MaxStringSize),
|
||||
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));
|
||||
return;
|
||||
}
|
||||
@ -76,13 +85,24 @@ public:
|
||||
const ColumnString * col_haystack_vector = checkAndGetColumn<ColumnString>(&*column_haystack);
|
||||
|
||||
if (col_haystack_vector)
|
||||
{
|
||||
const String & needle = col_needle_const->getValue<String>();
|
||||
if (needle.size() > Impl::MaxStringSize)
|
||||
{
|
||||
throw Exception(
|
||||
"String size of needle is too big for function " + getName() + ". Should be at most " + std::to_string(Impl::MaxStringSize),
|
||||
ErrorCodes::TOO_LARGE_STRING_SIZE);
|
||||
}
|
||||
Impl::vector_constant(
|
||||
col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), col_needle_const->getValue<String>(), vec_res);
|
||||
col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), needle, vec_res);
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception(
|
||||
"Illegal columns " + block.getByPosition(arguments[0]).column->getName() + " and "
|
||||
+ block.getByPosition(arguments[1]).column->getName() + " of arguments of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
}
|
||||
|
@ -1,29 +1,29 @@
|
||||
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 * distance(materialize(''), '')) from system.numbers limit 5;
|
||||
select round(1000 * distance(materialize('абв'), '')) from system.numbers limit 5;
|
||||
select round(1000 * distance(materialize(''), 'абв')) from system.numbers limit 5;
|
||||
select round(1000 * distance(materialize('абвгдеёжз'), 'абвгдеёжз')) from system.numbers limit 5;
|
||||
select round(1000 * distance(materialize('абвгдеёжз'), 'абвгдеёж')) from system.numbers limit 5;
|
||||
select round(1000 * distance(materialize('абвгдеёжз'), 'гдеёзд')) from system.numbers limit 5;
|
||||
select round(1000 * distance(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('абвгдеёжз', 'ёёёёёёёё'));
|
||||
select round(1000 * distance('', ''));
|
||||
select round(1000 * distance('абв', ''));
|
||||
select round(1000 * distance('', 'абв'));
|
||||
select round(1000 * distance('абвгдеёжз', 'абвгдеёжз'));
|
||||
select round(1000 * distance('абвгдеёжз', 'абвгдеёж'));
|
||||
select round(1000 * distance('абвгдеёжз', 'гдеёзд'));
|
||||
select round(1000 * distance('абвгдеёжз', 'ёёёёёёёё'));
|
||||
|
||||
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');
|
||||
SELECT Title FROM test.test_distance ORDER BY distance(Title, 'привет как дела');
|
||||
SELECT Title FROM test.test_distance ORDER BY distance(Title, 'как привет дела');
|
||||
SELECT Title FROM test.test_distance ORDER BY distance(Title, 'metrika');
|
||||
SELECT Title FROM test.test_distance ORDER BY distance(Title, 'metrica');
|
||||
SELECT Title FROM test.test_distance ORDER BY distance(Title, 'metriks');
|
||||
SELECT Title FROM test.test_distance ORDER BY distance(Title, 'metrics');
|
||||
SELECT Title FROM test.test_distance ORDER BY distance(Title, 'yandex');
|
||||
|
||||
drop table if exists test.test_distance;
|
||||
|
Loading…
Reference in New Issue
Block a user