mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-30 11:32:03 +00:00
Add function hammingDistance
This commit is contained in:
parent
5a6d37d8e7
commit
b7a3873265
@ -27,6 +27,9 @@ namespace ErrorCodes
|
||||
extern const int TOO_LARGE_STRING_SIZE;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
concept has_max_string_size = requires { T::max_string_size; };
|
||||
|
||||
template <typename Impl, typename Name>
|
||||
class FunctionsStringSimilarity : public IFunction
|
||||
{
|
||||
@ -68,10 +71,17 @@ public:
|
||||
{
|
||||
ResultType res{};
|
||||
const String & needle = col_needle_const->getValue<String>();
|
||||
if (needle.size() > Impl::max_string_size)
|
||||
if constexpr (has_max_string_size<Impl>)
|
||||
{
|
||||
throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "String size of needle is too big for function {}. "
|
||||
"Should be at most {}", getName(), Impl::max_string_size);
|
||||
if (needle.size() > Impl::max_string_size)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::TOO_LARGE_STRING_SIZE,
|
||||
"String size of needle is too big for function {}. "
|
||||
"Should be at most {}",
|
||||
getName(),
|
||||
Impl::max_string_size);
|
||||
}
|
||||
}
|
||||
Impl::constantConstant(col_haystack_const->getValue<String>(), needle, res);
|
||||
return result_type->createColumnConst(col_haystack_const->size(), toField(res));
|
||||
@ -88,10 +98,17 @@ public:
|
||||
if (col_haystack_vector && col_needle_const)
|
||||
{
|
||||
const String & needle = col_needle_const->getValue<String>();
|
||||
if (needle.size() > Impl::max_string_size)
|
||||
if constexpr (has_max_string_size<Impl>)
|
||||
{
|
||||
throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "String size of needle is too big for function {}. "
|
||||
"Should be at most {}", getName(), Impl::max_string_size);
|
||||
if (needle.size() > Impl::max_string_size)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::TOO_LARGE_STRING_SIZE,
|
||||
"String size of needle is too big for function {}. "
|
||||
"Should be at most {}",
|
||||
getName(),
|
||||
Impl::max_string_size);
|
||||
}
|
||||
}
|
||||
Impl::vectorConstant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), needle, vec_res);
|
||||
}
|
||||
@ -107,10 +124,17 @@ public:
|
||||
else if (col_haystack_const && col_needle_vector)
|
||||
{
|
||||
const String & haystack = col_haystack_const->getValue<String>();
|
||||
if (haystack.size() > Impl::max_string_size)
|
||||
if constexpr (has_max_string_size<Impl>)
|
||||
{
|
||||
throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "String size of haystack is too big for function {}. "
|
||||
"Should be at most {}", getName(), Impl::max_string_size);
|
||||
if (haystack.size() > Impl::max_string_size)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::TOO_LARGE_STRING_SIZE,
|
||||
"String size of haystack is too big for function {}. "
|
||||
"Should be at most {}",
|
||||
getName(),
|
||||
Impl::max_string_size);
|
||||
}
|
||||
}
|
||||
Impl::constantVector(haystack, col_needle_vector->getChars(), col_needle_vector->getOffsets(), vec_res);
|
||||
}
|
||||
|
106
src/Functions/stringHammingDistance.cpp
Normal file
106
src/Functions/stringHammingDistance.cpp
Normal file
@ -0,0 +1,106 @@
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionsStringSimilarity.h>
|
||||
|
||||
#ifdef __SSE4_2__
|
||||
# include <nmmintrin.h>
|
||||
#endif
|
||||
|
||||
namespace DB
|
||||
{
|
||||
struct StringHammingDistanceImpl
|
||||
{
|
||||
using ResultType = UInt64;
|
||||
|
||||
static void constantConstant(const std::string & haystack, const std::string & needle, UInt64 & res)
|
||||
{
|
||||
res = process(haystack.data(), haystack.size(), needle.data(), needle.size());
|
||||
}
|
||||
|
||||
static void vectorVector(
|
||||
const ColumnString::Chars & haystack_data,
|
||||
const ColumnString::Offsets & haystack_offsets,
|
||||
const ColumnString::Chars & needle_data,
|
||||
const ColumnString::Offsets & needle_offsets,
|
||||
PaddedPODArray<UInt64> & res)
|
||||
{
|
||||
size_t size = res.size();
|
||||
const char * haystack = reinterpret_cast<const char *>(haystack_data.data());
|
||||
const char * needle = reinterpret_cast<const char *>(needle_data.data());
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
res[i] = process(
|
||||
haystack + haystack_offsets[i - 1],
|
||||
haystack_offsets[i] - haystack_offsets[i - 1] - 1,
|
||||
needle + needle_offsets[i - 1],
|
||||
needle_offsets[i] - needle_offsets[i - 1] - 1);
|
||||
}
|
||||
}
|
||||
|
||||
static void constantVector(
|
||||
const std::string & haystack,
|
||||
const ColumnString::Chars & needle_data,
|
||||
const ColumnString::Offsets & needle_offsets,
|
||||
PaddedPODArray<UInt64> & res)
|
||||
{
|
||||
const char * haystack_data = haystack.data();
|
||||
size_t haystack_size = haystack.size();
|
||||
const char * needle = reinterpret_cast<const char *>(needle_data.data());
|
||||
size_t size = res.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
res[i] = process(haystack_data, haystack_size, needle + needle_offsets[i - 1], needle_offsets[i] - needle_offsets[i - 1] - 1);
|
||||
}
|
||||
}
|
||||
|
||||
static void vectorConstant(
|
||||
const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & needle, PaddedPODArray<UInt64> & res)
|
||||
{
|
||||
constantVector(needle, data, offsets, res);
|
||||
}
|
||||
|
||||
private:
|
||||
static UInt64 inline process(const char * haystack, size_t haystack_size, const char * needle, size_t needle_size)
|
||||
{
|
||||
UInt64 res = 0;
|
||||
const char * haystack_end = haystack + haystack_size;
|
||||
const char * needle_end = needle + needle_size;
|
||||
|
||||
#ifdef __SSE4_2__
|
||||
static constexpr auto mode = _SIDD_UBYTE_OPS | _SIDD_CMP_EQUAL_EACH | _SIDD_NEGATIVE_POLARITY;
|
||||
|
||||
const char * haystack_end16 = haystack + haystack_size / 16 * 16;
|
||||
const char * needle_end16 = needle + needle_size / 16 * 16;
|
||||
|
||||
for (; haystack < haystack_end16 && needle < needle_end16; haystack += 16, needle += 16)
|
||||
{
|
||||
__m128i s1 = _mm_loadu_si128(reinterpret_cast<const __m128i *>(haystack));
|
||||
__m128i s2 = _mm_loadu_si128(reinterpret_cast<const __m128i *>(needle));
|
||||
auto result_mask = _mm_cmpestrm(s1, 16, s2, 16, mode);
|
||||
const __m128i mask_hi = _mm_unpackhi_epi64(result_mask, result_mask);
|
||||
res += _mm_popcnt_u64(_mm_cvtsi128_si64(result_mask)) + _mm_popcnt_u64(_mm_cvtsi128_si64(mask_hi));
|
||||
}
|
||||
#endif
|
||||
for (; haystack != haystack_end && needle != needle_end; ++haystack, ++needle)
|
||||
res += *haystack != *needle;
|
||||
|
||||
res = res + (haystack_end - haystack) + (needle_end - needle);
|
||||
return res;
|
||||
}
|
||||
};
|
||||
|
||||
struct NameStringHammingDistance
|
||||
{
|
||||
static constexpr auto name = "hammingDistance";
|
||||
};
|
||||
|
||||
using FunctionStringHammingDistance = FunctionsStringSimilarity<StringHammingDistanceImpl, NameStringHammingDistance>;
|
||||
|
||||
REGISTER_FUNCTION(StringHammingDistance)
|
||||
{
|
||||
factory.registerFunction<FunctionStringHammingDistance>();
|
||||
}
|
||||
}
|
@ -0,0 +1,10 @@
|
||||
0
|
||||
1
|
||||
7
|
||||
7
|
||||
3
|
||||
6
|
||||
3
|
||||
6
|
||||
6
|
||||
3
|
17
tests/queries/0_stateless/02884_string_hamming_distance.sql
Normal file
17
tests/queries/0_stateless/02884_string_hamming_distance.sql
Normal file
@ -0,0 +1,17 @@
|
||||
select hammingDistance('abcd', 'abcd');
|
||||
drop table if exists t;
|
||||
create table t
|
||||
(
|
||||
s1 String,
|
||||
s2 String
|
||||
) engine = MergeTree order by s1;
|
||||
|
||||
insert into t values ('abcdefg', 'abcdef') ('abcdefg', 'bcdefg') ('abcdefg', '');
|
||||
|
||||
select hammingDistance(s1, s2) from t;
|
||||
|
||||
select hammingDistance('abc', s2) from t;
|
||||
|
||||
select hammingDistance(s2, 'def') from t;
|
||||
|
||||
drop table t;
|
Loading…
Reference in New Issue
Block a user