2011-09-26 04:00:46 +00:00
|
|
|
#pragma once
|
|
|
|
|
|
|
|
#include <city.h>
|
2014-02-02 09:08:06 +00:00
|
|
|
#include <type_traits>
|
2011-09-26 04:00:46 +00:00
|
|
|
|
2018-06-04 14:17:47 +00:00
|
|
|
#include <ext/bit_cast.h>
|
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <AggregateFunctions/UniquesHashSet.h>
|
2011-09-26 04:00:46 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <IO/WriteHelpers.h>
|
|
|
|
#include <IO/ReadHelpers.h>
|
2011-09-26 04:00:46 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <DataTypes/DataTypesNumber.h>
|
|
|
|
#include <DataTypes/DataTypeTuple.h>
|
2011-09-26 04:00:46 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/AggregationCommon.h>
|
|
|
|
#include <Common/HashTable/HashSet.h>
|
|
|
|
#include <Common/HyperLogLogWithSmallSetOptimization.h>
|
|
|
|
#include <Common/CombinedCardinalityEstimator.h>
|
2017-04-08 01:32:05 +00:00
|
|
|
#include <Common/MemoryTracker.h>
|
2014-02-02 09:08:06 +00:00
|
|
|
|
2017-07-13 20:58:19 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2013-06-30 11:38:46 +00:00
|
|
|
|
2017-12-20 07:36:30 +00:00
|
|
|
#include <AggregateFunctions/IAggregateFunction.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <AggregateFunctions/UniqCombinedBiasData.h>
|
|
|
|
#include <AggregateFunctions/UniqVariadicHash.h>
|
2011-09-26 04:00:46 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2015-08-04 13:37:56 +00:00
|
|
|
/// uniq
|
2011-09-26 04:00:46 +00:00
|
|
|
|
2013-08-21 13:26:42 +00:00
|
|
|
struct AggregateFunctionUniqUniquesHashSetData
|
2013-02-08 19:34:44 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
using Set = UniquesHashSet<DefaultHash<UInt64>>;
|
|
|
|
Set set;
|
2015-02-22 07:23:37 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
static String getName() { return "uniq"; }
|
2013-02-08 19:34:44 +00:00
|
|
|
};
|
|
|
|
|
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.
|
2015-10-29 04:02:22 +00:00
|
|
|
struct AggregateFunctionUniqUniquesHashSetDataForVariadic
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
using Set = UniquesHashSet<TrivialHash>;
|
|
|
|
Set set;
|
2015-10-29 04:02:22 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
static String getName() { return "uniq"; }
|
2015-10-29 04:02:22 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2015-08-04 13:37:56 +00:00
|
|
|
/// uniqHLL12
|
2014-02-02 09:08:06 +00:00
|
|
|
|
2015-02-22 07:23:37 +00:00
|
|
|
template <typename T>
|
2013-08-21 13:26:42 +00:00
|
|
|
struct AggregateFunctionUniqHLL12Data
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
using Set = HyperLogLogWithSmallSetOptimization<T, 16, 12>;
|
|
|
|
Set set;
|
2015-02-22 07:23:37 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
static String getName() { return "uniqHLL12"; }
|
2015-02-22 07:23:37 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
template <>
|
|
|
|
struct AggregateFunctionUniqHLL12Data<String>
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
using Set = HyperLogLogWithSmallSetOptimization<UInt64, 16, 12>;
|
|
|
|
Set set;
|
2015-02-22 07:23:37 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
static String getName() { return "uniqHLL12"; }
|
2013-08-21 13:26:42 +00:00
|
|
|
};
|
|
|
|
|
2017-11-09 23:37:57 +00:00
|
|
|
template <>
|
|
|
|
struct AggregateFunctionUniqHLL12Data<UInt128>
|
|
|
|
{
|
|
|
|
using Set = HyperLogLogWithSmallSetOptimization<UInt64, 16, 12>;
|
|
|
|
Set set;
|
|
|
|
|
|
|
|
static String getName() { return "uniqHLL12"; }
|
|
|
|
};
|
|
|
|
|
2015-10-29 04:13:18 +00:00
|
|
|
struct AggregateFunctionUniqHLL12DataForVariadic
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
using Set = HyperLogLogWithSmallSetOptimization<UInt64, 16, 12, TrivialHash>;
|
|
|
|
Set set;
|
2015-10-29 04:13:18 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
static String getName() { return "uniqHLL12"; }
|
2015-10-29 04:13:18 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2015-08-04 13:37:56 +00:00
|
|
|
/// uniqExact
|
2013-02-08 19:34:44 +00:00
|
|
|
|
2014-02-02 09:08:06 +00:00
|
|
|
template <typename T>
|
|
|
|
struct AggregateFunctionUniqExactData
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
using Key = T;
|
2014-02-02 09:08:06 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// When creating, the hash table must be small.
|
2017-05-11 22:21:06 +00:00
|
|
|
using Set = HashSet<
|
2017-04-01 07:20:54 +00:00
|
|
|
Key,
|
|
|
|
HashCRC32<Key>,
|
|
|
|
HashTableGrower<4>,
|
2017-05-11 22:21:06 +00:00
|
|
|
HashTableAllocatorWithStackMemory<sizeof(Key) * (1 << 4)>>;
|
2014-02-02 09:08:06 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
Set set;
|
2014-02-02 09:08:06 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
static String getName() { return "uniqExact"; }
|
2014-02-02 09:08:06 +00:00
|
|
|
};
|
|
|
|
|
2017-03-09 04:18:41 +00:00
|
|
|
/// For rows, we put the SipHash values (128 bits) into the hash table.
|
2014-02-02 09:08:06 +00:00
|
|
|
template <>
|
|
|
|
struct AggregateFunctionUniqExactData<String>
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
using Key = UInt128;
|
2014-02-02 09:08:06 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// When creating, the hash table must be small.
|
2017-05-11 22:21:06 +00:00
|
|
|
using Set = HashSet<
|
2017-04-01 07:20:54 +00:00
|
|
|
Key,
|
|
|
|
UInt128TrivialHash,
|
|
|
|
HashTableGrower<3>,
|
2017-05-11 22:21:06 +00:00
|
|
|
HashTableAllocatorWithStackMemory<sizeof(Key) * (1 << 3)>>;
|
2014-02-02 09:08:06 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
Set set;
|
2014-02-02 09:08:06 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
static String getName() { return "uniqExact"; }
|
2014-02-02 09:08:06 +00:00
|
|
|
};
|
|
|
|
|
2018-03-17 18:02:19 +00:00
|
|
|
template <typename T>
|
|
|
|
struct AggregateFunctionUniqCombinedData
|
2015-07-20 14:22:08 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
using Key = UInt32;
|
|
|
|
using Set = CombinedCardinalityEstimator<
|
|
|
|
Key,
|
2017-08-30 18:13:32 +00:00
|
|
|
HashSet<Key, TrivialHash, HashTableGrower<>>,
|
2017-04-01 07:20:54 +00:00
|
|
|
16,
|
|
|
|
14,
|
|
|
|
17,
|
|
|
|
TrivialHash,
|
|
|
|
UInt32,
|
|
|
|
HyperLogLogBiasEstimator<UniqCombinedBiasData>,
|
2018-03-17 18:02:19 +00:00
|
|
|
HyperLogLogMode::FullFeatured>;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
Set set;
|
2018-03-17 18:02:19 +00:00
|
|
|
|
|
|
|
static String getName() { return "uniqCombined"; }
|
2015-07-20 14:22:08 +00:00
|
|
|
};
|
|
|
|
|
2018-03-17 18:02:19 +00:00
|
|
|
template <>
|
|
|
|
struct AggregateFunctionUniqCombinedData<String>
|
2015-07-20 14:22:08 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
using Key = UInt64;
|
|
|
|
using Set = CombinedCardinalityEstimator<
|
|
|
|
Key,
|
2017-08-30 18:13:32 +00:00
|
|
|
HashSet<Key, TrivialHash, HashTableGrower<>>,
|
2017-04-01 07:20:54 +00:00
|
|
|
16,
|
|
|
|
14,
|
|
|
|
17,
|
|
|
|
TrivialHash,
|
|
|
|
UInt64,
|
|
|
|
HyperLogLogBiasEstimator<UniqCombinedBiasData>,
|
2018-03-17 18:02:19 +00:00
|
|
|
HyperLogLogMode::FullFeatured>;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
Set set;
|
2015-07-20 14:22:08 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
static String getName() { return "uniqCombined"; }
|
2015-08-04 13:37:56 +00:00
|
|
|
};
|
2015-07-29 17:06:39 +00:00
|
|
|
|
2018-03-17 18:02:19 +00:00
|
|
|
|
2015-08-31 13:52:17 +00:00
|
|
|
namespace detail
|
2015-08-04 13:37:56 +00:00
|
|
|
{
|
2015-07-29 17:06:39 +00:00
|
|
|
|
2017-03-09 00:56:38 +00:00
|
|
|
/** Hash function for uniq.
|
2015-08-04 13:37:56 +00:00
|
|
|
*/
|
|
|
|
template <typename T> struct AggregateFunctionUniqTraits
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
static UInt64 hash(T x) { return x; }
|
2015-08-04 13:37:56 +00:00
|
|
|
};
|
|
|
|
|
2017-11-09 23:37:57 +00:00
|
|
|
template <> struct AggregateFunctionUniqTraits<UInt128>
|
|
|
|
{
|
|
|
|
static UInt64 hash(UInt128 x)
|
|
|
|
{
|
2018-03-03 15:36:20 +00:00
|
|
|
return sipHash64(x);
|
2017-11-09 23:37:57 +00:00
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2015-08-04 13:37:56 +00:00
|
|
|
template <> struct AggregateFunctionUniqTraits<Float32>
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
static UInt64 hash(Float32 x)
|
|
|
|
{
|
2018-06-04 14:17:47 +00:00
|
|
|
return ext::bit_cast<UInt64>(x);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2015-08-04 13:37:56 +00:00
|
|
|
};
|
2015-07-29 17:06:39 +00:00
|
|
|
|
2015-08-04 13:37:56 +00:00
|
|
|
template <> struct AggregateFunctionUniqTraits<Float64>
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
static UInt64 hash(Float64 x)
|
|
|
|
{
|
2018-06-04 14:17:47 +00:00
|
|
|
return ext::bit_cast<UInt64>(x);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2015-08-04 13:37:56 +00:00
|
|
|
};
|
2015-08-04 12:33:08 +00:00
|
|
|
|
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
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
static UInt32 hash(T x) { return static_cast<UInt32>(intHash64(x)); }
|
2015-10-08 14:23:23 +00:00
|
|
|
};
|
|
|
|
|
2017-11-09 23:37:57 +00:00
|
|
|
template <> struct AggregateFunctionUniqCombinedTraits<UInt128>
|
|
|
|
{
|
|
|
|
static UInt32 hash(UInt128 x)
|
|
|
|
{
|
2018-03-03 15:36:20 +00:00
|
|
|
return sipHash64(x);
|
2017-11-09 23:37:57 +00:00
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2015-10-08 14:23:23 +00:00
|
|
|
template <> struct AggregateFunctionUniqCombinedTraits<Float32>
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
static UInt32 hash(Float32 x)
|
|
|
|
{
|
2018-06-04 15:01:24 +00:00
|
|
|
UInt64 res = ext::bit_cast<UInt64>(x);
|
2017-04-01 07:20:54 +00:00
|
|
|
return static_cast<UInt32>(intHash64(res));
|
|
|
|
}
|
2015-10-08 14:23:23 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
template <> struct AggregateFunctionUniqCombinedTraits<Float64>
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
static UInt32 hash(Float64 x)
|
|
|
|
{
|
2018-06-04 15:01:24 +00:00
|
|
|
UInt64 res = ext::bit_cast<UInt64>(x);
|
2017-04-01 07:20:54 +00:00
|
|
|
return static_cast<UInt32>(intHash64(res));
|
|
|
|
}
|
2015-10-08 14:23:23 +00:00
|
|
|
};
|
|
|
|
|
2017-12-02 03:42:21 +00:00
|
|
|
|
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>
|
2017-12-02 03:42:21 +00:00
|
|
|
struct OneAdder
|
2015-08-04 13:37:56 +00:00
|
|
|
{
|
2017-12-20 07:36:30 +00:00
|
|
|
static void ALWAYS_INLINE add(Data & data, const IColumn & column, size_t row_num)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-12-25 04:01:46 +00:00
|
|
|
if constexpr (std::is_same_v<Data, AggregateFunctionUniqUniquesHashSetData>
|
|
|
|
|| std::is_same_v<Data, AggregateFunctionUniqHLL12Data<T>>)
|
2017-12-02 03:42:21 +00:00
|
|
|
{
|
2017-12-25 04:01:46 +00:00
|
|
|
if constexpr (!std::is_same_v<T, String>)
|
2017-12-02 03:42:21 +00:00
|
|
|
{
|
|
|
|
const auto & value = static_cast<const ColumnVector<T> &>(column).getData()[row_num];
|
|
|
|
data.set.insert(AggregateFunctionUniqTraits<T>::hash(value));
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
StringRef value = column.getDataAt(row_num);
|
|
|
|
data.set.insert(CityHash_v1_0_2::CityHash64(value.data, value.size));
|
|
|
|
}
|
|
|
|
}
|
2018-03-17 18:02:19 +00:00
|
|
|
else if constexpr (std::is_same_v<Data, AggregateFunctionUniqCombinedData<T>>)
|
2017-12-02 03:42:21 +00:00
|
|
|
{
|
2017-12-25 04:01:46 +00:00
|
|
|
if constexpr (!std::is_same_v<T, String>)
|
2017-12-02 03:42:21 +00:00
|
|
|
{
|
|
|
|
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));
|
|
|
|
}
|
|
|
|
}
|
2017-12-25 04:01:46 +00:00
|
|
|
else if constexpr (std::is_same_v<Data, AggregateFunctionUniqExactData<T>>)
|
2017-12-02 03:42:21 +00:00
|
|
|
{
|
2017-12-25 04:01:46 +00:00
|
|
|
if constexpr (!std::is_same_v<T, String>)
|
2017-12-02 03:42:21 +00:00
|
|
|
{
|
|
|
|
data.set.insert(static_cast<const ColumnVector<T> &>(column).getData()[row_num]);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
StringRef value = column.getDataAt(row_num);
|
|
|
|
|
|
|
|
UInt128 key;
|
|
|
|
SipHash hash;
|
|
|
|
hash.update(value.data, value.size);
|
|
|
|
hash.get128(key.low, key.high);
|
|
|
|
|
|
|
|
data.set.insert(key);
|
|
|
|
}
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2015-08-04 13:37:56 +00:00
|
|
|
};
|
2015-07-20 14:22:08 +00:00
|
|
|
|
2014-02-02 09:08:06 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2017-03-09 04:18:41 +00:00
|
|
|
/// Calculates the number of different values approximately or exactly.
|
2013-08-21 13:26:42 +00:00
|
|
|
template <typename T, typename Data>
|
2017-12-20 07:36:30 +00:00
|
|
|
class AggregateFunctionUniq final : public IAggregateFunctionDataHelper<Data, AggregateFunctionUniq<T, Data>>
|
2011-09-26 04:00:46 +00:00
|
|
|
{
|
|
|
|
public:
|
2017-04-01 07:20:54 +00:00
|
|
|
String getName() const override { return Data::getName(); }
|
|
|
|
|
|
|
|
DataTypePtr getReturnType() const override
|
|
|
|
{
|
|
|
|
return std::make_shared<DataTypeUInt64>();
|
|
|
|
}
|
|
|
|
|
2017-12-20 07:36:30 +00:00
|
|
|
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-12-20 07:36:30 +00:00
|
|
|
detail::OneAdder<T, Data>::add(this->data(place), *columns[0], row_num);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2017-12-01 21:51:50 +00:00
|
|
|
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
this->data(place).set.merge(this->data(rhs).set);
|
|
|
|
}
|
|
|
|
|
|
|
|
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
|
|
|
|
{
|
|
|
|
this->data(place).set.write(buf);
|
|
|
|
}
|
|
|
|
|
|
|
|
void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena *) const override
|
|
|
|
{
|
|
|
|
this->data(place).set.read(buf);
|
|
|
|
}
|
|
|
|
|
|
|
|
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
|
|
|
|
{
|
|
|
|
static_cast<ColumnUInt64 &>(to).getData().push_back(this->data(place).set.size());
|
|
|
|
}
|
2017-09-17 20:22:39 +00:00
|
|
|
|
|
|
|
const char * getHeaderFilePath() const override { return __FILE__; }
|
2013-08-21 13:26:42 +00:00
|
|
|
};
|
2013-06-25 14:16:16 +00:00
|
|
|
|
2013-04-26 18:57:08 +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.
|
2018-03-17 18:02:19 +00:00
|
|
|
* But (for the possibility of efficient implementation), you can not pass several arguments, among which there are tuples.
|
2015-10-29 02:13:37 +00:00
|
|
|
*/
|
|
|
|
template <typename Data, bool argument_is_tuple>
|
2017-12-20 20:25:22 +00:00
|
|
|
class AggregateFunctionUniqVariadic final : public IAggregateFunctionDataHelper<Data, AggregateFunctionUniqVariadic<Data, argument_is_tuple>>
|
2015-10-29 02:13:37 +00:00
|
|
|
{
|
|
|
|
private:
|
2017-12-25 04:01:46 +00:00
|
|
|
static constexpr bool is_exact = std::is_same_v<Data, AggregateFunctionUniqExactData<String>>;
|
2015-10-29 04:02:22 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t num_args = 0;
|
2015-10-29 02:13:37 +00:00
|
|
|
|
|
|
|
public:
|
2017-12-20 20:25:22 +00:00
|
|
|
AggregateFunctionUniqVariadic(const DataTypes & arguments)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
if (argument_is_tuple)
|
|
|
|
num_args = typeid_cast<const DataTypeTuple &>(*arguments[0]).getElements().size();
|
|
|
|
else
|
|
|
|
num_args = arguments.size();
|
|
|
|
}
|
|
|
|
|
2017-12-20 07:36:30 +00:00
|
|
|
String getName() const override { return Data::getName(); }
|
|
|
|
|
|
|
|
DataTypePtr getReturnType() const override
|
|
|
|
{
|
|
|
|
return std::make_shared<DataTypeUInt64>();
|
|
|
|
}
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
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));
|
|
|
|
}
|
|
|
|
|
2017-12-01 21:51:50 +00:00
|
|
|
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
this->data(place).set.merge(this->data(rhs).set);
|
|
|
|
}
|
|
|
|
|
|
|
|
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
|
|
|
|
{
|
|
|
|
this->data(place).set.write(buf);
|
|
|
|
}
|
|
|
|
|
|
|
|
void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena *) const override
|
|
|
|
{
|
|
|
|
this->data(place).set.read(buf);
|
|
|
|
}
|
|
|
|
|
|
|
|
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
|
|
|
|
{
|
|
|
|
static_cast<ColumnUInt64 &>(to).getData().push_back(this->data(place).set.size());
|
|
|
|
}
|
|
|
|
|
2017-09-17 20:22:39 +00:00
|
|
|
const char * getHeaderFilePath() const override { return __FILE__; }
|
2015-10-29 02:13:37 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2011-09-26 04:00:46 +00:00
|
|
|
}
|