mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge pull request #7649 from ucasFL/master
Add simhash, minhash, and hammingdistance function
This commit is contained in:
commit
3fd0b69fab
166
src/Functions/ExtractString.h
Normal file
166
src/Functions/ExtractString.h
Normal file
@ -0,0 +1,166 @@
|
||||
#pragma once
|
||||
#include <Common/PODArray.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/UTF8Helpers.h>
|
||||
|
||||
#include <algorithm>
|
||||
#include <climits>
|
||||
#include <cstring>
|
||||
#include <memory>
|
||||
#include <utility>
|
||||
|
||||
#ifdef __SSE4_2__
|
||||
# include <nmmintrin.h>
|
||||
#endif
|
||||
|
||||
namespace DB
|
||||
{
|
||||
// used by FunctionsStringSimilarity and FunctionsStringHash
|
||||
// includes extracting ASCII ngram, UTF8 ngram, ASCII word and UTF8 word
|
||||
template <size_t N, bool CaseInsensitive>
|
||||
struct ExtractStringImpl
|
||||
{
|
||||
/// Padding form ColumnsString. It is a number of bytes we can always read starting from pos if pos < end.
|
||||
static constexpr size_t default_padding = 16;
|
||||
|
||||
/// Functions are read `default_padding - (N - 1)` bytes into the buffer. Window of size N is used.
|
||||
/// Read copies `N - 1` last bytes from buffer into beginning, and then reads new bytes.
|
||||
static constexpr size_t buffer_size = default_padding + N - 1;
|
||||
|
||||
// the length of code_points = buffer_size
|
||||
// pos: the current beginning location that we want to copy data
|
||||
// end: the end location of the string
|
||||
static ALWAYS_INLINE size_t readASCIICodePoints(UInt8 * code_points, const char *& pos, const char * end)
|
||||
{
|
||||
/// Offset before which we copy some data.
|
||||
constexpr size_t padding_offset = default_padding - N + 1;
|
||||
/// We have an array like this for ASCII (N == 4, other cases are similar)
|
||||
/// |a0|a1|a2|a3|a4|a5|a6|a7|a8|a9|a10|a11|a12|a13|a14|a15|a16|a17|a18|
|
||||
/// And we copy ^^^^^^^^^^^^^^^ these bytes to the start
|
||||
/// Actually it is enough to copy 3 bytes, but memcpy for 4 bytes translates into 1 instruction
|
||||
memcpy(code_points, code_points + padding_offset, roundUpToPowerOfTwoOrZero(N - 1) * sizeof(UInt8));
|
||||
/// Now we have an array
|
||||
/// |a13|a14|a15|a16|a4|a5|a6|a7|a8|a9|a10|a11|a12|a13|a14|a15|a16|a17|a18|
|
||||
/// ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
/// Doing unaligned read of 16 bytes and copy them like above
|
||||
/// 16 is also chosen to do two `movups`.
|
||||
/// Such copying allow us to have 3 codepoints from the previous read to produce the 4-grams with them.
|
||||
memcpy(code_points + (N - 1), pos, default_padding * sizeof(UInt8));
|
||||
|
||||
if constexpr (CaseInsensitive)
|
||||
{
|
||||
/// We really need template lambdas with C++20 to do it inline
|
||||
unrollLowering<N - 1>(code_points, std::make_index_sequence<padding_offset>());
|
||||
}
|
||||
pos += padding_offset;
|
||||
if (pos > end)
|
||||
return default_padding - (pos - end);
|
||||
return default_padding;
|
||||
}
|
||||
|
||||
// read a ASCII word
|
||||
static ALWAYS_INLINE inline size_t readOneASCIIWord(PaddedPODArray<UInt8> & word_buf, const char *& pos, const char * end)
|
||||
{
|
||||
// jump seperators
|
||||
while (pos < end && !isAlphaNumericASCII(*pos))
|
||||
++pos;
|
||||
|
||||
// word start from here
|
||||
const char * word_start = pos;
|
||||
while (pos < end && isAlphaNumericASCII(*pos))
|
||||
++pos;
|
||||
|
||||
word_buf.assign(word_start, pos);
|
||||
if (CaseInsensitive)
|
||||
{
|
||||
std::transform(word_buf.begin(), word_buf.end(), word_buf.begin(), [](UInt8 c) { return std::tolower(c); });
|
||||
}
|
||||
return word_buf.size();
|
||||
}
|
||||
|
||||
static ALWAYS_INLINE inline size_t readUTF8CodePoints(UInt32 * code_points, const char *& pos, const char * end)
|
||||
{
|
||||
memcpy(code_points, code_points + default_padding - N + 1, roundUpToPowerOfTwoOrZero(N - 1) * sizeof(UInt32));
|
||||
|
||||
size_t num = N - 1;
|
||||
while (num < default_padding && pos < end)
|
||||
{
|
||||
code_points[num++] = readOneUTF8Code(pos, end);
|
||||
}
|
||||
return num;
|
||||
}
|
||||
|
||||
// read one UTF8 word from pos to word
|
||||
static ALWAYS_INLINE inline size_t readOneUTF8Word(PaddedPODArray<UInt32> & word_buf, const char *& pos, const char * end)
|
||||
{
|
||||
// jump UTF8 seperator
|
||||
while (pos < end && isUTF8Sep(*pos))
|
||||
++pos;
|
||||
word_buf.clear();
|
||||
// UTF8 word's character number
|
||||
while (pos < end && !isUTF8Sep(*pos))
|
||||
{
|
||||
word_buf.push_back(readOneUTF8Code(pos, end));
|
||||
}
|
||||
return word_buf.size();
|
||||
}
|
||||
|
||||
private:
|
||||
template <size_t Offset, typename Container, size_t... I>
|
||||
static ALWAYS_INLINE inline void unrollLowering(Container & cont, const std::index_sequence<I...> &)
|
||||
{
|
||||
((cont[Offset + I] = std::tolower(cont[Offset + I])), ...);
|
||||
}
|
||||
|
||||
// we use ASCII non-alphanum character as UTF8 seperator
|
||||
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)
|
||||
{
|
||||
size_t length = UTF8::seqLength(*pos);
|
||||
|
||||
if (pos + length > end)
|
||||
length = end - pos;
|
||||
UInt32 res;
|
||||
switch (length)
|
||||
{
|
||||
case 1:
|
||||
res = 0;
|
||||
memcpy(&res, pos, 1);
|
||||
break;
|
||||
case 2:
|
||||
res = 0;
|
||||
memcpy(&res, pos, 2);
|
||||
break;
|
||||
case 3:
|
||||
res = 0;
|
||||
memcpy(&res, pos, 3);
|
||||
break;
|
||||
default:
|
||||
memcpy(&res, pos, 4);
|
||||
}
|
||||
|
||||
if constexpr (CaseInsensitive)
|
||||
{
|
||||
switch (length)
|
||||
{
|
||||
case 4:
|
||||
res &= ~(1u << (5 + 3 * CHAR_BIT));
|
||||
[[fallthrough]];
|
||||
case 3:
|
||||
res &= ~(1u << (5 + 2 * CHAR_BIT));
|
||||
[[fallthrough]];
|
||||
case 2:
|
||||
res &= ~(1u);
|
||||
res &= ~(1u << (5 + CHAR_BIT));
|
||||
[[fallthrough]];
|
||||
default:
|
||||
res &= ~(1u << 5);
|
||||
}
|
||||
}
|
||||
pos += length;
|
||||
return res;
|
||||
}
|
||||
};
|
||||
}
|
626
src/Functions/FunctionsStringHash.cpp
Normal file
626
src/Functions/FunctionsStringHash.cpp
Normal file
@ -0,0 +1,626 @@
|
||||
#include <Functions/FunctionsStringHash.h>
|
||||
|
||||
#include <Functions/ExtractString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionsHashing.h>
|
||||
#include <Common/HashTable/ClearableHashMap.h>
|
||||
#include <Common/HashTable/Hash.h>
|
||||
#include <Common/PODArray.h>
|
||||
|
||||
#include <Core/Defines.h>
|
||||
|
||||
#include <bitset>
|
||||
#include <functional>
|
||||
#include <memory>
|
||||
#include <tuple>
|
||||
#include <vector>
|
||||
#include <common/unaligned.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
struct Hash
|
||||
{
|
||||
static UInt64 crc32u64(UInt64 crc [[maybe_unused]], UInt64 val [[maybe_unused]])
|
||||
{
|
||||
#ifdef __SSE4_2__
|
||||
return _mm_crc32_u64(crc, val);
|
||||
#elif defined(__aarch64__) && defined(__ARM_FEATURE_CRC32)
|
||||
return __crc32cd(crc, val);
|
||||
#else
|
||||
throw Exception("String hash is not implemented without sse4.2 support", ErrorCodes::NOT_IMPLEMENTED);
|
||||
#endif
|
||||
}
|
||||
|
||||
static UInt64 crc32u32(UInt64 crc [[maybe_unused]], UInt32 val [[maybe_unused]])
|
||||
{
|
||||
#ifdef __SSE4_2__
|
||||
return _mm_crc32_u32(crc, val);
|
||||
#elif defined(__aarch64__) && defined(__ARM_FEATURE_CRC32)
|
||||
return __crc32cw(crc, val);
|
||||
#else
|
||||
throw Exception("String hash is not implemented without sse4.2 support", ErrorCodes::NOT_IMPLEMENTED);
|
||||
#endif
|
||||
}
|
||||
|
||||
static UInt64 crc32u8(UInt64 crc [[maybe_unused]], UInt8 val [[maybe_unused]])
|
||||
{
|
||||
#ifdef __SSE4_2__
|
||||
return _mm_crc32_u8(crc, val);
|
||||
#elif defined(__aarch64__) && defined(__ARM_FEATURE_CRC32)
|
||||
return __crc32cb(crc, val);
|
||||
#else
|
||||
throw Exception("String hash is not implemented without sse4.2 support", ErrorCodes::NOT_IMPLEMENTED);
|
||||
#endif
|
||||
}
|
||||
|
||||
static ALWAYS_INLINE inline UInt64 ngramASCIIHash(const UInt8 * code_points)
|
||||
{
|
||||
return crc32u64(-1ULL, unalignedLoad<UInt32>(code_points));
|
||||
}
|
||||
|
||||
static ALWAYS_INLINE inline UInt64 ngramUTF8Hash(const UInt32 * code_points)
|
||||
{
|
||||
UInt64 crc = -1ULL;
|
||||
crc = crc32u64(crc, code_points[0]);
|
||||
crc = crc32u64(crc, code_points[1]);
|
||||
crc = crc32u64(crc, code_points[2]);
|
||||
return crc;
|
||||
}
|
||||
|
||||
static ALWAYS_INLINE inline UInt64 wordShinglesHash(const UInt64 * hashes, size_t size, size_t offset)
|
||||
{
|
||||
UInt64 crc1 = -1ULL;
|
||||
UInt64 crc2 = -1ULL;
|
||||
|
||||
for (size_t i = offset; i < size; i += 2)
|
||||
crc1 = crc32u64(crc1, hashes[i]);
|
||||
for (size_t i = offset + 1; i < size; i += 2)
|
||||
crc2 = crc32u64(crc2, hashes[i]);
|
||||
|
||||
if ((size - offset) & 1)
|
||||
{
|
||||
for (size_t i = 0; i < offset; i += 2)
|
||||
crc2 = crc32u64(crc2, hashes[i]);
|
||||
for (size_t i = 1; i < offset; i += 2)
|
||||
crc1 = crc32u64(crc1, hashes[i]);
|
||||
}
|
||||
else
|
||||
{
|
||||
for (size_t i = 0; i < offset; i += 2)
|
||||
crc1 = crc32u64(crc1, hashes[i]);
|
||||
for (size_t i = 1; i < offset; i += 2)
|
||||
crc2 = crc32u64(crc2, hashes[i]);
|
||||
}
|
||||
|
||||
return crc1 | (crc2 << 32u);
|
||||
}
|
||||
|
||||
static ALWAYS_INLINE inline UInt64 hashSum(const UInt8 * hashes [[maybe_unused]], size_t K [[maybe_unused]])
|
||||
{
|
||||
UInt64 crc1 = -1ULL;
|
||||
UInt64 crc2 = -1ULL;
|
||||
|
||||
for (size_t i = 0; i < K; i += 2)
|
||||
crc1 = crc32u8(crc1, hashes[i]);
|
||||
for (size_t i = 1; i < K; i += 2)
|
||||
crc2 = crc32u8(crc2, hashes[i]);
|
||||
|
||||
return crc1 | (crc2 << 32u);
|
||||
}
|
||||
|
||||
static ALWAYS_INLINE inline UInt64 hashSum(const UInt32 * hashes [[maybe_unused]], size_t K [[maybe_unused]])
|
||||
{
|
||||
UInt64 crc1 = -1ULL;
|
||||
UInt64 crc2 = -1ULL;
|
||||
|
||||
for (size_t i = 0; i < K; i += 2)
|
||||
crc1 = crc32u32(crc1, hashes[i]);
|
||||
for (size_t i = 1; i < K; i += 2)
|
||||
crc2 = crc32u32(crc2, hashes[i]);
|
||||
|
||||
return crc1 | (crc2 << 32u);
|
||||
}
|
||||
|
||||
static ALWAYS_INLINE inline UInt64 hashSum(const UInt64 * hashes, size_t K)
|
||||
{
|
||||
UInt64 crc1 = -1ULL;
|
||||
UInt64 crc2 = -1ULL;
|
||||
|
||||
for (size_t i = 0; i < K; i += 2)
|
||||
crc1 = crc32u64(crc1, hashes[i]);
|
||||
for (size_t i = 1; i < K; i += 2)
|
||||
crc2 = crc32u64(crc2, hashes[i]);
|
||||
|
||||
return crc1 | (crc2 << 32u);
|
||||
}
|
||||
};
|
||||
|
||||
// Simhash String -> UInt64
|
||||
// N: the length of ngram or words shingles
|
||||
// CodePoint: UInt8(ASCII) or UInt32(UTF8)
|
||||
// UTF8: means ASCII or UTF8, these two parameters CodePoint and UTF8 can only be (UInt8, false) or (UInt32, true)
|
||||
// Ngram: means ngram(true) or words shingles(false)
|
||||
// CaseInsensitive: means should we consider about letter case or not
|
||||
template <size_t N, typename CodePoint, bool UTF8, bool Ngram, bool CaseInsensitive>
|
||||
struct SimhashImpl
|
||||
{
|
||||
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
|
||||
static constexpr size_t max_string_size = 1u << 15;
|
||||
static constexpr size_t simultaneously_codepoints_num = StrOp::buffer_size;
|
||||
|
||||
// Simhash ngram calculate function: String ->UInt64
|
||||
// this function extracting ngram from input string, and maintain a 64-dimensions vector
|
||||
// for each ngram, calculate a 64 bit hash value, and update the vector according the hash value
|
||||
// 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,
|
||||
size_t size,
|
||||
size_t (*read_code_points)(CodePoint *, const char *&, const char *),
|
||||
UInt64 (*hash_functor)(const CodePoint *))
|
||||
{
|
||||
const char * start = data;
|
||||
const char * end = data + size;
|
||||
// fingerprint vector, all dimensions initialized to zero at the first
|
||||
Int64 finger_vec[64] = {};
|
||||
CodePoint cp[simultaneously_codepoints_num] = {};
|
||||
|
||||
size_t found = read_code_points(cp, start, end);
|
||||
size_t iter = N - 1;
|
||||
|
||||
do
|
||||
{
|
||||
for (; iter + N <= found; ++iter)
|
||||
{
|
||||
// for each ngram, we can calculate an 64 bit hash
|
||||
// then update finger_vec according to this hash value
|
||||
// if the i'th bit is 1, finger_vec[i] plus 1, otherwise minus 1
|
||||
UInt64 hash_value = hash_functor(cp + iter);
|
||||
std::bitset<64> bits(hash_value);
|
||||
for (size_t i = 0; i < 64; ++i)
|
||||
{
|
||||
finger_vec[i] += ((bits.test(i)) ? 1 : -1);
|
||||
}
|
||||
}
|
||||
iter = 0;
|
||||
} while (start < end && (found = read_code_points(cp, start, end)));
|
||||
|
||||
// finally, we return a 64 bit value according to finger_vec
|
||||
// if finger_vec[i] > 0, the i'th bit of the value is 1, otherwise 0
|
||||
std::bitset<64> res_bit(0u);
|
||||
for (size_t i = 0; i < 64; ++i)
|
||||
{
|
||||
if (finger_vec[i] > 0)
|
||||
res_bit.set(i);
|
||||
}
|
||||
return res_bit.to_ullong();
|
||||
}
|
||||
|
||||
// Simhash word shingle calculate funtion: String -> UInt64
|
||||
// this function extracting n word shingle from input string, and maintain a 64-dimensions vector as well
|
||||
// for each word shingle, calculate a 64 bit hash value, and update the vector according the hash value
|
||||
// finally return a 64 bit value(UInt64), i'th bit is 1 means vector[i] > 0, otherwise, vector[i] < 0
|
||||
//
|
||||
// word shingle hash value calculate:
|
||||
// 1. at the first, extracts N word shingles and calculate N hash values, store into an array, use this N hash values
|
||||
// to calculate the first word shingle hash value
|
||||
// 2. next, we extrac one word each time, and calculate a new hash value of the new word,then use the latest N hash
|
||||
// values to caculate the next word shingle hash value
|
||||
static ALWAYS_INLINE inline UInt64 wordShinglesCalculateHashValue(
|
||||
const char * data,
|
||||
size_t size,
|
||||
size_t (*read_one_word)(PaddedPODArray<CodePoint> &, const char *&, const char *),
|
||||
UInt64 (*hash_functor)(const UInt64 *, size_t, size_t))
|
||||
{
|
||||
const char * start = data;
|
||||
const char * end = data + size;
|
||||
|
||||
// Also, a 64 bit vector initialized to zero
|
||||
Int64 finger_vec[64] = {};
|
||||
// a array to store N word hash values
|
||||
UInt64 nword_hashes[N] = {};
|
||||
// word buffer to store one word
|
||||
PaddedPODArray<CodePoint> word_buf;
|
||||
// get first word shingle
|
||||
for (size_t i = 0; i < N && start < end; ++i)
|
||||
{
|
||||
read_one_word(word_buf, start, end);
|
||||
if (!word_buf.empty())
|
||||
{
|
||||
// for each word, calculate a hash value and stored into the array
|
||||
nword_hashes[i++] = Hash::hashSum(word_buf.data(), word_buf.size());
|
||||
}
|
||||
}
|
||||
|
||||
// calculate the first word shingle 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] += ((first_bits.test(i)) ? 1 : -1);
|
||||
}
|
||||
|
||||
size_t offset = 0;
|
||||
while (start < end && read_one_word(word_buf, start, end))
|
||||
{
|
||||
// we need to store the new word hash value to the oldest location.
|
||||
// for example, N = 5, array |a0|a1|a2|a3|a4|, now , a0 is the oldest location,
|
||||
// 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|
|
||||
nword_hashes[offset] = Hash::hashSum(word_buf.data(), word_buf.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(nword_hashes, N, offset);
|
||||
std::bitset<64> bits(hash_value);
|
||||
for (size_t i = 0; i < 64; ++i)
|
||||
{
|
||||
finger_vec[i] += ((bits.test(i)) ? 1 : -1);
|
||||
}
|
||||
}
|
||||
|
||||
std::bitset<64> res_bit(0u);
|
||||
for (size_t i = 0; i < 64; ++i)
|
||||
{
|
||||
if (finger_vec[i] > 0)
|
||||
res_bit.set(i);
|
||||
}
|
||||
return res_bit.to_ullong();
|
||||
}
|
||||
|
||||
static void apply(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, PaddedPODArray<UInt64> & res)
|
||||
{
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
const char * one_data = reinterpret_cast<const char *>(&data[offsets[i - 1]]);
|
||||
const size_t data_size = offsets[i] - offsets[i - 1] - 1;
|
||||
if (data_size <= max_string_size)
|
||||
{
|
||||
if constexpr (Ngram)
|
||||
{
|
||||
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
|
||||
{
|
||||
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;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
template <typename F, size_t K, size_t v>
|
||||
class FixedHeap
|
||||
{
|
||||
public:
|
||||
FixedHeap() = delete;
|
||||
|
||||
explicit FixedHeap(F f_) : f(f_), data_t(std::make_shared<std::vector<UInt64>>(K, v))
|
||||
{
|
||||
std::make_heap(data_t->begin(), data_t->end(), f);
|
||||
}
|
||||
|
||||
void insertAndReplace(UInt64 new_v)
|
||||
{
|
||||
data_t->push_back(new_v);
|
||||
std::push_heap(data_t->begin(), data_t->end(), f);
|
||||
std::pop_heap(data_t->begin(), data_t->end(), f);
|
||||
data_t->pop_back();
|
||||
}
|
||||
|
||||
const UInt64 * data() { return data_t->data(); }
|
||||
|
||||
private:
|
||||
F f;
|
||||
std::shared_ptr<std::vector<UInt64>> data_t;
|
||||
};
|
||||
|
||||
|
||||
// Minhash: String -> Tuple(UInt64, UInt64)
|
||||
// for each string, we extract ngram or word shingle,
|
||||
// for each ngram or word shingle, calculate a hash value,
|
||||
// then we take the K minimum hash values to calculate a hashsum,
|
||||
// and take the K maximum hash values to calculate another hashsum,
|
||||
// return this two hashsum: Tuple(hashsum1, hashsum2)
|
||||
//
|
||||
// N: the length of ngram or words shingles
|
||||
// K: the number of minimum hashes and maximum hashes that we keep
|
||||
// CodePoint: UInt8(ASCII) or UInt32(UTF8)
|
||||
// UTF8: means ASCII or UTF8, these two parameters CodePoint and UTF8 can only be (UInt8, false) or (UInt32, true)
|
||||
// Ngram: means ngram(true) or words shingles(false)
|
||||
// CaseInsensitive: means should we consider about letter case or not
|
||||
template <size_t N, size_t K, typename CodePoint, bool UTF8, bool Ngram, bool CaseInsensitive>
|
||||
struct MinhashImpl
|
||||
{
|
||||
using Less = std::less<size_t>;
|
||||
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 StrOp = ExtractStringImpl<N, CaseInsensitive>;
|
||||
static constexpr size_t max_string_size = 1u << 15;
|
||||
static constexpr size_t simultaneously_codepoints_num = StrOp::buffer_size;
|
||||
|
||||
// Minhash ngram calculate function, String -> Tuple(UInt64, UInt64)
|
||||
// we extract ngram from input string, and calculate a hash value for each ngram
|
||||
// then we take the K minimum hash values to calculate a hashsum,
|
||||
// and take the K maximum hash values to calculate another hashsum,
|
||||
// return this two hashsum: Tuple(hashsum1, hashsum2)
|
||||
static ALWAYS_INLINE inline std::tuple<UInt64, UInt64> ngramCalculateHashValue(
|
||||
const char * data,
|
||||
size_t size,
|
||||
size_t (*read_code_points)(CodePoint *, const char *&, const char *),
|
||||
UInt64 (*hash_functor)(const CodePoint *))
|
||||
{
|
||||
const char * start = data;
|
||||
const char * end = data + size;
|
||||
// we just maintain the K minimu and K maximum hash values
|
||||
MaxHeap k_minimum_hashes(Less{});
|
||||
MinHeap k_maximum_hashes(Greater{});
|
||||
CodePoint cp[simultaneously_codepoints_num] = {};
|
||||
|
||||
size_t found = read_code_points(cp, start, end);
|
||||
size_t iter = N - 1;
|
||||
|
||||
do
|
||||
{
|
||||
for (; iter + N <= found; ++iter)
|
||||
{
|
||||
auto new_hash = hash_functor(cp + iter);
|
||||
// insert the new hash value into array used to store K minimum value
|
||||
// and K maximum value
|
||||
k_minimum_hashes.insertAndReplace(new_hash);
|
||||
k_maximum_hashes.insertAndReplace(new_hash);
|
||||
}
|
||||
iter = 0;
|
||||
} while (start < end && (found = read_code_points(cp, start, end)));
|
||||
|
||||
// calculate hashsum of the K minimum hash values and K maximum hash values
|
||||
UInt64 res1 = Hash::hashSum(k_minimum_hashes.data(), K);
|
||||
UInt64 res2 = Hash::hashSum(k_maximum_hashes.data(), K);
|
||||
return std::make_tuple(res1, res2);
|
||||
}
|
||||
|
||||
// Minhash word shingle hash value calculate function: String ->Tuple(UInt64, UInt64)
|
||||
// for each word shingle, we calculate a hash value, but in fact, we just maintain the
|
||||
// K minimum and K maximum hash value
|
||||
static ALWAYS_INLINE inline std::tuple<UInt64, UInt64> wordShinglesCalculateHashValue(
|
||||
const char * data,
|
||||
size_t size,
|
||||
size_t (*read_one_word)(PaddedPODArray<CodePoint> &, const char *&, const char *),
|
||||
UInt64 (*hash_functor)(const UInt64 *, size_t, size_t))
|
||||
{
|
||||
const char * start = data;
|
||||
const char * end = start + size;
|
||||
// also we just store the K minimu and K maximum hash values
|
||||
MaxHeap k_minimum_hashes(Less{});
|
||||
MinHeap k_maximum_hashes(Greater{});
|
||||
// array to store n word hashes
|
||||
UInt64 nword_hashes[N] = {};
|
||||
// word buffer to store one word
|
||||
PaddedPODArray<CodePoint> word_buf;
|
||||
// how word shingle hash value calculation and word hash storation is same as we
|
||||
// have descripted in Simhash wordShinglesCalculateHashValue function
|
||||
for (size_t i = 0; i < N && start < end; ++i)
|
||||
{
|
||||
read_one_word(word_buf, start, end);
|
||||
if (!word_buf.empty())
|
||||
{
|
||||
nword_hashes[i++] = Hash::hashSum(word_buf.data(), word_buf.size());
|
||||
}
|
||||
}
|
||||
|
||||
auto new_hash = hash_functor(nword_hashes, N, 0);
|
||||
k_minimum_hashes.insertAndReplace(new_hash);
|
||||
k_maximum_hashes.insertAndReplace(new_hash);
|
||||
|
||||
size_t offset = 0;
|
||||
while (start < end && read_one_word(word_buf, start, end))
|
||||
{
|
||||
nword_hashes[offset] = Hash::hashSum(word_buf.data(), word_buf.size());
|
||||
offset = (offset + 1) % N;
|
||||
new_hash = hash_functor(nword_hashes, N, offset);
|
||||
k_minimum_hashes.insertAndReplace(new_hash);
|
||||
k_maximum_hashes.insertAndReplace(new_hash);
|
||||
}
|
||||
|
||||
// calculate hashsum
|
||||
UInt64 res1 = Hash::hashSum(k_minimum_hashes.data(), K);
|
||||
UInt64 res2 = Hash::hashSum(k_maximum_hashes.data(), K);
|
||||
return std::make_tuple(res1, res2);
|
||||
}
|
||||
|
||||
static void apply(
|
||||
const ColumnString::Chars & data,
|
||||
const ColumnString::Offsets & offsets,
|
||||
PaddedPODArray<UInt64> & res1,
|
||||
PaddedPODArray<UInt64> & res2)
|
||||
{
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
const char * one_data = reinterpret_cast<const char *>(&data[offsets[i - 1]]);
|
||||
const size_t data_size = offsets[i] - offsets[i - 1] - 1;
|
||||
if (data_size <= max_string_size)
|
||||
{
|
||||
if constexpr (Ngram)
|
||||
{
|
||||
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
|
||||
{
|
||||
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);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
struct NameNgramSimhash
|
||||
{
|
||||
static constexpr auto name = "ngramSimhash";
|
||||
};
|
||||
|
||||
struct NameNgramSimhashCaseInsensitive
|
||||
{
|
||||
static constexpr auto name = "ngramSimhashCaseInsensitive";
|
||||
};
|
||||
|
||||
struct NameNgramSimhashUTF8
|
||||
{
|
||||
static constexpr auto name = "ngramSimhashUTF8";
|
||||
};
|
||||
|
||||
struct NameNgramSimhashCaseInsensitiveUTF8
|
||||
{
|
||||
static constexpr auto name = "ngramSimhashCaseInsensitiveUTF8";
|
||||
};
|
||||
|
||||
struct NameWordShingleSimhash
|
||||
{
|
||||
static constexpr auto name = "wordShingleSimhash";
|
||||
};
|
||||
|
||||
struct NameWordShingleSimhashCaseInsensitive
|
||||
{
|
||||
static constexpr auto name = "wordShingleSimhashCaseInsensitive";
|
||||
};
|
||||
|
||||
struct NameWordShingleSimhashUTF8
|
||||
{
|
||||
static constexpr auto name = "wordShingleSimhashUTF8";
|
||||
};
|
||||
|
||||
struct NameWordShingleSimhashCaseInsensitiveUTF8
|
||||
{
|
||||
static constexpr auto name = "wordShingleSimhashCaseInsensitiveUTF8";
|
||||
};
|
||||
|
||||
struct NameNgramMinhash
|
||||
{
|
||||
static constexpr auto name = "ngramMinhash";
|
||||
};
|
||||
|
||||
struct NameNgramMinhashCaseInsensitive
|
||||
{
|
||||
static constexpr auto name = "ngramMinhashCaseInsensitive";
|
||||
};
|
||||
|
||||
struct NameNgramMinhashUTF8
|
||||
{
|
||||
static constexpr auto name = "ngramMinhashUTF8";
|
||||
};
|
||||
|
||||
struct NameNgramMinhashCaseInsensitiveUTF8
|
||||
{
|
||||
static constexpr auto name = "ngramMinhashCaseInsensitiveUTF8";
|
||||
};
|
||||
|
||||
struct NameWordShingleMinhash
|
||||
{
|
||||
static constexpr auto name = "wordShingleMinhash";
|
||||
};
|
||||
|
||||
struct NameWordShingleMinhashCaseInsensitive
|
||||
{
|
||||
static constexpr auto name = "wordShingleMinhashCaseInsensitive";
|
||||
};
|
||||
|
||||
struct NameWordShingleMinhashUTF8
|
||||
{
|
||||
static constexpr auto name = "wordShingleMinhashUTF8";
|
||||
};
|
||||
|
||||
struct NameWordShingleMinhashCaseInsensitiveUTF8
|
||||
{
|
||||
static constexpr auto name = "wordShingleMinhashCaseInsensitiveUTF8";
|
||||
};
|
||||
|
||||
// Simhash
|
||||
using FunctionNgramSimhash = FunctionsStringHash<SimhashImpl<4, UInt8, false, true, false>, NameNgramSimhash, true>;
|
||||
|
||||
using FunctionNgramSimhashCaseInsensitive
|
||||
= FunctionsStringHash<SimhashImpl<4, UInt8, false, true, true>, NameNgramSimhashCaseInsensitive, true>;
|
||||
|
||||
using FunctionNgramSimhashUTF8 = FunctionsStringHash<SimhashImpl<3, UInt32, true, true, false>, NameNgramSimhashUTF8, true>;
|
||||
|
||||
using FunctionNgramSimhashCaseInsensitiveUTF8
|
||||
= FunctionsStringHash<SimhashImpl<3, UInt32, true, true, true>, NameNgramSimhashCaseInsensitiveUTF8, true>;
|
||||
|
||||
using FunctionWordShingleSimhash = FunctionsStringHash<SimhashImpl<3, UInt8, false, false, false>, NameWordShingleSimhash, true>;
|
||||
|
||||
using FunctionWordShingleSimhashCaseInsensitive
|
||||
= FunctionsStringHash<SimhashImpl<3, UInt8, false, false, true>, NameWordShingleSimhashCaseInsensitive, true>;
|
||||
|
||||
using FunctionWordShingleSimhashUTF8 = FunctionsStringHash<SimhashImpl<3, UInt32, true, false, false>, NameWordShingleSimhashUTF8, true>;
|
||||
|
||||
using FunctionWordShingleSimhashCaseInsensitiveUTF8
|
||||
= FunctionsStringHash<SimhashImpl<3, UInt32, true, false, true>, NameWordShingleSimhashCaseInsensitiveUTF8, true>;
|
||||
|
||||
// Minhash
|
||||
using FunctionNgramMinhash = FunctionsStringHash<MinhashImpl<4, 6, UInt8, false, true, false>, NameNgramMinhash, false>;
|
||||
|
||||
using FunctionNgramMinhashCaseInsensitive
|
||||
= FunctionsStringHash<MinhashImpl<4, 6, UInt8, false, true, true>, NameNgramMinhashCaseInsensitive, false>;
|
||||
|
||||
using FunctionNgramMinhashUTF8 = FunctionsStringHash<MinhashImpl<4, 6, UInt32, true, true, false>, NameNgramMinhashUTF8, false>;
|
||||
|
||||
using FunctionNgramMinhashCaseInsensitiveUTF8
|
||||
= FunctionsStringHash<MinhashImpl<4, 6, UInt32, true, true, true>, NameNgramMinhashCaseInsensitiveUTF8, false>;
|
||||
|
||||
using FunctionWordShingleMinhash = FunctionsStringHash<MinhashImpl<3, 6, UInt8, false, false, false>, NameWordShingleMinhash, false>;
|
||||
|
||||
using FunctionWordShingleMinhashCaseInsensitive
|
||||
= FunctionsStringHash<MinhashImpl<3, 6, UInt8, false, false, true>, NameWordShingleMinhashCaseInsensitive, false>;
|
||||
|
||||
using FunctionWordShingleMinhashUTF8
|
||||
= FunctionsStringHash<MinhashImpl<3, 6, UInt32, true, false, false>, NameWordShingleMinhashUTF8, false>;
|
||||
|
||||
using FunctionWordShingleMinhashCaseInsensitiveUTF8
|
||||
= FunctionsStringHash<MinhashImpl<3, 6, UInt32, true, false, true>, NameWordShingleMinhashCaseInsensitiveUTF8, false>;
|
||||
|
||||
void registerFunctionsStringHash(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionNgramSimhash>();
|
||||
factory.registerFunction<FunctionNgramSimhashCaseInsensitive>();
|
||||
factory.registerFunction<FunctionNgramSimhashUTF8>();
|
||||
factory.registerFunction<FunctionNgramSimhashCaseInsensitiveUTF8>();
|
||||
factory.registerFunction<FunctionWordShingleSimhash>();
|
||||
factory.registerFunction<FunctionWordShingleSimhashCaseInsensitive>();
|
||||
factory.registerFunction<FunctionWordShingleSimhashUTF8>();
|
||||
factory.registerFunction<FunctionWordShingleSimhashCaseInsensitiveUTF8>();
|
||||
|
||||
factory.registerFunction<FunctionNgramMinhash>();
|
||||
factory.registerFunction<FunctionNgramMinhashCaseInsensitive>();
|
||||
factory.registerFunction<FunctionNgramMinhashUTF8>();
|
||||
factory.registerFunction<FunctionNgramMinhashCaseInsensitiveUTF8>();
|
||||
factory.registerFunction<FunctionWordShingleMinhash>();
|
||||
factory.registerFunction<FunctionWordShingleMinhashCaseInsensitive>();
|
||||
factory.registerFunction<FunctionWordShingleMinhashUTF8>();
|
||||
factory.registerFunction<FunctionWordShingleMinhashCaseInsensitiveUTF8>();
|
||||
}
|
||||
}
|
||||
|
83
src/Functions/FunctionsStringHash.h
Normal file
83
src/Functions/FunctionsStringHash.h
Normal file
@ -0,0 +1,83 @@
|
||||
#pragma once
|
||||
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunctionImpl.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
// FunctionStringHash
|
||||
// Simhash: String -> UInt64
|
||||
// Minhash: String -> (UInt64, UInt64)
|
||||
template <typename Impl, typename Name, bool is_simhash>
|
||||
class FunctionsStringHash : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionsStringHash>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!isString(arguments[0]))
|
||||
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 type;
|
||||
|
||||
return std::make_shared<DataTypeTuple>(DataTypes{type, type});
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override
|
||||
{
|
||||
const ColumnPtr & column = arguments[0].column;
|
||||
|
||||
if constexpr (is_simhash)
|
||||
{
|
||||
// non const string, const case is handled by useDefaultImplementationForConstants.
|
||||
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 col_res;
|
||||
}
|
||||
else // Min hash
|
||||
{
|
||||
// non const string
|
||||
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);
|
||||
Impl::apply(col_str_vector->getChars(), col_str_vector->getOffsets(), vec_h1, vec_h2);
|
||||
MutableColumns tuple_columns;
|
||||
tuple_columns.emplace_back(std::move(col_h1));
|
||||
tuple_columns.emplace_back(std::move(col_h2));
|
||||
return ColumnTuple::create(std::move(tuple_columns));
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
160
src/Functions/bitHammingDistance.cpp
Normal file
160
src/Functions/bitHammingDistance.cpp
Normal file
@ -0,0 +1,160 @@
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/castTypeToEither.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
|
||||
template <typename A, typename B>
|
||||
struct BitHammingDistanceImpl
|
||||
{
|
||||
using ResultType = UInt8;
|
||||
|
||||
static void NO_INLINE vectorVector(const PaddedPODArray<A> & a, const PaddedPODArray<B> & b, PaddedPODArray<ResultType> & c)
|
||||
{
|
||||
size_t size = a.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
c[i] = apply(a[i], b[i]);
|
||||
}
|
||||
|
||||
static void NO_INLINE vectorConstant(const PaddedPODArray<A> & a, B b, PaddedPODArray<ResultType> & c)
|
||||
{
|
||||
size_t size = a.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
c[i] = apply(a[i], b);
|
||||
}
|
||||
|
||||
static void NO_INLINE constantVector(A a, const PaddedPODArray<B> & b, PaddedPODArray<ResultType> & c)
|
||||
{
|
||||
size_t size = b.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
c[i] = apply(a, b[i]);
|
||||
}
|
||||
|
||||
private:
|
||||
static inline UInt8 apply(UInt64 a, UInt64 b)
|
||||
{
|
||||
UInt64 res = a ^ b;
|
||||
return __builtin_popcountll(res);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename F>
|
||||
bool castType(const IDataType * type, F && f)
|
||||
{
|
||||
return castTypeToEither<
|
||||
DataTypeInt8,
|
||||
DataTypeInt16,
|
||||
DataTypeInt32,
|
||||
DataTypeInt64,
|
||||
DataTypeUInt8,
|
||||
DataTypeUInt16,
|
||||
DataTypeUInt32,
|
||||
DataTypeUInt64>(type, std::forward<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_); }); });
|
||||
}
|
||||
|
||||
// bitHammingDistance function: (Integer, Integer) -> UInt8
|
||||
class FunctionBitHammingDistance : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "bitHammingDistance";
|
||||
using ResultType = UInt8;
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionBitHammingDistance>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 2; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!isInteger(arguments[0]))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
if (!isInteger(arguments[1]))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
||||
{
|
||||
const auto * left_generic = arguments[0].type.get();
|
||||
const auto * right_generic = arguments[1].type.get();
|
||||
ColumnPtr result_column;
|
||||
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;
|
||||
using T1 = typename RightDataType::FieldType;
|
||||
using ColVecT0 = ColumnVector<T0>;
|
||||
using ColVecT1 = ColumnVector<T1>;
|
||||
using ColVecResult = ColumnVector<ResultType>;
|
||||
|
||||
using OpImpl = BitHammingDistanceImpl<T0, T1>;
|
||||
|
||||
const auto * const col_left_raw = arguments[0].column.get();
|
||||
const auto * const col_right_raw = arguments[1].column.get();
|
||||
|
||||
typename ColVecResult::MutablePtr col_res = nullptr;
|
||||
col_res = ColVecResult::create();
|
||||
|
||||
auto & vec_res = col_res->getData();
|
||||
vec_res.resize(input_rows_count);
|
||||
|
||||
if (auto col_left_const = checkAndGetColumnConst<ColVecT0>(col_left_raw))
|
||||
{
|
||||
if (auto col_right = checkAndGetColumn<ColVecT1>(col_right_raw))
|
||||
{
|
||||
// constant integer - non-constant integer
|
||||
OpImpl::constantVector(col_left_const->template getValue<T0>(), col_right->getData(), vec_res);
|
||||
}
|
||||
else
|
||||
return false;
|
||||
}
|
||||
else if (auto col_left = checkAndGetColumn<ColVecT0>(col_left_raw))
|
||||
{
|
||||
if (auto col_right = checkAndGetColumn<ColVecT1>(col_right_raw))
|
||||
// non-constant integer - non-constant integer
|
||||
OpImpl::vectorVector(col_left->getData(), col_right->getData(), vec_res);
|
||||
else if (auto col_right_const = checkAndGetColumnConst<ColVecT1>(col_right_raw))
|
||||
// non-constant integer - constant integer
|
||||
OpImpl::vectorConstant(col_left->getData(), col_right_const->template getValue<T1>(), vec_res);
|
||||
else
|
||||
return false;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
|
||||
result_column = std::move(col_res);
|
||||
return true;
|
||||
});
|
||||
if (!valid)
|
||||
throw Exception(getName() + "'s arguments do not match the expected data types", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
return result_column;
|
||||
}
|
||||
};
|
||||
|
||||
void registerFunctionBitHammingDistance(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionBitHammingDistance>();
|
||||
}
|
||||
}
|
@ -42,7 +42,9 @@ void registerFunctionsNull(FunctionFactory &);
|
||||
void registerFunctionsJSON(FunctionFactory &);
|
||||
void registerFunctionsConsistentHashing(FunctionFactory & factory);
|
||||
void registerFunctionsUnixTimestamp64(FunctionFactory & factory);
|
||||
|
||||
void registerFunctionBitHammingDistance(FunctionFactory & factory);
|
||||
void registerFunctionTupleHammingDistance(FunctionFactory & factory);
|
||||
void registerFunctionsStringHash(FunctionFactory & factory);
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
void registerFunctionBayesAB(FunctionFactory &);
|
||||
#endif
|
||||
@ -57,7 +59,6 @@ void registerFunctionAESDecryptMysql(FunctionFactory & factory);
|
||||
|
||||
#endif
|
||||
|
||||
|
||||
void registerFunctions()
|
||||
{
|
||||
auto & factory = FunctionFactory::instance();
|
||||
@ -99,6 +100,9 @@ void registerFunctions()
|
||||
registerFunctionsIntrospection(factory);
|
||||
registerFunctionsConsistentHashing(factory);
|
||||
registerFunctionsUnixTimestamp64(factory);
|
||||
registerFunctionBitHammingDistance(factory);
|
||||
registerFunctionTupleHammingDistance(factory);
|
||||
registerFunctionsStringHash(factory);
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
registerFunctionBayesAB(factory);
|
||||
|
220
src/Functions/tupleHammingDistance.cpp
Normal file
220
src/Functions/tupleHammingDistance.cpp
Normal file
@ -0,0 +1,220 @@
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/castTypeToEither.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
template <typename A, typename B>
|
||||
struct TupleHammingDistanceImpl
|
||||
{
|
||||
using ResultType = UInt8;
|
||||
|
||||
static void NO_INLINE vectorVector(
|
||||
const PaddedPODArray<A> & a1,
|
||||
const PaddedPODArray<A> & b1,
|
||||
const PaddedPODArray<B> & a2,
|
||||
const PaddedPODArray<B> & b2,
|
||||
PaddedPODArray<ResultType> & c)
|
||||
{
|
||||
size_t size = a1.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
c[i] = apply(a1[i], a2[i]) + apply(b1[i], b2[i]);
|
||||
}
|
||||
|
||||
static void NO_INLINE
|
||||
vectorConstant(const PaddedPODArray<A> & a1, const PaddedPODArray<A> & b1, UInt64 a2, UInt64 b2, PaddedPODArray<ResultType> & c)
|
||||
{
|
||||
size_t size = a1.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
c[i] = apply(a1[i], a2) + apply(b1[i], b2);
|
||||
}
|
||||
|
||||
static void NO_INLINE
|
||||
constantVector(UInt64 a1, UInt64 b1, const PaddedPODArray<B> & a2, const PaddedPODArray<B> & b2, PaddedPODArray<ResultType> & c)
|
||||
{
|
||||
size_t size = a2.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
c[i] = apply(a1, a2[i]) + apply(b1, b2[i]);
|
||||
}
|
||||
|
||||
static ResultType constantConstant(UInt64 a1, UInt64 b1, UInt64 a2, UInt64 b2) { return apply(a1, a2) + apply(b1, b2); }
|
||||
|
||||
private:
|
||||
static inline UInt8 apply(UInt64 a, UInt64 b) { return a != b; }
|
||||
};
|
||||
|
||||
template <typename F>
|
||||
bool castType(const IDataType * type, F && f)
|
||||
{
|
||||
return castTypeToEither<
|
||||
DataTypeInt8,
|
||||
DataTypeInt16,
|
||||
DataTypeInt32,
|
||||
DataTypeInt64,
|
||||
DataTypeUInt8,
|
||||
DataTypeUInt16,
|
||||
DataTypeUInt32,
|
||||
DataTypeUInt64>(type, std::forward<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_); }); });
|
||||
}
|
||||
|
||||
// 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
|
||||
class FunctionTupleHammingDistance : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "tupleHammingDistance";
|
||||
using ResultType = UInt8;
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionTupleHammingDistance>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 2; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!isTuple(arguments[0]))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
if (!isTuple(arguments[1]))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
||||
{
|
||||
const ColumnWithTypeAndName & arg1 = arguments[0];
|
||||
const ColumnWithTypeAndName & arg2 = arguments[1];
|
||||
const DataTypeTuple & type1 = static_cast<const DataTypeTuple &>(*arg1.type);
|
||||
const DataTypeTuple & type2 = static_cast<const DataTypeTuple &>(*arg2.type);
|
||||
const auto & left_elems = type1.getElements();
|
||||
const auto & right_elems = type2.getElements();
|
||||
if (left_elems.size() != 2 || right_elems.size() != 2)
|
||||
throw Exception(
|
||||
"Illegal column of arguments of function " + getName() + ", tuple should have exactly two elements.",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
ColumnPtr result_column;
|
||||
|
||||
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;
|
||||
using T1 = typename RightDataType::FieldType;
|
||||
using ColVecT0 = ColumnVector<T0>;
|
||||
using ColVecT1 = ColumnVector<T1>;
|
||||
using ColVecResult = ColumnVector<ResultType>;
|
||||
|
||||
using OpImpl = TupleHammingDistanceImpl<T0, T1>;
|
||||
|
||||
// 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()))
|
||||
{
|
||||
auto cols1 = convertConstTupleToConstantElements(*const_col_left);
|
||||
auto cols2 = convertConstTupleToConstantElements(*const_col_right);
|
||||
Field a1, b1, a2, b2;
|
||||
cols1[0]->get(0, a1);
|
||||
cols1[1]->get(0, b1);
|
||||
cols2[0]->get(0, a2);
|
||||
cols2[1]->get(0, b2);
|
||||
auto res = OpImpl::constantConstant(a1.get<UInt64>(), b1.get<UInt64>(), a2.get<UInt64>(), b2.get<UInt64>());
|
||||
result_column = DataTypeUInt8().createColumnConst(const_col_left->size(), toField(res));
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
typename ColVecResult::MutablePtr col_res = nullptr;
|
||||
col_res = ColVecResult::create();
|
||||
auto & vec_res = col_res->getData();
|
||||
vec_res.resize(input_rows_count);
|
||||
// constant tuple - non-constant tuple
|
||||
if (const ColumnConst * const_col_left = checkAndGetColumnConst<ColumnTuple>(arg1.column.get()))
|
||||
{
|
||||
if (const ColumnTuple * col_right = typeid_cast<const ColumnTuple *>(arg2.column.get()))
|
||||
{
|
||||
auto const_cols = convertConstTupleToConstantElements(*const_col_left);
|
||||
Field a1, b1;
|
||||
const_cols[0]->get(0, a1);
|
||||
const_cols[1]->get(0, b1);
|
||||
auto col_r1 = checkAndGetColumn<ColVecT1>(&col_right->getColumn(0));
|
||||
auto col_r2 = checkAndGetColumn<ColVecT1>(&col_right->getColumn(1));
|
||||
if (col_r1 && col_r2)
|
||||
OpImpl::constantVector(a1.get<UInt64>(), b1.get<UInt64>(), col_r1->getData(), col_r2->getData(), vec_res);
|
||||
else
|
||||
return false;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
}
|
||||
else if (const ColumnTuple * col_left = typeid_cast<const ColumnTuple *>(arg1.column.get()))
|
||||
{
|
||||
auto col_l1 = checkAndGetColumn<ColVecT0>(&col_left->getColumn(0));
|
||||
auto col_l2 = checkAndGetColumn<ColVecT0>(&col_left->getColumn(1));
|
||||
if (col_l1 && col_l2)
|
||||
{
|
||||
// non-constant tuple - constant tuple
|
||||
if (const ColumnConst * const_col_right = checkAndGetColumnConst<ColumnTuple>(arg2.column.get()))
|
||||
{
|
||||
auto const_cols = convertConstTupleToConstantElements(*const_col_right);
|
||||
Field a2, b2;
|
||||
const_cols[0]->get(0, a2);
|
||||
const_cols[1]->get(0, b2);
|
||||
OpImpl::vectorConstant(col_l1->getData(), col_l2->getData(), a2.get<UInt64>(), a2.get<UInt64>(), vec_res);
|
||||
}
|
||||
// non-constant tuple - non-constant tuple
|
||||
else if (const ColumnTuple * col_right = typeid_cast<const ColumnTuple *>(arg2.column.get()))
|
||||
{
|
||||
auto col_r1 = checkAndGetColumn<ColVecT1>(&col_right->getColumn(0));
|
||||
auto col_r2 = checkAndGetColumn<ColVecT1>(&col_right->getColumn(1));
|
||||
if (col_r1 && col_r2)
|
||||
OpImpl::vectorVector(col_l1->getData(), col_l2->getData(), col_r1->getData(), col_r2->getData(), vec_res);
|
||||
else
|
||||
return false;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
result_column = std::move(col_res);
|
||||
return true;
|
||||
});
|
||||
if (!valid)
|
||||
throw Exception(getName() + "'s arguments do not match the expected data types", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
return result_column;
|
||||
}
|
||||
};
|
||||
|
||||
void registerFunctionTupleHammingDistance(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionTupleHammingDistance>();
|
||||
}
|
||||
}
|
@ -53,6 +53,7 @@ SRCS(
|
||||
FunctionsRandom.cpp
|
||||
FunctionsRound.cpp
|
||||
FunctionsStringArray.cpp
|
||||
FunctionsStringHash.cpp
|
||||
FunctionsStringSimilarity.cpp
|
||||
GatherUtils/concat.cpp
|
||||
GatherUtils/createArraySink.cpp
|
||||
@ -185,6 +186,7 @@ SRCS(
|
||||
bitBoolMaskAnd.cpp
|
||||
bitBoolMaskOr.cpp
|
||||
bitCount.cpp
|
||||
bitHammingDistance.cpp
|
||||
bitNot.cpp
|
||||
bitOr.cpp
|
||||
bitRotateLeft.cpp
|
||||
@ -504,6 +506,7 @@ SRCS(
|
||||
tryBase64Decode.cpp
|
||||
tuple.cpp
|
||||
tupleElement.cpp
|
||||
tupleHammingDistance.cpp
|
||||
upper.cpp
|
||||
upperUTF8.cpp
|
||||
uptime.cpp
|
||||
|
144
tests/queries/0_stateless/01016_simhash_minhash.reference
Normal file
144
tests/queries/0_stateless/01016_simhash_minhash.reference
Normal file
@ -0,0 +1,144 @@
|
||||
0
|
||||
2718169299
|
||||
2718169299
|
||||
1315333491
|
||||
1099965843
|
||||
5746351769509927967
|
||||
5746351769509927967
|
||||
8347269581771603092
|
||||
6041373934059725027
|
||||
(17178276249054052155,8864230932371215121)
|
||||
(14133097226001036899,7985237721476952807)
|
||||
(14133097226001036899,7985237721476952807)
|
||||
(4661257206578284012,15229878657590021759)
|
||||
(3087743741749030713,11631667950302077749)
|
||||
(11923981719512934676,1193672187225825732)
|
||||
(11923981719512934676,1193672187225825732)
|
||||
(17970606678134635272,3825545538448404526)
|
||||
(9422952829151664974,568010773615758889)
|
||||
2548869326
|
||||
2548869326
|
||||
401385678
|
||||
401385710
|
||||
2652202579
|
||||
2652235347
|
||||
2984455347
|
||||
2984488115
|
||||
12804820948382413807
|
||||
12804820948919350245
|
||||
11651601468065149391
|
||||
11651600368014488527
|
||||
18377198011227067677
|
||||
18233505035951822655
|
||||
5501050600367972694
|
||||
5501050600367972692
|
||||
(8590465925632898311,12699049311112305995)
|
||||
(8590465925632898311,15828587343885202011)
|
||||
(8590465925632898311,15824051019631343049)
|
||||
(8590465925632898311,12699049311222825283)
|
||||
(217966158370437743,14452995911556652133)
|
||||
(217966158370437743,14452995911556652133)
|
||||
(2170210914777151141,5341809779339553313)
|
||||
(12469866236432988845,5341809779339553313)
|
||||
(12271157076799061825,5514511977572226426)
|
||||
(11639913962681153226,2767634094725305612)
|
||||
(12271157075024394466,17994666970078080114)
|
||||
(12271157077109587702,13572452308677868240)
|
||||
(6252006845407214340,13538761942960976531)
|
||||
(13795977174459370328,6392395597500134035)
|
||||
(16118993428517222971,13602445809406467)
|
||||
(16118993428517222971,13602445809406467)
|
||||
uniqExact 6
|
||||
ngramSimhash
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 938403918
|
||||
ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 904817231
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 904849486
|
||||
ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 938469966
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 2 938404430
|
||||
ngramSimhashCaseInsensitive
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 938453071
|
||||
ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 938453599
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 2 938404430
|
||||
ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 636382047
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 938388046
|
||||
ngramSimhashUTF8
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 2400625214
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 2669060670
|
||||
ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 2671174174
|
||||
ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 2669060798
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 2 2635506238
|
||||
ngramSimhashCaseInsensitiveUTF8
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 2984307934
|
||||
ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 2967514366
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 2715855070
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 2967529694
|
||||
ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 2984290526
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 2984306910
|
||||
wordShingleSimhash
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 2384813566025024242
|
||||
ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 2393820766427040734
|
||||
ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 2421405261516400471
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 2384883934767174398
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 2384813567165864670
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 2384813567098766070
|
||||
wordShingleSimhashCaseInsensitive
|
||||
ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 11635224793909957342
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 11617192803208139478
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 11617192803208151794
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 11617192803208151766
|
||||
ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 3006891407629799254
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 11617263171950236406
|
||||
wordShingleSimhashUTF8
|
||||
ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 9097818277104946605
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 9084246141658271116
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 9084247241171471628
|
||||
ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 9088752215857929613
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 9093255814816009484
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 9084247481822285196
|
||||
wordShingleSimhashCaseInsensitiveUTF8
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 14788772559981154978
|
||||
ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 14497164445320454820
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 14500537785782895266
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 14787646625647636642
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 14500016612976573090
|
||||
ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 14787956717160870888
|
||||
ngramMinhash
|
||||
ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 (15568933215262012353,16287411738807860353)
|
||||
ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 (9473589826959436958,14264235017873782379)
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 2 (261441656340606110,13387826928927239258)
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 2 (261441656340606110,3305790294064680121)
|
||||
ngramMinhashCaseInsensitive
|
||||
ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 (15568933215262012353,16287411738807860353)
|
||||
ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 (9473589826959436958,14264235017873782379)
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 2 (3051755284325985438,3305790294064680121)
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 2 (3051755284325985438,13387826928927239258)
|
||||
ngramMinhashUTF8
|
||||
ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 4 (309830857064065611,7476109060377919216)
|
||||
ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 (309830856946430871,7521913981442105351)
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 (309830857559697399,7476109060377919216)
|
||||
ngramMinhashCaseInsensitiveUTF8
|
||||
ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 (13010809262502929096,2266175201446733829)
|
||||
ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 5 (16827851229372179144,976408052548769549)
|
||||
wordShingleMinhash
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 2 (14343822344862533053,11776483993821900250)
|
||||
ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 (18417749332128868312,11776483993821900250)
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 (18417749329907528200,14156831980621923226)
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 (4600092690178227586,11776483993821900250)
|
||||
ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 (12998011837685887081,1565093152297016105)
|
||||
wordShingleMinhashCaseInsensitive
|
||||
ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 (12998011837880940480,1565093152297016105)
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 (1100751419997894255,15225006848401474458)
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 2 (1100751419777226283,12993805708561478711)
|
||||
ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 (1260401089202135898,12993805709529540523)
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 (1638964264353944555,12993805708561478711)
|
||||
wordShingleMinhashUTF8
|
||||
ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 (742280067319112377,14237963017046410351)
|
||||
ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 (7237654052534217600,14400297883226437452)
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 (742280067319112377,17574811665615962276)
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 3 (3458625375707825328,17574811665615962276)
|
||||
wordShingleMinhashCaseInsensitiveUTF8
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 (7032848390598450936,5104668712725998486)
|
||||
ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 (15582670464629505464,13034678298246801511)
|
||||
ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 (9935434838523508980,7648038926638343017)
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 2 (7032848390598450936,16870743692447971238)
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 (7302041809563941951,6856814412450461959)
|
111
tests/queries/0_stateless/01016_simhash_minhash.sql
Normal file
111
tests/queries/0_stateless/01016_simhash_minhash.sql
Normal file
@ -0,0 +1,111 @@
|
||||
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;
|
||||
|
||||
TRUNCATE TABLE defaults;
|
||||
INSERT INTO defaults SELECT arrayJoin(splitByString('\n\n',
|
||||
'ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.
|
||||
ClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.
|
||||
ClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems.
|
||||
|
||||
ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.
|
||||
ClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system''s read and write availability.
|
||||
ClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems.
|
||||
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.
|
||||
ClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system''s read / write availability.
|
||||
ClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.
|
||||
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.
|
||||
ClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.
|
||||
ClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.
|
||||
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.
|
||||
ClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.
|
||||
ClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.
|
||||
|
||||
ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.
|
||||
ClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.
|
||||
ClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.'
|
||||
));
|
||||
|
||||
SELECT 'uniqExact', uniqExact(s) FROM defaults;
|
||||
|
||||
|
||||
SELECT 'ngramSimhash';
|
||||
SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), ngramSimhash(s) as h FROM defaults GROUP BY h;
|
||||
SELECT 'ngramSimhashCaseInsensitive';
|
||||
SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), ngramSimhashCaseInsensitive(s) as h FROM defaults GROUP BY h;
|
||||
SELECT 'ngramSimhashUTF8';
|
||||
SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), ngramSimhashUTF8(s) as h FROM defaults GROUP BY h;
|
||||
SELECT 'ngramSimhashCaseInsensitiveUTF8';
|
||||
SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), ngramSimhashCaseInsensitiveUTF8(s) as h FROM defaults GROUP BY h;
|
||||
SELECT 'wordShingleSimhash';
|
||||
SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), wordShingleSimhash(s) as h FROM defaults GROUP BY h;
|
||||
SELECT 'wordShingleSimhashCaseInsensitive';
|
||||
SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), wordShingleSimhashCaseInsensitive(s) as h FROM defaults GROUP BY h;
|
||||
SELECT 'wordShingleSimhashUTF8';
|
||||
SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), wordShingleSimhashUTF8(s) as h FROM defaults GROUP BY h;
|
||||
SELECT 'wordShingleSimhashCaseInsensitiveUTF8';
|
||||
SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), wordShingleSimhashCaseInsensitiveUTF8(s) as h FROM defaults GROUP BY h;
|
||||
|
||||
SELECT 'ngramMinhash';
|
||||
SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), ngramMinhash(s) as h FROM defaults GROUP BY h;
|
||||
SELECT 'ngramMinhashCaseInsensitive';
|
||||
SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), ngramMinhashCaseInsensitive(s) as h FROM defaults GROUP BY h;
|
||||
SELECT 'ngramMinhashUTF8';
|
||||
SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), ngramMinhashUTF8(s) as h FROM defaults GROUP BY h;
|
||||
SELECT 'ngramMinhashCaseInsensitiveUTF8';
|
||||
SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), ngramMinhashCaseInsensitiveUTF8(s) as h FROM defaults GROUP BY h;
|
||||
SELECT 'wordShingleMinhash';
|
||||
SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), wordShingleMinhash(s) as h FROM defaults GROUP BY h;
|
||||
SELECT 'wordShingleMinhashCaseInsensitive';
|
||||
SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), wordShingleMinhashCaseInsensitive(s) as h FROM defaults GROUP BY h;
|
||||
SELECT 'wordShingleMinhashUTF8';
|
||||
SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), wordShingleMinhashUTF8(s) as h FROM defaults GROUP BY h;
|
||||
SELECT 'wordShingleMinhashCaseInsensitiveUTF8';
|
||||
SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), wordShingleMinhashCaseInsensitiveUTF8(s) as h FROM defaults GROUP BY h;
|
||||
|
||||
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