ClickHouse/src/Functions/FunctionsStringHash.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

800 lines
27 KiB
C++
Raw Normal View History

2019-11-06 10:34:13 +00:00
#include <Functions/FunctionsStringHash.h>
#include <Functions/ExtractString.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsHashing.h>
2020-06-10 15:02:58 +00:00
#include <Common/PODArray.h>
2019-11-06 10:34:13 +00:00
#include <Core/Defines.h>
2020-06-10 15:02:58 +00:00
#include <functional>
2019-11-06 10:34:13 +00:00
#include <tuple>
2020-06-10 15:02:58 +00:00
#include <vector>
2021-10-02 07:13:14 +00:00
#include <base/unaligned.h>
2019-11-06 10:34:13 +00:00
2020-12-25 14:18:41 +00:00
#include <city.h>
#if (defined(__PPC64__) || defined(__powerpc64__)) && __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__
#include "vec_crc32.h"
#endif
2023-09-19 14:12:14 +00:00
#if defined(__s390x__) && __BYTE_ORDER__==__ORDER_BIG_ENDIAN__
#include <crc32-s390x.h>
#endif
2019-11-06 10:34:13 +00:00
namespace DB
{
2020-12-21 11:39:17 +00:00
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
2020-12-25 16:04:55 +00:00
struct BytesRef
{
const UInt8 * data;
size_t size;
};
2019-11-06 10:34:13 +00:00
struct Hash
{
2020-12-22 15:29:32 +00:00
static UInt64 crc32u64(UInt64 crc [[maybe_unused]], UInt64 val [[maybe_unused]])
2019-11-06 10:34:13 +00:00
{
2020-12-21 11:39:17 +00:00
#ifdef __SSE4_2__
2020-12-22 15:29:32 +00:00
return _mm_crc32_u64(crc, val);
#elif defined(__aarch64__) && defined(__ARM_FEATURE_CRC32)
return __crc32cd(static_cast<UInt32>(crc), val);
#elif (defined(__PPC64__) || defined(__powerpc64__)) && __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__
return crc32_ppc(crc, reinterpret_cast<const unsigned char *>(&val), sizeof(val));
2023-01-12 01:00:10 +00:00
#elif defined(__s390x__) && __BYTE_ORDER__==__ORDER_BIG_ENDIAN__
2023-09-19 14:12:14 +00:00
return crc32c_le(static_cast<UInt32>(crc), reinterpret_cast<unsigned char *>(&val), sizeof(val));
2020-12-21 11:39:17 +00:00
#else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "String hash is not implemented without sse4.2 support");
2020-12-21 11:39:17 +00:00
#endif
2019-11-06 10:34:13 +00:00
}
static UInt64 crc32u32(UInt32 crc [[maybe_unused]], UInt32 val [[maybe_unused]])
2019-11-06 10:34:13 +00:00
{
#ifdef __SSE4_2__
2020-12-22 15:29:32 +00:00
return _mm_crc32_u32(crc, val);
#elif defined(__aarch64__) && defined(__ARM_FEATURE_CRC32)
return __crc32cw(crc, val);
#elif (defined(__PPC64__) || defined(__powerpc64__)) && __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__
return crc32_ppc(crc, reinterpret_cast<const unsigned char *>(&val), sizeof(val));
2023-01-12 01:00:10 +00:00
#elif defined(__s390x__) && __BYTE_ORDER__==__ORDER_BIG_ENDIAN__
2023-09-19 14:12:14 +00:00
return crc32c_le(static_cast<UInt32>(crc), reinterpret_cast<unsigned char *>(&val), sizeof(val));
2019-11-06 10:34:13 +00:00
#else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "String hash is not implemented without sse4.2 support");
2019-11-06 10:34:13 +00:00
#endif
2020-12-22 15:29:32 +00:00
}
static UInt64 crc32u16(UInt32 crc [[maybe_unused]], UInt16 val [[maybe_unused]])
2020-12-25 11:22:48 +00:00
{
#ifdef __SSE4_2__
return _mm_crc32_u16(crc, val);
#elif defined(__aarch64__) && defined(__ARM_FEATURE_CRC32)
return __crc32ch(crc, val);
#elif (defined(__PPC64__) || defined(__powerpc64__)) && __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__
return crc32_ppc(crc, reinterpret_cast<const unsigned char *>(&val), sizeof(val));
2023-01-12 01:00:10 +00:00
#elif defined(__s390x__) && __BYTE_ORDER__==__ORDER_BIG_ENDIAN__
2023-09-19 14:12:14 +00:00
return crc32c_le(static_cast<UInt32>(crc), reinterpret_cast<unsigned char *>(&val), sizeof(val));
2020-12-25 11:22:48 +00:00
#else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "String hash is not implemented without sse4.2 support");
2020-12-25 11:22:48 +00:00
#endif
}
static UInt64 crc32u8(UInt32 crc [[maybe_unused]], UInt8 val [[maybe_unused]])
2020-12-22 15:29:32 +00:00
{
#ifdef __SSE4_2__
return _mm_crc32_u8(crc, val);
#elif defined(__aarch64__) && defined(__ARM_FEATURE_CRC32)
return __crc32cb(crc, val);
#elif (defined(__PPC64__) || defined(__powerpc64__)) && __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__
return crc32_ppc(crc, reinterpret_cast<const unsigned char *>(&val), sizeof(val));
2023-01-12 01:00:10 +00:00
#elif defined(__s390x__) && __BYTE_ORDER__==__ORDER_BIG_ENDIAN__
2023-09-19 14:12:14 +00:00
return crc32c_le(static_cast<UInt32>(crc), reinterpret_cast<unsigned char *>(&val), sizeof(val));
2020-12-22 15:29:32 +00:00
#else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "String hash is not implemented without sse4.2 support");
2020-12-22 15:29:32 +00:00
#endif
}
2020-12-25 11:22:48 +00:00
template <bool CaseInsensitive>
static ALWAYS_INLINE inline UInt64 shingleHash(UInt64 crc, const UInt8 * start, size_t size)
2020-12-22 15:29:32 +00:00
{
2020-12-25 11:22:48 +00:00
if (size & 1)
{
UInt8 x = *start;
if constexpr (CaseInsensitive)
x |= 0x20u; /// see toLowerIfAlphaASCII from StringUtils.h
crc = crc32u8(static_cast<UInt32>(crc), x);
2020-12-25 11:22:48 +00:00
--size;
++start;
}
if (size & 2)
{
UInt16 x = unalignedLoad<UInt16>(start);
if constexpr (CaseInsensitive)
x |= 0x2020u;
crc = crc32u16(static_cast<UInt32>(crc), x);
2020-12-25 11:22:48 +00:00
size -= 2;
start += 2;
}
if (size & 4)
{
UInt32 x = unalignedLoad<UInt32>(start);
if constexpr (CaseInsensitive)
x |= 0x20202020u;
crc = crc32u32(static_cast<UInt32>(crc), x);
2020-12-25 11:22:48 +00:00
size -= 4;
start += 4;
}
while (size)
{
UInt64 x = unalignedLoad<UInt64>(start);
if constexpr (CaseInsensitive)
x |= 0x2020202020202020u;
crc = crc32u64(crc, x);
size -= 8;
start += 8;
}
return crc;
2020-12-22 15:29:32 +00:00
}
2020-12-25 11:22:48 +00:00
template <bool CaseInsensitive>
2020-12-25 16:04:55 +00:00
static ALWAYS_INLINE inline UInt64 shingleHash(const std::vector<BytesRef> & shingle, size_t offset = 0)
2020-12-22 15:29:32 +00:00
{
UInt64 crc = -1ULL;
2020-12-25 11:22:48 +00:00
for (size_t i = offset; i < shingle.size(); ++i)
crc = shingleHash<CaseInsensitive>(crc, shingle[i].data, shingle[i].size);
for (size_t i = 0; i < offset; ++i)
crc = shingleHash<CaseInsensitive>(crc, shingle[i].data, shingle[i].size);
2020-12-25 16:49:55 +00:00
return crc;
2019-11-06 10:34:13 +00:00
}
};
2020-12-24 15:49:05 +00:00
// SimHash String -> UInt64
2020-06-10 15:02:58 +00:00
// UTF8: means ASCII or UTF8, these two parameters CodePoint and UTF8 can only be (UInt8, false) or (UInt32, true)
// Ngram: means ngram(true) or words shingles(false)
// CaseInsensitive: means should we consider about letter case or not
2020-12-25 14:18:41 +00:00
template <bool UTF8, bool Ngram, bool CaseInsensitive>
2020-12-24 15:49:05 +00:00
struct SimHashImpl
2019-11-06 10:34:13 +00:00
{
2020-12-25 17:58:44 +00:00
static constexpr size_t min_word_size = 4;
2020-12-25 11:22:48 +00:00
/// Update fingerprint according to hash_value bits.
static ALWAYS_INLINE inline void updateFingerVector(Int64 * finger_vec, UInt64 hash_value)
{
for (size_t i = 0; i < 64; ++i)
finger_vec[i] += (hash_value & (1ULL << i)) ? 1 : -1;
}
/// Return a 64 bit value according to finger_vec.
static ALWAYS_INLINE inline UInt64 getSimHash(const Int64 * finger_vec)
{
UInt64 res = 0;
2019-11-06 10:34:13 +00:00
2020-12-25 11:22:48 +00:00
for (size_t i = 0; i < 64; ++i)
if (finger_vec[i] > 0)
res |= (1ULL << i);
return res;
}
// SimHash ngram calculate function: String -> UInt64
2019-11-06 10:34:13 +00:00
// this function extracting ngram from input string, and maintain a 64-dimensions vector
// for each ngram, calculate a 64 bit hash value, and update the vector according the hash value
// finally return a 64 bit value(UInt64), i'th bit is 1 means vector[i] > 0, otherwise, vector[i] < 0
2020-12-25 11:22:48 +00:00
2020-12-25 14:18:41 +00:00
static ALWAYS_INLINE inline UInt64 ngramHashASCII(const UInt8 * data, size_t size, size_t shingle_size)
2019-11-06 10:34:13 +00:00
{
2020-12-25 11:22:48 +00:00
if (size < shingle_size)
return Hash::shingleHash<CaseInsensitive>(-1ULL, data, size);
2019-11-06 10:34:13 +00:00
Int64 finger_vec[64] = {};
2020-12-25 11:22:48 +00:00
const UInt8 * end = data + size;
for (const UInt8 * pos = data; pos + shingle_size <= end; ++pos)
{
UInt64 hash_value = Hash::shingleHash<CaseInsensitive>(-1ULL, pos, shingle_size);
updateFingerVector(finger_vec, hash_value);
}
return getSimHash(finger_vec);
}
2020-12-25 14:18:41 +00:00
static ALWAYS_INLINE inline UInt64 ngramHashUTF8(const UInt8 * data, size_t size, size_t shingle_size)
2020-12-25 11:22:48 +00:00
{
const UInt8 * start = data;
const UInt8 * end = data + size;
2019-11-06 10:34:13 +00:00
2020-12-25 11:22:48 +00:00
const UInt8 * word_start = start;
const UInt8 * word_end = start;
2019-11-06 10:34:13 +00:00
2020-12-25 11:22:48 +00:00
for (size_t i = 0; i < shingle_size; ++i)
2019-11-06 10:34:13 +00:00
{
2020-12-25 11:22:48 +00:00
if (word_end >= end)
return Hash::shingleHash<CaseInsensitive>(-1ULL, data, size);
ExtractStringImpl::readOneUTF8Code(word_end, end);
}
Int64 finger_vec[64] = {};
while (word_end < end)
2019-11-06 10:34:13 +00:00
{
2020-12-25 11:22:48 +00:00
ExtractStringImpl::readOneUTF8Code(word_start, word_end);
ExtractStringImpl::readOneUTF8Code(word_end, end);
size_t length = word_end - word_start;
UInt64 hash_value = Hash::shingleHash<CaseInsensitive>(-1ULL, word_start, length);
updateFingerVector(finger_vec, hash_value);
2019-11-06 10:34:13 +00:00
}
2020-12-25 11:22:48 +00:00
return getSimHash(finger_vec);
2019-11-06 10:34:13 +00:00
}
2020-12-24 15:49:05 +00:00
// SimHash word shingle calculate function: String -> UInt64
2019-11-06 10:34:13 +00:00
// this function extracting n word shingle from input string, and maintain a 64-dimensions vector as well
// for each word shingle, calculate a 64 bit hash value, and update the vector according the hash value
// finally return a 64 bit value(UInt64), i'th bit is 1 means vector[i] > 0, otherwise, vector[i] < 0
//
// word shingle hash value calculate:
// 1. at the first, extracts N word shingles and calculate N hash values, store into an array, use this N hash values
// to calculate the first word shingle hash value
2020-12-24 10:48:10 +00:00
// 2. next, we extract one word each time, and calculate a new hash value of the new word,then use the latest N hash
// values to calculate the next word shingle hash value
2020-12-25 11:22:48 +00:00
2020-12-25 14:18:41 +00:00
static ALWAYS_INLINE inline UInt64 wordShingleHash(const UInt8 * data, size_t size, size_t shingle_size)
2020-12-25 11:22:48 +00:00
{
const UInt8 * start = data;
const UInt8 * end = data + size;
// A 64 bit vector initialized to zero.
Int64 finger_vec[64] = {};
// An array to store N words.
2020-12-25 16:04:55 +00:00
std::vector<BytesRef> words;
2020-12-25 11:22:48 +00:00
words.reserve(shingle_size);
// get first word shingle
while (start < end && words.size() < shingle_size)
{
2020-12-29 10:16:22 +00:00
const UInt8 * word_start = ExtractStringImpl::readOneWord(start, end);
2020-12-25 11:22:48 +00:00
size_t length = start - word_start;
2020-12-25 17:58:44 +00:00
if (length >= min_word_size)
2020-12-25 16:11:05 +00:00
words.emplace_back(BytesRef{word_start, length});
2020-12-25 11:22:48 +00:00
}
2020-12-27 09:54:59 +00:00
if (words.empty())
return 0;
2020-12-25 11:22:48 +00:00
UInt64 hash_value = Hash::shingleHash<CaseInsensitive>(words);
updateFingerVector(finger_vec, hash_value);
size_t offset = 0;
while (start < end)
{
2020-12-29 10:16:22 +00:00
const UInt8 * word_start = ExtractStringImpl::readOneWord(start, end);
2020-12-25 11:22:48 +00:00
size_t length = start - word_start;
2020-12-25 17:58:44 +00:00
if (length < min_word_size)
2020-12-25 11:22:48 +00:00
continue;
// we need to store the new word hash value to the oldest location.
2023-07-25 04:10:04 +00:00
// for example, N = 5, array |a0|a1|a2|a3|a4|, now, a0 is the oldest location,
// so we need to store new word hash into location of a0, then this array become
2020-12-25 11:22:48 +00:00
// |a5|a1|a2|a3|a4|, next time, a1 become the oldest location, we need to store new
// word hash value into location of a1, then array become |a5|a6|a2|a3|a4|
2020-12-25 16:04:55 +00:00
words[offset] = BytesRef{word_start, length};
2020-12-25 11:22:48 +00:00
++offset;
if (offset >= shingle_size)
offset = 0;
// according to the word hash storation way, in order to not lose the word shingle's
// sequence information, when calculation word shingle hash value, we need provide the offset
// information, which is the offset of the first word's hash value of the word shingle
hash_value = Hash::shingleHash<CaseInsensitive>(words, offset);
updateFingerVector(finger_vec, hash_value);
}
return getSimHash(finger_vec);
}
2020-12-25 14:18:41 +00:00
static void apply(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, size_t shingle_size, PaddedPODArray<UInt64> & res)
2019-11-06 10:34:13 +00:00
{
for (size_t i = 0; i < offsets.size(); ++i)
{
2020-12-25 14:18:41 +00:00
const UInt8 * one_data = &data[offsets[i - 1]];
2019-11-06 10:34:13 +00:00
const size_t data_size = offsets[i] - offsets[i - 1] - 1;
2020-12-24 15:49:05 +00:00
if constexpr (Ngram)
2019-11-06 10:34:13 +00:00
{
2020-12-24 15:49:05 +00:00
if constexpr (!UTF8)
2020-12-25 14:18:41 +00:00
res[i] = ngramHashASCII(one_data, data_size, shingle_size);
2019-11-06 10:34:13 +00:00
else
2020-12-25 14:18:41 +00:00
res[i] = ngramHashUTF8(one_data, data_size, shingle_size);
2019-11-06 10:34:13 +00:00
}
2020-06-10 15:02:58 +00:00
else
2020-12-24 15:49:05 +00:00
{
2020-12-25 14:18:41 +00:00
res[i] = wordShingleHash(one_data, data_size, shingle_size);
2020-12-24 15:49:05 +00:00
}
2019-11-06 10:34:13 +00:00
}
}
};
2020-12-24 15:49:05 +00:00
// MinHash: String -> Tuple(UInt64, UInt64)
// for each string, we extract ngram or word shingle,
// for each ngram or word shingle, calculate a hash value,
// then we take the K minimum hash values to calculate a hashsum,
// and take the K maximum hash values to calculate another hashsum,
// return this two hashsum: Tuple(hashsum1, hashsum2)
2020-06-10 15:02:58 +00:00
//
// UTF8: means ASCII or UTF8, these two parameters CodePoint and UTF8 can only be (UInt8, false) or (UInt32, true)
// Ngram: means ngram(true) or words shingles(false)
// CaseInsensitive: means should we consider about letter case or not
2020-12-25 14:18:41 +00:00
template <bool UTF8, bool Ngram, bool CaseInsensitive>
2020-12-24 15:49:05 +00:00
struct MinHashImpl
2019-11-06 10:34:13 +00:00
{
2020-12-25 18:03:08 +00:00
static constexpr size_t min_word_size = 4;
2020-12-25 17:58:44 +00:00
2020-12-29 10:16:22 +00:00
template <typename Comp>
2020-12-25 14:18:41 +00:00
struct Heap
2019-11-06 10:34:13 +00:00
{
2020-12-25 16:04:55 +00:00
void update(UInt64 hash, BytesRef ref, size_t limit)
2020-12-25 14:18:41 +00:00
{
if (values.contains(hash))
2020-12-25 14:18:41 +00:00
return;
values[hash] = ref;
2019-11-06 10:34:13 +00:00
2020-12-25 14:18:41 +00:00
if (values.size() > limit)
values.erase(values.begin());
}
UInt64 getHash()
{
if (values.empty())
return 0;
2020-12-25 16:11:05 +00:00
UInt64 res = 0;
for (auto it = values.begin(); it != values.end(); ++it)
2020-12-25 14:18:41 +00:00
res = CityHash_v1_0_2::Hash128to64(CityHash_v1_0_2::uint128(res, it->first));
return res;
}
void fill(ColumnTuple & strings)
2019-11-06 10:34:13 +00:00
{
2020-12-25 14:18:41 +00:00
auto it = values.begin();
2020-12-25 17:30:44 +00:00
for (size_t i = 0; i < strings.tupleSize(); ++i)
2019-11-06 10:34:13 +00:00
{
2020-12-25 14:18:41 +00:00
auto & col_string = static_cast<ColumnString &>(strings.getColumn(i));
if (it != values.end())
{
2020-12-25 16:56:51 +00:00
col_string.insertData(reinterpret_cast<const char *>(it->second.data), it->second.size);
2020-12-25 14:18:41 +00:00
++it;
}
else
col_string.insertDefault();
2019-11-06 10:34:13 +00:00
}
2020-12-25 14:18:41 +00:00
}
2020-12-25 16:04:55 +00:00
std::map<UInt64, BytesRef, Comp> values;
2020-12-25 14:18:41 +00:00
};
Activated a bunch of LLVM 12/13/14 clang-tidy warnings Omitted new checks which produce too many matches or which are controversial (e.g. readability-identifier-length). New checks: - misc-misleading-bidirectional + misc-misleading-identifier Detects potential attack as described in the Trojan Source attack - modernize-macro-to-enum Replaces groups of adjacent macros with an unscoped anonymous enum - modernize-shrink-to-fit Replace copy and swap tricks on shrinkable containers with the shrink_to_fit() method call - modernize-use-transparent-functors Prefer transparent functors to non-transparent ones - modernize-use-uncaught-exceptions This check will warn on calls to std::uncaught_exception and replace them with calls to std::uncaught_exceptions (uncaught_exception was deprecated with C++17) - performance-no-int-to-ptr Diagnoses every integer to pointer cast - readability-duplicate-include Looks for duplicate includes and removes them - readability-redundant-preprocessor Finds potentially redundant preprocessor directives - bugprone-lambda-function-name Checks for attempts to get the name of a function from within a lambda expression - bugprone-redundant-branch-condition Finds condition variables in nested if statements that were also checked in the outer if statement and were not changed - bugprone-shared-ptr-array-mismatch Finds initializations of C++ shared pointers to non-array type that are initialized with an array - bugprone-stringview-nullptr Checks for various ways that the const CharT* constructor of std::basic_string_view can be passed a null argument and replaces them with the default constructor in most cases - bugprone-suspicious-memory-comparison Finds potentially incorrect calls to memcmp() based on properties of the arguments
2022-04-30 18:11:06 +00:00
using MaxHeap = Heap<std::less<>>;
using MinHeap = Heap<std::greater<>>;
2020-12-25 14:18:41 +00:00
2020-12-25 16:55:15 +00:00
static ALWAYS_INLINE inline void ngramHashASCII(
2020-12-25 14:18:41 +00:00
MinHeap & min_heap,
MaxHeap & max_heap,
const UInt8 * data,
size_t size,
size_t shingle_size,
size_t heap_size)
{
if (size < shingle_size)
2020-12-25 16:55:15 +00:00
{
UInt64 hash_value = Hash::shingleHash<CaseInsensitive>(-1ULL, data, size);
min_heap.update(hash_value, BytesRef{data, size}, heap_size);
max_heap.update(hash_value, BytesRef{data, size}, heap_size);
return;
}
2020-12-25 14:18:41 +00:00
const UInt8 * end = data + size;
for (const UInt8 * pos = data; pos + shingle_size <= end; ++pos)
{
UInt64 hash_value = Hash::shingleHash<CaseInsensitive>(-1ULL, pos, shingle_size);
// insert the new hash value into array used to store K minimum value
// and K maximum value
2020-12-25 16:04:55 +00:00
min_heap.update(hash_value, BytesRef{pos, shingle_size}, heap_size);
max_heap.update(hash_value, BytesRef{pos, shingle_size}, heap_size);
2020-12-25 14:18:41 +00:00
}
}
2020-12-25 16:55:15 +00:00
static ALWAYS_INLINE inline void ngramHashUTF8(
2020-12-25 14:18:41 +00:00
MinHeap & min_heap,
MaxHeap & max_heap,
const UInt8 * data,
size_t size,
size_t shingle_size,
size_t heap_size)
{
const UInt8 * start = data;
const UInt8 * end = data + size;
const UInt8 * word_start = start;
const UInt8 * word_end = start;
for (size_t i = 0; i < shingle_size; ++i)
{
if (word_end >= end)
2020-12-25 16:55:15 +00:00
{
auto hash_value = Hash::shingleHash<CaseInsensitive>(-1ULL, data, size);
min_heap.update(hash_value, BytesRef{data, size}, heap_size);
max_heap.update(hash_value, BytesRef{data, size}, heap_size);
return;
}
2020-12-25 14:18:41 +00:00
ExtractStringImpl::readOneUTF8Code(word_end, end);
}
while (word_end < end)
{
ExtractStringImpl::readOneUTF8Code(word_start, word_end);
ExtractStringImpl::readOneUTF8Code(word_end, end);
2019-11-06 10:34:13 +00:00
2020-12-25 14:18:41 +00:00
size_t length = word_end - word_start;
UInt64 hash_value = Hash::shingleHash<CaseInsensitive>(-1ULL, word_start, length);
2020-12-25 16:04:55 +00:00
min_heap.update(hash_value, BytesRef{word_start, length}, heap_size);
max_heap.update(hash_value, BytesRef{word_start, length}, heap_size);
2020-12-25 14:18:41 +00:00
}
2019-11-06 10:34:13 +00:00
}
2020-12-24 15:49:05 +00:00
// MinHash word shingle hash value calculate function: String ->Tuple(UInt64, UInt64)
// for each word shingle, we calculate a hash value, but in fact, we just maintain the
// K minimum and K maximum hash value
2020-12-25 16:55:15 +00:00
static ALWAYS_INLINE inline void wordShingleHash(
2020-12-25 14:18:41 +00:00
MinHeap & min_heap,
MaxHeap & max_heap,
const UInt8 * data,
2020-05-22 13:23:49 +00:00
size_t size,
2020-12-25 14:18:41 +00:00
size_t shingle_size,
size_t heap_size)
2019-11-06 10:34:13 +00:00
{
2020-12-25 14:18:41 +00:00
const UInt8 * start = data;
const UInt8 * end = data + size;
// An array to store N words.
2020-12-25 16:04:55 +00:00
std::vector<BytesRef> words;
2020-12-25 14:18:41 +00:00
words.reserve(shingle_size);
// get first word shingle
while (start < end && words.size() < shingle_size)
2019-11-06 10:34:13 +00:00
{
2020-12-29 10:16:22 +00:00
const UInt8 * word_start = ExtractStringImpl::readOneWord(start, end);
2020-12-25 14:18:41 +00:00
size_t length = start - word_start;
2020-12-25 17:58:44 +00:00
if (length >= min_word_size)
2020-12-25 17:00:42 +00:00
words.emplace_back(BytesRef{word_start, length});
2019-11-06 10:34:13 +00:00
}
2020-12-27 09:54:59 +00:00
if (words.empty())
return;
2020-12-25 14:18:41 +00:00
UInt64 hash_value = Hash::shingleHash<CaseInsensitive>(words);
{
2020-12-25 16:04:55 +00:00
const UInt8 * shingle_start = words.front().data;
const UInt8 * shingle_end = words.back().data + words.back().size;
BytesRef ref{shingle_start, static_cast<size_t>(shingle_end - shingle_start)};
2020-12-25 14:18:41 +00:00
min_heap.update(hash_value, ref, heap_size);
max_heap.update(hash_value, ref, heap_size);
}
2019-11-06 10:34:13 +00:00
size_t offset = 0;
2020-12-25 14:18:41 +00:00
while (start < end)
2019-11-06 10:34:13 +00:00
{
2020-12-29 10:16:22 +00:00
const UInt8 * word_start = ExtractStringImpl::readOneWord(start, end);
2020-12-25 14:18:41 +00:00
size_t length = start - word_start;
2020-12-25 17:58:44 +00:00
if (length < min_word_size)
2020-12-25 14:18:41 +00:00
continue;
2020-12-25 16:04:55 +00:00
words[offset] = BytesRef{word_start, length};
const UInt8 * shingle_end = words[offset].data + length;
2020-12-25 14:18:41 +00:00
++offset;
if (offset >= shingle_size)
offset = 0;
2020-12-25 16:04:55 +00:00
const UInt8 * shingle_start = words[offset].data;
2020-12-25 14:18:41 +00:00
hash_value = Hash::shingleHash<CaseInsensitive>(words, offset);
2020-12-25 16:04:55 +00:00
BytesRef ref{shingle_start, static_cast<size_t>(shingle_end - shingle_start)};
2020-12-25 14:18:41 +00:00
min_heap.update(hash_value, ref, heap_size);
max_heap.update(hash_value, ref, heap_size);
}
2019-11-06 10:34:13 +00:00
}
2020-06-10 15:02:58 +00:00
static void apply(
2019-11-06 10:34:13 +00:00
const ColumnString::Chars & data,
const ColumnString::Offsets & offsets,
2020-12-25 14:18:41 +00:00
size_t shingle_size,
size_t heap_size,
PaddedPODArray<UInt64> * res1,
PaddedPODArray<UInt64> * res2,
ColumnTuple * res1_strings,
ColumnTuple * res2_strings)
2019-11-06 10:34:13 +00:00
{
2020-12-25 14:18:41 +00:00
MinHeap min_heap;
MaxHeap max_heap;
2019-11-06 10:34:13 +00:00
for (size_t i = 0; i < offsets.size(); ++i)
{
2020-12-25 16:11:05 +00:00
const UInt8 * one_data = &data[offsets[i - 1]];
2019-11-06 10:34:13 +00:00
const size_t data_size = offsets[i] - offsets[i - 1] - 1;
2020-12-24 15:49:05 +00:00
2020-12-25 14:18:41 +00:00
min_heap.values.clear();
max_heap.values.clear();
2020-12-24 15:49:05 +00:00
if constexpr (Ngram)
2019-11-06 10:34:13 +00:00
{
2020-12-24 15:49:05 +00:00
if constexpr (!UTF8)
2020-12-25 14:18:41 +00:00
ngramHashASCII(min_heap, max_heap, one_data, data_size, shingle_size, heap_size);
2019-11-06 10:34:13 +00:00
else
2020-12-25 14:18:41 +00:00
ngramHashUTF8(min_heap, max_heap, one_data, data_size, shingle_size, heap_size);
2019-11-06 10:34:13 +00:00
}
2020-06-10 15:02:58 +00:00
else
2020-12-24 15:49:05 +00:00
{
2020-12-25 14:18:41 +00:00
wordShingleHash(min_heap, max_heap, one_data, data_size, shingle_size, heap_size);
2020-12-24 15:49:05 +00:00
}
2020-12-25 14:18:41 +00:00
if (res1)
(*res1)[i] = min_heap.getHash();
if (res2)
(*res2)[i] = max_heap.getHash();
if (res1_strings)
min_heap.fill(*res1_strings);
if (res2_strings)
max_heap.fill(*res2_strings);
2019-11-06 10:34:13 +00:00
}
}
};
2020-12-24 15:49:05 +00:00
struct NameNgramSimHash
2019-11-06 10:34:13 +00:00
{
2020-12-24 15:49:05 +00:00
static constexpr auto name = "ngramSimHash";
2019-11-06 10:34:13 +00:00
};
2020-12-24 15:49:05 +00:00
struct NameNgramSimHashCaseInsensitive
2019-11-06 10:34:13 +00:00
{
2020-12-24 15:49:05 +00:00
static constexpr auto name = "ngramSimHashCaseInsensitive";
2019-11-06 10:34:13 +00:00
};
2020-12-24 15:49:05 +00:00
struct NameNgramSimHashUTF8
2019-11-06 10:34:13 +00:00
{
2020-12-24 15:49:05 +00:00
static constexpr auto name = "ngramSimHashUTF8";
2019-11-06 10:34:13 +00:00
};
2020-12-24 15:49:05 +00:00
struct NameNgramSimHashCaseInsensitiveUTF8
2019-11-06 10:34:13 +00:00
{
2020-12-24 15:49:05 +00:00
static constexpr auto name = "ngramSimHashCaseInsensitiveUTF8";
2019-11-06 10:34:13 +00:00
};
2020-12-24 15:49:05 +00:00
struct NameWordShingleSimHash
2019-11-06 10:34:13 +00:00
{
2020-12-24 15:49:05 +00:00
static constexpr auto name = "wordShingleSimHash";
2019-11-06 10:34:13 +00:00
};
2020-12-24 15:49:05 +00:00
struct NameWordShingleSimHashCaseInsensitive
2019-11-06 10:34:13 +00:00
{
2020-12-24 15:49:05 +00:00
static constexpr auto name = "wordShingleSimHashCaseInsensitive";
2019-11-06 10:34:13 +00:00
};
2020-12-24 15:49:05 +00:00
struct NameWordShingleSimHashUTF8
2019-11-06 10:34:13 +00:00
{
2020-12-24 15:49:05 +00:00
static constexpr auto name = "wordShingleSimHashUTF8";
2019-11-06 10:34:13 +00:00
};
2020-12-24 15:49:05 +00:00
struct NameWordShingleSimHashCaseInsensitiveUTF8
2019-11-06 10:34:13 +00:00
{
2020-12-24 15:49:05 +00:00
static constexpr auto name = "wordShingleSimHashCaseInsensitiveUTF8";
2019-11-06 10:34:13 +00:00
};
2020-12-24 15:49:05 +00:00
struct NameNgramMinHash
2019-11-06 10:34:13 +00:00
{
2020-12-24 15:49:05 +00:00
static constexpr auto name = "ngramMinHash";
2019-11-06 10:34:13 +00:00
};
2020-12-24 15:49:05 +00:00
struct NameNgramMinHashCaseInsensitive
2019-11-06 10:34:13 +00:00
{
2020-12-24 15:49:05 +00:00
static constexpr auto name = "ngramMinHashCaseInsensitive";
2019-11-06 10:34:13 +00:00
};
2020-12-24 15:49:05 +00:00
struct NameNgramMinHashUTF8
2019-11-06 10:34:13 +00:00
{
2020-12-24 15:49:05 +00:00
static constexpr auto name = "ngramMinHashUTF8";
2019-11-06 10:34:13 +00:00
};
2020-12-24 15:49:05 +00:00
struct NameNgramMinHashCaseInsensitiveUTF8
2019-11-06 10:34:13 +00:00
{
2020-12-24 15:49:05 +00:00
static constexpr auto name = "ngramMinHashCaseInsensitiveUTF8";
2019-11-06 10:34:13 +00:00
};
2020-12-24 15:49:05 +00:00
struct NameWordShingleMinHash
2019-11-06 10:34:13 +00:00
{
2020-12-24 15:49:05 +00:00
static constexpr auto name = "wordShingleMinHash";
2019-11-06 10:34:13 +00:00
};
2020-12-24 15:49:05 +00:00
struct NameWordShingleMinHashCaseInsensitive
2019-11-06 10:34:13 +00:00
{
2020-12-24 15:49:05 +00:00
static constexpr auto name = "wordShingleMinHashCaseInsensitive";
2019-11-06 10:34:13 +00:00
};
2020-12-24 15:49:05 +00:00
struct NameWordShingleMinHashUTF8
2019-11-06 10:34:13 +00:00
{
2020-12-24 15:49:05 +00:00
static constexpr auto name = "wordShingleMinHashUTF8";
2019-11-06 10:34:13 +00:00
};
2020-12-24 15:49:05 +00:00
struct NameWordShingleMinHashCaseInsensitiveUTF8
2019-11-06 10:34:13 +00:00
{
2020-12-24 15:49:05 +00:00
static constexpr auto name = "wordShingleMinHashCaseInsensitiveUTF8";
2019-11-06 10:34:13 +00:00
};
2020-12-25 14:18:41 +00:00
struct NameNgramMinHashArg
{
static constexpr auto name = "ngramMinHashArg";
};
struct NameNgramMinHashArgCaseInsensitive
{
static constexpr auto name = "ngramMinHashArgCaseInsensitive";
};
struct NameNgramMinHashArgUTF8
{
static constexpr auto name = "ngramMinHashArgUTF8";
};
struct NameNgramMinHashArgCaseInsensitiveUTF8
{
static constexpr auto name = "ngramMinHashArgCaseInsensitiveUTF8";
};
struct NameWordShingleMinHashArg
{
static constexpr auto name = "wordShingleMinHashArg";
};
struct NameWordShingleMinHashArgCaseInsensitive
{
static constexpr auto name = "wordShingleMinHashArgCaseInsensitive";
};
struct NameWordShingleMinHashArgUTF8
{
static constexpr auto name = "wordShingleMinHashArgUTF8";
};
struct NameWordShingleMinHashArgCaseInsensitiveUTF8
{
static constexpr auto name = "wordShingleMinHashArgCaseInsensitiveUTF8";
};
2020-12-24 15:49:05 +00:00
// SimHash
2020-12-25 14:18:41 +00:00
using FunctionNgramSimHash = FunctionsStringHash<SimHashImpl<false, true, false>, NameNgramSimHash, true>;
2019-11-06 10:34:13 +00:00
2020-12-24 15:49:05 +00:00
using FunctionNgramSimHashCaseInsensitive
2020-12-25 14:18:41 +00:00
= FunctionsStringHash<SimHashImpl<false, true, true>, NameNgramSimHashCaseInsensitive, true>;
2019-11-06 10:34:13 +00:00
2020-12-25 14:18:41 +00:00
using FunctionNgramSimHashUTF8 = FunctionsStringHash<SimHashImpl<true, true, false>, NameNgramSimHashUTF8, true>;
2019-11-06 10:34:13 +00:00
2020-12-24 15:49:05 +00:00
using FunctionNgramSimHashCaseInsensitiveUTF8
2020-12-25 14:18:41 +00:00
= FunctionsStringHash<SimHashImpl<true, true, true>, NameNgramSimHashCaseInsensitiveUTF8, true>;
2019-11-06 10:34:13 +00:00
2020-12-25 14:18:41 +00:00
using FunctionWordShingleSimHash = FunctionsStringHash<SimHashImpl<false, false, false>, NameWordShingleSimHash, true>;
2019-11-06 10:34:13 +00:00
2020-12-24 15:49:05 +00:00
using FunctionWordShingleSimHashCaseInsensitive
2020-12-25 14:18:41 +00:00
= FunctionsStringHash<SimHashImpl<false, false, true>, NameWordShingleSimHashCaseInsensitive, true>;
2019-11-06 10:34:13 +00:00
2020-12-25 14:18:41 +00:00
using FunctionWordShingleSimHashUTF8 = FunctionsStringHash<SimHashImpl<true, false, false>, NameWordShingleSimHashUTF8, true>;
2019-11-06 10:34:13 +00:00
2020-12-24 15:49:05 +00:00
using FunctionWordShingleSimHashCaseInsensitiveUTF8
2020-12-25 14:18:41 +00:00
= FunctionsStringHash<SimHashImpl<true, false, true>, NameWordShingleSimHashCaseInsensitiveUTF8, true>;
2019-11-06 10:34:13 +00:00
2020-12-24 15:49:05 +00:00
// MinHash
2020-12-25 14:18:41 +00:00
using FunctionNgramMinHash = FunctionsStringHash<MinHashImpl<false, true, false>, NameNgramMinHash, false>;
2019-11-06 10:34:13 +00:00
2020-12-24 15:49:05 +00:00
using FunctionNgramMinHashCaseInsensitive
2020-12-25 14:18:41 +00:00
= FunctionsStringHash<MinHashImpl<false, true, true>, NameNgramMinHashCaseInsensitive, false>;
2019-11-06 10:34:13 +00:00
2020-12-25 14:18:41 +00:00
using FunctionNgramMinHashUTF8 = FunctionsStringHash<MinHashImpl<true, true, false>, NameNgramMinHashUTF8, false>;
2019-11-06 10:34:13 +00:00
2020-12-24 15:49:05 +00:00
using FunctionNgramMinHashCaseInsensitiveUTF8
2020-12-25 14:18:41 +00:00
= FunctionsStringHash<MinHashImpl<true, true, true>, NameNgramMinHashCaseInsensitiveUTF8, false>;
2019-11-06 10:34:13 +00:00
2020-12-25 14:18:41 +00:00
using FunctionWordShingleMinHash = FunctionsStringHash<MinHashImpl<false, false, false>, NameWordShingleMinHash, false>;
2019-11-06 10:34:13 +00:00
2020-12-24 15:49:05 +00:00
using FunctionWordShingleMinHashCaseInsensitive
2020-12-25 14:18:41 +00:00
= FunctionsStringHash<MinHashImpl<false, false, true>, NameWordShingleMinHashCaseInsensitive, false>;
2019-11-06 10:34:13 +00:00
2020-12-24 15:49:05 +00:00
using FunctionWordShingleMinHashUTF8
2020-12-25 14:18:41 +00:00
= FunctionsStringHash<MinHashImpl<true, false, false>, NameWordShingleMinHashUTF8, false>;
2019-11-06 10:34:13 +00:00
2020-12-24 15:49:05 +00:00
using FunctionWordShingleMinHashCaseInsensitiveUTF8
2020-12-25 14:18:41 +00:00
= FunctionsStringHash<MinHashImpl<true, false, true>, NameWordShingleMinHashCaseInsensitiveUTF8, false>;
// MinHasArg
2020-12-25 15:19:30 +00:00
using FunctionNgramMinHashArg = FunctionsStringHash<MinHashImpl<false, true, false>, NameNgramMinHashArg, false, true>;
2020-12-25 14:18:41 +00:00
using FunctionNgramMinHashArgCaseInsensitive
2020-12-25 15:19:30 +00:00
= FunctionsStringHash<MinHashImpl<false, true, true>, NameNgramMinHashArgCaseInsensitive, false, true>;
2020-12-25 14:18:41 +00:00
2020-12-25 15:19:30 +00:00
using FunctionNgramMinHashArgUTF8 = FunctionsStringHash<MinHashImpl<true, true, false>, NameNgramMinHashArgUTF8, false, true>;
2020-12-25 14:18:41 +00:00
using FunctionNgramMinHashArgCaseInsensitiveUTF8
2020-12-25 15:19:30 +00:00
= FunctionsStringHash<MinHashImpl<true, true, true>, NameNgramMinHashArgCaseInsensitiveUTF8, false, true>;
2020-12-25 14:18:41 +00:00
2020-12-25 15:19:30 +00:00
using FunctionWordShingleMinHashArg = FunctionsStringHash<MinHashImpl<false, false, false>, NameWordShingleMinHashArg, false, true>;
2020-12-25 14:18:41 +00:00
using FunctionWordShingleMinHashArgCaseInsensitive
2020-12-25 15:19:30 +00:00
= FunctionsStringHash<MinHashImpl<false, false, true>, NameWordShingleMinHashArgCaseInsensitive, false, true>;
2020-12-25 14:18:41 +00:00
using FunctionWordShingleMinHashArgUTF8
2020-12-25 15:19:30 +00:00
= FunctionsStringHash<MinHashImpl<true, false, false>, NameWordShingleMinHashArgUTF8, false, true>;
2020-12-25 14:18:41 +00:00
using FunctionWordShingleMinHashArgCaseInsensitiveUTF8
2020-12-25 15:19:30 +00:00
= FunctionsStringHash<MinHashImpl<true, false, true>, NameWordShingleMinHashArgCaseInsensitiveUTF8, false, true>;
2019-11-06 10:34:13 +00:00
REGISTER_FUNCTION(StringHash)
2019-11-06 10:34:13 +00:00
{
2020-12-24 15:49:05 +00:00
factory.registerFunction<FunctionNgramSimHash>();
factory.registerFunction<FunctionNgramSimHashCaseInsensitive>();
factory.registerFunction<FunctionNgramSimHashUTF8>();
factory.registerFunction<FunctionNgramSimHashCaseInsensitiveUTF8>();
factory.registerFunction<FunctionWordShingleSimHash>();
factory.registerFunction<FunctionWordShingleSimHashCaseInsensitive>();
factory.registerFunction<FunctionWordShingleSimHashUTF8>();
factory.registerFunction<FunctionWordShingleSimHashCaseInsensitiveUTF8>();
factory.registerFunction<FunctionNgramMinHash>();
factory.registerFunction<FunctionNgramMinHashCaseInsensitive>();
factory.registerFunction<FunctionNgramMinHashUTF8>();
factory.registerFunction<FunctionNgramMinHashCaseInsensitiveUTF8>();
factory.registerFunction<FunctionWordShingleMinHash>();
factory.registerFunction<FunctionWordShingleMinHashCaseInsensitive>();
factory.registerFunction<FunctionWordShingleMinHashUTF8>();
factory.registerFunction<FunctionWordShingleMinHashCaseInsensitiveUTF8>();
2020-12-25 14:18:41 +00:00
factory.registerFunction<FunctionNgramMinHashArg>();
factory.registerFunction<FunctionNgramMinHashArgCaseInsensitive>();
factory.registerFunction<FunctionNgramMinHashArgUTF8>();
factory.registerFunction<FunctionNgramMinHashArgCaseInsensitiveUTF8>();
factory.registerFunction<FunctionWordShingleMinHashArg>();
factory.registerFunction<FunctionWordShingleMinHashArgCaseInsensitive>();
factory.registerFunction<FunctionWordShingleMinHashArgUTF8>();
factory.registerFunction<FunctionWordShingleMinHashArgCaseInsensitiveUTF8>();
2019-11-06 10:34:13 +00:00
}
}