Optimize distance function, some allocation, hashmap tricks

This commit is contained in:
Danila Kutenin 2019-02-22 06:02:10 +03:00
parent 3f636b3c34
commit ada849263b
3 changed files with 151 additions and 61 deletions

View File

@ -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>();
}
}

View File

@ -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);
}

View File

@ -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;