#pragma once #include #include #include #include #include #include #include #include #include "config_functions.h" #if USE_XXHASH # include #endif #include "config_core.h" #if USE_SSL # include # include #endif #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include namespace DB { namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int NOT_IMPLEMENTED; extern const int ILLEGAL_COLUMN; } /** Hashing functions. * * halfMD5: String -> UInt64 * * A faster cryptographic hash function: * sipHash64: String -> UInt64 * * Fast non-cryptographic hash function for strings: * cityHash64: String -> UInt64 * * A non-cryptographic hashes from a tuple of values of any types (uses respective function for strings and intHash64 for numbers): * cityHash64: any* -> UInt64 * sipHash64: any* -> UInt64 * halfMD5: any* -> UInt64 * * Fast non-cryptographic hash function from any integer: * intHash32: number -> UInt32 * intHash64: number -> UInt64 * */ struct IntHash32Impl { using ReturnType = UInt32; static UInt32 apply(UInt64 x) { /// seed is taken from /dev/urandom. It allows you to avoid undesirable dependencies with hashes in different data structures. return intHash32<0x75D9543DE018BF45ULL>(x); } }; struct IntHash64Impl { using ReturnType = UInt64; static UInt64 apply(UInt64 x) { return intHash64(x ^ 0x4CF2D2BAAE6DA887ULL); } }; #if USE_SSL struct HalfMD5Impl { static constexpr auto name = "halfMD5"; using ReturnType = UInt64; static UInt64 apply(const char * begin, size_t size) { union { unsigned char char_data[16]; uint64_t uint64_data; } buf; MD5_CTX ctx; MD5_Init(&ctx); MD5_Update(&ctx, reinterpret_cast(begin), size); MD5_Final(buf.char_data, &ctx); return Poco::ByteOrder::flipBytes(static_cast(buf.uint64_data)); /// Compatibility with existing code. Cast need for old poco AND macos where UInt64 != uint64_t } static UInt64 combineHashes(UInt64 h1, UInt64 h2) { UInt64 hashes[] = {h1, h2}; return apply(reinterpret_cast(hashes), 16); } /// If true, it will use intHash32 or intHash64 to hash POD types. This behaviour is intended for better performance of some functions. /// Otherwise it will hash bytes in memory as a string using corresponding hash function. static constexpr bool use_int_hash_for_pods = false; }; struct MD5Impl { static constexpr auto name = "MD5"; enum { length = 16 }; static void apply(const char * begin, const size_t size, unsigned char * out_char_data) { MD5_CTX ctx; MD5_Init(&ctx); MD5_Update(&ctx, reinterpret_cast(begin), size); MD5_Final(out_char_data, &ctx); } }; struct SHA1Impl { static constexpr auto name = "SHA1"; enum { length = 20 }; static void apply(const char * begin, const size_t size, unsigned char * out_char_data) { SHA_CTX ctx; SHA1_Init(&ctx); SHA1_Update(&ctx, reinterpret_cast(begin), size); SHA1_Final(out_char_data, &ctx); } }; struct SHA224Impl { static constexpr auto name = "SHA224"; enum { length = 28 }; static void apply(const char * begin, const size_t size, unsigned char * out_char_data) { SHA256_CTX ctx; SHA224_Init(&ctx); SHA224_Update(&ctx, reinterpret_cast(begin), size); SHA224_Final(out_char_data, &ctx); } }; struct SHA256Impl { static constexpr auto name = "SHA256"; enum { length = 32 }; static void apply(const char * begin, const size_t size, unsigned char * out_char_data) { SHA256_CTX ctx; SHA256_Init(&ctx); SHA256_Update(&ctx, reinterpret_cast(begin), size); SHA256_Final(out_char_data, &ctx); } }; #endif struct SipHash64Impl { static constexpr auto name = "sipHash64"; using ReturnType = UInt64; static UInt64 apply(const char * begin, size_t size) { return sipHash64(begin, size); } static UInt64 combineHashes(UInt64 h1, UInt64 h2) { UInt64 hashes[] = {h1, h2}; return apply(reinterpret_cast(hashes), 16); } static constexpr bool use_int_hash_for_pods = false; }; struct SipHash128Impl { static constexpr auto name = "sipHash128"; enum { length = 16 }; static void apply(const char * begin, const size_t size, unsigned char * out_char_data) { sipHash128(begin, size, reinterpret_cast(out_char_data)); } }; /** Why we need MurmurHash2? * MurmurHash2 is an outdated hash function, superseded by MurmurHash3 and subsequently by CityHash, xxHash, HighwayHash. * Usually there is no reason to use MurmurHash. * It is needed for the cases when you already have MurmurHash in some applications and you want to reproduce it * in ClickHouse as is. For example, it is needed to reproduce the behaviour * for NGINX a/b testing module: https://nginx.ru/en/docs/http/ngx_http_split_clients_module.html */ struct MurmurHash2Impl32 { static constexpr auto name = "murmurHash2_32"; using ReturnType = UInt32; static UInt32 apply(const char * data, const size_t size) { return MurmurHash2(data, size, 0); } static UInt32 combineHashes(UInt32 h1, UInt32 h2) { return IntHash32Impl::apply(h1) ^ h2; } static constexpr bool use_int_hash_for_pods = false; }; struct MurmurHash2Impl64 { static constexpr auto name = "murmurHash2_64"; using ReturnType = UInt64; static UInt64 apply(const char * data, const size_t size) { return MurmurHash64A(data, size, 0); } static UInt64 combineHashes(UInt64 h1, UInt64 h2) { return IntHash64Impl::apply(h1) ^ h2; } static constexpr bool use_int_hash_for_pods = false; }; /// To be compatible with gcc: https://github.com/gcc-mirror/gcc/blob/41d6b10e96a1de98e90a7c0378437c3255814b16/libstdc%2B%2B-v3/include/bits/functional_hash.h#L191 struct GccMurmurHashImpl { static constexpr auto name = "gccMurmurHash"; using ReturnType = UInt64; static UInt64 apply(const char * data, const size_t size) { return MurmurHash64A(data, size, 0xc70f6907UL); } static UInt64 combineHashes(UInt64 h1, UInt64 h2) { return IntHash64Impl::apply(h1) ^ h2; } static constexpr bool use_int_hash_for_pods = false; }; struct MurmurHash3Impl32 { static constexpr auto name = "murmurHash3_32"; using ReturnType = UInt32; static UInt32 apply(const char * data, const size_t size) { union { UInt32 h; char bytes[sizeof(h)]; }; MurmurHash3_x86_32(data, size, 0, bytes); return h; } static UInt32 combineHashes(UInt32 h1, UInt32 h2) { return IntHash32Impl::apply(h1) ^ h2; } static constexpr bool use_int_hash_for_pods = false; }; struct MurmurHash3Impl64 { static constexpr auto name = "murmurHash3_64"; using ReturnType = UInt64; static UInt64 apply(const char * data, const size_t size) { union { UInt64 h[2]; char bytes[16]; }; MurmurHash3_x64_128(data, size, 0, bytes); return h[0] ^ h[1]; } static UInt64 combineHashes(UInt64 h1, UInt64 h2) { return IntHash64Impl::apply(h1) ^ h2; } static constexpr bool use_int_hash_for_pods = false; }; /// http://hg.openjdk.java.net/jdk8u/jdk8u/jdk/file/478a4add975b/src/share/classes/java/lang/String.java#l1452 /// Care should be taken to do all calculation in unsigned integers (to avoid undefined behaviour on overflow) /// but obtain the same result as it is done in singed integers with two's complement arithmetic. struct JavaHashImpl { static constexpr auto name = "javaHash"; using ReturnType = Int32; static Int32 apply(const char * data, const size_t size) { UInt32 h = 0; for (size_t i = 0; i < size; ++i) h = 31 * h + static_cast(static_cast(data[i])); return static_cast(h); } static Int32 combineHashes(Int32, Int32) { throw Exception("Java hash is not combineable for multiple arguments", ErrorCodes::NOT_IMPLEMENTED); } static constexpr bool use_int_hash_for_pods = false; }; struct JavaHashUTF16LEImpl { static constexpr auto name = "javaHashUTF16LE"; using ReturnType = Int32; static Int32 apply(const char * raw_data, const size_t raw_size) { char * data = const_cast(raw_data); size_t size = raw_size; // Remove Byte-order-mark(0xFFFE) for UTF-16LE if (size >= 2 && data[0] == '\xFF' && data[1] == '\xFE') { data += 2; size -= 2; } if (size % 2 != 0) throw Exception("Arguments for javaHashUTF16LE must be in the form of UTF-16", ErrorCodes::LOGICAL_ERROR); UInt32 h = 0; for (size_t i = 0; i < size; i += 2) h = 31 * h + static_cast(static_cast(data[i]) | static_cast(data[i + 1]) << 8); return static_cast(h); } static Int32 combineHashes(Int32, Int32) { throw Exception("Java hash is not combineable for multiple arguments", ErrorCodes::NOT_IMPLEMENTED); } static constexpr bool use_int_hash_for_pods = false; }; /// This is just JavaHash with zeroed out sign bit. /// This function is used in Hive for versions before 3.0, /// after 3.0, Hive uses murmur-hash3. struct HiveHashImpl { static constexpr auto name = "hiveHash"; using ReturnType = Int32; static Int32 apply(const char * data, const size_t size) { return static_cast(0x7FFFFFFF & static_cast(JavaHashImpl::apply(data, size))); } static Int32 combineHashes(Int32, Int32) { throw Exception("Hive hash is not combineable for multiple arguments", ErrorCodes::NOT_IMPLEMENTED); } static constexpr bool use_int_hash_for_pods = false; }; struct MurmurHash3Impl128 { static constexpr auto name = "murmurHash3_128"; enum { length = 16 }; static void apply(const char * begin, const size_t size, unsigned char * out_char_data) { MurmurHash3_x64_128(begin, size, 0, out_char_data); } }; struct ImplCityHash64 { static constexpr auto name = "cityHash64"; using ReturnType = UInt64; using uint128_t = CityHash_v1_0_2::uint128; static auto combineHashes(UInt64 h1, UInt64 h2) { return CityHash_v1_0_2::Hash128to64(uint128_t(h1, h2)); } static auto apply(const char * s, const size_t len) { return CityHash_v1_0_2::CityHash64(s, len); } static constexpr bool use_int_hash_for_pods = true; }; // see farmhash.h for definition of NAMESPACE_FOR_HASH_FUNCTIONS struct ImplFarmHash64 { static constexpr auto name = "farmHash64"; using ReturnType = UInt64; using uint128_t = NAMESPACE_FOR_HASH_FUNCTIONS::uint128_t; static auto combineHashes(UInt64 h1, UInt64 h2) { return NAMESPACE_FOR_HASH_FUNCTIONS::Hash128to64(uint128_t(h1, h2)); } static auto apply(const char * s, const size_t len) { return NAMESPACE_FOR_HASH_FUNCTIONS::Hash64(s, len); } static constexpr bool use_int_hash_for_pods = true; }; struct ImplMetroHash64 { static constexpr auto name = "metroHash64"; using ReturnType = UInt64; using uint128_t = CityHash_v1_0_2::uint128; static auto combineHashes(UInt64 h1, UInt64 h2) { return CityHash_v1_0_2::Hash128to64(uint128_t(h1, h2)); } static auto apply(const char * s, const size_t len) { union { UInt64 u64; UInt8 u8[sizeof(u64)]; }; metrohash64_1(reinterpret_cast(s), len, 0, u8); return u64; } static constexpr bool use_int_hash_for_pods = true; }; #if USE_XXHASH struct ImplXxHash32 { static constexpr auto name = "xxHash32"; using ReturnType = UInt32; static auto apply(const char * s, const size_t len) { return XXH32(s, len, 0); } /** * With current implementation with more than 1 arguments it will give the results * non-reproducable from outside of CH. * * Proper way of combining several input is to use streaming mode of hash function * https://github.com/Cyan4973/xxHash/issues/114#issuecomment-334908566 * * In common case doable by init_state / update_state / finalize_state */ static auto combineHashes(UInt32 h1, UInt32 h2) { return IntHash32Impl::apply(h1) ^ h2; } static constexpr bool use_int_hash_for_pods = false; }; struct ImplXxHash64 { static constexpr auto name = "xxHash64"; using ReturnType = UInt64; using uint128_t = CityHash_v1_0_2::uint128; static auto apply(const char * s, const size_t len) { return XXH64(s, len, 0); } /* With current implementation with more than 1 arguments it will give the results non-reproducable from outside of CH. (see comment on ImplXxHash32). */ static auto combineHashes(UInt64 h1, UInt64 h2) { return CityHash_v1_0_2::Hash128to64(uint128_t(h1, h2)); } static constexpr bool use_int_hash_for_pods = false; }; #endif template class FunctionStringHashFixedString : public IFunction { public: static constexpr auto name = Impl::name; static FunctionPtr create(const Context &) { return std::make_shared(); } 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("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return std::make_shared(Impl::length); } bool useDefaultImplementationForConstants() const override { return true; } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override { if (const ColumnString * col_from = checkAndGetColumn(block.getByPosition(arguments[0]).column.get())) { auto col_to = ColumnFixedString::create(Impl::length); const typename ColumnString::Chars & data = col_from->getChars(); const typename ColumnString::Offsets & offsets = col_from->getOffsets(); auto & chars_to = col_to->getChars(); const auto size = offsets.size(); chars_to.resize(size * Impl::length); ColumnString::Offset current_offset = 0; for (size_t i = 0; i < size; ++i) { Impl::apply( reinterpret_cast(&data[current_offset]), offsets[i] - current_offset - 1, &chars_to[i * Impl::length]); current_offset = offsets[i]; } block.getByPosition(result).column = std::move(col_to); } else throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); } }; template class FunctionIntHash : public IFunction { public: static constexpr auto name = Name::name; static FunctionPtr create(const Context &) { return std::make_shared(); } private: using ToType = typename Impl::ReturnType; template void executeType(Block & block, const ColumnNumbers & arguments, size_t result) { if (auto col_from = checkAndGetColumn>(block.getByPosition(arguments[0]).column.get())) { auto col_to = ColumnVector::create(); const typename ColumnVector::Container & vec_from = col_from->getData(); typename ColumnVector::Container & vec_to = col_to->getData(); size_t size = vec_from.size(); vec_to.resize(size); for (size_t i = 0; i < size; ++i) vec_to[i] = Impl::apply(vec_from[i]); block.getByPosition(result).column = std::move(col_to); } else throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + Name::name, ErrorCodes::ILLEGAL_COLUMN); } public: String getName() const override { return name; } size_t getNumberOfArguments() const override { return 1; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (!arguments[0]->isValueRepresentedByNumber()) throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return std::make_shared>(); } bool useDefaultImplementationForConstants() const override { return true; } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override { const IDataType * from_type = block.getByPosition(arguments[0]).type.get(); WhichDataType which(from_type); if (which.isUInt8()) executeType(block, arguments, result); else if (which.isUInt16()) executeType(block, arguments, result); else if (which.isUInt32()) executeType(block, arguments, result); else if (which.isUInt64()) executeType(block, arguments, result); else if (which.isInt8()) executeType(block, arguments, result); else if (which.isInt16()) executeType(block, arguments, result); else if (which.isInt32()) executeType(block, arguments, result); else if (which.isInt64()) executeType(block, arguments, result); else if (which.isDate()) executeType(block, arguments, result); else if (which.isDateTime()) executeType(block, arguments, result); else throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } }; template class FunctionAnyHash : public IFunction { public: static constexpr auto name = Impl::name; static FunctionPtr create(const Context &) { return std::make_shared(); } private: using ToType = typename Impl::ReturnType; template void executeIntType(const IColumn * column, typename ColumnVector::Container & vec_to) { if (const ColumnVector * col_from = checkAndGetColumn>(column)) { const typename ColumnVector::Container & vec_from = col_from->getData(); size_t size = vec_from.size(); for (size_t i = 0; i < size; ++i) { ToType h; if constexpr (Impl::use_int_hash_for_pods) { if constexpr (std::is_same_v) h = IntHash64Impl::apply(ext::bit_cast(vec_from[i])); else h = IntHash32Impl::apply(ext::bit_cast(vec_from[i])); } else { h = Impl::apply(reinterpret_cast(&vec_from[i]), sizeof(vec_from[i])); } if (first) vec_to[i] = h; else vec_to[i] = Impl::combineHashes(vec_to[i], h); } } else if (auto col_from_const = checkAndGetColumnConst>(column)) { auto value = col_from_const->template getValue(); ToType hash; if constexpr (std::is_same_v) hash = IntHash64Impl::apply(ext::bit_cast(value)); else hash = IntHash32Impl::apply(ext::bit_cast(value)); size_t size = vec_to.size(); if (first) { vec_to.assign(size, hash); } else { for (size_t i = 0; i < size; ++i) vec_to[i] = Impl::combineHashes(vec_to[i], hash); } } else throw Exception("Illegal column " + column->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); } template void executeGeneric(const IColumn * column, typename ColumnVector::Container & vec_to) { for (size_t i = 0, size = column->size(); i < size; ++i) { StringRef bytes = column->getDataAt(i); const ToType h = Impl::apply(bytes.data, bytes.size); if (first) vec_to[i] = h; else vec_to[i] = Impl::combineHashes(vec_to[i], h); } } template void executeString(const IColumn * column, typename ColumnVector::Container & vec_to) { if (const ColumnString * col_from = checkAndGetColumn(column)) { const typename ColumnString::Chars & data = col_from->getChars(); const typename ColumnString::Offsets & offsets = col_from->getOffsets(); size_t size = offsets.size(); ColumnString::Offset current_offset = 0; for (size_t i = 0; i < size; ++i) { const ToType h = Impl::apply( reinterpret_cast(&data[current_offset]), offsets[i] - current_offset - 1); if (first) vec_to[i] = h; else vec_to[i] = Impl::combineHashes(vec_to[i], h); current_offset = offsets[i]; } } else if (const ColumnFixedString * col_from_fixed = checkAndGetColumn(column)) { const typename ColumnString::Chars & data = col_from_fixed->getChars(); size_t n = col_from_fixed->getN(); size_t size = data.size() / n; for (size_t i = 0; i < size; ++i) { const ToType h = Impl::apply(reinterpret_cast(&data[i * n]), n); if (first) vec_to[i] = h; else vec_to[i] = Impl::combineHashes(vec_to[i], h); } } else if (const ColumnConst * col_from_const = checkAndGetColumnConstStringOrFixedString(column)) { String value = col_from_const->getValue().data(); const ToType hash = Impl::apply(value.data(), value.size()); const size_t size = vec_to.size(); if (first) { vec_to.assign(size, hash); } else { for (size_t i = 0; i < size; ++i) { vec_to[i] = Impl::combineHashes(vec_to[i], hash); } } } else throw Exception("Illegal column " + column->getName() + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); } template void executeArray(const IDataType * type, const IColumn * column, typename ColumnVector::Container & vec_to) { const IDataType * nested_type = typeid_cast(type)->getNestedType().get(); if (const ColumnArray * col_from = checkAndGetColumn(column)) { const IColumn * nested_column = &col_from->getData(); const ColumnArray::Offsets & offsets = col_from->getOffsets(); const size_t nested_size = nested_column->size(); typename ColumnVector::Container vec_temp(nested_size); executeAny(nested_type, nested_column, vec_temp); const size_t size = offsets.size(); ColumnArray::Offset current_offset = 0; for (size_t i = 0; i < size; ++i) { ColumnArray::Offset next_offset = offsets[i]; ToType h; if constexpr (std::is_same_v) h = IntHash64Impl::apply(next_offset - current_offset); else h = IntHash32Impl::apply(next_offset - current_offset); if (first) vec_to[i] = h; else vec_to[i] = Impl::combineHashes(vec_to[i], h); for (size_t j = current_offset; j < next_offset; ++j) vec_to[i] = Impl::combineHashes(vec_to[i], vec_temp[j]); current_offset = offsets[i]; } } else if (const ColumnConst * col_from_const = checkAndGetColumnConst(column)) { /// NOTE: here, of course, you can do without the materialization of the column. ColumnPtr full_column = col_from_const->convertToFullColumn(); executeArray(type, &*full_column, vec_to); } else throw Exception("Illegal column " + column->getName() + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); } template void executeAny(const IDataType * from_type, const IColumn * icolumn, typename ColumnVector::Container & vec_to) { WhichDataType which(from_type); if (which.isUInt8()) executeIntType(icolumn, vec_to); else if (which.isUInt16()) executeIntType(icolumn, vec_to); else if (which.isUInt32()) executeIntType(icolumn, vec_to); else if (which.isUInt64()) executeIntType(icolumn, vec_to); else if (which.isInt8()) executeIntType(icolumn, vec_to); else if (which.isInt16()) executeIntType(icolumn, vec_to); else if (which.isInt32()) executeIntType(icolumn, vec_to); else if (which.isInt64()) executeIntType(icolumn, vec_to); else if (which.isEnum8()) executeIntType(icolumn, vec_to); else if (which.isEnum16()) executeIntType(icolumn, vec_to); else if (which.isDate()) executeIntType(icolumn, vec_to); else if (which.isDateTime()) executeIntType(icolumn, vec_to); else if (which.isFloat32()) executeIntType(icolumn, vec_to); else if (which.isFloat64()) executeIntType(icolumn, vec_to); else if (which.isString()) executeString(icolumn, vec_to); else if (which.isFixedString()) executeString(icolumn, vec_to); else if (which.isArray()) executeArray(from_type, icolumn, vec_to); else executeGeneric(icolumn, vec_to); } void executeForArgument(const IDataType * type, const IColumn * column, typename ColumnVector::Container & vec_to, bool & is_first) { /// Flattening of tuples. if (const ColumnTuple * tuple = typeid_cast(column)) { const auto & tuple_columns = tuple->getColumns(); const DataTypes & tuple_types = typeid_cast(*type).getElements(); size_t tuple_size = tuple_columns.size(); for (size_t i = 0; i < tuple_size; ++i) executeForArgument(tuple_types[i].get(), tuple_columns[i].get(), vec_to, is_first); } else if (const ColumnTuple * tuple_const = checkAndGetColumnConstData(column)) { const auto & tuple_columns = tuple_const->getColumns(); const DataTypes & tuple_types = typeid_cast(*type).getElements(); size_t tuple_size = tuple_columns.size(); for (size_t i = 0; i < tuple_size; ++i) { auto tmp = ColumnConst::create(tuple_columns[i], column->size()); executeForArgument(tuple_types[i].get(), tmp.get(), vec_to, is_first); } } else { if (is_first) executeAny(type, column, vec_to); else executeAny(type, column, vec_to); } is_first = false; } public: String getName() const override { return name; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } bool useDefaultImplementationForConstants() const override { return true; } DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override { return std::make_shared>(); } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override { size_t rows = input_rows_count; auto col_to = ColumnVector::create(rows); typename ColumnVector::Container & vec_to = col_to->getData(); if (arguments.empty()) { /// Constant random number from /dev/urandom is used as a hash value of empty list of arguments. vec_to.assign(rows, static_cast(0xe28dbde7fe22e41c)); } /// The function supports arbitrary number of arguments of arbitrary types. bool is_first_argument = true; for (size_t i = 0; i < arguments.size(); ++i) { const ColumnWithTypeAndName & col = block.getByPosition(arguments[i]); executeForArgument(col.type.get(), col.column.get(), vec_to, is_first_argument); } block.getByPosition(result).column = std::move(col_to); } }; struct URLHashImpl { static UInt64 apply(const char * data, const size_t size) { /// do not take last slash, '?' or '#' character into account if (size > 0 && (data[size - 1] == '/' || data[size - 1] == '?' || data[size - 1] == '#')) return CityHash_v1_0_2::CityHash64(data, size - 1); return CityHash_v1_0_2::CityHash64(data, size); } }; struct URLHierarchyHashImpl { static size_t findLevelLength(const UInt64 level, const char * begin, const char * end) { auto pos = begin; /// Let's parse everything that goes before the path /// Suppose that the protocol has already been changed to lowercase. while (pos < end && ((*pos > 'a' && *pos < 'z') || (*pos > '0' && *pos < '9'))) ++pos; /** We will calculate the hierarchy only for URLs in which there is a protocol, and after it there are two slashes. * (http, file - fit, mailto, magnet - do not fit), and after two slashes there is still something * For the rest, simply return the full URL as the only element of the hierarchy. */ if (pos == begin || pos == end || !(*pos++ == ':' && pos < end && *pos++ == '/' && pos < end && *pos++ == '/' && pos < end)) { pos = end; return 0 == level ? pos - begin : 0; } /// The domain for simplicity is everything that after the protocol and the two slashes, until the next slash or before `?` or `#` while (pos < end && !(*pos == '/' || *pos == '?' || *pos == '#')) ++pos; if (pos != end) ++pos; if (0 == level) return pos - begin; UInt64 current_level = 0; while (current_level != level && pos < end) { /// We go to the next `/` or `?` or `#`, skipping all at the beginning. while (pos < end && (*pos == '/' || *pos == '?' || *pos == '#')) ++pos; if (pos == end) break; while (pos < end && !(*pos == '/' || *pos == '?' || *pos == '#')) ++pos; if (pos != end) ++pos; ++current_level; } return current_level == level ? pos - begin : 0; } static UInt64 apply(const UInt64 level, const char * data, const size_t size) { return URLHashImpl::apply(data, findLevelLength(level, data, data + size)); } }; class FunctionURLHash : public IFunction { public: static constexpr auto name = "URLHash"; static FunctionPtr create(const Context &) { return std::make_shared(); } String getName() const override { return name; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { const auto arg_count = arguments.size(); if (arg_count != 1 && arg_count != 2) throw Exception{"Number of arguments for function " + getName() + " doesn't match: passed " + toString(arg_count) + ", should be 1 or 2.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; const auto first_arg = arguments.front().get(); if (!WhichDataType(first_arg).isString()) throw Exception{"Illegal type " + first_arg->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; if (arg_count == 2) { const auto & second_arg = arguments.back(); if (!isInteger(second_arg)) throw Exception{"Illegal type " + second_arg->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } return std::make_shared(); } bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override { const auto arg_count = arguments.size(); if (arg_count == 1) executeSingleArg(block, arguments, result); else if (arg_count == 2) executeTwoArgs(block, arguments, result); else throw Exception{"got into IFunction::execute with unexpected number of arguments", ErrorCodes::LOGICAL_ERROR}; } private: void executeSingleArg(Block & block, const ColumnNumbers & arguments, const size_t result) const { const auto col_untyped = block.getByPosition(arguments.front()).column.get(); if (const auto col_from = checkAndGetColumn(col_untyped)) { const auto size = col_from->size(); auto col_to = ColumnUInt64::create(size); const auto & chars = col_from->getChars(); const auto & offsets = col_from->getOffsets(); auto & out = col_to->getData(); ColumnString::Offset current_offset = 0; for (size_t i = 0; i < size; ++i) { out[i] = URLHashImpl::apply( reinterpret_cast(&chars[current_offset]), offsets[i] - current_offset - 1); current_offset = offsets[i]; } block.getByPosition(result).column = std::move(col_to); } else throw Exception{"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; } void executeTwoArgs(Block & block, const ColumnNumbers & arguments, const size_t result) const { const auto level_col = block.getByPosition(arguments.back()).column.get(); if (!isColumnConst(*level_col)) throw Exception{"Second argument of function " + getName() + " must be an integral constant", ErrorCodes::ILLEGAL_COLUMN}; const auto level = level_col->get64(0); const auto col_untyped = block.getByPosition(arguments.front()).column.get(); if (const auto col_from = checkAndGetColumn(col_untyped)) { const auto size = col_from->size(); auto col_to = ColumnUInt64::create(size); const auto & chars = col_from->getChars(); const auto & offsets = col_from->getOffsets(); auto & out = col_to->getData(); ColumnString::Offset current_offset = 0; for (size_t i = 0; i < size; ++i) { out[i] = URLHierarchyHashImpl::apply( level, reinterpret_cast(&chars[current_offset]), offsets[i] - current_offset - 1); current_offset = offsets[i]; } block.getByPosition(result).column = std::move(col_to); } else throw Exception{"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; } }; struct NameIntHash32 { static constexpr auto name = "intHash32"; }; struct NameIntHash64 { static constexpr auto name = "intHash64"; }; #if USE_SSL using FunctionHalfMD5 = FunctionAnyHash; #endif using FunctionSipHash64 = FunctionAnyHash; using FunctionIntHash32 = FunctionIntHash; using FunctionIntHash64 = FunctionIntHash; #if USE_SSL using FunctionMD5 = FunctionStringHashFixedString; using FunctionSHA1 = FunctionStringHashFixedString; using FunctionSHA224 = FunctionStringHashFixedString; using FunctionSHA256 = FunctionStringHashFixedString; #endif using FunctionSipHash128 = FunctionStringHashFixedString; using FunctionCityHash64 = FunctionAnyHash; using FunctionFarmHash64 = FunctionAnyHash; using FunctionMetroHash64 = FunctionAnyHash; using FunctionMurmurHash2_32 = FunctionAnyHash; using FunctionMurmurHash2_64 = FunctionAnyHash; using FunctionGccMurmurHash = FunctionAnyHash; using FunctionMurmurHash3_32 = FunctionAnyHash; using FunctionMurmurHash3_64 = FunctionAnyHash; using FunctionMurmurHash3_128 = FunctionStringHashFixedString; using FunctionJavaHash = FunctionAnyHash; using FunctionJavaHashUTF16LE = FunctionAnyHash; using FunctionHiveHash = FunctionAnyHash; #if USE_XXHASH using FunctionXxHash32 = FunctionAnyHash; using FunctionXxHash64 = FunctionAnyHash; #endif }