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

223 lines
6.3 KiB
C++
Raw Normal View History

#pragma once
#include <DB/Common/PODArray.h>
#include <DB/IO/WriteHelpers.h>
#include <DB/IO/ReadHelpers.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/AggregateFunctions/IUnaryAggregateFunction.h>
#include <DB/AggregateFunctions/QuantilesCommon.h>
#include <DB/Columns/ColumnArray.h>
namespace DB
{
2017-03-09 00:56:38 +00:00
/** The state is an array, into which all values are added.
* NOTE If there are few different values then this is not optimal.
* For 8 and 16-bit values it might be better to use a lookup table.
*/
template <typename T>
struct AggregateFunctionQuantileExactData
{
2017-03-09 00:56:38 +00:00
/// The memory will be allocated to several elements at once, so that the state occupies 64 bytes.
static constexpr size_t bytes_in_arena = 64 - sizeof(PODArray<T>);
using Array = PODArray<T, bytes_in_arena, AllocatorWithStackMemory<Allocator<false>, bytes_in_arena>>;
Array array;
};
2017-03-09 00:56:38 +00:00
/** Exactly calculates the quantile.
* The argument type can only be a numeric type (including date and date-time).
* The result type is the same as the argument type.
*/
template <typename T>
class AggregateFunctionQuantileExact final
: public IUnaryAggregateFunction<AggregateFunctionQuantileExactData<T>, AggregateFunctionQuantileExact<T>>
{
private:
double level;
DataTypePtr type;
public:
AggregateFunctionQuantileExact(double level_ = 0.5) : level(level_) {}
String getName() const override { return "quantileExact"; }
DataTypePtr getReturnType() const override
{
return type;
}
void setArgument(const DataTypePtr & argument)
{
type = argument;
}
void setParameters(const Array & params) override
{
if (params.size() != 1)
throw Exception("Aggregate function " + getName() + " requires exactly one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
Squashed commit of the following: commit c567d4e1fe8d54e6363e47548f1e3927cc5ee78f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Jan 6 20:35:01 2017 +0300 Style [#METR-2944]. commit 26bf3e1228e03f46c29b13edb0e3770bd453e3f1 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Jan 6 20:33:11 2017 +0300 Miscellaneous [#METR-2944]. commit eb946f4c6fd4bb0e9e5c7fb1468d36be3dfca5a5 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Jan 6 20:30:19 2017 +0300 Miscellaneous [#METR-2944]. commit 78c867a14744b5af2db8d37caf7804fc2057ea51 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Jan 6 20:11:41 2017 +0300 Miscellaneous [#METR-2944]. commit 6604c5c83cfcedc81c8da4da026711920d5963b4 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Jan 6 19:56:15 2017 +0300 Miscellaneous [#METR-2944]. commit 23fbf05c1d4bead636458ec21b05a101b1152e33 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Jan 6 19:47:52 2017 +0300 Miscellaneous [#METR-2944]. commit 98772faf11a7d450d473f7fa84f8a9ae24f7b59b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Jan 6 19:46:05 2017 +0300 Miscellaneous [#METR-2944]. commit 3dc636ab9f9359dbeac2e8d997ae563d4ca147e2 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Jan 6 19:39:46 2017 +0300 Miscellaneous [#METR-2944]. commit 3e16aee95482f374ee3eda1a4dbe9ba5cdce02e8 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Jan 6 19:38:03 2017 +0300 Miscellaneous [#METR-2944]. commit ae7e7e90eb1f82bd0fe0f887708d08b9e7755612 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Jan 6 19:34:15 2017 +0300 Miscellaneous [#METR-2944].
2017-01-06 17:41:19 +00:00
level = applyVisitor(FieldVisitorConvertToNumber<Float64>(), params[0]);
}
void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num, Arena *) const
{
this->data(place).array.push_back(static_cast<const ColumnVector<T> &>(column).getData()[row_num]);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override
{
this->data(place).array.insert(this->data(rhs).array.begin(), this->data(rhs).array.end());
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
const auto & array = this->data(place).array;
size_t size = array.size();
writeVarUInt(size, buf);
buf.write(reinterpret_cast<const char *>(&array[0]), size * sizeof(array[0]));
}
2016-09-22 23:26:08 +00:00
void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena *) const override
{
auto & array = this->data(place).array;
size_t size = 0;
readVarUInt(size, buf);
2016-03-12 04:01:03 +00:00
array.resize(size);
buf.read(reinterpret_cast<char *>(&array[0]), size * sizeof(array[0]));
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
2017-03-09 00:56:38 +00:00
/// Sorting an array will not be considered a violation of constancy.
auto & array = const_cast<typename AggregateFunctionQuantileExactData<T>::Array &>(this->data(place).array);
T quantile = T();
if (!array.empty())
{
size_t n = level < 1
? level * array.size()
: (array.size() - 1);
2017-03-09 00:56:38 +00:00
std::nth_element(array.begin(), array.begin() + n, array.end()); /// NOTE You can think of the radix-select algorithm.
quantile = array[n];
}
static_cast<ColumnVector<T> &>(to).getData().push_back(quantile);
}
};
2017-03-09 00:56:38 +00:00
/** The same, but allows you to calculate several quantiles at once.
* To do this, takes several levels as parameters. Example: quantilesExact(0.5, 0.8, 0.9, 0.95)(ConnectTiming).
* Returns an array of results.
*/
template <typename T>
class AggregateFunctionQuantilesExact final
: public IUnaryAggregateFunction<AggregateFunctionQuantileExactData<T>, AggregateFunctionQuantilesExact<T>>
{
private:
QuantileLevels<double> levels;
DataTypePtr type;
public:
String getName() const override { return "quantilesExact"; }
DataTypePtr getReturnType() const override
{
return std::make_shared<DataTypeArray>(type);
}
void setArgument(const DataTypePtr & argument)
{
type = argument;
}
void setParameters(const Array & params) override
{
levels.set(params);
}
void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num, Arena *) const
{
this->data(place).array.push_back(static_cast<const ColumnVector<T> &>(column).getData()[row_num]);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override
{
this->data(place).array.insert(this->data(rhs).array.begin(), this->data(rhs).array.end());
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
const auto & array = this->data(place).array;
size_t size = array.size();
writeVarUInt(size, buf);
buf.write(reinterpret_cast<const char *>(&array[0]), size * sizeof(array[0]));
}
2016-09-22 23:26:08 +00:00
void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena *) const override
{
auto & array = this->data(place).array;
size_t size = 0;
readVarUInt(size, buf);
2016-03-12 04:01:03 +00:00
array.resize(size);
buf.read(reinterpret_cast<char *>(&array[0]), size * sizeof(array[0]));
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
2017-03-09 00:56:38 +00:00
/// Sorting an array will not be considered a violation of constancy.
auto & array = const_cast<typename AggregateFunctionQuantileExactData<T>::Array &>(this->data(place).array);
ColumnArray & arr_to = static_cast<ColumnArray &>(to);
ColumnArray::Offsets_t & offsets_to = arr_to.getOffsets();
size_t num_levels = levels.size();
offsets_to.push_back((offsets_to.size() == 0 ? 0 : offsets_to.back()) + num_levels);
typename ColumnVector<T>::Container_t & data_to = static_cast<ColumnVector<T> &>(arr_to.getData()).getData();
size_t old_size = data_to.size();
data_to.resize(old_size + num_levels);
if (!array.empty())
{
size_t prev_n = 0;
for (auto level_index : levels.permutation)
{
auto level = levels.levels[level_index];
size_t n = level < 1
? level * array.size()
: (array.size() - 1);
std::nth_element(array.begin() + prev_n, array.begin() + n, array.end());
data_to[old_size + level_index] = array[n];
prev_n = n;
}
}
else
{
for (size_t i = 0; i < num_levels; ++i)
data_to[old_size + i] = T();
}
}
};
}