ClickHouse/src/AggregateFunctions/AggregateFunctionBoundingRatio.h

162 lines
4.1 KiB
C++
Raw Normal View History

2018-09-14 18:26:43 +00:00
#pragma once
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnsNumber.h>
2018-09-14 18:26:43 +00:00
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <AggregateFunctions/Helpers.h>
2018-11-01 17:55:11 +00:00
#include <AggregateFunctions/IAggregateFunction.h>
#include <Common/assert_cast.h>
2018-09-14 18:26:43 +00:00
namespace DB
{
struct Settings;
2018-09-14 18:26:43 +00:00
namespace ErrorCodes
2018-09-14 18:26:43 +00:00
{
extern const int BAD_ARGUMENTS;
}
2018-11-01 17:55:11 +00:00
/** Tracks the leftmost and rightmost (x, y) data points.
*/
2021-05-08 14:01:25 +00:00
struct AggregateFunctionBoundingRatioData //-V730
{
struct Point
{
Float64 x;
Float64 y;
};
2018-11-01 17:55:11 +00:00
bool empty = true;
Point left;
Point right;
2018-11-01 17:55:11 +00:00
void add(Float64 x, Float64 y)
2018-11-01 17:55:11 +00:00
{
Point point{x, y};
if (empty)
2018-11-01 17:55:11 +00:00
{
left = point;
right = point;
empty = false;
2018-11-01 17:55:11 +00:00
}
else if (point.x < left.x)
{
left = point;
}
else if (point.x > right.x)
2018-11-01 17:55:11 +00:00
{
right = point;
2018-11-01 17:55:11 +00:00
}
}
void merge(const AggregateFunctionBoundingRatioData & other)
{
if (empty)
2018-11-01 17:55:11 +00:00
{
*this = other;
2018-11-01 17:55:11 +00:00
}
else
2018-11-01 17:55:11 +00:00
{
if (other.left.x < left.x)
left = other.left;
if (other.right.x > right.x)
right = other.right;
2018-11-01 17:55:11 +00:00
}
}
void serialize(WriteBuffer & buf) const
{
writeBinary(empty, buf);
2018-11-01 17:55:11 +00:00
if (!empty)
{
writePODBinary(left, buf);
writePODBinary(right, buf);
}
2018-11-01 17:55:11 +00:00
}
void deserialize(ReadBuffer & buf)
{
readBinary(empty, buf);
2018-11-01 17:55:11 +00:00
if (!empty)
{
readPODBinary(left, buf);
readPODBinary(right, buf);
}
2018-11-01 17:55:11 +00:00
}
2018-09-14 18:26:43 +00:00
};
2018-11-01 17:55:11 +00:00
class AggregateFunctionBoundingRatio final : public IAggregateFunctionDataHelper<AggregateFunctionBoundingRatioData, AggregateFunctionBoundingRatio>
2018-09-14 18:26:43 +00:00
{
private:
/** Calculates the slope of a line between leftmost and rightmost data points.
* (y2 - y1) / (x2 - x1)
*/
static Float64 NO_SANITIZE_UNDEFINED getBoundingRatio(const AggregateFunctionBoundingRatioData & data)
2018-11-01 17:55:11 +00:00
{
if (data.empty)
return std::numeric_limits<Float64>::quiet_NaN();
return (data.right.y - data.left.y) / (data.right.x - data.left.x);
2018-11-01 17:55:11 +00:00
}
2018-09-14 18:26:43 +00:00
public:
2018-11-01 17:55:11 +00:00
String getName() const override
{
return "boundingRatio";
}
explicit AggregateFunctionBoundingRatio(const DataTypes & arguments)
2019-02-11 19:26:32 +00:00
: IAggregateFunctionDataHelper<AggregateFunctionBoundingRatioData, AggregateFunctionBoundingRatio>(arguments, {})
2018-11-01 17:55:11 +00:00
{
const auto * x_arg = arguments.at(0).get();
const auto * y_arg = arguments.at(1).get();
2018-11-01 17:55:11 +00:00
if (!x_arg->isValueRepresentedByNumber() || !y_arg->isValueRepresentedByNumber())
throw Exception("Illegal types of arguments of aggregate function " + getName() + ", must have number representation.",
ErrorCodes::BAD_ARGUMENTS);
2018-11-01 17:55:11 +00:00
}
DataTypePtr getReturnType() const override
{
return std::make_shared<DataTypeFloat64>();
}
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, const size_t row_num, Arena *) const override
2018-11-01 17:55:11 +00:00
{
/// NOTE Slightly inefficient.
const auto x = columns[0]->getFloat64(row_num);
const auto y = columns[1]->getFloat64(row_num);
data(place).add(x, y);
2018-11-01 17:55:11 +00:00
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
2018-11-01 17:55:11 +00:00
{
data(place).merge(data(rhs));
}
2021-05-30 13:57:30 +00:00
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
2018-11-01 17:55:11 +00:00
{
data(place).serialize(buf);
}
2021-05-31 14:44:57 +00:00
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
2018-11-01 17:55:11 +00:00
{
data(place).deserialize(buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
2018-11-01 17:55:11 +00:00
{
assert_cast<ColumnFloat64 &>(to).getData().push_back(getBoundingRatio(data(place)));
2018-11-01 17:55:11 +00:00
}
2018-09-14 18:26:43 +00:00
};
}