mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Move-away "uniqCombined" as a separate aggregated function with HLL precision
This commit is contained in:
parent
8623cb232c
commit
6d783ef013
@ -6,6 +6,6 @@ do
|
||||
do
|
||||
n=$(( 10**p * i ))
|
||||
echo -n "$n "
|
||||
clickhouse-client -q "select uniqHLL12(number), uniq(number), uniqCombined(number) from numbers($n);"
|
||||
clickhouse-client -q "select uniqHLL12(number), uniq(number), uniqCombined(15)(number) from numbers($n);"
|
||||
done
|
||||
done
|
||||
|
@ -130,9 +130,6 @@ void registerAggregateFunctionsUniq(AggregateFunctionFactory & factory)
|
||||
|
||||
factory.registerFunction("uniqExact",
|
||||
createAggregateFunctionUniq<true, AggregateFunctionUniqExactData, AggregateFunctionUniqExactData<String>>);
|
||||
|
||||
factory.registerFunction("uniqCombined",
|
||||
createAggregateFunctionUniq<false, AggregateFunctionUniqCombinedData, AggregateFunctionUniqCombinedData<UInt64>>);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -22,7 +22,6 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <AggregateFunctions/UniqCombinedBiasData.h>
|
||||
#include <AggregateFunctions/UniqVariadicHash.h>
|
||||
|
||||
|
||||
@ -124,46 +123,6 @@ struct AggregateFunctionUniqExactData<String>
|
||||
static String getName() { return "uniqExact"; }
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
struct AggregateFunctionUniqCombinedData
|
||||
{
|
||||
using Key = UInt32;
|
||||
using Set = CombinedCardinalityEstimator<
|
||||
Key,
|
||||
HashSet<Key, TrivialHash, HashTableGrower<>>,
|
||||
16,
|
||||
14,
|
||||
17,
|
||||
TrivialHash,
|
||||
UInt32,
|
||||
HyperLogLogBiasEstimator<UniqCombinedBiasData>,
|
||||
HyperLogLogMode::FullFeatured>;
|
||||
|
||||
Set set;
|
||||
|
||||
static String getName() { return "uniqCombined"; }
|
||||
};
|
||||
|
||||
template <>
|
||||
struct AggregateFunctionUniqCombinedData<String>
|
||||
{
|
||||
using Key = UInt64;
|
||||
using Set = CombinedCardinalityEstimator<
|
||||
Key,
|
||||
HashSet<Key, TrivialHash, HashTableGrower<>>,
|
||||
16,
|
||||
14,
|
||||
17,
|
||||
TrivialHash,
|
||||
UInt64,
|
||||
HyperLogLogBiasEstimator<UniqCombinedBiasData>,
|
||||
HyperLogLogMode::FullFeatured>;
|
||||
|
||||
Set set;
|
||||
|
||||
static String getName() { return "uniqCombined"; }
|
||||
};
|
||||
|
||||
|
||||
namespace detail
|
||||
{
|
||||
@ -199,39 +158,6 @@ template <> struct AggregateFunctionUniqTraits<Float64>
|
||||
}
|
||||
};
|
||||
|
||||
/** Hash function for uniqCombined.
|
||||
*/
|
||||
template <typename T> struct AggregateFunctionUniqCombinedTraits
|
||||
{
|
||||
static UInt32 hash(T x) { return static_cast<UInt32>(intHash64(x)); }
|
||||
};
|
||||
|
||||
template <> struct AggregateFunctionUniqCombinedTraits<UInt128>
|
||||
{
|
||||
static UInt32 hash(UInt128 x)
|
||||
{
|
||||
return sipHash64(x);
|
||||
}
|
||||
};
|
||||
|
||||
template <> struct AggregateFunctionUniqCombinedTraits<Float32>
|
||||
{
|
||||
static UInt32 hash(Float32 x)
|
||||
{
|
||||
UInt64 res = ext::bit_cast<UInt64>(x);
|
||||
return static_cast<UInt32>(intHash64(res));
|
||||
}
|
||||
};
|
||||
|
||||
template <> struct AggregateFunctionUniqCombinedTraits<Float64>
|
||||
{
|
||||
static UInt32 hash(Float64 x)
|
||||
{
|
||||
UInt64 res = ext::bit_cast<UInt64>(x);
|
||||
return static_cast<UInt32>(intHash64(res));
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/** The structure for the delegation work to add one element to the `uniq` aggregate functions.
|
||||
* Used for partial specialization to add strings.
|
||||
@ -255,19 +181,6 @@ struct OneAdder
|
||||
data.set.insert(CityHash_v1_0_2::CityHash64(value.data, value.size));
|
||||
}
|
||||
}
|
||||
else if constexpr (std::is_same_v<Data, AggregateFunctionUniqCombinedData<T>>)
|
||||
{
|
||||
if constexpr (!std::is_same_v<T, String>)
|
||||
{
|
||||
const auto & value = static_cast<const ColumnVector<T> &>(column).getData()[row_num];
|
||||
data.set.insert(AggregateFunctionUniqCombinedTraits<T>::hash(value));
|
||||
}
|
||||
else
|
||||
{
|
||||
StringRef value = column.getDataAt(row_num);
|
||||
data.set.insert(CityHash_v1_0_2::CityHash64(value.data, value.size));
|
||||
}
|
||||
}
|
||||
else if constexpr (std::is_same_v<Data, AggregateFunctionUniqExactData<T>>)
|
||||
{
|
||||
if constexpr (!std::is_same_v<T, String>)
|
||||
@ -387,5 +300,4 @@ public:
|
||||
const char * getHeaderFilePath() const override { return __FILE__; }
|
||||
};
|
||||
|
||||
|
||||
}
|
||||
|
@ -0,0 +1,90 @@
|
||||
#include <AggregateFunctions/AggregateFunctionUniqCombined.h>
|
||||
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/Helpers.h>
|
||||
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int ARGUMENT_OUT_OF_BOUND;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
AggregateFunctionPtr createAggregateFunctionUniqCombined(
|
||||
const std::string & name, const DataTypes & argument_types, const Array & params)
|
||||
{
|
||||
UInt8 precision = 17; /// default value - must correlate with default ctor of |AggregateFunctionUniqCombinedData|
|
||||
|
||||
if (!params.empty())
|
||||
{
|
||||
if (params.size() != 1)
|
||||
throw Exception(
|
||||
"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 into |AggregateFunctionUniqCombinedData|
|
||||
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);
|
||||
|
||||
precision = precision_param;
|
||||
}
|
||||
|
||||
if (argument_types.empty())
|
||||
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
/// We use exact hash function if the user wants it;
|
||||
/// or if the arguments are not contiguous in memory, because only exact hash function have support for this case.
|
||||
bool use_exact_hash_function = !isAllArgumentsContiguousInMemory(argument_types);
|
||||
|
||||
if (argument_types.size() == 1)
|
||||
{
|
||||
const IDataType & argument_type = *argument_types[0];
|
||||
|
||||
AggregateFunctionPtr res(createWithNumericType<AggregateFunctionUniqCombined>(*argument_types[0], precision));
|
||||
|
||||
WhichDataType which(argument_type);
|
||||
if (res)
|
||||
return res;
|
||||
else if (which.isDate())
|
||||
return std::make_shared<AggregateFunctionUniqCombined<DataTypeDate::FieldType>>(precision);
|
||||
else if (which.isDateTime())
|
||||
return std::make_shared<AggregateFunctionUniqCombined<DataTypeDateTime::FieldType>>(precision);
|
||||
else if (which.isStringOrFixedString())
|
||||
return std::make_shared<AggregateFunctionUniqCombined<String>>(precision);
|
||||
else if (which.isUUID())
|
||||
return std::make_shared<AggregateFunctionUniqCombined<DataTypeUUID::FieldType>>(precision);
|
||||
else if (which.isTuple())
|
||||
{
|
||||
if (use_exact_hash_function)
|
||||
return std::make_shared<AggregateFunctionUniqCombinedVariadic<true, true>>(argument_types, precision);
|
||||
else
|
||||
return std::make_shared<AggregateFunctionUniqCombinedVariadic<false, true>>(argument_types, precision);
|
||||
}
|
||||
}
|
||||
|
||||
/// "Variadic" method also works as a fallback generic case for single argument.
|
||||
if (use_exact_hash_function)
|
||||
return std::make_shared<AggregateFunctionUniqCombinedVariadic<true, false>>(argument_types, precision);
|
||||
else
|
||||
return std::make_shared<AggregateFunctionUniqCombinedVariadic<false, false>>(argument_types, precision);
|
||||
}
|
||||
|
||||
} // namespace
|
||||
|
||||
void registerAggregateFunctionUniqCombined(AggregateFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction("uniqCombined", createAggregateFunctionUniqCombined);
|
||||
}
|
||||
|
||||
} // namespace DB
|
429
dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h
Normal file
429
dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h
Normal file
@ -0,0 +1,429 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/CombinedCardinalityEstimator.h>
|
||||
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <AggregateFunctions/UniqCombinedBiasData.h>
|
||||
#include <AggregateFunctions/UniqVariadicHash.h>
|
||||
|
||||
#include <ext/bit_cast.h>
|
||||
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/SipHash.h>
|
||||
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace detail
|
||||
{
|
||||
/** Hash function for uniqCombined.
|
||||
*/
|
||||
template <typename T>
|
||||
struct AggregateFunctionUniqCombinedTraits
|
||||
{
|
||||
static UInt32 hash(T x)
|
||||
{
|
||||
return static_cast<UInt32>(intHash64(x));
|
||||
}
|
||||
};
|
||||
|
||||
template <>
|
||||
struct AggregateFunctionUniqCombinedTraits<UInt128>
|
||||
{
|
||||
static UInt32 hash(UInt128 x)
|
||||
{
|
||||
return sipHash64(x);
|
||||
}
|
||||
};
|
||||
|
||||
template <>
|
||||
struct AggregateFunctionUniqCombinedTraits<Float32>
|
||||
{
|
||||
static UInt32 hash(Float32 x)
|
||||
{
|
||||
UInt64 res = ext::bit_cast<UInt64>(x);
|
||||
return static_cast<UInt32>(intHash64(res));
|
||||
}
|
||||
};
|
||||
|
||||
template <>
|
||||
struct AggregateFunctionUniqCombinedTraits<Float64>
|
||||
{
|
||||
static UInt32 hash(Float64 x)
|
||||
{
|
||||
UInt64 res = ext::bit_cast<UInt64>(x);
|
||||
return static_cast<UInt32>(intHash64(res));
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace detail
|
||||
|
||||
|
||||
template <typename Key>
|
||||
struct __attribute__((__packed__)) AggregateFunctionUniqCombinedDataWithKey
|
||||
{
|
||||
template <UInt8 K>
|
||||
using Set = CombinedCardinalityEstimator<Key,
|
||||
HashSet<Key, TrivialHash, HashTableGrower<>>,
|
||||
16,
|
||||
K - 3,
|
||||
K,
|
||||
TrivialHash,
|
||||
Key,
|
||||
HyperLogLogBiasEstimator<UniqCombinedBiasData>,
|
||||
HyperLogLogMode::FullFeatured>;
|
||||
|
||||
mutable UInt8 inited = 0;
|
||||
union
|
||||
{
|
||||
Set<12> set_12;
|
||||
Set<13> set_13;
|
||||
Set<14> set_14;
|
||||
Set<15> set_15;
|
||||
Set<16> set_16;
|
||||
Set<17> set_17;
|
||||
Set<18> set_18;
|
||||
Set<19> set_19;
|
||||
Set<20> set_20;
|
||||
};
|
||||
|
||||
AggregateFunctionUniqCombinedDataWithKey() : set_17() {}
|
||||
|
||||
~AggregateFunctionUniqCombinedDataWithKey()
|
||||
{
|
||||
switch (inited)
|
||||
{
|
||||
case 12:
|
||||
set_12.~CombinedCardinalityEstimator();
|
||||
break;
|
||||
case 13:
|
||||
set_13.~CombinedCardinalityEstimator();
|
||||
break;
|
||||
case 14:
|
||||
set_14.~CombinedCardinalityEstimator();
|
||||
break;
|
||||
case 15:
|
||||
set_15.~CombinedCardinalityEstimator();
|
||||
break;
|
||||
case 16:
|
||||
set_16.~CombinedCardinalityEstimator();
|
||||
break;
|
||||
case 0:
|
||||
case 17:
|
||||
set_17.~CombinedCardinalityEstimator();
|
||||
break;
|
||||
case 18:
|
||||
set_18.~CombinedCardinalityEstimator();
|
||||
break;
|
||||
case 19:
|
||||
set_19.~CombinedCardinalityEstimator();
|
||||
break;
|
||||
case 20:
|
||||
set_20.~CombinedCardinalityEstimator();
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
void init(UInt8 precision) const
|
||||
{
|
||||
if (inited || precision == 17)
|
||||
return;
|
||||
|
||||
// TODO: assert "inited == precision"
|
||||
|
||||
set_17.~CombinedCardinalityEstimator();
|
||||
|
||||
switch (precision)
|
||||
{
|
||||
case 12:
|
||||
new (&set_12) Set<12>;
|
||||
break;
|
||||
case 13:
|
||||
new (&set_13) Set<13>;
|
||||
break;
|
||||
case 14:
|
||||
new (&set_14) Set<14>;
|
||||
break;
|
||||
case 15:
|
||||
new (&set_15) Set<15>;
|
||||
break;
|
||||
case 16:
|
||||
new (&set_16) Set<16>;
|
||||
break;
|
||||
case 18:
|
||||
new (&set_18) Set<18>;
|
||||
break;
|
||||
case 19:
|
||||
new (&set_19) Set<19>;
|
||||
break;
|
||||
case 20:
|
||||
new (&set_20) Set<20>;
|
||||
break;
|
||||
}
|
||||
inited = precision;
|
||||
}
|
||||
|
||||
#define SET_METHOD(method) \
|
||||
switch (inited) \
|
||||
{ \
|
||||
case 12: \
|
||||
set_12.method; \
|
||||
break; \
|
||||
case 13: \
|
||||
set_13.method; \
|
||||
break; \
|
||||
case 14: \
|
||||
set_14.method; \
|
||||
break; \
|
||||
case 15: \
|
||||
set_15.method; \
|
||||
break; \
|
||||
case 16: \
|
||||
set_16.method; \
|
||||
break; \
|
||||
case 17: \
|
||||
set_17.method; \
|
||||
break; \
|
||||
case 18: \
|
||||
set_18.method; \
|
||||
break; \
|
||||
case 19: \
|
||||
set_19.method; \
|
||||
break; \
|
||||
case 20: \
|
||||
set_20.method; \
|
||||
break; \
|
||||
}
|
||||
|
||||
#define SET_RETURN_METHOD(method) \
|
||||
switch (inited) \
|
||||
{ \
|
||||
case 12: \
|
||||
return set_12.method; \
|
||||
case 13: \
|
||||
return set_13.method; \
|
||||
case 14: \
|
||||
return set_14.method; \
|
||||
case 15: \
|
||||
return set_15.method; \
|
||||
case 16: \
|
||||
return set_16.method; \
|
||||
case 18: \
|
||||
return set_18.method; \
|
||||
case 19: \
|
||||
return set_19.method; \
|
||||
case 20: \
|
||||
return set_20.method; \
|
||||
case 17: \
|
||||
default: \
|
||||
return set_17.method; \
|
||||
}
|
||||
|
||||
void insert(Key value, UInt8 precision)
|
||||
{
|
||||
init(precision);
|
||||
SET_METHOD(insert(value));
|
||||
}
|
||||
|
||||
void merge(const AggregateFunctionUniqCombinedDataWithKey<Key> & rhs, UInt8 precision)
|
||||
{
|
||||
init(precision);
|
||||
switch (inited)
|
||||
{
|
||||
case 12:
|
||||
set_12.merge(rhs.set_12);
|
||||
break;
|
||||
case 13:
|
||||
set_13.merge(rhs.set_13);
|
||||
break;
|
||||
case 14:
|
||||
set_14.merge(rhs.set_14);
|
||||
break;
|
||||
case 15:
|
||||
set_15.merge(rhs.set_15);
|
||||
break;
|
||||
case 16:
|
||||
set_16.merge(rhs.set_16);
|
||||
break;
|
||||
case 17:
|
||||
set_17.merge(rhs.set_17);
|
||||
break;
|
||||
case 18:
|
||||
set_18.merge(rhs.set_18);
|
||||
break;
|
||||
case 19:
|
||||
set_19.merge(rhs.set_19);
|
||||
break;
|
||||
case 20:
|
||||
set_20.merge(rhs.set_20);
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
void write(DB::WriteBuffer & out, UInt8 precision) const
|
||||
{
|
||||
init(precision);
|
||||
SET_METHOD(write(out));
|
||||
}
|
||||
|
||||
void read(DB::ReadBuffer & in, UInt8 precision)
|
||||
{
|
||||
init(precision);
|
||||
SET_METHOD(read(in));
|
||||
}
|
||||
|
||||
UInt32 size(UInt8 precision) const
|
||||
{
|
||||
init(precision);
|
||||
SET_RETURN_METHOD(size());
|
||||
}
|
||||
|
||||
#undef SET_METHOD
|
||||
#undef SET_RETURN_METHOD
|
||||
};
|
||||
|
||||
|
||||
template <typename T>
|
||||
struct __attribute__((__packed__)) AggregateFunctionUniqCombinedData : public AggregateFunctionUniqCombinedDataWithKey<UInt32>
|
||||
{
|
||||
};
|
||||
|
||||
|
||||
template <>
|
||||
struct __attribute__((__packed__)) AggregateFunctionUniqCombinedData<String> : public AggregateFunctionUniqCombinedDataWithKey<UInt64>
|
||||
{
|
||||
};
|
||||
|
||||
|
||||
template <typename T>
|
||||
class AggregateFunctionUniqCombined final
|
||||
: public IAggregateFunctionDataHelper<AggregateFunctionUniqCombinedData<T>, AggregateFunctionUniqCombined<T>>
|
||||
{
|
||||
private:
|
||||
const UInt8 precision;
|
||||
|
||||
public:
|
||||
explicit AggregateFunctionUniqCombined(UInt8 precision) : precision(precision) {}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return "uniqCombined";
|
||||
}
|
||||
|
||||
DataTypePtr getReturnType() const override
|
||||
{
|
||||
return std::make_shared<DataTypeUInt64>();
|
||||
}
|
||||
|
||||
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override
|
||||
{
|
||||
if constexpr (!std::is_same_v<T, String>)
|
||||
{
|
||||
const auto & value = static_cast<const ColumnVector<T> &>(*columns[0]).getData()[row_num];
|
||||
this->data(place).insert(detail::AggregateFunctionUniqCombinedTraits<T>::hash(value), precision);
|
||||
}
|
||||
else
|
||||
{
|
||||
StringRef value = columns[0]->getDataAt(row_num);
|
||||
this->data(place).insert(CityHash_v1_0_2::CityHash64(value.data, value.size), precision);
|
||||
}
|
||||
}
|
||||
|
||||
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override
|
||||
{
|
||||
this->data(place).merge(this->data(rhs), precision);
|
||||
}
|
||||
|
||||
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
|
||||
{
|
||||
this->data(place).write(buf, precision);
|
||||
}
|
||||
|
||||
void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena *) const override
|
||||
{
|
||||
this->data(place).read(buf, precision);
|
||||
}
|
||||
|
||||
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
|
||||
{
|
||||
static_cast<ColumnUInt64 &>(to).getData().push_back(this->data(place).size(precision));
|
||||
}
|
||||
|
||||
const char * getHeaderFilePath() const override
|
||||
{
|
||||
return __FILE__;
|
||||
}
|
||||
};
|
||||
|
||||
/** For multiple arguments. To compute, hashes them.
|
||||
* 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>
|
||||
class AggregateFunctionUniqCombinedVariadic final : public IAggregateFunctionDataHelper<AggregateFunctionUniqCombinedData<UInt64>,
|
||||
AggregateFunctionUniqCombinedVariadic<is_exact, argument_is_tuple>>
|
||||
{
|
||||
private:
|
||||
size_t num_args = 0;
|
||||
UInt8 precision;
|
||||
|
||||
public:
|
||||
AggregateFunctionUniqCombinedVariadic(const DataTypes & arguments, UInt8 precision) : precision(precision)
|
||||
{
|
||||
if (argument_is_tuple)
|
||||
num_args = typeid_cast<const DataTypeTuple &>(*arguments[0]).getElements().size();
|
||||
else
|
||||
num_args = arguments.size();
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return "uniqCombined";
|
||||
}
|
||||
|
||||
DataTypePtr getReturnType() const override
|
||||
{
|
||||
return std::make_shared<DataTypeUInt64>();
|
||||
}
|
||||
|
||||
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override
|
||||
{
|
||||
this->data(place).insert(typename AggregateFunctionUniqCombinedData<UInt64>::Set<12>::value_type(
|
||||
UniqVariadicHash<is_exact, argument_is_tuple>::apply(num_args, columns, row_num)),
|
||||
precision);
|
||||
}
|
||||
|
||||
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override
|
||||
{
|
||||
this->data(place).merge(this->data(rhs), precision);
|
||||
}
|
||||
|
||||
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
|
||||
{
|
||||
this->data(place).write(buf, precision);
|
||||
}
|
||||
|
||||
void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena *) const override
|
||||
{
|
||||
this->data(place).read(buf, precision);
|
||||
}
|
||||
|
||||
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
|
||||
{
|
||||
static_cast<ColumnUInt64 &>(to).getData().push_back(this->data(place).size(precision));
|
||||
}
|
||||
|
||||
const char * getHeaderFilePath() const override
|
||||
{
|
||||
return __FILE__;
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace DB
|
@ -21,6 +21,7 @@ void registerAggregateFunctionsStatisticsSimple(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionSum(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionSumMap(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionsUniq(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionUniqCombined(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionUniqUpTo(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionTopK(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionsBitwise(AggregateFunctionFactory &);
|
||||
@ -55,6 +56,7 @@ void registerAggregateFunctions()
|
||||
registerAggregateFunctionSum(factory);
|
||||
registerAggregateFunctionSumMap(factory);
|
||||
registerAggregateFunctionsUniq(factory);
|
||||
registerAggregateFunctionUniqCombined(factory);
|
||||
registerAggregateFunctionUniqUpTo(factory);
|
||||
registerAggregateFunctionTopK(factory);
|
||||
registerAggregateFunctionsBitwise(factory);
|
||||
|
@ -86,8 +86,8 @@ PageCharset тоже почти всегда непуст, но его сред
|
||||
<query>SELECT uniq(UserID) FROM test.hits SETTINGS max_threads = 1</query>
|
||||
<query>SELECT uniq(UserID) FROM test.hits</query>
|
||||
<!-- 43. Разные алгоритмы вычисления кардинальности. -->
|
||||
<query>SELECT uniqCombined(UserID) FROM test.hits SETTINGS max_threads = 1</query>
|
||||
<query>SELECT uniqCombined(UserID) FROM test.hits</query>
|
||||
<query>SELECT uniqCombined(15)(UserID) FROM test.hits SETTINGS max_threads = 1</query>
|
||||
<query>SELECT uniqCombined(15)(UserID) FROM test.hits</query>
|
||||
<!-- 44. Разные алгоритмы вычисления кардинальности. -->
|
||||
<query>SELECT uniqExact(UserID) FROM test.hits SETTINGS max_threads = 1</query>
|
||||
<query>SELECT uniqExact(UserID) FROM test.hits</query>
|
||||
|
@ -20,20 +20,20 @@ SELECT uniqHLL12(dummy) FROM remote('127.0.0.{2,3}', system.one);
|
||||
|
||||
/* uniqCombined */
|
||||
|
||||
SELECT Y, uniqCombined(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
|
||||
|
||||
SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
|
||||
|
||||
SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
|
||||
|
||||
SELECT Y, uniqCombined(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(15)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(15)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
|
||||
SELECT Y, uniqCombined(15)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
|
||||
|
||||
SELECT uniqCombined(dummy) FROM remote('127.0.0.{2,3}', system.one);
|
||||
SELECT uniqCombined(15)(dummy) FROM remote('127.0.0.{2,3}', system.one);
|
||||
|
@ -1,12 +1,12 @@
|
||||
SELECT
|
||||
SELECT
|
||||
uniq(x), uniq((x)), uniq(x, y), uniq((x, y)), uniq(x, y, z), uniq((x, y, z)),
|
||||
uniqCombined(x), uniqCombined((x)), uniqCombined(x, y), uniqCombined((x, y)), uniqCombined(x, y, z), uniqCombined((x, y, z)),
|
||||
uniqCombined(15)(x), uniqCombined(15)((x)), uniqCombined(15)(x, y), uniqCombined(15)((x, y)), uniqCombined(15)(x, y, z), uniqCombined(15)((x, y, z)),
|
||||
uniqHLL12(x), uniqHLL12((x)), uniqHLL12(x, y), uniqHLL12((x, y)), uniqHLL12(x, y, z), uniqHLL12((x, y, z)),
|
||||
uniqExact(x), uniqExact((x)), uniqExact(x, y), uniqExact((x, y)), uniqExact(x, y, z), uniqExact((x, y, z)),
|
||||
uniqUpTo(5)(x), uniqUpTo(5)((x)), uniqUpTo(5)(x, y), uniqUpTo(5)((x, y)), uniqUpTo(5)(x, y, z), uniqUpTo(5)((x, y, z))
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
SELECT
|
||||
number % 10 AS x,
|
||||
intDiv(number, 10) % 10 AS y,
|
||||
toString(intDiv(number, 100) % 10) AS z
|
||||
@ -16,14 +16,14 @@ FROM
|
||||
|
||||
SELECT k,
|
||||
uniq(x), uniq((x)), uniq(x, y), uniq((x, y)), uniq(x, y, z), uniq((x, y, z)),
|
||||
uniqCombined(x), uniqCombined((x)), uniqCombined(x, y), uniqCombined((x, y)), uniqCombined(x, y, z), uniqCombined((x, y, z)),
|
||||
uniqCombined(15)(x), uniqCombined(15)((x)), uniqCombined(15)(x, y), uniqCombined(15)((x, y)), uniqCombined(15)(x, y, z), uniqCombined(15)((x, y, z)),
|
||||
uniqHLL12(x), uniqHLL12((x)), uniqHLL12(x, y), uniqHLL12((x, y)), uniqHLL12(x, y, z), uniqHLL12((x, y, z)),
|
||||
uniqExact(x), uniqExact((x)), uniqExact(x, y), uniqExact((x, y)), uniqExact(x, y, z), uniqExact((x, y, z)),
|
||||
uniqUpTo(5)(x), uniqUpTo(5)((x)), uniqUpTo(5)(x, y), uniqUpTo(5)((x, y)), uniqUpTo(5)(x, y, z), uniqUpTo(5)((x, y, z)),
|
||||
count() AS c
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
SELECT
|
||||
(number + 0x8ffcbd8257219a26) * 0x66bb3430c06d2353 % 131 AS k,
|
||||
number % 10 AS x,
|
||||
intDiv(number, 10) % 10 AS y,
|
||||
|
@ -28,7 +28,7 @@ SELECT avg(a), avg(b), avg(c) FROM test.decimal WHERE a > 0;
|
||||
SELECT avg(a), avg(b), avg(c) FROM test.decimal WHERE a < 0;
|
||||
|
||||
SELECT (uniq(a), uniq(b), uniq(c)),
|
||||
(uniqCombined(a), uniqCombined(b), uniqCombined(c)),
|
||||
(uniqCombined(15)(a), uniqCombined(15)(b), uniqCombined(15)(c)),
|
||||
(uniqExact(a), uniqExact(b), uniqExact(c)),
|
||||
(uniqHLL12(a), uniqHLL12(b), uniqHLL12(c))
|
||||
FROM (SELECT * FROM test.decimal ORDER BY a);
|
||||
|
@ -1,3 +1,3 @@
|
||||
SELECT RegionID, uniqHLL12(WatchID) AS X FROM remote('127.0.0.{1,2}', test, hits) GROUP BY RegionID HAVING X > 100000 ORDER BY RegionID ASC;
|
||||
SELECT RegionID, uniqCombined(WatchID) AS X FROM remote('127.0.0.{1,2}', test, hits) GROUP BY RegionID HAVING X > 100000 ORDER BY RegionID ASC;
|
||||
SELECT RegionID, uniqCombined(15)(WatchID) AS X FROM remote('127.0.0.{1,2}', test, hits) GROUP BY RegionID HAVING X > 100000 ORDER BY RegionID ASC;
|
||||
SELECT abs(uniq(WatchID) - uniqExact(WatchID)) FROM test.hits;
|
||||
|
@ -146,7 +146,7 @@ The result is determinate (it doesn't depend on the order of query processing).
|
||||
|
||||
This function provides excellent accuracy even for data sets with extremely high cardinality (over 10 billion elements). It is recommended for default use.
|
||||
|
||||
## uniqCombined(x)
|
||||
## uniqCombined(HLL_precision)(x)
|
||||
|
||||
Calculates the approximate number of different values of the argument. Works for numbers, strings, dates, date-with-time, and for multiple arguments and tuple arguments.
|
||||
|
||||
|
@ -156,7 +156,7 @@ GROUP BY timeslot
|
||||
Данная функция обеспечивает отличную точность даже для множеств огромной кардинальности (10B+ элементов) и рекомендуется к использованию по умолчанию.
|
||||
|
||||
|
||||
## uniqCombined(x)
|
||||
## uniqCombined(HLL_precision)(x)
|
||||
|
||||
Приближённо вычисляет количество различных значений аргумента. Работает для чисел, строк, дат, дат-с-временем, для нескольких аргументов и аргументов-кортежей.
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user