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

78 lines
1.9 KiB
C++
Raw Normal View History

2011-09-26 04:00:46 +00:00
#pragma once
#include <DB/IO/WriteHelpers.h>
#include <DB/IO/ReadHelpers.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/AggregateFunctions/IUnaryAggregateFunction.h>
namespace DB
{
template <typename T>
struct AggregateFunctionAvgData
2011-09-26 04:00:46 +00:00
{
T sum;
2011-09-26 04:00:46 +00:00
UInt64 count;
AggregateFunctionAvgData() : sum(0), count(0) {}
};
/// Считает арифметическое среднее значение чисел.
template <typename T>
class AggregateFunctionAvg final : public IUnaryAggregateFunction<AggregateFunctionAvgData<typename NearestFieldType<T>::Type>, AggregateFunctionAvg<T> >
{
public:
2015-11-11 02:04:23 +00:00
String getName() const override { return "avg"; }
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<DataTypeFloat64>();
2011-09-26 04:00:46 +00:00
}
void setArgument(const DataTypePtr & argument)
2011-09-26 04:00:46 +00:00
{
if (!argument->isNumeric())
throw Exception("Illegal type " + argument->getName() + " of argument for aggregate function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num, Arena *) const
2011-09-26 04:00:46 +00:00
{
this->data(place).sum += static_cast<const ColumnVector<T> &>(column).getData()[row_num];
++this->data(place).count;
2011-09-26 04:00:46 +00:00
}
2015-11-11 02:04:23 +00:00
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
2011-09-26 04:00:46 +00:00
{
this->data(place).sum += this->data(rhs).sum;
this->data(place).count += this->data(rhs).count;
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
{
writeBinary(this->data(place).sum, buf);
writeVarUInt(this->data(place).count, 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
readBinary(this->data(place).sum, buf);
readVarUInt(this->data(place).count, 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<ColumnFloat64 &>(to).getData().push_back(
static_cast<Float64>(this->data(place).sum) / this->data(place).count);
2011-09-26 04:00:46 +00:00
}
};
2011-09-26 04:00:46 +00:00
}