mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Refactor some code.
This commit is contained in:
parent
654b8528b2
commit
c3a99e21bd
@ -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);
|
||||
|
@ -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);
|
||||
|
Loading…
Reference in New Issue
Block a user