mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
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:
parent
15deedb420
commit
f46c5a47c0
@ -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));
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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)));
|
||||
}
|
||||
|
||||
|
@ -0,0 +1,2 @@
|
||||
10021957
|
||||
10021969
|
9
dbms/tests/queries/0_stateless/01016_uniqCombined64.sql
Normal file
9
dbms/tests/queries/0_stateless/01016_uniqCombined64.sql
Normal 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));
|
@ -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}
|
||||
|
||||
|
@ -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).
|
||||
|
Loading…
Reference in New Issue
Block a user