diff --git a/dbms/src/Functions/FunctionsHashing.h b/dbms/src/Functions/FunctionsHashing.h index 9ea16bc09fe..84b61062398 100644 --- a/dbms/src/Functions/FunctionsHashing.h +++ b/dbms/src/Functions/FunctionsHashing.h @@ -54,8 +54,10 @@ namespace ErrorCodes * Fast non-cryptographic hash function for strings: * cityHash64: String -> UInt64 * - * A non-cryptographic hash from a tuple of values of any types (uses cityHash64 for strings and intHash64 for numbers): + * 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 @@ -63,8 +65,31 @@ namespace ErrorCodes * */ +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); + } +}; + + struct HalfMD5Impl { + static constexpr auto name = "halfMD5"; using ReturnType = UInt64; static UInt64 apply(const char * begin, size_t size) @@ -82,6 +107,11 @@ struct HalfMD5Impl return Poco::ByteOrder::flipBytes(buf.uint64_data); /// Compatibility with existing code. } + + static UInt64 mergeHashes(UInt64 h1, UInt64 h2) + { + return IntHash64Impl::apply(h1) ^ h2; + } }; struct MD5Impl @@ -142,12 +172,18 @@ struct SHA256Impl 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 mergeHashes(UInt64 h1, UInt64 h2){ + return IntHash64Impl::apply(h1) ^ h2; + } }; @@ -162,27 +198,6 @@ struct SipHash128Impl } }; -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); - } -}; - template class FunctionStringHashFixedString : public IFunction @@ -322,19 +337,18 @@ public: }; -/** We use hash functions called CityHash, FarmHash, MetroHash. - * In this regard, this template is named with the words `NeighborhoodHash`. - */ template -class FunctionNeighbourhoodHash64 : public IFunction +class FunctionAnyHash : public IFunction { public: static constexpr auto name = Impl::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(const Context &) { return std::make_shared(); } private: + using ToType = typename Impl::ReturnType; + template - void executeIntType(const IColumn * column, ColumnUInt64::Container & vec_to) + void executeIntType(const IColumn * column, typename ColumnVector::Container & vec_to) { if (const ColumnVector * col_from = checkAndGetColumn>(column)) { @@ -342,16 +356,26 @@ private: size_t size = vec_from.size(); for (size_t i = 0; i < size; ++i) { - UInt64 h = IntHash64Impl::apply(ext::bit_cast(vec_from[i])); + ToType h; + 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])); if (first) vec_to[i] = h; else - vec_to[i] = Impl::Hash128to64(typename Impl::uint128_t(vec_to[i], h)); + vec_to[i] = Impl::mergeHashes(vec_to[i], h); } } else if (auto col_from = checkAndGetColumnConst>(column)) { - const UInt64 hash = IntHash64Impl::apply(ext::bit_cast(col_from->template getValue())); + auto value = col_from->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) { @@ -360,7 +384,7 @@ private: else { for (size_t i = 0; i < size; ++i) - vec_to[i] = Impl::Hash128to64(typename Impl::uint128_t(vec_to[i], hash)); + vec_to[i] = Impl::mergeHashes(vec_to[i], hash); } } else @@ -370,7 +394,7 @@ private: } template - void executeString(const IColumn * column, ColumnUInt64::Container & vec_to) + void executeString(const IColumn * column, typename ColumnVector::Container & vec_to) { if (const ColumnString * col_from = checkAndGetColumn(column)) { @@ -381,14 +405,14 @@ private: ColumnString::Offset current_offset = 0; for (size_t i = 0; i < size; ++i) { - const UInt64 h = Impl::Hash64( + 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::Hash128to64(typename Impl::uint128_t(vec_to[i], h)); + vec_to[i] = Impl::mergeHashes(vec_to[i], h); current_offset = offsets[i]; } @@ -401,17 +425,17 @@ private: for (size_t i = 0; i < size; ++i) { - const UInt64 h = Impl::Hash64(reinterpret_cast(&data[i * n]), n); + const ToType h = Impl::apply(reinterpret_cast(&data[i * n]), n); if (first) vec_to[i] = h; else - vec_to[i] = Impl::Hash128to64(typename Impl::uint128_t(vec_to[i], h)); + vec_to[i] = Impl::mergeHashes(vec_to[i], h); } } else if (const ColumnConst * col_from = checkAndGetColumnConstStringOrFixedString(column)) { String value = col_from->getValue().data(); - const UInt64 hash = Impl::Hash64(value.data(), value.size()); + const ToType hash = Impl::apply(value.data(), value.size()); const size_t size = vec_to.size(); if (first) @@ -422,7 +446,7 @@ private: { for (size_t i = 0; i < size; ++i) { - vec_to[i] = Impl::Hash128to64(typename Impl::uint128_t(vec_to[i], hash)); + vec_to[i] = Impl::mergeHashes(vec_to[i], hash); } } } @@ -433,7 +457,7 @@ private: } template - void executeArray(const IDataType * type, const IColumn * column, ColumnUInt64::Container & vec_to) + void executeArray(const IDataType * type, const IColumn * column, typename ColumnVector::Container & vec_to) { const IDataType * nested_type = typeid_cast(type)->getNestedType().get(); @@ -443,7 +467,7 @@ private: const ColumnArray::Offsets & offsets = col_from->getOffsets(); const size_t nested_size = nested_column->size(); - ColumnUInt64::Container vec_temp(nested_size); + typename ColumnVector::Container vec_temp(nested_size); executeAny(nested_type, nested_column, vec_temp); const size_t size = offsets.size(); @@ -453,14 +477,19 @@ private: { ColumnArray::Offset next_offset = offsets[i]; - UInt64 h = IntHash64Impl::apply(next_offset - current_offset); + 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::Hash128to64(typename Impl::uint128_t(vec_to[i], h)); + vec_to[i] = Impl::mergeHashes(vec_to[i], h); for (size_t j = current_offset; j < next_offset; ++j) - vec_to[i] = Impl::Hash128to64(typename Impl::uint128_t(vec_to[i], vec_temp[j])); + vec_to[i] = Impl::mergeHashes(vec_to[i], vec_temp[j]); current_offset = offsets[i]; } @@ -478,7 +507,7 @@ private: } template - void executeAny(const IDataType * from_type, const IColumn * icolumn, ColumnUInt64::Container & vec_to) + void executeAny(const IDataType * from_type, const IColumn * icolumn, typename ColumnVector::Container & vec_to) { WhichDataType which(from_type); @@ -504,7 +533,7 @@ private: ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } - void executeForArgument(const IDataType * type, const IColumn * column, ColumnUInt64::Container & vec_to, bool & is_first) + 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)) @@ -549,20 +578,20 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override { - return std::make_shared(); + 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 = ColumnUInt64::create(rows); + auto col_to = ColumnVector::create(rows); - ColumnUInt64::Container & vec_to = col_to->getData(); + 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)); + vec_to.assign(rows, static_cast(0xe28dbde7fe22e41c)); } /// The function supports arbitrary number of arguments of arbitrary types. @@ -579,110 +608,6 @@ public: }; -template -class FunctionStringHash : public IFunction -{ -public: - static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } - - String getName() const override { return name; } - - bool isVariadic() const override { return false; } - - size_t getNumberOfArguments() const override { return 1; } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments */) const override - { 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 - { - auto col_to = ColumnVector::create(input_rows_count); - typename ColumnVector::Container & vec_to = col_to->getData(); - - const ColumnWithTypeAndName & col = block.getByPosition(arguments[0]); - const IDataType * from_type = col.type.get(); - const IColumn * icolumn = col.column.get(); - 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.isStringOrFixedString()) executeString(icolumn, vec_to); - else - throw Exception("Unexpected type " + from_type->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - block.getByPosition(result).column = std::move(col_to); - } -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) - { - vec_to[i] = Impl::apply(reinterpret_cast(&vec_from[i]), sizeof(FromType)); - } - } - else - throw Exception("Illegal column " + column->getName() - + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); - } - - void executeString(const IColumn * column, typename ColumnVector::Container & vec_to) - { - if (const ColumnString * col_from = checkAndGetColumn(column)) - { - const typename ColumnString::Chars_t & 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) - { - vec_to[i] = Impl::apply( - reinterpret_cast(&data[current_offset]), - offsets[i] - current_offset - 1); - - current_offset = offsets[i]; - } - } - else if (const ColumnFixedString * col_from = checkAndGetColumn(column)) - { - const typename ColumnString::Chars_t & data = col_from->getChars(); - size_t n = col_from->getN(); - size_t size = data.size() / n; - for (size_t i = 0; i < size; ++i) - vec_to[i] = Impl::apply(reinterpret_cast(&data[i * n]), n); - } - else - throw Exception("Illegal column " + column->getName() - + " of first argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); - } -}; - - /** 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. @@ -692,26 +617,41 @@ private: */ 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 mergeHashes(UInt32 h1, UInt32 h2) + { + return IntHash32Impl::apply(h1) ^ h2; + } }; 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 mergeHashes(UInt64 h1, UInt64 h2){ + return IntHash64Impl::apply(h1) ^ h2; + } }; struct MurmurHash3Impl32 { + static constexpr auto name = "murmurHash3_32"; + using ReturnType = UInt32; static UInt32 apply(const char * data, const size_t size) @@ -724,10 +664,16 @@ struct MurmurHash3Impl32 MurmurHash3_x86_32(data, size, 0, bytes); return h; } + + static UInt32 mergeHashes(UInt32 h1, UInt32 h2) + { + return IntHash32Impl::apply(h1) ^ h2; + } }; struct MurmurHash3Impl64 { + static constexpr auto name = "murmurHash3_64"; using ReturnType = UInt64; static UInt64 apply(const char * data, const size_t size) @@ -740,6 +686,11 @@ struct MurmurHash3Impl64 MurmurHash3_x64_128(data, size, 0, bytes); return h[0] ^ h[1]; } + + static UInt64 mergeHashes(UInt64 h1, UInt64 h2) + { + return IntHash64Impl::apply(h1) ^ h2; + } }; struct MurmurHash3Impl128 @@ -943,43 +894,39 @@ private: }; -struct NameHalfMD5 { static constexpr auto name = "halfMD5"; }; -struct NameSipHash64 { static constexpr auto name = "sipHash64"; }; struct NameIntHash32 { static constexpr auto name = "intHash32"; }; struct NameIntHash64 { static constexpr auto name = "intHash64"; }; -struct NameMurmurHash2_32 { static constexpr auto name = "murmurHash2_32"; }; -struct NameMurmurHash2_64 { static constexpr auto name = "murmurHash2_64"; }; -struct NameMurmurHash3_32 { static constexpr auto name = "murmurHash3_32"; }; -struct NameMurmurHash3_64 { static constexpr auto name = "murmurHash3_64"; }; -struct NameMurmurHash3_128 { static constexpr auto name = "murmurHash3_128"; }; struct ImplCityHash64 { static constexpr auto name = "cityHash64"; + using ReturnType = UInt64; using uint128_t = CityHash_v1_0_2::uint128; - static auto Hash128to64(const uint128_t & x) { return CityHash_v1_0_2::Hash128to64(x); } - static auto Hash64(const char * s, const size_t len) { return CityHash_v1_0_2::CityHash64(s, len); } + static auto mergeHashes(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); } }; // 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 Hash128to64(const uint128_t & x) { return NAMESPACE_FOR_HASH_FUNCTIONS::Hash128to64(x); } - static auto Hash64(const char * s, const size_t len) { return NAMESPACE_FOR_HASH_FUNCTIONS::Hash64(s, len); } + static auto mergeHashes(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); } }; struct ImplMetroHash64 { static constexpr auto name = "metroHash64"; + using ReturnType = UInt64; using uint128_t = CityHash_v1_0_2::uint128; - static auto Hash128to64(const uint128_t & x) { return CityHash_v1_0_2::Hash128to64(x); } - static auto Hash64(const char * s, const size_t len) + static auto mergeHashes(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 { @@ -993,8 +940,8 @@ struct ImplMetroHash64 } }; -using FunctionHalfMD5 = FunctionStringHash; -using FunctionSipHash64 = FunctionStringHash; +using FunctionHalfMD5 = FunctionAnyHash; +using FunctionSipHash64 = FunctionAnyHash; using FunctionIntHash32 = FunctionIntHash; using FunctionIntHash64 = FunctionIntHash; using FunctionMD5 = FunctionStringHashFixedString; @@ -1002,12 +949,12 @@ using FunctionSHA1 = FunctionStringHashFixedString; using FunctionSHA224 = FunctionStringHashFixedString; using FunctionSHA256 = FunctionStringHashFixedString; using FunctionSipHash128 = FunctionStringHashFixedString; -using FunctionCityHash64 = FunctionNeighbourhoodHash64; -using FunctionFarmHash64 = FunctionNeighbourhoodHash64; -using FunctionMetroHash64 = FunctionNeighbourhoodHash64; -using FunctionMurmurHash2_32 = FunctionStringHash; -using FunctionMurmurHash2_64 = FunctionStringHash; -using FunctionMurmurHash3_32 = FunctionStringHash; -using FunctionMurmurHash3_64 = FunctionStringHash; +using FunctionCityHash64 = FunctionAnyHash; +using FunctionFarmHash64 = FunctionAnyHash; +using FunctionMetroHash64 = FunctionAnyHash; +using FunctionMurmurHash2_32 = FunctionAnyHash; +using FunctionMurmurHash2_64 = FunctionAnyHash; +using FunctionMurmurHash3_32 = FunctionAnyHash; +using FunctionMurmurHash3_64 = FunctionAnyHash; using FunctionMurmurHash3_128 = FunctionStringHashFixedString; } diff --git a/dbms/tests/queries/0_stateless/00678_murmurhash.reference b/dbms/tests/queries/0_stateless/00678_murmurhash.reference index 548c5c1cae6..f9d3ed38508 100644 --- a/dbms/tests/queries/0_stateless/00678_murmurhash.reference +++ b/dbms/tests/queries/0_stateless/00678_murmurhash.reference @@ -1,26 +1,26 @@ -623211862 -3533626746 -2388617433 -2708309598 +3012058918 +1298551497 +864444010 +367840556 2414502773 670491991 +1343103100 0 0 0 0 -0 -0 -0 -1 +1343103100 +1996614413 +kek 1 14834356025302342401 12725806677685968135 -12725806677685968135 +10577349846663553072 4138058784 3831157163 -3831157163 +1343103100 11303473983767132390 956517343494314387 -956517343494314387 +10577349846663553072 6145F501578671E2877DBA2BE487AF7E 16FE7483905CCE7A85670E43E4678877 diff --git a/dbms/tests/queries/0_stateless/00678_murmurhash.sql b/dbms/tests/queries/0_stateless/00678_murmurhash.sql index 9d20b56aa93..1b800c7eb9c 100644 --- a/dbms/tests/queries/0_stateless/00678_murmurhash.sql +++ b/dbms/tests/queries/0_stateless/00678_murmurhash.sql @@ -13,7 +13,6 @@ SELECT murmurHash2_32('\x03\0\0'); SELECT murmurHash2_32(1); SELECT murmurHash2_32(toUInt16(2)); -SELECT murmurHash2_32(2) = bitXor(toUInt32(0x5bd1e995 * bitXor(toUInt32(3 * 0x5bd1e995) AS a, bitShiftRight(a, 13))) AS b, bitShiftRight(b, 15)); SELECT murmurHash2_32('\x02') = bitXor(toUInt32(0x5bd1e995 * bitXor(toUInt32(3 * 0x5bd1e995) AS a, bitShiftRight(a, 13))) AS b, bitShiftRight(b, 15)); SELECT murmurHash2_64('foo'); diff --git a/dbms/tests/queries/0_stateless/00746_hashing_tuples.reference b/dbms/tests/queries/0_stateless/00746_hashing_tuples.reference new file mode 100644 index 00000000000..b38b216a8d6 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00746_hashing_tuples.reference @@ -0,0 +1,12 @@ +6847376565456338547 +15499510486101262177 +3822366986039497337 +617416965 +3293554683 +15433379 +6847376565456338547 +15499510486101262177 +3137889964064254064 +6847376565456338547 +15499510486101262177 +6284898493105666575 diff --git a/dbms/tests/queries/0_stateless/00746_hashing_tuples.sql b/dbms/tests/queries/0_stateless/00746_hashing_tuples.sql new file mode 100644 index 00000000000..ce26225d730 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00746_hashing_tuples.sql @@ -0,0 +1,15 @@ +SELECT sipHash64(1, 2, 3); +SELECT sipHash64(1, 3, 2); +SELECT sipHash64('a', [1, 2, 3], 4); + +SELECT murmurHash2_32(1, 2, 3); +SELECT murmurHash2_32(1, 3, 2); +SELECT murmurHash2_32('a', [1, 2, 3], 4); + +SELECT murmurHash2_64(1, 2, 3); +SELECT murmurHash2_64(1, 3, 2); +SELECT murmurHash2_64('a', [1, 2, 3], 4); + +SELECT murmurHash3_64(1, 2, 3); +SELECT murmurHash3_64(1, 3, 2); +SELECT murmurHash3_64('a', [1, 2, 3], 4);