2019-12-12 13:28:28 +00:00
|
|
|
#pragma once
|
|
|
|
|
|
|
|
#include <AggregateFunctions/IAggregateFunction.h>
|
|
|
|
#include <Columns/ColumnArray.h>
|
|
|
|
#include <Columns/ColumnsNumber.h>
|
|
|
|
#include <DataTypes/DataTypeArray.h>
|
|
|
|
#include <DataTypes/DataTypesNumber.h>
|
|
|
|
#include <IO/ReadHelpers.h>
|
|
|
|
#include <IO/WriteHelpers.h>
|
|
|
|
|
2021-10-02 07:13:14 +00:00
|
|
|
#include <base/range.h>
|
2019-12-12 13:28:28 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
2021-05-26 11:32:14 +00:00
|
|
|
struct Settings;
|
2019-12-12 13:28:28 +00:00
|
|
|
|
|
|
|
template <typename T = UInt64>
|
|
|
|
class AggregateFunctionCategoricalIV final : public IAggregateFunctionHelper<AggregateFunctionCategoricalIV<T>>
|
|
|
|
{
|
|
|
|
private:
|
|
|
|
size_t category_count;
|
|
|
|
|
|
|
|
public:
|
|
|
|
AggregateFunctionCategoricalIV(const DataTypes & arguments_, const Array & params_) :
|
|
|
|
IAggregateFunctionHelper<AggregateFunctionCategoricalIV<T>> {arguments_, params_},
|
|
|
|
category_count {arguments_.size() - 1}
|
|
|
|
{
|
|
|
|
// notice: argument types has been checked before
|
|
|
|
}
|
|
|
|
|
|
|
|
String getName() const override
|
|
|
|
{
|
|
|
|
return "categoricalInformationValue";
|
|
|
|
}
|
|
|
|
|
2021-04-09 17:12:06 +00:00
|
|
|
bool allocatesMemoryInArena() const override { return false; }
|
|
|
|
|
2021-02-01 17:12:12 +00:00
|
|
|
void create(AggregateDataPtr __restrict place) const override
|
2019-12-12 13:28:28 +00:00
|
|
|
{
|
|
|
|
memset(place, 0, sizeOfData());
|
|
|
|
}
|
|
|
|
|
2022-06-15 12:11:10 +00:00
|
|
|
void destroy(AggregateDataPtr __restrict) const noexcept override
|
2019-12-12 13:28:28 +00:00
|
|
|
{
|
|
|
|
// nothing
|
|
|
|
}
|
|
|
|
|
|
|
|
bool hasTrivialDestructor() const override
|
|
|
|
{
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
|
|
|
size_t sizeOfData() const override
|
|
|
|
{
|
|
|
|
return sizeof(T) * (category_count + 1) * 2;
|
|
|
|
}
|
|
|
|
|
|
|
|
size_t alignOfData() const override
|
|
|
|
{
|
|
|
|
return alignof(T);
|
|
|
|
}
|
|
|
|
|
2022-06-15 12:11:10 +00:00
|
|
|
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
|
2019-12-12 13:28:28 +00:00
|
|
|
{
|
2022-03-11 17:55:41 +00:00
|
|
|
const auto * y_col = static_cast<const ColumnUInt8 *>(columns[category_count]);
|
2019-12-12 13:28:28 +00:00
|
|
|
bool y = y_col->getData()[row_num];
|
|
|
|
|
2021-06-15 19:55:21 +00:00
|
|
|
for (size_t i : collections::range(0, category_count))
|
2019-12-12 13:28:28 +00:00
|
|
|
{
|
2022-03-11 17:55:41 +00:00
|
|
|
const auto * x_col = static_cast<const ColumnUInt8 *>(columns[i]);
|
2019-12-12 13:28:28 +00:00
|
|
|
bool x = x_col->getData()[row_num];
|
|
|
|
|
|
|
|
if (x)
|
|
|
|
reinterpret_cast<T *>(place)[i * 2 + size_t(y)] += 1;
|
|
|
|
}
|
|
|
|
|
|
|
|
reinterpret_cast<T *>(place)[category_count * 2 + size_t(y)] += 1;
|
|
|
|
}
|
|
|
|
|
2022-06-15 12:11:10 +00:00
|
|
|
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
|
2019-12-12 13:28:28 +00:00
|
|
|
{
|
2021-06-15 19:55:21 +00:00
|
|
|
for (size_t i : collections::range(0, category_count + 1))
|
2019-12-12 13:28:28 +00:00
|
|
|
{
|
|
|
|
reinterpret_cast<T *>(place)[i * 2] += reinterpret_cast<const T *>(rhs)[i * 2];
|
|
|
|
reinterpret_cast<T *>(place)[i * 2 + 1] += reinterpret_cast<const T *>(rhs)[i * 2 + 1];
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-06-15 12:11:10 +00:00
|
|
|
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
|
2019-12-12 13:28:28 +00:00
|
|
|
{
|
|
|
|
buf.write(place, sizeOfData());
|
|
|
|
}
|
|
|
|
|
2022-06-15 12:11:10 +00:00
|
|
|
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
|
2019-12-12 13:28:28 +00:00
|
|
|
{
|
|
|
|
buf.read(place, sizeOfData());
|
|
|
|
}
|
|
|
|
|
|
|
|
DataTypePtr getReturnType() const override
|
|
|
|
{
|
|
|
|
return std::make_shared<DataTypeArray>(
|
|
|
|
std::make_shared<DataTypeNumber<Float64>>()
|
|
|
|
);
|
|
|
|
}
|
|
|
|
|
2022-03-11 17:55:41 +00:00
|
|
|
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override /// NOLINT
|
2019-12-12 13:28:28 +00:00
|
|
|
{
|
|
|
|
auto & col = static_cast<ColumnArray &>(to);
|
|
|
|
auto & data_col = static_cast<ColumnFloat64 &>(col.getData());
|
|
|
|
auto & offset_col = static_cast<ColumnArray::ColumnOffsets &>(
|
|
|
|
col.getOffsetsColumn()
|
|
|
|
);
|
|
|
|
|
|
|
|
data_col.reserve(data_col.size() + category_count);
|
|
|
|
|
|
|
|
T sum_no = reinterpret_cast<const T *>(place)[category_count * 2];
|
|
|
|
T sum_yes = reinterpret_cast<const T *>(place)[category_count * 2 + 1];
|
|
|
|
|
|
|
|
Float64 rev_no = 1. / sum_no;
|
|
|
|
Float64 rev_yes = 1. / sum_yes;
|
|
|
|
|
2021-06-15 19:55:21 +00:00
|
|
|
for (size_t i : collections::range(0, category_count))
|
2019-12-12 13:28:28 +00:00
|
|
|
{
|
|
|
|
T no = reinterpret_cast<const T *>(place)[i * 2];
|
|
|
|
T yes = reinterpret_cast<const T *>(place)[i * 2 + 1];
|
|
|
|
|
|
|
|
data_col.insertValue((no * rev_no - yes * rev_yes) * (log(no * rev_no) - log(yes * rev_yes)));
|
|
|
|
}
|
|
|
|
|
|
|
|
offset_col.insertValue(data_col.size());
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
}
|