ClickHouse/dbms/include/DB/AggregateFunctions/AggregateFunctionUniq.h

433 lines
11 KiB
C++
Raw Normal View History

2011-09-26 04:00:46 +00:00
#pragma once
#include <city.h>
#include <type_traits>
2011-09-26 04:00:46 +00:00
#include <DB/AggregateFunctions/UniquesHashSet.h>
2011-09-26 04:00:46 +00:00
#include <DB/IO/WriteHelpers.h>
#include <DB/IO/ReadHelpers.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypeTuple.h>
2011-09-26 04:00:46 +00:00
#include <DB/Interpreters/AggregationCommon.h>
#include <DB/Common/HashTable/HashSet.h>
#include <DB/Common/HyperLogLogWithSmallSetOptimization.h>
#include <DB/Common/CombinedCardinalityEstimator.h>
#include <DB/Columns/ColumnString.h>
#include <DB/Columns/ColumnTuple.h>
2011-09-26 04:00:46 +00:00
#include <DB/AggregateFunctions/IUnaryAggregateFunction.h>
2015-08-31 13:52:17 +00:00
#include <DB/AggregateFunctions/UniqCombinedBiasData.h>
2015-10-29 04:04:43 +00:00
#include <DB/AggregateFunctions/UniqVariadicHash.h>
2011-09-26 04:00:46 +00:00
namespace DB
{
/// uniq
2011-09-26 04:00:46 +00:00
struct AggregateFunctionUniqUniquesHashSetData
{
using Set = UniquesHashSet<DefaultHash<UInt64>>;
Set set;
static String getName() { return "uniq"; }
};
2017-03-09 00:56:38 +00:00
/// For a function that takes multiple arguments. Such a function pre-hashes them in advance, so TrivialHash is used here.
struct AggregateFunctionUniqUniquesHashSetDataForVariadic
{
using Set = UniquesHashSet<TrivialHash>;
Set set;
static String getName() { return "uniq"; }
};
/// uniqHLL12
template <typename T>
struct AggregateFunctionUniqHLL12Data
{
using Set = HyperLogLogWithSmallSetOptimization<T, 16, 12>;
Set set;
static String getName() { return "uniqHLL12"; }
};
template <>
struct AggregateFunctionUniqHLL12Data<String>
{
using Set = HyperLogLogWithSmallSetOptimization<UInt64, 16, 12>;
Set set;
static String getName() { return "uniqHLL12"; }
};
2015-10-29 04:13:18 +00:00
struct AggregateFunctionUniqHLL12DataForVariadic
{
using Set = HyperLogLogWithSmallSetOptimization<UInt64, 16, 12, TrivialHash>;
2015-10-29 04:13:18 +00:00
Set set;
static String getName() { return "uniqHLL12"; }
};
/// uniqExact
template <typename T>
struct AggregateFunctionUniqExactData
{
using Key = T;
2017-03-09 00:56:38 +00:00
/// When creating, the hash table must be small.
typedef HashSet<
Key,
HashCRC32<Key>,
2014-05-03 16:03:49 +00:00
HashTableGrower<4>,
HashTableAllocatorWithStackMemory<sizeof(Key) * (1 << 4)>
> Set;
Set set;
static String getName() { return "uniqExact"; }
};
/// For rows, we put the SipHash values (128 bits) into the hash table.
template <>
struct AggregateFunctionUniqExactData<String>
{
using Key = UInt128;
2017-03-09 00:56:38 +00:00
/// When creating, the hash table must be small.
typedef HashSet<
Key,
UInt128TrivialHash,
2014-05-03 16:03:49 +00:00
HashTableGrower<3>,
HashTableAllocatorWithStackMemory<sizeof(Key) * (1 << 3)>
> Set;
Set set;
static String getName() { return "uniqExact"; }
};
2015-08-31 13:52:17 +00:00
template <typename T, HyperLogLogMode mode>
struct BaseUniqCombinedData
{
2015-10-08 14:23:23 +00:00
using Key = UInt32;
2015-08-31 13:52:17 +00:00
using Set = CombinedCardinalityEstimator<
Key,
2015-10-08 14:23:23 +00:00
HashSet<Key, TrivialHash, HashTableGrower<> >,
2015-08-31 13:52:17 +00:00
16,
14,
17,
2015-10-08 14:23:23 +00:00
TrivialHash,
UInt32,
2015-08-31 13:52:17 +00:00
HyperLogLogBiasEstimator<UniqCombinedBiasData>,
mode
>;
2015-08-31 13:52:17 +00:00
Set set;
};
2015-08-31 13:52:17 +00:00
template <HyperLogLogMode mode>
struct BaseUniqCombinedData<String, mode>
{
using Key = UInt64;
2015-08-31 13:52:17 +00:00
using Set = CombinedCardinalityEstimator<
Key,
HashSet<Key, TrivialHash, HashTableGrower<> >,
16,
14,
17,
TrivialHash,
UInt64,
HyperLogLogBiasEstimator<UniqCombinedBiasData>,
mode
>;
2015-08-31 13:52:17 +00:00
Set set;
};
2017-03-09 00:56:38 +00:00
/// Aggregate functions uniqCombinedRaw, uniqCombinedLinearCounting, and uniqCombinedBiasCorrected
/// are intended for development of new versions of the uniqCombined function.
/// Users should only use uniqCombined.
2015-08-31 13:52:17 +00:00
template <typename T>
struct AggregateFunctionUniqCombinedRawData
: public BaseUniqCombinedData<T, HyperLogLogMode::Raw>
{
2015-08-31 13:52:17 +00:00
static String getName() { return "uniqCombinedRaw"; }
};
2015-08-31 13:52:17 +00:00
template <typename T>
struct AggregateFunctionUniqCombinedLinearCountingData
: public BaseUniqCombinedData<T, HyperLogLogMode::LinearCounting>
{
2015-08-31 13:52:17 +00:00
static String getName() { return "uniqCombinedLinearCounting"; }
};
2015-08-31 13:52:17 +00:00
template <typename T>
struct AggregateFunctionUniqCombinedBiasCorrectedData
: public BaseUniqCombinedData<T, HyperLogLogMode::BiasCorrected>
{
2015-08-31 13:52:17 +00:00
static String getName() { return "uniqCombinedBiasCorrected"; }
};
2015-08-31 13:52:17 +00:00
template <typename T>
struct AggregateFunctionUniqCombinedData
: public BaseUniqCombinedData<T, HyperLogLogMode::FullFeatured>
{
2015-08-31 13:52:17 +00:00
static String getName() { return "uniqCombined"; }
};
2015-08-31 13:52:17 +00:00
namespace detail
{
2017-03-09 00:56:38 +00:00
/** Hash function for uniq.
*/
template <typename T> struct AggregateFunctionUniqTraits
{
static UInt64 hash(T x) { return x; }
};
template <> struct AggregateFunctionUniqTraits<Float32>
{
static UInt64 hash(Float32 x)
{
UInt64 res = 0;
memcpy(reinterpret_cast<char *>(&res), reinterpret_cast<char *>(&x), sizeof(x));
return res;
}
};
template <> struct AggregateFunctionUniqTraits<Float64>
{
static UInt64 hash(Float64 x)
{
UInt64 res = 0;
memcpy(reinterpret_cast<char *>(&res), reinterpret_cast<char *>(&x), sizeof(x));
return res;
}
};
2017-03-09 00:56:38 +00:00
/** Hash function for uniqCombined.
2015-10-08 14:23:23 +00:00
*/
template <typename T> struct AggregateFunctionUniqCombinedTraits
{
static UInt32 hash(T x) { return static_cast<UInt32>(intHash64(x)); }
};
template <> struct AggregateFunctionUniqCombinedTraits<Float32>
{
static UInt32 hash(Float32 x)
{
UInt64 res = 0;
memcpy(reinterpret_cast<char *>(&res), reinterpret_cast<char *>(&x), sizeof(x));
return static_cast<UInt32>(intHash64(res));
}
};
template <> struct AggregateFunctionUniqCombinedTraits<Float64>
{
static UInt32 hash(Float64 x)
{
UInt64 res = 0;
memcpy(reinterpret_cast<char *>(&res), reinterpret_cast<char *>(&x), sizeof(x));
return static_cast<UInt32>(intHash64(res));
}
};
2017-03-09 00:56:38 +00:00
/** The structure for the delegation work to add one element to the `uniq` aggregate functions.
* Used for partial specialization to add strings.
2015-08-31 13:52:17 +00:00
*/
template <typename T, typename Data, typename Enable = void>
struct OneAdder;
template <typename T, typename Data>
struct OneAdder<T, Data, typename std::enable_if<
std::is_same<Data, AggregateFunctionUniqUniquesHashSetData>::value ||
2015-10-08 14:23:23 +00:00
std::is_same<Data, AggregateFunctionUniqHLL12Data<T> >::value>::type>
{
template <typename T2 = T>
2015-11-15 06:23:44 +00:00
static void addImpl(Data & data, const IColumn & column, size_t row_num,
2015-10-08 14:23:23 +00:00
typename std::enable_if<!std::is_same<T2, String>::value>::type * = nullptr)
{
const auto & value = static_cast<const ColumnVector<T2> &>(column).getData()[row_num];
data.set.insert(AggregateFunctionUniqTraits<T2>::hash(value));
}
template <typename T2 = T>
2015-11-15 06:23:44 +00:00
static void addImpl(Data & data, const IColumn & column, size_t row_num,
2015-10-08 14:23:23 +00:00
typename std::enable_if<std::is_same<T2, String>::value>::type * = nullptr)
{
StringRef value = column.getDataAt(row_num);
data.set.insert(CityHash64(value.data, value.size));
}
};
template <typename T, typename Data>
struct OneAdder<T, Data, typename std::enable_if<
2015-08-31 13:52:17 +00:00
std::is_same<Data, AggregateFunctionUniqCombinedRawData<T> >::value ||
std::is_same<Data, AggregateFunctionUniqCombinedLinearCountingData<T> >::value ||
std::is_same<Data, AggregateFunctionUniqCombinedBiasCorrectedData<T> >::value ||
std::is_same<Data, AggregateFunctionUniqCombinedData<T> >::value>::type>
{
2015-08-31 13:52:17 +00:00
template <typename T2 = T>
2015-11-15 06:23:44 +00:00
static void addImpl(Data & data, const IColumn & column, size_t row_num,
2015-08-31 13:52:17 +00:00
typename std::enable_if<!std::is_same<T2, String>::value>::type * = nullptr)
{
2015-08-31 13:52:17 +00:00
const auto & value = static_cast<const ColumnVector<T2> &>(column).getData()[row_num];
2015-10-08 14:23:23 +00:00
data.set.insert(AggregateFunctionUniqCombinedTraits<T2>::hash(value));
}
2015-08-31 13:52:17 +00:00
template <typename T2 = T>
2015-11-15 06:23:44 +00:00
static void addImpl(Data & data, const IColumn & column, size_t row_num,
2015-08-31 13:52:17 +00:00
typename std::enable_if<std::is_same<T2, String>::value>::type * = nullptr)
{
StringRef value = column.getDataAt(row_num);
data.set.insert(CityHash64(value.data, value.size));
}
};
2015-08-31 13:52:17 +00:00
template <typename T, typename Data>
struct OneAdder<T, Data, typename std::enable_if<
std::is_same<Data, AggregateFunctionUniqExactData<T> >::value>::type>
{
2015-08-31 13:52:17 +00:00
template <typename T2 = T>
2015-11-15 06:23:44 +00:00
static void addImpl(Data & data, const IColumn & column, size_t row_num,
2015-08-31 13:52:17 +00:00
typename std::enable_if<!std::is_same<T2, String>::value>::type * = nullptr)
{
2015-08-31 13:52:17 +00:00
data.set.insert(static_cast<const ColumnVector<T2> &>(column).getData()[row_num]);
}
2015-08-31 13:52:17 +00:00
template <typename T2 = T>
2015-11-15 06:23:44 +00:00
static void addImpl(Data & data, const IColumn & column, size_t row_num,
2015-08-31 13:52:17 +00:00
typename std::enable_if<std::is_same<T2, String>::value>::type * = nullptr)
{
StringRef value = column.getDataAt(row_num);
UInt128 key;
SipHash hash;
hash.update(value.data, value.size);
hash.get128(key.first, key.second);
data.set.insert(key);
}
};
}
/// Calculates the number of different values approximately or exactly.
template <typename T, typename Data>
class AggregateFunctionUniq final : public IUnaryAggregateFunction<Data, AggregateFunctionUniq<T, Data> >
2011-09-26 04:00:46 +00:00
{
public:
2015-11-11 02:04:23 +00:00
String getName() const override { return Data::getName(); }
2011-09-26 04:00:46 +00:00
2015-11-11 02:04:23 +00:00
DataTypePtr getReturnType() const override
2011-09-26 04:00:46 +00:00
{
return std::make_shared<DataTypeUInt64>();
2011-09-26 04:00:46 +00:00
}
void setArgument(const DataTypePtr & argument)
2011-09-26 04:00:46 +00:00
{
}
void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num, Arena *) const
2011-09-26 04:00:46 +00:00
{
2015-11-15 06:23:44 +00:00
detail::OneAdder<T, Data>::addImpl(this->data(place), column, row_num);
2011-09-26 04:00:46 +00:00
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override
2011-09-26 04:00:46 +00:00
{
this->data(place).set.merge(this->data(rhs).set);
2011-09-26 04:00:46 +00:00
}
2015-11-11 02:04:23 +00:00
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
2011-09-26 04:00:46 +00:00
{
this->data(place).set.write(buf);
2011-09-26 04:00:46 +00:00
}
2016-09-22 23:26:08 +00:00
void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena *) const override
2011-09-26 04:00:46 +00:00
{
2016-03-12 04:01:03 +00:00
this->data(place).set.read(buf);
2011-09-26 04:00:46 +00:00
}
2015-11-11 02:04:23 +00:00
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
2011-09-26 04:00:46 +00:00
{
static_cast<ColumnUInt64 &>(to).getData().push_back(this->data(place).set.size());
2011-09-26 04:00:46 +00:00
}
};
2017-03-09 00:56:38 +00:00
/** 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 effective implementation), you can not pass several arguments, among which there are tuples.
*/
template <typename Data, bool argument_is_tuple>
class AggregateFunctionUniqVariadic final : public IAggregateFunctionHelper<Data>
{
private:
static constexpr bool is_exact = std::is_same<Data, AggregateFunctionUniqExactData<String>>::value;
size_t num_args = 0;
public:
2015-11-11 02:04:23 +00:00
String getName() const override { return Data::getName(); }
2015-11-11 02:04:23 +00:00
DataTypePtr getReturnType() const override
{
return std::make_shared<DataTypeUInt64>();
}
2015-11-11 02:04:23 +00:00
void setArguments(const DataTypes & arguments) override
{
if (argument_is_tuple)
num_args = typeid_cast<const DataTypeTuple &>(*arguments[0]).getElements().size();
else
num_args = arguments.size();
}
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override
{
this->data(place).set.insert(UniqVariadicHash<is_exact, argument_is_tuple>::apply(num_args, columns, row_num));
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override
{
this->data(place).set.merge(this->data(rhs).set);
}
2015-11-11 02:04:23 +00:00
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
this->data(place).set.write(buf);
}
2016-09-22 23:26:08 +00:00
void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena *) const override
{
2016-03-12 04:01:03 +00:00
this->data(place).set.read(buf);
}
2015-11-11 02:04:23 +00:00
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
static_cast<ColumnUInt64 &>(to).getData().push_back(this->data(place).set.size());
}
static void addFree(const IAggregateFunction * that, AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena)
{
static_cast<const AggregateFunctionUniqVariadic &>(*that).add(place, columns, row_num, arena);
}
IAggregateFunction::AddFunc getAddressOfAddFunction() const override final { return &addFree; }
};
2011-09-26 04:00:46 +00:00
}