add more functions

This commit is contained in:
flynn 2023-09-23 17:17:35 +00:00
parent 6ca4a286f2
commit 661ba85865
7 changed files with 322 additions and 121 deletions

View File

@ -705,3 +705,56 @@ Result:
│ 1 │
└──────────────────────────────────┘
```
- Alias: mismatches
## byteJaccardIndex
Calculates the jaccard similarity index between two byte strings.
**Syntax**
```sql
byteJaccardIndex(haystack, needle)
```
**Examples**
``` sql
SELECT byteJaccardIndex('clickhouse', 'mouse');
```
Result:
``` text
┌─byteJaccardIndex('clickhouse', 'mouse')─┐
│ 0.4 │
└─────────────────────────────────────────┘
```
## byteEditDistance
Calcultes the edit distance between two byte strings.
**Syntax**
```sql
byteEidtDistance(haystack, needle)
```
**Examples**
``` sql
SELECT byteEditDistance('clickhouse', 'mouse');
```
Result:
``` text
┌─byteEditDistance('clickhouse', 'mouse')─┐
│ 6 │
└─────────────────────────────────────────┘
```
- Alias: byteLevenshteinDistance

View File

@ -0,0 +1,218 @@
#include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsStringSimilarity.h>
#include <Common/PODArray.h>
#ifdef __SSE4_2__
# include <nmmintrin.h>
#endif
namespace DB
{
namespace ErrorCodes
{
extern const int TOO_LARGE_STRING_SIZE;
}
template <typename Op>
struct FunctionDistanceImpl
{
using ResultType = typename Op::ResultType;
static void constantConstant(const std::string & haystack, const std::string & needle, ResultType & res)
{
res = Op::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<ResultType> & 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] = Op::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<ResultType> & 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]
= Op::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<ResultType> & res)
{
constantVector(needle, data, offsets, res);
}
};
struct ByteHammingDistanceImpl
{
using ResultType = UInt64;
static ResultType inline process(
const char * __restrict haystack, size_t haystack_size, const char * __restrict 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 ByteJaccardIndexImpl
{
using ResultType = Float64;
static ResultType inline process(
const char * __restrict haystack, size_t haystack_size, const char * __restrict needle, size_t needle_size)
{
if (haystack_size == 0 || needle_size == 0)
return 0;
std::unordered_set<char> haystack_set(haystack, haystack + haystack_size);
std::unordered_set<char> needle_set(needle, needle + needle_size);
size_t intersect = 0;
for (auto elem : haystack_set)
{
intersect += needle_set.contains(elem);
}
return static_cast<Float64>(intersect) / (haystack_set.size() + needle_set.size() - intersect);
}
};
struct ByteEditDistanceImpl
{
using ResultType = UInt64;
static constexpr size_t max_string_size = 1u << 16;
static ResultType inline process(
const char * __restrict haystack, size_t haystack_size, const char * __restrict needle, size_t needle_size)
{
if (haystack_size == 0 || needle_size == 0)
return haystack_size + needle_size;
/// Safety threshold against DoS, since we use two array to calculate the distance.
if (haystack_size > max_string_size || needle_size > max_string_size)
throw Exception(
ErrorCodes::TOO_LARGE_STRING_SIZE,
"The string size is too big for function byteEditDistance. "
"Should be at most {}",
max_string_size);
PaddedPODArray<ResultType> distances0(haystack_size + 1, 0);
PaddedPODArray<ResultType> distances1(haystack_size + 1, 0);
ResultType substitution = 0;
ResultType insertion = 0;
ResultType deletion = 0;
for (size_t i = 0; i <= haystack_size; ++i)
distances0[i] = i;
for (size_t pos_needle = 0; pos_needle < needle_size; ++pos_needle)
{
distances1[0] = pos_needle + 1;
for (size_t pos_haystack = 0; pos_haystack < haystack_size; pos_haystack++)
{
deletion = distances0[pos_haystack + 1] + 1;
insertion = distances1[pos_haystack] + 1;
substitution = distances0[pos_haystack];
if (*(needle + pos_needle) != *(haystack + pos_haystack))
substitution += 1;
distances1[pos_haystack + 1] = std::min(deletion, std::min(substitution, insertion));
}
distances0.swap(distances1);
}
return distances0[haystack_size];
}
};
struct NameByteHammingDistance
{
static constexpr auto name = "byteHammingDistance";
};
struct NameByteJaccardIndex
{
static constexpr auto name = "byteJaccardIndex";
};
struct NameByteEditDistance
{
static constexpr auto name = "byteEditDistance";
};
using FunctionByteHammingDistance
= FunctionsStringSimilarity<FunctionDistanceImpl<ByteHammingDistanceImpl>, NameByteHammingDistance>;
using FunctionByteJaccardIndex = FunctionsStringSimilarity<FunctionDistanceImpl<ByteJaccardIndexImpl>, NameByteJaccardIndex>;
using FunctionByteEditDistance = FunctionsStringSimilarity<FunctionDistanceImpl<ByteEditDistanceImpl>, NameByteEditDistance>;
REGISTER_FUNCTION(StringHammingDistance)
{
factory.registerFunction<FunctionByteHammingDistance>(
FunctionDocumentation{.description = R"(Calculates the hamming distance between two bytes strings.)"});
factory.registerAlias("mismatches", NameByteHammingDistance::name);
factory.registerFunction<FunctionByteJaccardIndex>(
FunctionDocumentation{.description = R"(Calculates the jaccard similarity index between two bytes strings.)"});
factory.registerFunction<FunctionByteEditDistance>(
FunctionDocumentation{.description = R"(Calculates the edit distance between two bytes strings.)"});
factory.registerAlias("byteLevenshteinDistance", NameByteEditDistance::name);
}
}

View File

@ -1,107 +0,0 @@
#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 ByteHammingDistanceImpl
{
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 NameByteHammingDistance
{
static constexpr auto name = "byteHammingDistance";
};
using FunctionByteHammingDistance = FunctionsStringSimilarity<ByteHammingDistanceImpl, NameByteHammingDistance>;
REGISTER_FUNCTION(StringHammingDistance)
{
factory.registerFunction<FunctionByteHammingDistance>(
FunctionDocumentation{.description = R"(Calculates the hamming distance between two bytes strings.)"});
}
}

View File

@ -1,10 +0,0 @@
0
1
7
7
3
6
3
6
6
3

View File

@ -0,0 +1,37 @@
0
1
7
7
10
3
6
3
10
6
6
3
10
1
7
7
10
3
6
3
10
6
6
3
10
0.8571428571428571
0.8571428571428571
0
0.4
1
1
7
6
1
1
7
6

View File

@ -6,12 +6,19 @@ create table t
s2 String
) engine = MergeTree order by s1;
insert into t values ('abcdefg', 'abcdef') ('abcdefg', 'bcdefg') ('abcdefg', '');
insert into t values ('abcdefg', 'abcdef') ('abcdefg', 'bcdefg') ('abcdefg', '') ('mouse', 'clickhouse');
select byteHammingDistance(s1, s2) from t;
select byteHammingDistance('abc', s2) from t;
select byteHammingDistance(s2, 'def') from t;
select mismatches(s1, s2) from t;
select mismatches('abc', s2) from t;
select mismatches(s2, 'def') from t;
select byteJaccardIndex(s1, s2) from t;
select byteEditDistance(s1, s2) from t;
select byteLevenshteinDistance(s1, s2) from t;
SELECT byteEditDistance(randomString(power(2, 17)), 'abc'); -- { serverError 131 }
drop table t;

View File

@ -1179,7 +1179,10 @@ builtins
byteSize
bytebase
bytesToCutForIPv
byteEditDistance
byteHammingDistance
byteJaccardIndex
byteLevenshteinDistance
cLoki
caConfig
cacheSessions