Introduce uniqCombined64() to get sane results for cardinality > UINT_MAX

By default uniqCombined() uses 32-bit hash for all types except String,
so this means that you cannot use uniqCombined() with i.e UInt64 and
cardinality > UINT_MAX, although you can use uniqCombined(toString())
and this will lead to 64-bit hash, but will not have good performance.

So uniqCombined64() had been introduced to fix this.

Requires: #7213
This commit is contained in:
Azat Khuzhin 2019-10-08 00:44:37 +03:00
parent 15deedb420
commit f46c5a47c0
6 changed files with 105 additions and 60 deletions

View File

@ -6,6 +6,8 @@
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <functional>
namespace DB
{
namespace ErrorCodes
@ -17,17 +19,17 @@ namespace ErrorCodes
namespace
{
template <UInt8 K>
template <UInt8 K, typename HashValueType>
struct WithK
{
template <typename T>
using AggregateFunction = AggregateFunctionUniqCombined<T, K>;
using AggregateFunction = AggregateFunctionUniqCombined<T, K, HashValueType>;
template <bool is_exact, bool argument_is_tuple>
using AggregateFunctionVariadic = AggregateFunctionUniqCombinedVariadic<is_exact, argument_is_tuple, K>;
using AggregateFunctionVariadic = AggregateFunctionUniqCombinedVariadic<is_exact, argument_is_tuple, K, HashValueType>;
};
template <UInt8 K>
template <UInt8 K, typename HashValueType>
AggregateFunctionPtr createAggregateFunctionWithK(const DataTypes & argument_types, const Array & params)
{
/// We use exact hash function if the arguments are not contiguous in memory, because only exact hash function has support for this case.
@ -37,36 +39,45 @@ namespace
{
const IDataType & argument_type = *argument_types[0];
AggregateFunctionPtr res(createWithNumericType<WithK<K>::template AggregateFunction>(*argument_types[0], argument_types, params));
AggregateFunctionPtr res(createWithNumericType<WithK<K, HashValueType>::template AggregateFunction>(*argument_types[0], argument_types, params));
WhichDataType which(argument_type);
if (res)
return res;
else if (which.isDate())
return std::make_shared<typename WithK<K>::template AggregateFunction<DataTypeDate::FieldType>>(argument_types, params);
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<DataTypeDate::FieldType>>(argument_types, params);
else if (which.isDateTime())
return std::make_shared<typename WithK<K>::template AggregateFunction<DataTypeDateTime::FieldType>>(argument_types, params);
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<DataTypeDateTime::FieldType>>(argument_types, params);
else if (which.isStringOrFixedString())
return std::make_shared<typename WithK<K>::template AggregateFunction<String>>(argument_types, params);
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<String>>(argument_types, params);
else if (which.isUUID())
return std::make_shared<typename WithK<K>::template AggregateFunction<DataTypeUUID::FieldType>>(argument_types, params);
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<DataTypeUUID::FieldType>>(argument_types, params);
else if (which.isTuple())
{
if (use_exact_hash_function)
return std::make_shared<typename WithK<K>::template AggregateFunctionVariadic<true, true>>(argument_types, params);
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunctionVariadic<true, true>>(argument_types, params);
else
return std::make_shared<typename WithK<K>::template AggregateFunctionVariadic<false, true>>(argument_types, params);
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunctionVariadic<false, true>>(argument_types, params);
}
}
/// "Variadic" method also works as a fallback generic case for a single argument.
if (use_exact_hash_function)
return std::make_shared<typename WithK<K>::template AggregateFunctionVariadic<true, false>>(argument_types, params);
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunctionVariadic<true, false>>(argument_types, params);
else
return std::make_shared<typename WithK<K>::template AggregateFunctionVariadic<false, false>>(argument_types, params);
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunctionVariadic<false, false>>(argument_types, params);
}
AggregateFunctionPtr createAggregateFunctionUniqCombined(
template <UInt8 K>
AggregateFunctionPtr createAggregateFunctionWithHashType(bool use_64_bit_hash, const DataTypes & argument_types, const Array & params)
{
if (use_64_bit_hash)
return createAggregateFunctionWithK<K, UInt64>(argument_types, params);
else
return createAggregateFunctionWithK<K, UInt32>(argument_types, params);
}
AggregateFunctionPtr createAggregateFunctionUniqCombined(bool use_64_bit_hash,
const std::string & name, const DataTypes & argument_types, const Array & params)
{
/// log2 of the number of cells in HyperLogLog.
@ -80,12 +91,10 @@ namespace
"Aggregate function " + name + " requires one parameter or less.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
UInt64 precision_param = applyVisitor(FieldVisitorConvertToNumber<UInt64>(), params[0]);
// This range is hardcoded below
if (precision_param > 20 || precision_param < 12)
throw Exception(
"Parameter for aggregate function " + name + "is out or range: [12, 20].", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
"Parameter for aggregate function " + name + " is out or range: [12, 20].", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
precision = precision_param;
}
@ -95,23 +104,23 @@ namespace
switch (precision)
{
case 12:
return createAggregateFunctionWithK<12>(argument_types, params);
return createAggregateFunctionWithHashType<12>(use_64_bit_hash, argument_types, params);
case 13:
return createAggregateFunctionWithK<13>(argument_types, params);
return createAggregateFunctionWithHashType<13>(use_64_bit_hash, argument_types, params);
case 14:
return createAggregateFunctionWithK<14>(argument_types, params);
return createAggregateFunctionWithHashType<14>(use_64_bit_hash, argument_types, params);
case 15:
return createAggregateFunctionWithK<15>(argument_types, params);
return createAggregateFunctionWithHashType<15>(use_64_bit_hash, argument_types, params);
case 16:
return createAggregateFunctionWithK<16>(argument_types, params);
return createAggregateFunctionWithHashType<16>(use_64_bit_hash, argument_types, params);
case 17:
return createAggregateFunctionWithK<17>(argument_types, params);
return createAggregateFunctionWithHashType<17>(use_64_bit_hash, argument_types, params);
case 18:
return createAggregateFunctionWithK<18>(argument_types, params);
return createAggregateFunctionWithHashType<18>(use_64_bit_hash, argument_types, params);
case 19:
return createAggregateFunctionWithK<19>(argument_types, params);
return createAggregateFunctionWithHashType<19>(use_64_bit_hash, argument_types, params);
case 20:
return createAggregateFunctionWithK<20>(argument_types, params);
return createAggregateFunctionWithHashType<20>(use_64_bit_hash, argument_types, params);
}
__builtin_unreachable();
@ -121,7 +130,9 @@ namespace
void registerAggregateFunctionUniqCombined(AggregateFunctionFactory & factory)
{
factory.registerFunction("uniqCombined", createAggregateFunctionUniqCombined);
using namespace std::placeholders;
factory.registerFunction("uniqCombined", std::bind(createAggregateFunctionUniqCombined, false, _1, _2, _3));
factory.registerFunction("uniqCombined64", std::bind(createAggregateFunctionUniqCombined, true, _1, _2, _3));
}
}

View File

@ -24,43 +24,43 @@ namespace DB
{
namespace detail
{
/** Hash function for uniqCombined.
/** Hash function for uniqCombined/uniqCombined64 (based on Ret).
*/
template <typename T>
template <typename T, typename Ret>
struct AggregateFunctionUniqCombinedTraits
{
static UInt32 hash(T x)
static Ret hash(T x)
{
return static_cast<UInt32>(intHash64(x));
return static_cast<Ret>(intHash64(x));
}
};
template <>
struct AggregateFunctionUniqCombinedTraits<UInt128>
template <typename Ret>
struct AggregateFunctionUniqCombinedTraits<UInt128, Ret>
{
static UInt32 hash(UInt128 x)
static Ret hash(UInt128 x)
{
return sipHash64(x);
}
};
template <>
struct AggregateFunctionUniqCombinedTraits<Float32>
template <typename Ret>
struct AggregateFunctionUniqCombinedTraits<Float32, Ret>
{
static UInt32 hash(Float32 x)
static Ret hash(Float32 x)
{
UInt64 res = ext::bit_cast<UInt64>(x);
return static_cast<UInt32>(intHash64(res));
return static_cast<Ret>(intHash64(res));
}
};
template <>
struct AggregateFunctionUniqCombinedTraits<Float64>
template <typename Ret>
struct AggregateFunctionUniqCombinedTraits<Float64, Ret>
{
static UInt32 hash(Float64 x)
static Ret hash(Float64 x)
{
UInt64 res = ext::bit_cast<UInt64>(x);
return static_cast<UInt32>(intHash64(res));
return static_cast<Ret>(intHash64(res));
}
};
@ -98,29 +98,32 @@ struct AggregateFunctionUniqCombinedDataWithKey<Key, 17>
};
template <typename T, UInt8 K>
struct AggregateFunctionUniqCombinedData : public AggregateFunctionUniqCombinedDataWithKey<UInt32, K>
template <typename T, UInt8 K, typename HashValueType>
struct AggregateFunctionUniqCombinedData : public AggregateFunctionUniqCombinedDataWithKey<HashValueType, K>
{
};
template <UInt8 K>
struct AggregateFunctionUniqCombinedData<String, K> : public AggregateFunctionUniqCombinedDataWithKey<UInt64, K>
template <UInt8 K, typename HashValueType>
struct AggregateFunctionUniqCombinedData<String, K, HashValueType> : public AggregateFunctionUniqCombinedDataWithKey<UInt64 /*always*/, K>
{
};
template <typename T, UInt8 K>
template <typename T, UInt8 K, typename HashValueType>
class AggregateFunctionUniqCombined final
: public IAggregateFunctionDataHelper<AggregateFunctionUniqCombinedData<T, K>, AggregateFunctionUniqCombined<T, K>>
: public IAggregateFunctionDataHelper<AggregateFunctionUniqCombinedData<T, K, HashValueType>, AggregateFunctionUniqCombined<T, K, HashValueType>>
{
public:
AggregateFunctionUniqCombined(const DataTypes & argument_types_, const Array & params_)
: IAggregateFunctionDataHelper<AggregateFunctionUniqCombinedData<T, K>, AggregateFunctionUniqCombined<T, K>>(argument_types_, params_) {}
: IAggregateFunctionDataHelper<AggregateFunctionUniqCombinedData<T, K, HashValueType>, AggregateFunctionUniqCombined<T, K, HashValueType>>(argument_types_, params_) {}
String getName() const override
{
return "uniqCombined";
if constexpr (std::is_same_v<HashValueType, UInt64>)
return "uniqCombined64";
else
return "uniqCombined";
}
DataTypePtr getReturnType() const override
@ -133,7 +136,7 @@ public:
if constexpr (!std::is_same_v<T, String>)
{
const auto & value = assert_cast<const ColumnVector<T> &>(*columns[0]).getElement(row_num);
this->data(place).set.insert(detail::AggregateFunctionUniqCombinedTraits<T>::hash(value));
this->data(place).set.insert(detail::AggregateFunctionUniqCombinedTraits<T, HashValueType>::hash(value));
}
else
{
@ -172,17 +175,17 @@ public:
* You can pass multiple arguments as is; You can also pass one argument - a tuple.
* But (for the possibility of efficient implementation), you can not pass several arguments, among which there are tuples.
*/
template <bool is_exact, bool argument_is_tuple, UInt8 K>
class AggregateFunctionUniqCombinedVariadic final : public IAggregateFunctionDataHelper<AggregateFunctionUniqCombinedData<UInt64, K>,
AggregateFunctionUniqCombinedVariadic<is_exact, argument_is_tuple, K>>
template <bool is_exact, bool argument_is_tuple, UInt8 K, typename HashValueType>
class AggregateFunctionUniqCombinedVariadic final : public IAggregateFunctionDataHelper<AggregateFunctionUniqCombinedData<UInt64, K, HashValueType>,
AggregateFunctionUniqCombinedVariadic<is_exact, argument_is_tuple, K, HashValueType>>
{
private:
size_t num_args = 0;
public:
explicit AggregateFunctionUniqCombinedVariadic(const DataTypes & arguments, const Array & params)
: IAggregateFunctionDataHelper<AggregateFunctionUniqCombinedData<UInt64, K>,
AggregateFunctionUniqCombinedVariadic<is_exact, argument_is_tuple, K>>(arguments, params)
: IAggregateFunctionDataHelper<AggregateFunctionUniqCombinedData<UInt64, K, HashValueType>,
AggregateFunctionUniqCombinedVariadic<is_exact, argument_is_tuple, K, HashValueType>>(arguments, params)
{
if (argument_is_tuple)
num_args = typeid_cast<const DataTypeTuple &>(*arguments[0]).getElements().size();
@ -192,7 +195,10 @@ public:
String getName() const override
{
return "uniqCombined";
if constexpr (std::is_same_v<HashValueType, UInt64>)
return "uniqCombined64";
else
return "uniqCombined";
}
DataTypePtr getReturnType() const override
@ -202,7 +208,7 @@ public:
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override
{
this->data(place).set.insert(typename AggregateFunctionUniqCombinedData<UInt64, K>::Set::value_type(
this->data(place).set.insert(typename AggregateFunctionUniqCombinedData<UInt64, K, HashValueType>::Set::value_type(
UniqVariadicHash<is_exact, argument_is_tuple>::apply(num_args, columns, row_num)));
}

View File

@ -0,0 +1,2 @@
10021957
10021969

View File

@ -0,0 +1,9 @@
-- for small cardinality the 64 bit hash perform worse, but for 1e10:
-- 4 byte hash: 2.8832809652e10
-- 8 byte hash: 0.9998568925e10
-- but hence checking with 1e10 values takes too much time (~45 secs), this
-- test is just to ensure that the result is different (and to document the
-- outcome).
SELECT uniqCombined(number) FROM numbers(toUInt64(1e7));
SELECT uniqCombined64(number) FROM numbers(toUInt64(1e7));

View File

@ -546,6 +546,7 @@ We recommend using this function in almost all scenarios.
**See Also**
- [uniqCombined](#agg_function-uniqcombined)
- [uniqCombined64](#agg_function-uniqcombined64)
- [uniqHLL12](#agg_function-uniqhll12)
- [uniqExact](#agg_function-uniqexact)
@ -573,13 +574,16 @@ The function takes a variable number of parameters. Parameters can be `Tuple`, `
Function:
- Calculates a hash for all parameters in the aggregate, then uses it in calculations.
- Calculates a hash (64-bit hash for `String` and 32-bit otherwise) for all parameters in the aggregate, then uses it in calculations.
- Uses a combination of three algorithms: array, hash table, and HyperLogLog with an error correction table.
For a small number of distinct elements, an array is used. When the set size is larger, a hash table is used. For a larger number of elements, HyperLogLog is used, which will occupy a fixed amount of memory.
- Provides the result deterministically (it doesn't depend on the query processing order).
!! note "Note"
Since it uses 32-bit hash for non-`String` type, the result will have very high error for cardinalities > `UINT_MAX`, hence in this case you should use [uniqCombined64](#agg_function-uniqcombined64)
Compared to the [uniq](#agg_function-uniq) function, the `uniqCombined`:
- Consumes several times less memory.
@ -589,9 +593,13 @@ Compared to the [uniq](#agg_function-uniq) function, the `uniqCombined`:
**See Also**
- [uniq](#agg_function-uniq)
- [uniqCombined64](#agg_function-uniqcombined64)
- [uniqHLL12](#agg_function-uniqhll12)
- [uniqExact](#agg_function-uniqexact)
## uniqCombined64 {#agg_function-uniqcombined64}
Same as [uniqCombined](#agg_function-uniqcombined), but uses 64-bit hash for all data types.
## uniqHLL12 {#agg_function-uniqhll12}

View File

@ -556,6 +556,7 @@ uniq(x[, ...])
**Смотрите также**
- [uniqCombined](#agg_function-uniqcombined)
- [uniqCombined64](#agg_function-uniqcombined64)
- [uniqHLL12](#agg_function-uniqhll12)
- [uniqExact](#agg_function-uniqexact)
@ -583,7 +584,7 @@ uniqCombined(HLL_precision)(x[, ...])
Функция:
- Вычисляет хэш для всех параметров агрегации, а затем использует его в вычислениях.
- Вычисляет хэш (64-битный для `String` и 32-битный для всех остальных типов) для всех параметров агрегации, а затем использует его в вычислениях.
- Используется комбинация трёх алгоритмов: массив, хэш-таблица и HyperLogLog с таблицей коррекции погрешности.
@ -591,6 +592,9 @@ uniqCombined(HLL_precision)(x[, ...])
- Результат детерминирован (не зависит от порядка выполнения запроса).
!! note "Note"
Так как используется 32-битный хэш для не-`String` типов, результат будет иметь очень очень большую ошибку для количества элементов > `UINT_MAX`, таким образом в этом случае нужно использовать [uniqCombined64](#agg_function-uniqcombined64)
По сравнению с функцией [uniq](#agg_function-uniq), `uniqCombined`:
- Потребляет в несколько раз меньше памяти.
@ -600,9 +604,14 @@ uniqCombined(HLL_precision)(x[, ...])
**Смотрите также**
- [uniq](#agg_function-uniq)
- [uniqCombined64](#agg_function-uniqcombined64)
- [uniqHLL12](#agg_function-uniqhll12)
- [uniqExact](#agg_function-uniqexact)
## uniqCombined64 {#agg_function-uniqcombined64}
Использует 64-битный хэш для всех типов, в отличие от [uniqCombined](#agg_function-uniqcombined).
## uniqHLL12 {#agg_function-uniqhll12}
Вычисляет приблизительное число различных значений аргументов, используя алгоритм [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog).