mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
update
update name
This commit is contained in:
parent
64207abd0a
commit
83c0807b43
@ -72,18 +72,18 @@ struct ExtractStringImpl
|
||||
// read a ASCII word from pos to word
|
||||
// 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
|
||||
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
|
||||
while (pos < end && !isAlphaNum(*pos))
|
||||
while (pos < end && !isAlphaNumericASCII(*pos))
|
||||
++pos;
|
||||
|
||||
// word start from here
|
||||
const char * word_start = pos;
|
||||
while (pos < end && isAlphaNum(*pos))
|
||||
while (pos < end && isAlphaNumericASCII(*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);
|
||||
if (CaseInsensitive)
|
||||
@ -107,7 +107,7 @@ struct ExtractStringImpl
|
||||
|
||||
// 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
|
||||
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
|
||||
while (pos < end && isUTF8Sep(*pos))
|
||||
@ -122,7 +122,7 @@ struct ExtractStringImpl
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
@ -134,7 +134,7 @@ private:
|
||||
}
|
||||
|
||||
// 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
|
||||
static ALWAYS_INLINE inline UInt32 readOneUTF8Code(const char *& pos, const char * end)
|
||||
|
@ -31,47 +31,35 @@ struct Hash
|
||||
#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;
|
||||
UInt8 flag = 0;
|
||||
UInt64 crc = -1ULL;
|
||||
#ifdef __SSE4_2__
|
||||
for (size_t i = offset; i < size; ++i)
|
||||
{
|
||||
if (flag)
|
||||
res &= intHashCRC32(hashes[i]);
|
||||
else
|
||||
res |= intHashCRC32(hashes[i]);
|
||||
flag = (flag + 1) % 2;
|
||||
}
|
||||
crc = _mm_crc32_u64(crc, hashes[i]);
|
||||
for (size_t i = 0; i < offset; ++i)
|
||||
{
|
||||
if (flag)
|
||||
res &= intHashCRC32(hashes[i]);
|
||||
else
|
||||
res |= intHashCRC32(hashes[i]);
|
||||
flag = (flag + 1) % 2;
|
||||
}
|
||||
return res;
|
||||
crc = _mm_crc32_u64(crc, hashes[i]);
|
||||
#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]);
|
||||
#endif
|
||||
return crc;
|
||||
}
|
||||
|
||||
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 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]);
|
||||
UInt64 crc = -1ULL;
|
||||
#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
|
||||
return (intHashCRC32(even) ^ intHashCRC32(odd));
|
||||
for (size_t i = 0; i < K; ++i)
|
||||
crc = intHashCRC32(crc) ^ intHashCRC32(hashes[i]);
|
||||
#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
|
||||
static ALWAYS_INLINE inline UInt64 ngramCalculateHashValue(
|
||||
const char * data,
|
||||
const size_t size,
|
||||
size_t size,
|
||||
size_t (*read_code_points)(CodePoint *, const char *&, const char *),
|
||||
UInt64 (*hash_functor)(const CodePoint *))
|
||||
{
|
||||
@ -146,9 +134,9 @@ struct SimhashImpl
|
||||
// values to caculate the next word shingle hash value
|
||||
static ALWAYS_INLINE inline UInt64 wordShinglesCalculateHashValue(
|
||||
const char * data,
|
||||
const size_t size,
|
||||
size_t (*read_one_word)(CodePoint *, const char *&, const char *, const size_t &),
|
||||
UInt64 (*hash_functor)(const UInt64 *, const size_t &, const size_t &))
|
||||
size_t size,
|
||||
size_t (*read_one_word)(CodePoint *, const char *&, const char *, size_t),
|
||||
UInt64 (*hash_functor)(const UInt64 *, size_t, size_t))
|
||||
{
|
||||
const char * start = data;
|
||||
const char * end = data + size;
|
||||
@ -156,7 +144,7 @@ struct SimhashImpl
|
||||
// Also, a 64 bit vector initialized to zero
|
||||
Int64 finger_vec[64] = {};
|
||||
// a array to store N word hash values
|
||||
UInt64 nwordHashes[N] = {};
|
||||
UInt64 nword_hashes[N] = {};
|
||||
// word buffer to store one word
|
||||
CodePoint word_buf[max_word_size] = {};
|
||||
size_t word_size;
|
||||
@ -167,16 +155,16 @@ struct SimhashImpl
|
||||
if (word_size)
|
||||
{
|
||||
// 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
|
||||
UInt64 hash_value = hash_functor(nwordHashes, N, 0);
|
||||
std::bitset<64> bits_(hash_value);
|
||||
UInt64 hash_value = hash_functor(nword_hashes, N, 0);
|
||||
std::bitset<64> first_bits(hash_value);
|
||||
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;
|
||||
@ -187,12 +175,12 @@ struct SimhashImpl
|
||||
// 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
|
||||
// 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;
|
||||
// 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
|
||||
// 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);
|
||||
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
|
||||
// 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;
|
||||
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
|
||||
// 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;
|
||||
for (; i >= 0 && hashes[i] >= v; --i)
|
||||
@ -305,7 +293,7 @@ struct MinhashImpl
|
||||
// return this two hashsum: Tuple(hashsum1, hashsum2)
|
||||
static ALWAYS_INLINE inline std::tuple<UInt64, UInt64> ngramCalculateHashValue(
|
||||
const char * data,
|
||||
const size_t size,
|
||||
size_t size,
|
||||
size_t (*read_code_points)(CodePoint *, const char *&, const char *),
|
||||
UInt64 (*hash_functor)(const CodePoint *))
|
||||
{
|
||||
@ -326,8 +314,8 @@ struct MinhashImpl
|
||||
auto new_hash = hash_functor(cp + iter);
|
||||
// insert the new hash value into array used to store K minimum value
|
||||
// and K maximum value
|
||||
insert_minValue(k_minimum, new_hash);
|
||||
insert_maxValue(k_maxinum, new_hash);
|
||||
insertMinValue(k_minimum, new_hash);
|
||||
insertMaxValue(k_maxinum, new_hash);
|
||||
}
|
||||
iter = 0;
|
||||
} while (start < end && (found = read_code_points(cp, start, end)));
|
||||
@ -343,9 +331,9 @@ struct MinhashImpl
|
||||
// K minimum and K maximum hash value
|
||||
static ALWAYS_INLINE inline std::tuple<UInt64, UInt64> wordShinglesCalculateHashValue(
|
||||
const char * data,
|
||||
const size_t size,
|
||||
size_t (*read_one_word)(CodePoint *, const char *&, const char *, const size_t &),
|
||||
UInt64 (*hash_functor)(const UInt64 *, const size_t &, const size_t &))
|
||||
size_t size,
|
||||
size_t (*read_one_word)(CodePoint *, const char *&, const char *, size_t),
|
||||
UInt64 (*hash_functor)(const UInt64 *, size_t, size_t))
|
||||
{
|
||||
const char * start = data;
|
||||
const char * end = start + size;
|
||||
@ -353,7 +341,7 @@ struct MinhashImpl
|
||||
UInt64 k_minimum[K] = {};
|
||||
UInt64 k_maxinum[K] = {};
|
||||
// array to store n word hashes
|
||||
UInt64 nwordHashes[N] = {};
|
||||
UInt64 nword_hashes[N] = {};
|
||||
// word buffer to store one word
|
||||
CodePoint word_buf[max_word_size] = {};
|
||||
size_t word_size;
|
||||
@ -364,22 +352,22 @@ struct MinhashImpl
|
||||
word_size = read_one_word(word_buf, start, end, max_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);
|
||||
insert_minValue(k_minimum, new_hash);
|
||||
insert_maxValue(k_maxinum, new_hash);
|
||||
auto new_hash = hash_functor(nword_hashes, N, 0);
|
||||
insertMinValue(k_minimum, new_hash);
|
||||
insertMaxValue(k_maxinum, new_hash);
|
||||
|
||||
size_t offset = 0;
|
||||
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;
|
||||
new_hash = hash_functor(nwordHashes, N, offset);
|
||||
insert_minValue(k_minimum, new_hash);
|
||||
insert_maxValue(k_maxinum, new_hash);
|
||||
new_hash = hash_functor(nword_hashes, N, offset);
|
||||
insertMinValue(k_minimum, new_hash);
|
||||
insertMaxValue(k_maxinum, new_hash);
|
||||
}
|
||||
|
||||
// calculate hashsum
|
||||
|
@ -8,7 +8,7 @@
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/IFunctionImpl.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -21,7 +21,7 @@ namespace ErrorCodes
|
||||
// FunctionStringHash
|
||||
// Simhash: String -> UInt64
|
||||
// Minhash: String -> (UInt64, UInt64)
|
||||
template <typename Impl, typename Name, bool IsSimhash>
|
||||
template <typename Impl, typename Name, bool is_simhash>
|
||||
class FunctionsStringHash : public IFunction
|
||||
{
|
||||
public:
|
||||
@ -38,7 +38,7 @@ public:
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception(
|
||||
"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>>();
|
||||
auto element = DataTypeFactory::instance().get("UInt64");
|
||||
return std::make_shared<DataTypeTuple>(DataTypes{element, element});
|
||||
@ -49,7 +49,7 @@ public:
|
||||
const ColumnPtr & column = block.getByPosition(arguments[0]).column;
|
||||
const ColumnConst * col_const = typeid_cast<const ColumnConst *>(&*column);
|
||||
using ResultType = typename Impl::ResultType;
|
||||
if constexpr (IsSimhash)
|
||||
if constexpr (is_simhash)
|
||||
{
|
||||
if (col_const)
|
||||
{
|
||||
|
@ -40,21 +40,11 @@ struct BitHammingDistanceImpl
|
||||
c[i] = apply(a, b[i]);
|
||||
}
|
||||
|
||||
static ResultType constant_constant(A a, B b) { return apply(a, b); }
|
||||
|
||||
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)
|
||||
{
|
||||
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>
|
||||
static bool castBothTypes(const IDataType * left, const IDataType * right, F && f)
|
||||
{
|
||||
return castType(left, [&](const auto & left_) {
|
||||
return castType(right, [&](const auto & right_) { return f(left_, right_); });
|
||||
});
|
||||
return castType(left, [&](const auto & left_) { return castType(right, [&](const auto & right_) { return f(left_, right_); }); });
|
||||
}
|
||||
|
||||
// bitHammingDistance function: (Integer, Integer) -> UInt8
|
||||
@ -103,12 +91,13 @@ public:
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) override
|
||||
{
|
||||
auto * left_generic = block.getByPosition(arguments[0]).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 RightDataType = std::decay_t<decltype(right)>;
|
||||
using T0 = typename LeftDataType::FieldType;
|
||||
@ -121,16 +110,6 @@ public:
|
||||
|
||||
auto col_left_raw = block.getByPosition(arguments[0]).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;
|
||||
col_res = ColVecResult::create();
|
||||
|
@ -3,7 +3,6 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void registerFunctionsArithmetic(FunctionFactory &);
|
||||
void registerFunctionsArray(FunctionFactory &);
|
||||
void registerFunctionsTuple(FunctionFactory &);
|
||||
@ -37,6 +36,9 @@ void registerFunctionsIntrospection(FunctionFactory &);
|
||||
void registerFunctionsNull(FunctionFactory &);
|
||||
void registerFunctionsJSON(FunctionFactory &);
|
||||
void registerFunctionsConsistentHashing(FunctionFactory & factory);
|
||||
void registerFunctionBitHammingDistance(FunctionFactory & factory);
|
||||
void registerFunctionTupleHammingDistance(FunctionFactory & factory);
|
||||
void registerFunctionsStringHash(FunctionFactory & factory);
|
||||
|
||||
|
||||
void registerFunctions()
|
||||
@ -78,6 +80,9 @@ void registerFunctions()
|
||||
registerFunctionsJSON(factory);
|
||||
registerFunctionsIntrospection(factory);
|
||||
registerFunctionsConsistentHashing(factory);
|
||||
registerFunctionBitHammingDistance(factory);
|
||||
registerFunctionTupleHammingDistance(factory);
|
||||
registerFunctionsStringHash(factory);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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); }
|
||||
|
||||
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)
|
||||
{
|
||||
UInt64 res = a ^ b;
|
||||
return pop_cnt(res);
|
||||
}
|
||||
static inline UInt8 apply(UInt64 a, UInt64 b) { return a != b; }
|
||||
};
|
||||
|
||||
template <typename F>
|
||||
@ -83,12 +71,10 @@ bool castType(const IDataType * type, F && f)
|
||||
template <typename F>
|
||||
static bool castBothTypes(const IDataType * left, const IDataType * right, F && f)
|
||||
{
|
||||
return castType(left, [&](const auto & left_) {
|
||||
return castType(right, [&](const auto & right_) { return f(left_, right_); });
|
||||
});
|
||||
return castType(left, [&](const auto & left_) { 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 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
|
||||
@ -126,8 +112,7 @@ public:
|
||||
throw Exception(
|
||||
"Illegal column of arguments of function " + getName() + ", tuple should have exactly two elements.",
|
||||
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 RightDataType = std::decay_t<decltype(right)>;
|
||||
using T0 = typename LeftDataType::FieldType;
|
||||
@ -138,7 +123,9 @@ public:
|
||||
|
||||
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_right = checkAndGetColumnConst<ColumnTuple>(arg2.column.get()))
|
||||
|
59
tests/queries/0_stateless/01016_simhash_minhash.reference
Normal file
59
tests/queries/0_stateless/01016_simhash_minhash.reference
Normal 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)
|
47
tests/queries/0_stateless/01016_simhash_minhash.sql
Normal file
47
tests/queries/0_stateless/01016_simhash_minhash.sql
Normal 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;
|
@ -0,0 +1,15 @@
|
||||
1
|
||||
7
|
||||
63
|
||||
2
|
||||
1
|
||||
3
|
||||
5
|
||||
4
|
||||
6
|
||||
6
|
||||
6
|
||||
3
|
||||
5
|
||||
9
|
||||
9
|
20
tests/queries/0_stateless/01017_bithamming_distance.sql
Normal file
20
tests/queries/0_stateless/01017_bithamming_distance.sql
Normal 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;
|
@ -0,0 +1,15 @@
|
||||
2
|
||||
1
|
||||
1
|
||||
0
|
||||
2
|
||||
2
|
||||
2
|
||||
2
|
||||
1
|
||||
2
|
||||
2
|
||||
2
|
||||
0
|
||||
2
|
||||
2
|
19
tests/queries/0_stateless/01017_tuplehamming_distance.sql
Normal file
19
tests/queries/0_stateless/01017_tuplehamming_distance.sql
Normal 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;
|
Loading…
Reference in New Issue
Block a user