Refactor some code.

This commit is contained in:
Nikolai Kochetov 2020-12-21 14:39:17 +03:00
parent 654b8528b2
commit c3a99e21bd
2 changed files with 108 additions and 84 deletions

View File

@ -18,52 +18,94 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
struct Hash
{
static ALWAYS_INLINE inline UInt64 ngramASCIIHash(const UInt8 * code_points)
{
return intHashCRC32(unalignedLoad<UInt32>(code_points));
#ifdef __SSE4_2__
return _mm_crc32_u64(-1ULL, unalignedLoad<UInt32>(code_points));
#else
throw Exception("ngramASCIIHash is not implemented without sse4.2 support", ErrorCodes::NOT_IMPLEMENTED);
#endif
}
static ALWAYS_INLINE inline UInt64 ngramUTF8Hash(const UInt32 * code_points)
{
UInt64 combined = (static_cast<UInt64>(code_points[0]) << 32) | code_points[1];
#ifdef __SSE4_2__
return _mm_crc32_u64(code_points[2], combined);
#else
return (intHashCRC32(combined) ^ intHashCRC32(code_points[2]));
#endif
}
static ALWAYS_INLINE inline UInt64 wordShinglesHash(const UInt64 * hashes, size_t size, size_t offset)
{
UInt64 crc = -1ULL;
#ifdef __SSE4_2__
for (size_t i = offset; i < size; ++i)
crc = _mm_crc32_u64(crc, hashes[i]);
for (size_t i = 0; i < offset; ++i)
crc = _mm_crc32_u64(crc, hashes[i]);
crc = _mm_crc32_u64(crc, code_points[0]);
crc = _mm_crc32_u64(crc, code_points[1]);
crc = _mm_crc32_u64(crc, code_points[2]);
#else
for (size_t i = offset; i < size; ++i)
crc = intHashCRC32(crc) ^ intHashCRC32(hashes[i]);
for (size_t i = 0; i < offset; ++i)
crc = intHashCRC32(crc) ^ intHashCRC32(hashes[i]);
throw Exception("ngramUTF8Hash is not implemented without sse4.2 support", ErrorCodes::NOT_IMPLEMENTED);
#endif
return crc;
}
template <typename CodePoint>
static ALWAYS_INLINE inline UInt64 hashSum(const CodePoint * hashes, size_t K)
static ALWAYS_INLINE inline UInt64 wordShinglesHash(const UInt32 * hashes, size_t size, size_t offset)
{
UInt64 crc = -1ULL;
UInt64 crc1 = -1ULL;
UInt64 crc2 = -1ULL;
#ifdef __SSE4_2__
for (size_t i = 0; i < K; ++i)
crc = _mm_crc32_u64(crc, hashes[i]);
for (size_t i = offset; i < size; i += 2)
crc1 = _mm_crc32_u64(crc1, hashes[i]);
for (size_t i = offset + 1; i < size; i += 2)
crc2 = _mm_crc32_u64(crc2, hashes[i]);
if ((size - offset) & 1)
{
for (size_t i = 0; i < offset; i += 2)
crc2 = _mm_crc32_u64(crc2, hashes[i]);
for (size_t i = 1; i < offset; i += 2)
crc1 = _mm_crc32_u64(crc1, hashes[i]);
}
else
{
for (size_t i = 0; i < offset; i += 2)
crc1 = _mm_crc32_u64(crc1, hashes[i]);
for (size_t i = 1; i < offset; i += 2)
crc2 = _mm_crc32_u64(crc2, hashes[i]);
}
#else
for (size_t i = 0; i < K; ++i)
crc = intHashCRC32(crc) ^ intHashCRC32(hashes[i]);
throw Exception("wordShinglesHash is not implemented without sse4.2 support", ErrorCodes::NOT_IMPLEMENTED);
#endif
return crc;
return crc1 | (crc2 << 32u);
}
static ALWAYS_INLINE inline UInt64 hashSum(const UInt8 * hashes, size_t K)
{
UInt64 crc1 = -1ULL;
UInt64 crc2 = -1ULL;
#ifdef __SSE4_2__
for (size_t i = 0; i < K; i += 2)
crc1 = _mm_crc32_u8(crc1, hashes[i]);
for (size_t i = 1; i < K; i += 2)
crc2 = _mm_crc32_u8(crc2, hashes[i]);
#else
throw Exception("hashSum is not implemented without sse4.2 support", ErrorCodes::NOT_IMPLEMENTED);
#endif
return crc1 | (crc2 << 32u);
}
static ALWAYS_INLINE inline UInt64 hashSum(const UInt32 * hashes, size_t K)
{
UInt64 crc1 = -1ULL;
UInt64 crc2 = -1ULL;
#ifdef __SSE4_2__
for (size_t i = 0; i < K; i += 2)
crc1 = _mm_crc32_u32(crc1, hashes[i]);
for (size_t i = 1; i < K; i += 2)
crc2 = _mm_crc32_u32(crc2, hashes[i]);
#else
throw Exception("hashSum is not implemented without sse4.2 support", ErrorCodes::NOT_IMPLEMENTED);
#endif
return crc1 | (crc2 << 32u);
}
};
@ -76,7 +118,6 @@ struct Hash
template <size_t N, typename CodePoint, bool UTF8, bool Ngram, bool CaseInsensitive>
struct SimhashImpl
{
using ResultType = UInt64;
using StrOp = ExtractStringImpl<N, CaseInsensitive>;
// we made an assumption that the size of one word cann't exceed 128, which may not true
// if some word's size exceed 128, it would be cut up to several word
@ -204,25 +245,6 @@ struct SimhashImpl
return res_bit.to_ullong();
}
template <typename CalcFunc, typename... Args>
static ALWAYS_INLINE inline auto dispatch(CalcFunc calc_func, Args &&... args)
{
if constexpr (Ngram)
{
if constexpr (!UTF8)
return calc_func(std::forward<Args>(args)..., StrOp::readASCIICodePoints, Hash::ngramASCIIHash);
else
return calc_func(std::forward<Args>(args)..., StrOp::readUTF8CodePoints, Hash::ngramUTF8Hash);
}
else
{
if constexpr (!UTF8)
return calc_func(std::forward<Args>(args)..., StrOp::readOneASCIIWord, Hash::wordShinglesHash);
else
return calc_func(std::forward<Args>(args)..., StrOp::readOneUTF8Word, Hash::wordShinglesHash);
}
}
static void apply(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, PaddedPODArray<UInt64> & res)
{
for (size_t i = 0; i < offsets.size(); ++i)
@ -232,9 +254,19 @@ struct SimhashImpl
if (data_size <= max_string_size)
{
if constexpr (Ngram)
res[i] = dispatch(ngramCalculateHashValue, one_data, data_size);
{
if constexpr (!UTF8)
res[i] = ngramCalculateHashValue(one_data, data_size, StrOp::readASCIICodePoints, Hash::ngramASCIIHash);
else
res[i] = ngramCalculateHashValue(one_data, data_size, StrOp::readUTF8CodePoints, Hash::ngramUTF8Hash);
}
else
res[i] = dispatch(wordShinglesCalculateHashValue, one_data, data_size);
{
if constexpr (!UTF8)
res[i] = wordShinglesCalculateHashValue(one_data, data_size, StrOp::readOneASCIIWord, Hash::wordShinglesHash);
else
res[i] = wordShinglesCalculateHashValue(one_data, data_size, StrOp::readOneUTF8Word, Hash::wordShinglesHash);
}
}
else
res[i] = -1ull;
@ -289,7 +321,6 @@ struct MinhashImpl
using Greater = std::greater<size_t>;
using MaxHeap = FixedHeap<std::less<size_t>, K, -1ULL>;
using MinHeap = FixedHeap<std::greater<size_t>, K, 0>;
using ResultType = UInt64;
using StrOp = ExtractStringImpl<N, CaseInsensitive>;
static constexpr size_t max_string_size = 1u << 15;
static constexpr size_t simultaneously_codepoints_num = StrOp::default_padding + N - 1;
@ -383,25 +414,6 @@ struct MinhashImpl
return std::make_tuple(res1, res2);
}
template <typename CalcFunc, typename... Args>
static ALWAYS_INLINE inline auto dispatch(CalcFunc calc_func, Args &&... args)
{
if constexpr (Ngram)
{
if constexpr (!UTF8)
return calc_func(std::forward<Args>(args)..., StrOp::readASCIICodePoints, Hash::ngramASCIIHash);
else
return calc_func(std::forward<Args>(args)..., StrOp::readUTF8CodePoints, Hash::ngramUTF8Hash);
}
else
{
if constexpr (!UTF8)
return calc_func(std::forward<Args>(args)..., StrOp::readOneASCIIWord, Hash::wordShinglesHash);
else
return calc_func(std::forward<Args>(args)..., StrOp::readOneUTF8Word, Hash::wordShinglesHash);
}
}
static void apply(
const ColumnString::Chars & data,
const ColumnString::Offsets & offsets,
@ -415,9 +427,19 @@ struct MinhashImpl
if (data_size <= max_string_size)
{
if constexpr (Ngram)
std::tie(res1[i], res2[i]) = dispatch(ngramCalculateHashValue, one_data, data_size);
{
if constexpr (!UTF8)
std::tie(res1[i], res2[i]) = ngramCalculateHashValue(one_data, data_size, StrOp::readASCIICodePoints, Hash::ngramASCIIHash);
else
std::tie(res1[i], res2[i]) = ngramCalculateHashValue(one_data, data_size, StrOp::readUTF8CodePoints, Hash::ngramUTF8Hash);
}
else
std::tie(res1[i], res2[i]) = dispatch(wordShinglesCalculateHashValue, one_data, data_size);
{
if constexpr (!UTF8)
std::tie(res1[i], res2[i]) = wordShinglesCalculateHashValue(one_data, data_size, StrOp::readOneASCIIWord, Hash::wordShinglesHash);
else
std::tie(res1[i], res2[i]) = wordShinglesCalculateHashValue(one_data, data_size, StrOp::readOneUTF8Word, Hash::wordShinglesHash);
}
}
else
std::tie(res1[i], res2[i]) = std::make_tuple(-1ull, -1ull);

View File

@ -35,12 +35,14 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!isString(arguments[0]))
throw Exception(
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Function {} expect single String argument, got {}", getName(), arguments[0]->getName());
auto type = std::make_shared<DataTypeUInt64>();
if constexpr (is_simhash)
return std::make_shared<DataTypeNumber<typename Impl::ResultType>>();
auto element = DataTypeFactory::instance().get("UInt64");
return std::make_shared<DataTypeTuple>(DataTypes{element, element});
return type;
return std::make_shared<DataTypeTuple>(DataTypes{type, type});
}
bool useDefaultImplementationForConstants() const override { return true; }
@ -48,24 +50,24 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override
{
const ColumnPtr & column = arguments[0].column;
using ResultType = typename Impl::ResultType;
if constexpr (is_simhash)
{
// non const string, const case is handled by useDefaultImplementationForConstants.
auto col_res = ColumnVector<ResultType>::create();
typename ColumnVector<ResultType>::Container & vec_res = col_res->getData();
auto col_res = ColumnVector<UInt64>::create();
auto & vec_res = col_res->getData();
vec_res.resize(column->size());
const ColumnString * col_str_vector = checkAndGetColumn<ColumnString>(&*column);
Impl::apply(col_str_vector->getChars(), col_str_vector->getOffsets(), vec_res);
return std::move(col_res);
return col_res;
}
else // Min hash
{
// non const string
auto col_h1 = ColumnVector<ResultType>::create();
auto col_h2 = ColumnVector<ResultType>::create();
typename ColumnVector<ResultType>::Container & vec_h1 = col_h1->getData();
typename ColumnVector<ResultType>::Container & vec_h2 = col_h2->getData();
auto col_h1 = ColumnVector<UInt64>::create();
auto col_h2 = ColumnVector<UInt64>::create();
auto & vec_h1 = col_h1->getData();
auto & vec_h2 = col_h2->getData();
vec_h1.resize(column->size());
vec_h2.resize(column->size());
const ColumnString * col_str_vector = checkAndGetColumn<ColumnString>(&*column);