ClickHouse/dbms/src/AggregateFunctions/AggregateFunctionHistogram.h

364 lines
10 KiB
C++
Raw Normal View History

2018-06-22 18:30:09 +00:00
#pragma once
2018-06-15 23:13:47 +00:00
#include <Common/Arena.h>
2018-06-22 18:30:09 +00:00
#include <Common/PODArray.h>
#include <Common/AutoArray.h>
2018-06-22 18:30:09 +00:00
#include <Columns/ColumnVector.h>
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnArray.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <IO/WriteBuffer.h>
#include <IO/ReadBuffer.h>
#include <IO/VarInt.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <queue>
2018-06-22 18:30:09 +00:00
namespace DB {
namespace ErrorCodes
{
extern const int TOO_LARGE_ARRAY_SIZE;
}
/**
* distance compression algorigthm implementation
* http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
*/
class AggregateFunctionHistogramData
{
public:
using Mean = Float64;
using Weight = Float64;
2018-06-22 18:30:09 +00:00
private:
struct WeightedValue {
Mean mean;
Weight weight;
WeightedValue operator + (const WeightedValue& other)
{
return {mean + other.weight * (other.mean - mean) / (other.weight + weight), other.weight + weight};
2018-06-22 18:30:09 +00:00
}
};
private:
UInt32 max_bins;
// quantity of stored weighted-values
2018-06-15 23:13:47 +00:00
UInt32 size;
// calculated lower and upper bounds of seen points
2018-06-22 18:30:09 +00:00
Mean lower_bound;
Mean upper_bound;
static constexpr Mean epsilon = 1e-8;
2018-06-22 18:30:09 +00:00
// Weighted values representation of histogram.
// We allow up to max_bins * 2 values stored in intermediate states
// Is allocated in arena, so there is no explicit management.
2018-06-15 23:13:47 +00:00
WeightedValue* points;
2018-06-22 18:30:09 +00:00
private:
void sort()
{
std::sort(points, points + size,
[](const WeightedValue& first, const WeightedValue& second)
{
return first.mean < second.mean;
});
2018-06-22 18:30:09 +00:00
}
/**
* Repeatedly fuse most close values until max_bins bins left
2018-06-22 18:30:09 +00:00
*/
void compress()
{
sort();
auto newsz = size;
if (size <= max_bins) return;
// Maintain doubly-linked list of "active" points
// and store neighbour pairs in priority queue by distance
AutoArray<int> previous(size + 1);
AutoArray<int> next(size + 1);
AutoArray<bool> active(size + 1, true);
active[size] = false;
auto delete_node = [&](int i)
2018-06-22 18:30:09 +00:00
{
previous[next[i]] = previous[i];
next[previous[i]] = next[i];
active[i] = false;
};
for (size_t i = 0; i <= size; i++)
{
previous[i] = i - 1;
next[i] = i + 1;
}
next[size] = 0;
previous[0] = size;
2018-06-22 18:30:09 +00:00
using QueueItem = std::pair<Mean, int>;
std::vector<QueueItem> storage;
storage.reserve(2 * size - max_bins);
std::priority_queue<QueueItem, std::vector<QueueItem>, std::greater<QueueItem>> queue;
auto quality = [&](int i) { return points[next[i]].mean - points[i].mean; };
for (size_t i = 0; i + 1 < size; i++)
queue.push({quality(i), i});
while (newsz > max_bins && !queue.empty())
{
auto min_item = queue.top();
queue.pop();
auto l = min_item.second;
auto r = next[l];
if (!active[l] || !active[r] || quality(l) > min_item.first)
continue;
points[l] = points[l] + points[r];
delete_node(r);
if (active[next[l]])
queue.push({quality(l), l});
if (active[previous[l]])
queue.push({quality(previous[l]), previous[l]});
newsz--;
2018-06-22 18:30:09 +00:00
}
size_t l = 0;
for (size_t r = 0; r < size; r++)
if (active[r])
points[l++] = points[r];
size = newsz;
2018-06-22 18:30:09 +00:00
}
/***
2018-06-15 23:13:47 +00:00
* Delete too close points from histogram.
* Assumes that points are sorted.
2018-06-22 18:30:09 +00:00
*/
void unique()
{
2018-06-15 23:13:47 +00:00
size_t l = 0;
for (auto r = l + 1; r < size; r++)
2018-06-22 18:30:09 +00:00
{
if (points[l].mean + epsilon >= points[r].mean)
2018-06-22 18:30:09 +00:00
{
2018-06-15 23:13:47 +00:00
points[l] = points[l] + points[r];
2018-06-22 18:30:09 +00:00
}
else
{
l++;
2018-06-15 23:13:47 +00:00
points[l] = points[r];
2018-06-22 18:30:09 +00:00
}
}
2018-06-15 23:13:47 +00:00
size = l + 1;
}
void init(Arena* arena)
{
points = reinterpret_cast<WeightedValue*>(arena->alloc(max_bins * 2 * sizeof(WeightedValue)));
2018-06-22 18:30:09 +00:00
}
public:
AggregateFunctionHistogramData(UInt32 max_bins)
: max_bins(max_bins)
2018-06-15 23:13:47 +00:00
, size(0)
, lower_bound(std::numeric_limits<Mean>::max())
, upper_bound(std::numeric_limits<Mean>::min())
2018-06-15 23:13:47 +00:00
, points(nullptr)
2018-06-22 18:30:09 +00:00
{
}
void insertResultInto(ColumnVector<Mean>& to_lower, ColumnVector<Mean>& to_upper, ColumnVector<Weight>& to_weights) {
if (!points) return;
2018-06-22 18:30:09 +00:00
compress();
unique();
2018-06-15 23:13:47 +00:00
for (size_t i = 0; i < size; i++)
2018-06-22 18:30:09 +00:00
{
2018-06-15 21:48:13 +00:00
to_lower.insert((i == 0) ? lower_bound : (points[i].mean + points[i - 1].mean) / 2);
2018-06-15 23:13:47 +00:00
to_upper.insert((i + 1 == size) ? upper_bound : (points[i].mean + points[i + 1].mean) / 2);
2018-06-22 18:30:09 +00:00
2018-06-15 21:48:13 +00:00
// linear density approximation
Weight lower_weight = (i == 0) ? points[i].weight : ((points[i - 1].weight) + points[i].weight * 3) / 4;
2018-06-15 23:13:47 +00:00
Weight upper_weight = (i + 1 == size) ? points[i].weight : (points[i + 1].weight + points[i].weight * 3) / 4;
2018-06-15 21:48:13 +00:00
to_weights.insert((lower_weight + upper_weight) / 2);
2018-06-22 18:30:09 +00:00
}
}
2018-06-15 23:13:47 +00:00
void add(Mean value, Weight weight, Arena* arena)
2018-06-22 18:30:09 +00:00
{
2018-06-15 23:13:47 +00:00
if (!points)
init(arena);
points[size++] = {value, weight};
2018-06-22 18:30:09 +00:00
lower_bound = std::min(lower_bound, value);
upper_bound = std::max(upper_bound, value);
if (size >= max_bins * 2)
2018-06-22 18:30:09 +00:00
{
compress();
}
}
2018-06-15 23:13:47 +00:00
void merge(const AggregateFunctionHistogramData& other, Arena* arena)
2018-06-22 18:30:09 +00:00
{
lower_bound = std::min(lower_bound, other.lower_bound);
upper_bound = std::max(lower_bound, other.upper_bound);
2018-06-15 23:13:47 +00:00
for (size_t i = 0; i < other.size; i++)
{
add(other.points[i].mean, other.points[i].weight, arena);
2018-06-22 18:30:09 +00:00
}
}
void write(WriteBuffer & buf) const
{
buf.write(reinterpret_cast<const char *>(&lower_bound), sizeof(lower_bound));
buf.write(reinterpret_cast<const char *>(&upper_bound), sizeof(upper_bound));
2018-06-15 23:13:47 +00:00
writeVarUInt(size, buf);
buf.write(reinterpret_cast<const char *>(points), size * sizeof(WeightedValue));
2018-06-22 18:30:09 +00:00
}
2018-06-15 23:13:47 +00:00
void read(ReadBuffer & buf, Arena* arena)
2018-06-22 18:30:09 +00:00
{
buf.read(reinterpret_cast<char *>(&lower_bound), sizeof(lower_bound));
buf.read(reinterpret_cast<char *>(&upper_bound), sizeof(upper_bound));
readVarUInt(size, buf);
if (size > max_bins * 2)
2018-06-22 18:30:09 +00:00
throw Exception("Too many bins", ErrorCodes::TOO_LARGE_ARRAY_SIZE);
2018-06-15 23:13:47 +00:00
if (!points)
init(arena);
buf.read(reinterpret_cast<char *>(points), size * sizeof(points[0]));
2018-06-22 18:30:09 +00:00
}
};
template <typename T>
class AggregateFunctionHistogram final: public IAggregateFunctionHelper<AggregateFunctionHistogram<T>>
{
private:
using Data = AggregateFunctionHistogramData;
UInt32 max_bins;
Data& data(AggregateDataPtr place) const
{
return *reinterpret_cast<Data*>(place);
}
const Data& data(ConstAggregateDataPtr place) const
{
return *reinterpret_cast<const Data*>(place);
}
public:
AggregateFunctionHistogram(UInt32 max_bins)
: max_bins(max_bins)
{
}
void destroy(AggregateDataPtr place) const noexcept override
{
data(place).~Data();
}
bool hasTrivialDestructor() const override
{
return std::is_trivially_destructible_v<Data>;
}
size_t alignOfData() const override
{
return alignof(Data);
}
size_t sizeOfData() const override
{
return sizeof(Data);
}
void create(AggregateDataPtr place) const override
{
new (place) Data(max_bins);
}
DataTypePtr getReturnType() const override
{
DataTypes types;
auto mean = std::make_shared<DataTypeNumber<Data::Mean>>();
auto weight = std::make_shared<DataTypeNumber<Data::Weight>>();
// lower bound
types.emplace_back(mean);
// upper bound
types.emplace_back(mean);
// weight
types.emplace_back(weight);
auto tuple = std::make_shared<DataTypeTuple>(types);
return std::make_shared<DataTypeArray>(tuple);
}
2018-06-15 23:13:47 +00:00
bool allocatesMemoryInArena() const override
{
return true;
}
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override
2018-06-22 18:30:09 +00:00
{
auto val = static_cast<const ColumnVector<T> &>(*columns[0]).getData()[row_num];
2018-06-15 23:13:47 +00:00
this->data(place).add(static_cast<Data::Mean>(val), 1, arena);
2018-06-22 18:30:09 +00:00
}
2018-06-15 23:13:47 +00:00
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override
2018-06-22 18:30:09 +00:00
{
2018-06-15 23:13:47 +00:00
this->data(place).merge(this->data(rhs), arena);
2018-06-22 18:30:09 +00:00
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
this->data(place).write(buf);
}
2018-06-15 23:13:47 +00:00
void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena * arena) const override
2018-06-22 18:30:09 +00:00
{
2018-06-15 23:13:47 +00:00
this->data(place).read(buf, arena);
2018-06-22 18:30:09 +00:00
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
auto& data = this->data(const_cast<AggregateDataPtr>(place));
auto & to_array = static_cast<ColumnArray &>(to);
ColumnArray::Offsets & offsets_to = to_array.getOffsets();
auto & to_tuple = static_cast<ColumnTuple &>(to_array.getData());
auto & to_lower = static_cast<ColumnVector<Data::Mean> &>(to_tuple.getColumn(0));
auto & to_upper = static_cast<ColumnVector<Data::Mean> &>(to_tuple.getColumn(1));
auto & to_weights = static_cast<ColumnVector<Data::Weight> &>(to_tuple.getColumn(2));
data.insertResultInto(to_lower, to_upper, to_weights);
offsets_to.push_back(to_tuple.size());
}
const char * getHeaderFilePath() const override { return __FILE__; }
String getName() const override { return "histogram"; }
};
}