Merge pull request #7649 from ucasFL/master

Add simhash, minhash, and hammingdistance function
This commit is contained in:
Nikolai Kochetov 2020-12-23 12:22:38 +03:00 committed by GitHub
commit 3fd0b69fab
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 1588 additions and 2 deletions

View 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;
}
};
}

View 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>();
}
}

View 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));
}
}
};
}

View 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>();
}
}

View File

@ -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);

View 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>();
}
}

View File

@ -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

View 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)

View 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;

View File

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

View File

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

View File

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

View File

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