update name
This commit is contained in:
feng lv 2020-05-22 21:23:49 +08:00
parent 64207abd0a
commit 83c0807b43
12 changed files with 251 additions and 117 deletions

View File

@ -72,18 +72,18 @@ struct ExtractStringImpl
// read a ASCII word from pos to word // read a ASCII word from pos to word
// if the word size exceeds max_word_size, only read max_word_size byte // if the word size exceeds max_word_size, only read max_word_size byte
// in FuntionsStringHash, the default value of max_word_size is 128 // in FuntionsStringHash, the default value of max_word_size is 128
static ALWAYS_INLINE inline size_t readOneASCIIWord(UInt8 * word, const char *& pos, const char * end, const size_t & max_word_size) static ALWAYS_INLINE inline size_t readOneASCIIWord(UInt8 * word, const char *& pos, const char * end, size_t max_word_size)
{ {
// jump seperators // jump seperators
while (pos < end && !isAlphaNum(*pos)) while (pos < end && !isAlphaNumericASCII(*pos))
++pos; ++pos;
// word start from here // word start from here
const char * word_start = pos; const char * word_start = pos;
while (pos < end && isAlphaNum(*pos)) while (pos < end && isAlphaNumericASCII(*pos))
++pos; ++pos;
size_t word_size = (static_cast<size_t>(pos - word_start) <= max_word_size) ? pos - word_start : max_word_size; size_t word_size = std::min<size_t>(pos - word_start, max_word_size);
memcpy(word, word_start, word_size); memcpy(word, word_start, word_size);
if (CaseInsensitive) if (CaseInsensitive)
@ -107,7 +107,7 @@ struct ExtractStringImpl
// read one UTF8 word from pos to word // read one UTF8 word from pos to word
// also, we assume that one word size cann't exceed max_word_size with default value 128 // also, we assume that one word size cann't exceed max_word_size with default value 128
static ALWAYS_INLINE inline size_t readOneUTF8Word(UInt32 * word, const char *& pos, const char * end, const size_t & max_word_size) static ALWAYS_INLINE inline size_t readOneUTF8Word(UInt32 * word, const char *& pos, const char * end, size_t max_word_size)
{ {
// jump UTF8 seperator // jump UTF8 seperator
while (pos < end && isUTF8Sep(*pos)) while (pos < end && isUTF8Sep(*pos))
@ -122,7 +122,7 @@ struct ExtractStringImpl
} }
private: private:
static ALWAYS_INLINE inline bool isAlphaNum(const UInt8 c) static ALWAYS_INLINE inline bool isAlphaNumericASCII(const UInt8 c)
{ {
return (c >= 48 && c <= 57) || (c >= 65 && c <= 90) || (c >= 97 && c <= 122); return (c >= 48 && c <= 57) || (c >= 65 && c <= 90) || (c >= 97 && c <= 122);
} }
@ -134,7 +134,7 @@ private:
} }
// we use ASCII non-alphanum character as UTF8 seperator // we use ASCII non-alphanum character as UTF8 seperator
static ALWAYS_INLINE inline bool isUTF8Sep(const UInt8 c) { return c < 128 && !isAlphaNum(c); } static ALWAYS_INLINE inline bool isUTF8Sep(const UInt8 c) { return c < 128 && !isAlphaNumericASCII(c); }
// read one UTF8 character and return it // read one UTF8 character and return it
static ALWAYS_INLINE inline UInt32 readOneUTF8Code(const char *& pos, const char * end) static ALWAYS_INLINE inline UInt32 readOneUTF8Code(const char *& pos, const char * end)

View File

@ -31,47 +31,35 @@ struct Hash
#endif #endif
} }
static ALWAYS_INLINE inline UInt64 wordShinglesHash(const UInt64 * hashes, const size_t & size, const size_t & offset) static ALWAYS_INLINE inline UInt64 wordShinglesHash(const UInt64 * hashes, size_t size, size_t offset)
{ {
UInt64 res = 0; UInt64 crc = -1ULL;
UInt8 flag = 0; #ifdef __SSE4_2__
for (size_t i = offset; i < size; ++i) for (size_t i = offset; i < size; ++i)
{ crc = _mm_crc32_u64(crc, hashes[i]);
if (flag)
res &= intHashCRC32(hashes[i]);
else
res |= intHashCRC32(hashes[i]);
flag = (flag + 1) % 2;
}
for (size_t i = 0; i < offset; ++i) for (size_t i = 0; i < offset; ++i)
{ crc = _mm_crc32_u64(crc, hashes[i]);
if (flag) #else
res &= intHashCRC32(hashes[i]); for (size_t i = offset; i < size; ++i)
else crc = intHashCRC32(crc) ^ intHashCRC32(hashes[i]);
res |= intHashCRC32(hashes[i]); for (size_t i = 0; i < offset; ++i)
flag = (flag + 1) % 2; crc = intHashCRC32(crc) ^ intHashCRC32(hashes[i]);
} #endif
return res; return crc;
} }
template <typename CodePoint> template <typename CodePoint>
static ALWAYS_INLINE inline UInt64 hashSum(const CodePoint * hashes, const size_t & K) static ALWAYS_INLINE inline UInt64 hashSum(const CodePoint * hashes, size_t K)
{ {
UInt64 even = 0; UInt64 crc = -1ULL;
UInt64 odd = 0;
size_t i = 0;
for (; i + 1 < K; i += 2)
{
even |= intHashCRC32(hashes[i]);
odd |= intHashCRC32(hashes[i + 1]);
}
if (i < K)
even |= intHashCRC32(hashes[K - 1]);
#ifdef __SSE4_2__ #ifdef __SSE4_2__
return _mm_crc32_u64(even, odd); for (size_t i = 0; i < K; ++i)
crc = _mm_crc32_u64(crc, hashes[i]);
#else #else
return (intHashCRC32(even) ^ intHashCRC32(odd)); for (size_t i = 0; i < K; ++i)
crc = intHashCRC32(crc) ^ intHashCRC32(hashes[i]);
#endif #endif
return crc;
} }
}; };
@ -93,7 +81,7 @@ struct SimhashImpl
// finally return a 64 bit value(UInt64), i'th bit is 1 means vector[i] > 0, otherwise, vector[i] < 0 // finally return a 64 bit value(UInt64), i'th bit is 1 means vector[i] > 0, otherwise, vector[i] < 0
static ALWAYS_INLINE inline UInt64 ngramCalculateHashValue( static ALWAYS_INLINE inline UInt64 ngramCalculateHashValue(
const char * data, const char * data,
const size_t size, size_t size,
size_t (*read_code_points)(CodePoint *, const char *&, const char *), size_t (*read_code_points)(CodePoint *, const char *&, const char *),
UInt64 (*hash_functor)(const CodePoint *)) UInt64 (*hash_functor)(const CodePoint *))
{ {
@ -146,9 +134,9 @@ struct SimhashImpl
// values to caculate the next word shingle hash value // values to caculate the next word shingle hash value
static ALWAYS_INLINE inline UInt64 wordShinglesCalculateHashValue( static ALWAYS_INLINE inline UInt64 wordShinglesCalculateHashValue(
const char * data, const char * data,
const size_t size, size_t size,
size_t (*read_one_word)(CodePoint *, const char *&, const char *, const size_t &), size_t (*read_one_word)(CodePoint *, const char *&, const char *, size_t),
UInt64 (*hash_functor)(const UInt64 *, const size_t &, const size_t &)) UInt64 (*hash_functor)(const UInt64 *, size_t, size_t))
{ {
const char * start = data; const char * start = data;
const char * end = data + size; const char * end = data + size;
@ -156,7 +144,7 @@ struct SimhashImpl
// Also, a 64 bit vector initialized to zero // Also, a 64 bit vector initialized to zero
Int64 finger_vec[64] = {}; Int64 finger_vec[64] = {};
// a array to store N word hash values // a array to store N word hash values
UInt64 nwordHashes[N] = {}; UInt64 nword_hashes[N] = {};
// word buffer to store one word // word buffer to store one word
CodePoint word_buf[max_word_size] = {}; CodePoint word_buf[max_word_size] = {};
size_t word_size; size_t word_size;
@ -167,16 +155,16 @@ struct SimhashImpl
if (word_size) if (word_size)
{ {
// for each word, calculate a hash value and stored into the array // for each word, calculate a hash value and stored into the array
nwordHashes[i++] = Hash::hashSum(word_buf, word_size); nword_hashes[i++] = Hash::hashSum(word_buf, word_size);
} }
} }
// calculate the first word shingle hash value // calculate the first word shingle hash value
UInt64 hash_value = hash_functor(nwordHashes, N, 0); UInt64 hash_value = hash_functor(nword_hashes, N, 0);
std::bitset<64> bits_(hash_value); std::bitset<64> first_bits(hash_value);
for (size_t i = 0; i < 64; ++i) for (size_t i = 0; i < 64; ++i)
{ {
finger_vec[i] += ((bits_.test(i)) ? 1 : -1); finger_vec[i] += ((first_bits.test(i)) ? 1 : -1);
} }
size_t offset = 0; size_t offset = 0;
@ -187,12 +175,12 @@ struct SimhashImpl
// so we need to store new word hash into location of a0, then ,this array become // so we need to store new word hash into location of a0, then ,this array become
// |a5|a1|a2|a3|a4|, next time, a1 become the oldest location, we need to store new // |a5|a1|a2|a3|a4|, next time, a1 become the oldest location, we need to store new
// word hash value into locaion of a1, then array become |a5|a6|a2|a3|a4| // word hash value into locaion of a1, then array become |a5|a6|a2|a3|a4|
nwordHashes[offset] = Hash::hashSum(word_buf, word_size); nword_hashes[offset] = Hash::hashSum(word_buf, word_size);
offset = (offset + 1) % N; offset = (offset + 1) % N;
// according to the word hash storation way, in order to not lose the word shingle's // 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 // sequence information, when calculation word shingle hash value, we need provide the offset
// inforation, which is the offset of the first word's hash value of the word shingle // inforation, which is the offset of the first word's hash value of the word shingle
hash_value = hash_functor(nwordHashes, N, offset); hash_value = hash_functor(nword_hashes, N, offset);
std::bitset<64> bits(hash_value); std::bitset<64> bits(hash_value);
for (size_t i = 0; i < 64; ++i) for (size_t i = 0; i < 64; ++i)
{ {
@ -272,7 +260,7 @@ struct MinhashImpl
// insert a new value into K minimum hash array if this value // insert a new value into K minimum hash array if this value
// is smaller than the greatest value in the array // is smaller than the greatest value in the array
static ALWAYS_INLINE inline void insert_minValue(UInt64 * hashes, UInt64 v) static ALWAYS_INLINE inline void insertMinValue(UInt64 * hashes, UInt64 v)
{ {
size_t i = 0; size_t i = 0;
for (; i < K && hashes[i] <= v; ++i) for (; i < K && hashes[i] <= v; ++i)
@ -286,7 +274,7 @@ struct MinhashImpl
// insert a new value into K maximum hash array if this value // insert a new value into K maximum hash array if this value
// is greater than the smallest value in the array // is greater than the smallest value in the array
static ALWAYS_INLINE inline void insert_maxValue(UInt64 * hashes, UInt64 v) static ALWAYS_INLINE inline void insertMaxValue(UInt64 * hashes, UInt64 v)
{ {
int i = K - 1; int i = K - 1;
for (; i >= 0 && hashes[i] >= v; --i) for (; i >= 0 && hashes[i] >= v; --i)
@ -305,7 +293,7 @@ struct MinhashImpl
// return this two hashsum: Tuple(hashsum1, hashsum2) // return this two hashsum: Tuple(hashsum1, hashsum2)
static ALWAYS_INLINE inline std::tuple<UInt64, UInt64> ngramCalculateHashValue( static ALWAYS_INLINE inline std::tuple<UInt64, UInt64> ngramCalculateHashValue(
const char * data, const char * data,
const size_t size, size_t size,
size_t (*read_code_points)(CodePoint *, const char *&, const char *), size_t (*read_code_points)(CodePoint *, const char *&, const char *),
UInt64 (*hash_functor)(const CodePoint *)) UInt64 (*hash_functor)(const CodePoint *))
{ {
@ -326,8 +314,8 @@ struct MinhashImpl
auto new_hash = hash_functor(cp + iter); auto new_hash = hash_functor(cp + iter);
// insert the new hash value into array used to store K minimum value // insert the new hash value into array used to store K minimum value
// and K maximum value // and K maximum value
insert_minValue(k_minimum, new_hash); insertMinValue(k_minimum, new_hash);
insert_maxValue(k_maxinum, new_hash); insertMaxValue(k_maxinum, new_hash);
} }
iter = 0; iter = 0;
} while (start < end && (found = read_code_points(cp, start, end))); } while (start < end && (found = read_code_points(cp, start, end)));
@ -343,9 +331,9 @@ struct MinhashImpl
// K minimum and K maximum hash value // K minimum and K maximum hash value
static ALWAYS_INLINE inline std::tuple<UInt64, UInt64> wordShinglesCalculateHashValue( static ALWAYS_INLINE inline std::tuple<UInt64, UInt64> wordShinglesCalculateHashValue(
const char * data, const char * data,
const size_t size, size_t size,
size_t (*read_one_word)(CodePoint *, const char *&, const char *, const size_t &), size_t (*read_one_word)(CodePoint *, const char *&, const char *, size_t),
UInt64 (*hash_functor)(const UInt64 *, const size_t &, const size_t &)) UInt64 (*hash_functor)(const UInt64 *, size_t, size_t))
{ {
const char * start = data; const char * start = data;
const char * end = start + size; const char * end = start + size;
@ -353,7 +341,7 @@ struct MinhashImpl
UInt64 k_minimum[K] = {}; UInt64 k_minimum[K] = {};
UInt64 k_maxinum[K] = {}; UInt64 k_maxinum[K] = {};
// array to store n word hashes // array to store n word hashes
UInt64 nwordHashes[N] = {}; UInt64 nword_hashes[N] = {};
// word buffer to store one word // word buffer to store one word
CodePoint word_buf[max_word_size] = {}; CodePoint word_buf[max_word_size] = {};
size_t word_size; size_t word_size;
@ -364,22 +352,22 @@ struct MinhashImpl
word_size = read_one_word(word_buf, start, end, max_word_size); word_size = read_one_word(word_buf, start, end, max_word_size);
if (word_size) if (word_size)
{ {
nwordHashes[i++] = Hash::hashSum(word_buf, word_size); nword_hashes[i++] = Hash::hashSum(word_buf, word_size);
} }
} }
auto new_hash = hash_functor(nwordHashes, N, 0); auto new_hash = hash_functor(nword_hashes, N, 0);
insert_minValue(k_minimum, new_hash); insertMinValue(k_minimum, new_hash);
insert_maxValue(k_maxinum, new_hash); insertMaxValue(k_maxinum, new_hash);
size_t offset = 0; size_t offset = 0;
while (start < end && (word_size = read_one_word(word_buf, start, end, max_word_size))) while (start < end && (word_size = read_one_word(word_buf, start, end, max_word_size)))
{ {
nwordHashes[offset] = Hash::hashSum(word_buf, word_size); nword_hashes[offset] = Hash::hashSum(word_buf, word_size);
offset = (offset + 1) % N; offset = (offset + 1) % N;
new_hash = hash_functor(nwordHashes, N, offset); new_hash = hash_functor(nword_hashes, N, offset);
insert_minValue(k_minimum, new_hash); insertMinValue(k_minimum, new_hash);
insert_maxValue(k_maxinum, new_hash); insertMaxValue(k_maxinum, new_hash);
} }
// calculate hashsum // calculate hashsum

View File

@ -8,7 +8,7 @@
#include <DataTypes/DataTypeTuple.h> #include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypesNumber.h> #include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionHelpers.h> #include <Functions/FunctionHelpers.h>
#include <Functions/IFunction.h> #include <Functions/IFunctionImpl.h>
namespace DB namespace DB
{ {
@ -21,7 +21,7 @@ namespace ErrorCodes
// FunctionStringHash // FunctionStringHash
// Simhash: String -> UInt64 // Simhash: String -> UInt64
// Minhash: String -> (UInt64, UInt64) // Minhash: String -> (UInt64, UInt64)
template <typename Impl, typename Name, bool IsSimhash> template <typename Impl, typename Name, bool is_simhash>
class FunctionsStringHash : public IFunction class FunctionsStringHash : public IFunction
{ {
public: public:
@ -38,7 +38,7 @@ public:
if (!isString(arguments[0])) if (!isString(arguments[0]))
throw Exception( throw Exception(
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (IsSimhash) if constexpr (is_simhash)
return std::make_shared<DataTypeNumber<typename Impl::ResultType>>(); return std::make_shared<DataTypeNumber<typename Impl::ResultType>>();
auto element = DataTypeFactory::instance().get("UInt64"); auto element = DataTypeFactory::instance().get("UInt64");
return std::make_shared<DataTypeTuple>(DataTypes{element, element}); return std::make_shared<DataTypeTuple>(DataTypes{element, element});
@ -49,7 +49,7 @@ public:
const ColumnPtr & column = block.getByPosition(arguments[0]).column; const ColumnPtr & column = block.getByPosition(arguments[0]).column;
const ColumnConst * col_const = typeid_cast<const ColumnConst *>(&*column); const ColumnConst * col_const = typeid_cast<const ColumnConst *>(&*column);
using ResultType = typename Impl::ResultType; using ResultType = typename Impl::ResultType;
if constexpr (IsSimhash) if constexpr (is_simhash)
{ {
if (col_const) if (col_const)
{ {

View File

@ -40,21 +40,11 @@ struct BitHammingDistanceImpl
c[i] = apply(a, b[i]); c[i] = apply(a, b[i]);
} }
static ResultType constant_constant(A a, B b) { return apply(a, b); }
private: private:
static UInt8 pop_cnt(UInt64 res)
{
UInt8 count = 0;
for (; res; res >>= 1)
count += res & 1u;
return count;
}
static inline UInt8 apply(UInt64 a, UInt64 b) static inline UInt8 apply(UInt64 a, UInt64 b)
{ {
UInt64 res = a ^ b; UInt64 res = a ^ b;
return pop_cnt(res); return __builtin_popcountll(res);
} }
}; };
@ -75,9 +65,7 @@ bool castType(const IDataType * type, F && f)
template <typename F> template <typename F>
static bool castBothTypes(const IDataType * left, const IDataType * right, F && f) static bool castBothTypes(const IDataType * left, const IDataType * right, F && f)
{ {
return castType(left, [&](const auto & left_) { return castType(left, [&](const auto & left_) { return castType(right, [&](const auto & right_) { return f(left_, right_); }); });
return castType(right, [&](const auto & right_) { return f(left_, right_); });
});
} }
// bitHammingDistance function: (Integer, Integer) -> UInt8 // bitHammingDistance function: (Integer, Integer) -> UInt8
@ -103,12 +91,13 @@ public:
return std::make_shared<DataTypeUInt8>(); return std::make_shared<DataTypeUInt8>();
} }
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) override void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) override
{ {
auto * left_generic = block.getByPosition(arguments[0]).type.get(); auto * left_generic = block.getByPosition(arguments[0]).type.get();
auto * right_generic = block.getByPosition(arguments[1]).type.get(); auto * right_generic = block.getByPosition(arguments[1]).type.get();
bool valid = castBothTypes(left_generic, right_generic, [&](const auto & left, const auto & right) bool valid = castBothTypes(left_generic, right_generic, [&](const auto & left, const auto & right) {
{
using LeftDataType = std::decay_t<decltype(left)>; using LeftDataType = std::decay_t<decltype(left)>;
using RightDataType = std::decay_t<decltype(right)>; using RightDataType = std::decay_t<decltype(right)>;
using T0 = typename LeftDataType::FieldType; using T0 = typename LeftDataType::FieldType;
@ -121,16 +110,6 @@ public:
auto col_left_raw = block.getByPosition(arguments[0]).column.get(); auto col_left_raw = block.getByPosition(arguments[0]).column.get();
auto col_right_raw = block.getByPosition(arguments[1]).column.get(); auto col_right_raw = block.getByPosition(arguments[1]).column.get();
if (auto col_left = checkAndGetColumnConst<ColVecT0>(col_left_raw))
{
if (auto col_right = checkAndGetColumnConst<ColVecT1>(col_right_raw))
{
// constant integer - constant integer
auto res = OpImpl::constant_constant(col_left->template getValue<T0>(), col_right->template getValue<T1>());
block.getByPosition(result).column = DataTypeUInt8().createColumnConst(col_left->size(), toField(res));
return true;
}
}
typename ColVecResult::MutablePtr col_res = nullptr; typename ColVecResult::MutablePtr col_res = nullptr;
col_res = ColVecResult::create(); col_res = ColVecResult::create();

View File

@ -3,7 +3,6 @@
namespace DB namespace DB
{ {
void registerFunctionsArithmetic(FunctionFactory &); void registerFunctionsArithmetic(FunctionFactory &);
void registerFunctionsArray(FunctionFactory &); void registerFunctionsArray(FunctionFactory &);
void registerFunctionsTuple(FunctionFactory &); void registerFunctionsTuple(FunctionFactory &);
@ -37,6 +36,9 @@ void registerFunctionsIntrospection(FunctionFactory &);
void registerFunctionsNull(FunctionFactory &); void registerFunctionsNull(FunctionFactory &);
void registerFunctionsJSON(FunctionFactory &); void registerFunctionsJSON(FunctionFactory &);
void registerFunctionsConsistentHashing(FunctionFactory & factory); void registerFunctionsConsistentHashing(FunctionFactory & factory);
void registerFunctionBitHammingDistance(FunctionFactory & factory);
void registerFunctionTupleHammingDistance(FunctionFactory & factory);
void registerFunctionsStringHash(FunctionFactory & factory);
void registerFunctions() void registerFunctions()
@ -78,6 +80,9 @@ void registerFunctions()
registerFunctionsJSON(factory); registerFunctionsJSON(factory);
registerFunctionsIntrospection(factory); registerFunctionsIntrospection(factory);
registerFunctionsConsistentHashing(factory); registerFunctionsConsistentHashing(factory);
registerFunctionBitHammingDistance(factory);
registerFunctionTupleHammingDistance(factory);
registerFunctionsStringHash(factory);
} }
} }

View File

@ -51,19 +51,7 @@ struct TupleHammingDistanceImpl
static ResultType constant_constant(UInt64 a1, UInt64 b1, UInt64 a2, UInt64 b2) { return apply(a1, a2) + apply(b1, b2); } static ResultType constant_constant(UInt64 a1, UInt64 b1, UInt64 a2, UInt64 b2) { return apply(a1, a2) + apply(b1, b2); }
private: private:
static UInt8 pop_cnt(UInt64 res) static inline UInt8 apply(UInt64 a, UInt64 b) { return a != b; }
{
UInt8 count = 0;
for (; res; res >>= 1)
count += res & 1u;
return count;
}
static inline UInt8 apply(UInt64 a, UInt64 b)
{
UInt64 res = a ^ b;
return pop_cnt(res);
}
}; };
template <typename F> template <typename F>
@ -83,12 +71,10 @@ bool castType(const IDataType * type, F && f)
template <typename F> template <typename F>
static bool castBothTypes(const IDataType * left, const IDataType * right, F && f) static bool castBothTypes(const IDataType * left, const IDataType * right, F && f)
{ {
return castType(left, [&](const auto & left_) { return castType(left, [&](const auto & left_) { return castType(right, [&](const auto & right_) { return f(left_, right_); }); });
return castType(right, [&](const auto & right_) { return f(left_, right_); });
});
} }
// tupleHammingDistance function: (Tuple(Integer, Integer), Tuple(Integer, Integer))->UInt8 // tupleHammingDistance function: (Tuple(Integer, Integer), Tuple(Integer, Integer))->0/1/2
// in order to avoid code bloating, for non-constant tuple, we make sure that the elements // in order to avoid code bloating, for non-constant tuple, we make sure that the elements
// in the tuple should have same data type, and for constant tuple, elements can be any integer // in the tuple should have same data type, and for constant tuple, elements can be any integer
// data type, we cast all of them into UInt64 // data type, we cast all of them into UInt64
@ -126,8 +112,7 @@ public:
throw Exception( throw Exception(
"Illegal column of arguments of function " + getName() + ", tuple should have exactly two elements.", "Illegal column of arguments of function " + getName() + ", tuple should have exactly two elements.",
ErrorCodes::ILLEGAL_COLUMN); ErrorCodes::ILLEGAL_COLUMN);
bool valid = castBothTypes(left_elems[0].get(), right_elems[0].get(), [&](const auto & left, const auto & right) bool valid = castBothTypes(left_elems[0].get(), right_elems[0].get(), [&](const auto & left, const auto & right) {
{
using LeftDataType = std::decay_t<decltype(left)>; using LeftDataType = std::decay_t<decltype(left)>;
using RightDataType = std::decay_t<decltype(right)>; using RightDataType = std::decay_t<decltype(right)>;
using T0 = typename LeftDataType::FieldType; using T0 = typename LeftDataType::FieldType;
@ -138,7 +123,9 @@ public:
using OpImpl = TupleHammingDistanceImpl<T0, T1>; using OpImpl = TupleHammingDistanceImpl<T0, T1>;
// constant tuple - constant tuple // we can not useDefaultImplementationForConstants,
// because with that, tupleHammingDistance((10, 300), (10, 20)) does not work,
// since 10 has data type UInt8, and 300 has data type UInt16
if (const ColumnConst * const_col_left = checkAndGetColumnConst<ColumnTuple>(arg1.column.get())) if (const ColumnConst * const_col_left = checkAndGetColumnConst<ColumnTuple>(arg1.column.get()))
{ {
if (const ColumnConst * const_col_right = checkAndGetColumnConst<ColumnTuple>(arg2.column.get())) if (const ColumnConst * const_col_right = checkAndGetColumnConst<ColumnTuple>(arg2.column.get()))

View File

@ -0,0 +1,59 @@
0
2718169299
2718169299
3333471646
26585365
4124079607
4124079607
4124079607
979945684
(3614688582,3614688582)
(3614688582,3614688582)
(765622645,765622645)
(765622645,765622645)
(765622645,765622645)
(765622645,765622645)
(3573094983,3573094983)
(3573094983,3573094983)
(3604768422,3604768422)
(3604768422,3604768422)
(3614688582,1599892600)
(3614688582,1599892600)
(3614688582,1599892600)
(3614688582,1599892600)
(3614688582,1599892600)
(3614688582,1599892600)
(3614688582,996508363)
(3614688582,996508363)
2548869326
2548869326
401385678
401385710
4258739090
4260836242
718415633
718681881
2314703251
1238864275
3900085650
3907425682
2314703251
1238864275
3569207545
3568143609
(1436198067,1436198067)
(1436198067,1436198067)
(3846780865,3846780865)
(1956854492,1956854492)
(2929435161,2929435161)
(2929435161,2929435161)
(3310088565,3310088565)
(3310088565,3310088565)
(3614688582,1294895121)
(3614688582,1294895121)
(3614688582,1138551650)
(3614688582,1138551650)
(3614688582,1294895121)
(3614688582,1294895121)
(3614688582,2840007763)
(3614688582,929186815)

View File

@ -0,0 +1,47 @@
SELECT ngramSimhash('');
SELECT ngramSimhash('what a cute cat.');
SELECT ngramSimhashCaseInsensitive('what a cute cat.');
SELECT ngramSimhashUTF8('what a cute cat.');
SELECT ngramSimhashCaseInsensitiveUTF8('what a cute cat.');
SELECT wordShingleSimhash('what a cute cat.');
SELECT wordShingleSimhashCaseInsensitive('what a cute cat.');
SELECT wordShingleSimhashUTF8('what a cute cat.');
SELECT wordShingleSimhashCaseInsensitiveUTF8('what a cute cat.');
SELECT ngramMinhash('');
SELECT ngramMinhash('what a cute cat.');
SELECT ngramMinhashCaseInsensitive('what a cute cat.');
SELECT ngramMinhashUTF8('what a cute cat.');
SELECT ngramMinhashCaseInsensitiveUTF8('what a cute cat.');
SELECT wordShingleMinhash('what a cute cat.');
SELECT wordShingleMinhashCaseInsensitive('what a cute cat.');
SELECT wordShingleMinhashUTF8('what a cute cat.');
SELECT wordShingleMinhashCaseInsensitiveUTF8('what a cute cat.');
DROP TABLE IF EXISTS defaults;
CREATE TABLE defaults
(
s String
)ENGINE = Memory();
INSERT INTO defaults values ('It is the latest occurrence of the Southeast European haze, the issue that occurs in constant intensity during every wet season. It has mainly been caused by forest fires resulting from illegal slash-and-burn clearing performed on behalf of the palm oil industry in Kazakhstan, principally on the islands, which then spread quickly in the dry season.') ('It is the latest occurrence of the Southeast Asian haze, the issue that occurs in constant intensity during every wet season. It has mainly been caused by forest fires resulting from illegal slash-and-burn clearing performed on behalf of the palm oil industry in Kazakhstan, principally on the islands, which then spread quickly in the dry season.');
SELECT ngramSimhash(s) FROM defaults;
SELECT ngramSimhashCaseInsensitive(s) FROM defaults;
SELECT ngramSimhashUTF8(s) FROM defaults;
SELECT ngramSimhashCaseInsensitiveUTF8(s) FROM defaults;
SELECT wordShingleSimhash(s) FROM defaults;
SELECT wordShingleSimhashCaseInsensitive(s) FROM defaults;
SELECT wordShingleSimhashUTF8(s) FROM defaults;
SELECT wordShingleSimhashCaseInsensitiveUTF8(s) FROM defaults;
SELECT ngramMinhash(s) FROM defaults;
SELECT ngramMinhashCaseInsensitive(s) FROM defaults;
SELECT ngramMinhashUTF8(s) FROM defaults;
SELECT ngramMinhashCaseInsensitiveUTF8(s) FROM defaults;
SELECT wordShingleMinhash(s) FROM defaults;
SELECT wordShingleMinhashCaseInsensitive(s) FROM defaults;
SELECT wordShingleMinhashUTF8(s) FROM defaults;
SELECT wordShingleMinhashCaseInsensitiveUTF8(s) FROM defaults;
DROP TABLE defaults;

View File

@ -0,0 +1,15 @@
1
7
63
2
1
3
5
4
6
6
6
3
5
9
9

View File

@ -0,0 +1,20 @@
SELECT bitHammingDistance(1, 5);
SELECT bitHammingDistance(100, 100000);
SELECT bitHammingDistance(-1, 1);
DROP TABLE IF EXISTS defaults;
CREATE TABLE defaults
(
n1 UInt8,
n2 UInt16,
n3 UInt32,
n4 UInt64
)ENGINE = Memory();
INSERT INTO defaults VALUES (1, 2, 3, 4) (12, 4345, 435, 1233) (45, 675, 32343, 54566) (90, 784, 9034, 778752);
SELECT bitHammingDistance(4, n1) FROM defaults;
SELECT bitHammingDistance(n2, 100) FROM defaults;
SELECT bitHammingDistance(n3, n4) FROM defaults;
DROP TABLE defaults;

View File

@ -0,0 +1,15 @@
2
1
1
0
2
2
2
2
1
2
2
2
0
2
2

View File

@ -0,0 +1,19 @@
SELECT tupleHammingDistance((1, 2), (3, 4));
SELECT tupleHammingDistance((120, 243), (120, 434));
SELECT tupleHammingDistance((-12, 434), (434, 434));
DROP TABLE IF EXISTS defaults;
CREATE TABLE defaults
(
t1 Tuple(UInt16, UInt16),
t2 Tuple(UInt32, UInt32),
t3 Tuple(Int64, Int64)
)ENGINE = Memory();
INSERT INTO defaults VALUES ((12, 43), (12312, 43453) ,(-10, 32)) ((1, 4), (546, 12345), (546, 12345)) ((90, 9875), (43456, 234203), (1231, -123)) ((87, 987), (545645, 768354634), (9123, 909));
SELECT tupleHammingDistance((12, 43), t1) FROM defaults;
SELECT tupleHammingDistance(t2, (546, 456)) FROM defaults;
SELECT tupleHammingDistance(t2, t3) FROM defaults;
DROP TABLE defaults;