mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 07:01:59 +00:00
Merge branch 'master' into remove-trash-5
This commit is contained in:
commit
bafc5f24c7
@ -1,12 +1,18 @@
|
|||||||
#include <AggregateFunctions/AggregateFunctionCategoricalInformationValue.h>
|
#include <AggregateFunctions/IAggregateFunction.h>
|
||||||
|
|
||||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||||
#include <AggregateFunctions/FactoryHelpers.h>
|
#include <AggregateFunctions/FactoryHelpers.h>
|
||||||
#include <AggregateFunctions/Helpers.h>
|
#include <AggregateFunctions/Helpers.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>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
struct Settings;
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
@ -15,6 +21,136 @@ namespace ErrorCodes
|
|||||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/** The function takes arguments x1, x2, ... xn, y. All arguments are bool.
|
||||||
|
* x arguments represents the fact that some category is true.
|
||||||
|
*
|
||||||
|
* It calculates how many times y was true and how many times y was false when every n-th category was true
|
||||||
|
* and the total number of times y was true and false.
|
||||||
|
*
|
||||||
|
* So, the size of the state is (n + 1) * 2 cells.
|
||||||
|
*/
|
||||||
|
class AggregateFunctionCategoricalIV final : public IAggregateFunctionHelper<AggregateFunctionCategoricalIV>
|
||||||
|
{
|
||||||
|
private:
|
||||||
|
using Counter = UInt64;
|
||||||
|
size_t category_count;
|
||||||
|
|
||||||
|
Counter & counter(AggregateDataPtr __restrict place, size_t i, bool what) const
|
||||||
|
{
|
||||||
|
return reinterpret_cast<Counter *>(place)[i * 2 + (what ? 1 : 0)];
|
||||||
|
}
|
||||||
|
|
||||||
|
const Counter & counter(ConstAggregateDataPtr __restrict place, size_t i, bool what) const
|
||||||
|
{
|
||||||
|
return reinterpret_cast<const Counter *>(place)[i * 2 + (what ? 1 : 0)];
|
||||||
|
}
|
||||||
|
|
||||||
|
public:
|
||||||
|
AggregateFunctionCategoricalIV(const DataTypes & arguments_, const Array & params_) :
|
||||||
|
IAggregateFunctionHelper<AggregateFunctionCategoricalIV>{arguments_, params_},
|
||||||
|
category_count{arguments_.size() - 1}
|
||||||
|
{
|
||||||
|
// notice: argument types has been checked before
|
||||||
|
}
|
||||||
|
|
||||||
|
String getName() const override
|
||||||
|
{
|
||||||
|
return "categoricalInformationValue";
|
||||||
|
}
|
||||||
|
|
||||||
|
bool allocatesMemoryInArena() const override { return false; }
|
||||||
|
|
||||||
|
void create(AggregateDataPtr __restrict place) const override
|
||||||
|
{
|
||||||
|
memset(place, 0, sizeOfData());
|
||||||
|
}
|
||||||
|
|
||||||
|
void destroy(AggregateDataPtr __restrict) const noexcept override
|
||||||
|
{
|
||||||
|
// nothing
|
||||||
|
}
|
||||||
|
|
||||||
|
bool hasTrivialDestructor() const override
|
||||||
|
{
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
size_t sizeOfData() const override
|
||||||
|
{
|
||||||
|
return sizeof(Counter) * (category_count + 1) * 2;
|
||||||
|
}
|
||||||
|
|
||||||
|
size_t alignOfData() const override
|
||||||
|
{
|
||||||
|
return alignof(Counter);
|
||||||
|
}
|
||||||
|
|
||||||
|
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
|
||||||
|
{
|
||||||
|
const auto * y_col = static_cast<const ColumnUInt8 *>(columns[category_count]);
|
||||||
|
bool y = y_col->getData()[row_num];
|
||||||
|
|
||||||
|
for (size_t i = 0; i < category_count; ++i)
|
||||||
|
{
|
||||||
|
const auto * x_col = static_cast<const ColumnUInt8 *>(columns[i]);
|
||||||
|
bool x = x_col->getData()[row_num];
|
||||||
|
|
||||||
|
if (x)
|
||||||
|
++counter(place, i, y);
|
||||||
|
}
|
||||||
|
|
||||||
|
++counter(place, category_count, y);
|
||||||
|
}
|
||||||
|
|
||||||
|
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
|
||||||
|
{
|
||||||
|
for (size_t i = 0; i <= category_count; ++i)
|
||||||
|
{
|
||||||
|
counter(place, i, false) += counter(rhs, i, false);
|
||||||
|
counter(place, i, true) += counter(rhs, i, true);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
|
||||||
|
{
|
||||||
|
buf.write(place, sizeOfData());
|
||||||
|
}
|
||||||
|
|
||||||
|
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
|
||||||
|
{
|
||||||
|
buf.read(place, sizeOfData());
|
||||||
|
}
|
||||||
|
|
||||||
|
DataTypePtr getReturnType() const override
|
||||||
|
{
|
||||||
|
return std::make_shared<DataTypeArray>(
|
||||||
|
std::make_shared<DataTypeNumber<Float64>>());
|
||||||
|
}
|
||||||
|
|
||||||
|
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override /// NOLINT
|
||||||
|
{
|
||||||
|
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);
|
||||||
|
|
||||||
|
Float64 sum_no = static_cast<Float64>(counter(place, category_count, false));
|
||||||
|
Float64 sum_yes = static_cast<Float64>(counter(place, category_count, true));
|
||||||
|
|
||||||
|
for (size_t i = 0; i < category_count; ++i)
|
||||||
|
{
|
||||||
|
Float64 no = static_cast<Float64>(counter(place, i, false));
|
||||||
|
Float64 yes = static_cast<Float64>(counter(place, i, true));
|
||||||
|
|
||||||
|
data_col.insertValue((no / sum_no - yes / sum_yes) * (log((no / sum_no) / (yes / sum_yes))));
|
||||||
|
}
|
||||||
|
|
||||||
|
offset_col.insertValue(data_col.size());
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
namespace
|
namespace
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -39,16 +175,15 @@ AggregateFunctionPtr createAggregateFunctionCategoricalIV(
|
|||||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
}
|
}
|
||||||
|
|
||||||
return std::make_shared<AggregateFunctionCategoricalIV<>>(arguments, params);
|
return std::make_shared<AggregateFunctionCategoricalIV>(arguments, params);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
void registerAggregateFunctionCategoricalIV(
|
void registerAggregateFunctionCategoricalIV(AggregateFunctionFactory & factory)
|
||||||
AggregateFunctionFactory & factory
|
|
||||||
)
|
|
||||||
{
|
{
|
||||||
factory.registerFunction("categoricalInformationValue", createAggregateFunctionCategoricalIV);
|
AggregateFunctionProperties properties = { .returns_default_when_only_null = true };
|
||||||
|
factory.registerFunction("categoricalInformationValue", { createAggregateFunctionCategoricalIV, properties });
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -1,135 +0,0 @@
|
|||||||
#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>
|
|
||||||
|
|
||||||
#include <base/range.h>
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
struct Settings;
|
|
||||||
|
|
||||||
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";
|
|
||||||
}
|
|
||||||
|
|
||||||
bool allocatesMemoryInArena() const override { return false; }
|
|
||||||
|
|
||||||
void create(AggregateDataPtr __restrict place) const override
|
|
||||||
{
|
|
||||||
memset(place, 0, sizeOfData());
|
|
||||||
}
|
|
||||||
|
|
||||||
void destroy(AggregateDataPtr __restrict) const noexcept override
|
|
||||||
{
|
|
||||||
// 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);
|
|
||||||
}
|
|
||||||
|
|
||||||
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
|
|
||||||
{
|
|
||||||
const auto * y_col = static_cast<const ColumnUInt8 *>(columns[category_count]);
|
|
||||||
bool y = y_col->getData()[row_num];
|
|
||||||
|
|
||||||
for (size_t i : collections::range(0, category_count))
|
|
||||||
{
|
|
||||||
const auto * x_col = static_cast<const ColumnUInt8 *>(columns[i]);
|
|
||||||
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;
|
|
||||||
}
|
|
||||||
|
|
||||||
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
|
|
||||||
{
|
|
||||||
for (size_t i : collections::range(0, category_count + 1))
|
|
||||||
{
|
|
||||||
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];
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
|
|
||||||
{
|
|
||||||
buf.write(place, sizeOfData());
|
|
||||||
}
|
|
||||||
|
|
||||||
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
|
|
||||||
{
|
|
||||||
buf.read(place, sizeOfData());
|
|
||||||
}
|
|
||||||
|
|
||||||
DataTypePtr getReturnType() const override
|
|
||||||
{
|
|
||||||
return std::make_shared<DataTypeArray>(
|
|
||||||
std::make_shared<DataTypeNumber<Float64>>()
|
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override /// NOLINT
|
|
||||||
{
|
|
||||||
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;
|
|
||||||
|
|
||||||
for (size_t i : collections::range(0, category_count))
|
|
||||||
{
|
|
||||||
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());
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
}
|
|
@ -174,7 +174,7 @@ namespace
|
|||||||
{
|
{
|
||||||
auto current_offset = offsets_data[i];
|
auto current_offset = offsets_data[i];
|
||||||
sizes_data[i] = current_offset - prev_offset;
|
sizes_data[i] = current_offset - prev_offset;
|
||||||
prev_offset = current_offset;
|
prev_offset = current_offset;
|
||||||
}
|
}
|
||||||
|
|
||||||
return column_sizes;
|
return column_sizes;
|
||||||
|
@ -0,0 +1,12 @@
|
|||||||
|
0.347
|
||||||
|
0.5
|
||||||
|
0.347
|
||||||
|
0.347
|
||||||
|
[nan]
|
||||||
|
[nan]
|
||||||
|
[nan]
|
||||||
|
[nan]
|
||||||
|
[0]
|
||||||
|
\N
|
||||||
|
[nan]
|
||||||
|
[0,0]
|
@ -0,0 +1,14 @@
|
|||||||
|
SELECT round(arrayJoin(categoricalInformationValue(x.1, x.2)), 3) FROM (SELECT arrayJoin([(0, 0), (NULL, 2), (1, 0), (1, 1)]) AS x);
|
||||||
|
SELECT corr(c1, c2) FROM VALUES((0, 0), (NULL, 2), (1, 0), (1, 1));
|
||||||
|
SELECT round(arrayJoin(categoricalInformationValue(c1, c2)), 3) FROM VALUES((0, 0), (NULL, 2), (1, 0), (1, 1));
|
||||||
|
SELECT round(arrayJoin(categoricalInformationValue(c1, c2)), 3) FROM VALUES((0, 0), (NULL, 1), (1, 0), (1, 1));
|
||||||
|
SELECT categoricalInformationValue(c1, c2) FROM VALUES((0, 0), (NULL, 1));
|
||||||
|
SELECT categoricalInformationValue(c1, c2) FROM VALUES((NULL, 1)); -- { serverError 43 }
|
||||||
|
SELECT categoricalInformationValue(dummy, dummy);
|
||||||
|
SELECT categoricalInformationValue(dummy, dummy) WHERE 0;
|
||||||
|
SELECT categoricalInformationValue(c1, c2) FROM VALUES((toNullable(0), 0));
|
||||||
|
SELECT groupUniqArray(*) FROM VALUES(toNullable(0));
|
||||||
|
SELECT groupUniqArray(*) FROM VALUES(NULL);
|
||||||
|
SELECT categoricalInformationValue(c1, c2) FROM VALUES((NULL, NULL)); -- { serverError 43 }
|
||||||
|
SELECT categoricalInformationValue(c1, c2) FROM VALUES((0, 0), (NULL, 0));
|
||||||
|
SELECT quantiles(0.5, 0.9)(c1) FROM VALUES(0::Nullable(UInt8));
|
Loading…
Reference in New Issue
Block a user